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

[18/23] lucene-solr:feature/autoscaling: Squash-merge from master.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/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 d74650f..8b8269b 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -82,6 +82,7 @@ import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.search.SolrFieldCacheBean;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.AuthorizationPlugin;
 import org.apache.solr.security.HttpClientBuilderPlugin;
@@ -122,7 +123,7 @@ public class CoreContainer {
     public final Exception exception;
 
     public CoreLoadFailure(CoreDescriptor cd, Exception loadFailure) {
-      this.cd = cd;
+      this.cd = new CoreDescriptor(cd.getName(), cd);
       this.exception = loadFailure;
     }
   }
@@ -216,8 +217,6 @@ public class CoreContainer {
     return this.containerHandlers;
   }
 
- // private ClientConnectionManager clientConnectionManager = new PoolingClientConnectionManager();
-
   {
     log.debug("New CoreContainer " + System.identityHashCode(this));
   }
@@ -485,18 +484,18 @@ public class CoreContainer {
     metricManager = new SolrMetricManager();
 
     coreContainerWorkExecutor = MetricUtils.instrumentedExecutorService(
-        coreContainerWorkExecutor,
-        metricManager.registry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node)),
-        SolrMetricManager.mkName("coreContainerWorkExecutor", SolrInfoMBean.Category.CONTAINER.toString(), "threadPool"));
+        coreContainerWorkExecutor, null,
+        metricManager.registry(SolrMetricManager.getRegistryName(SolrInfoBean.Group.node)),
+        SolrMetricManager.mkName("coreContainerWorkExecutor", SolrInfoBean.Category.CONTAINER.toString(), "threadPool"));
 
     shardHandlerFactory = ShardHandlerFactory.newInstance(cfg.getShardHandlerFactoryPluginInfo(), loader);
     if (shardHandlerFactory instanceof SolrMetricProducer) {
       SolrMetricProducer metricProducer = (SolrMetricProducer) shardHandlerFactory;
-      metricProducer.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), "httpShardHandler");
+      metricProducer.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), "httpShardHandler");
     }
 
     updateShardHandler = new UpdateShardHandler(cfg.getUpdateShardHandlerConfig());
-    updateShardHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), "updateShardHandler");
+    updateShardHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), "updateShardHandler");
 
     transientCoreCache = TransientSolrCoreCacheFactory.newInstance(loader, this);
 
@@ -526,14 +525,14 @@ public class CoreContainer {
     autoScalingHandler = createHandler(AutoScalingHandler.HANDLER_PATH, AutoScalingHandler.class.getName(), AutoScalingHandler.class);
 
     containerHandlers.put(AUTHZ_PATH, securityConfHandler);
-    securityConfHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), AUTHZ_PATH);
+    securityConfHandler.initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), AUTHZ_PATH);
     containerHandlers.put(AUTHC_PATH, securityConfHandler);
     if(pkiAuthenticationPlugin != null)
       containerHandlers.put(PKIAuthenticationPlugin.PATH, pkiAuthenticationPlugin.getRequestHandler());
 
-    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, null, SolrInfoMBean.Group.node);
-    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, null, SolrInfoMBean.Group.jvm);
-    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, null, SolrInfoMBean.Group.jetty);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, null, SolrInfoBean.Group.node);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, null, SolrInfoBean.Group.jvm);
+    metricManager.loadReporters(cfg.getMetricReporterPlugins(), loader, null, SolrInfoBean.Group.jetty);
 
     coreConfigService = ConfigSetService.createConfigSetService(cfg, loader, zkSys.zkController);
 
@@ -541,17 +540,25 @@ public class CoreContainer {
 
     // initialize gauges for reporting the number of cores and disk total/free
 
-    String registryName = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node);
-    metricManager.registerGauge(registryName, () -> solrCores.getCores().size(),
-        true, "loaded", SolrInfoMBean.Category.CONTAINER.toString(), "cores");
-    metricManager.registerGauge(registryName, () -> solrCores.getLoadedCoreNames().size() - solrCores.getCores().size(),
-        true, "lazy",SolrInfoMBean.Category.CONTAINER.toString(), "cores");
-    metricManager.registerGauge(registryName, () -> solrCores.getAllCoreNames().size() - solrCores.getLoadedCoreNames().size(),
-        true, "unloaded",SolrInfoMBean.Category.CONTAINER.toString(), "cores");
-    metricManager.registerGauge(registryName, () -> cfg.getCoreRootDirectory().toFile().getTotalSpace(),
-        true, "totalSpace", SolrInfoMBean.Category.CONTAINER.toString(), "fs");
-    metricManager.registerGauge(registryName, () -> cfg.getCoreRootDirectory().toFile().getUsableSpace(),
-        true, "usableSpace", SolrInfoMBean.Category.CONTAINER.toString(), "fs");
+    String registryName = SolrMetricManager.getRegistryName(SolrInfoBean.Group.node);
+    metricManager.registerGauge(null, registryName, () -> solrCores.getCores().size(),
+        true, "loaded", SolrInfoBean.Category.CONTAINER.toString(), "cores");
+    metricManager.registerGauge(null, registryName, () -> solrCores.getLoadedCoreNames().size() - solrCores.getCores().size(),
+        true, "lazy", SolrInfoBean.Category.CONTAINER.toString(), "cores");
+    metricManager.registerGauge(null, registryName, () -> solrCores.getAllCoreNames().size() - solrCores.getLoadedCoreNames().size(),
+        true, "unloaded", SolrInfoBean.Category.CONTAINER.toString(), "cores");
+    metricManager.registerGauge(null, registryName, () -> cfg.getCoreRootDirectory().toFile().getTotalSpace(),
+        true, "totalSpace", SolrInfoBean.Category.CONTAINER.toString(), "fs");
+    metricManager.registerGauge(null, registryName, () -> cfg.getCoreRootDirectory().toFile().getUsableSpace(),
+        true, "usableSpace", SolrInfoBean.Category.CONTAINER.toString(), "fs");
+    // add version information
+    metricManager.registerGauge(null, registryName, () -> this.getClass().getPackage().getSpecificationVersion(),
+        true, "specification", SolrInfoBean.Category.CONTAINER.toString(), "version");
+    metricManager.registerGauge(null, registryName, () -> this.getClass().getPackage().getImplementationVersion(),
+        true, "implementation", SolrInfoBean.Category.CONTAINER.toString(), "version");
+
+    SolrFieldCacheBean fieldCacheBean = new SolrFieldCacheBean();
+    fieldCacheBean.initializeMetrics(metricManager, registryName, null);
 
     if (isZooKeeperAware()) {
       metricManager.loadClusterReporters(cfg.getMetricReporterPlugins(), this);
@@ -561,9 +568,9 @@ public class CoreContainer {
     ExecutorService coreLoadExecutor = MetricUtils.instrumentedExecutorService(
         ExecutorUtil.newMDCAwareFixedThreadPool(
             cfg.getCoreLoadThreadCount(isZooKeeperAware()),
-            new DefaultSolrThreadFactory("coreLoadExecutor")),
-        metricManager.registry(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node)),
-        SolrMetricManager.mkName("coreLoadExecutor",SolrInfoMBean.Category.CONTAINER.toString(), "threadPool"));
+            new DefaultSolrThreadFactory("coreLoadExecutor")), null,
+        metricManager.registry(SolrMetricManager.getRegistryName(SolrInfoBean.Group.node)),
+        SolrMetricManager.mkName("coreLoadExecutor", SolrInfoBean.Category.CONTAINER.toString(), "threadPool"));
     final List<Future<SolrCore>> futures = new ArrayList<>();
     try {
       List<CoreDescriptor> cds = coresLocator.discover(this);
@@ -577,7 +584,7 @@ public class CoreContainer {
 
       for (final CoreDescriptor cd : cds) {
         if (cd.isTransient() || !cd.isLoadOnStartup()) {
-          solrCores.putDynamicDescriptor(cd.getName(), cd);
+          getTransientCacheHandler().addTransientDescriptor(cd.getName(), cd);
         } else if (asyncSolrCoreLoad) {
           solrCores.markCoreAsLoading(cd);
         }
@@ -691,14 +698,16 @@ public class CoreContainer {
 
     ExecutorUtil.shutdownAndAwaitTermination(coreContainerWorkExecutor);
     if (metricManager != null) {
-      metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.node));
+      metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoBean.Group.node));
+      metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoBean.Group.jvm));
+      metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoBean.Group.jetty));
     }
 
     if (isZooKeeperAware()) {
       cancelCoreRecoveries();
       zkSys.zkController.publishNodeAsDown(zkSys.zkController.getNodeName());
       if (metricManager != null) {
-        metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoMBean.Group.cluster));
+        metricManager.closeReporters(SolrMetricManager.getRegistryName(SolrInfoBean.Group.cluster));
       }
     }
 
@@ -810,50 +819,35 @@ public class CoreContainer {
     return coresLocator;
   }
 
-  protected SolrCore registerCore(String name, SolrCore core, boolean registerInZk, boolean skipRecovery) {
+  protected SolrCore registerCore(CoreDescriptor cd, SolrCore core, boolean registerInZk, boolean skipRecovery) {
     if( core == null ) {
       throw new RuntimeException( "Can not register a null core." );
     }
-
-    // We can register a core when creating them via the admin UI, so we need to ensure that the dynamic descriptors
-    // are up to date
-    CoreDescriptor cd = core.getCoreDescriptor();
-    if ((cd.isTransient() || ! cd.isLoadOnStartup())
-        && solrCores.getDynamicDescriptor(name) == null) {
-      // Store it away for later use. includes non-transient but not
-      // loaded at startup cores.
-      solrCores.putDynamicDescriptor(name, cd);
-    }
-
-    SolrCore old;
-
+    
     if (isShutDown) {
       core.close();
       throw new IllegalStateException("This CoreContainer has been closed");
     }
-    if (cd.isTransient()) {
-      old = solrCores.putTransientCore(cfg, name, core, loader);
-    } else {
-      old = solrCores.putCore(name, core);
-    }
+    SolrCore old = solrCores.putCore(cd, core);
       /*
       * set both the name of the descriptor and the name of the
       * core, since the descriptors name is used for persisting.
       */
 
-    core.setName(name);
+    solrCores.addCoreDescriptor(new CoreDescriptor(cd.getName(), cd));
+    core.setName(cd.getName());
 
-    coreInitFailures.remove(name);
+    coreInitFailures.remove(cd.getName());
 
     if( old == null || old == core) {
-      log.debug( "registering core: "+name );
+      log.debug( "registering core: " + cd.getName() );
       if (registerInZk) {
         zkSys.registerInZk(core, false, skipRecovery);
       }
       return null;
     }
     else {
-      log.debug( "replacing core: "+name );
+      log.debug( "replacing core: " + cd.getName() );
       old.close();
       if (registerInZk) {
         zkSys.registerInZk(core, false, skipRecovery);
@@ -881,10 +875,10 @@ public class CoreContainer {
    */
   public SolrCore create(String coreName, Path instancePath, Map<String, String> parameters, boolean newCollection) {
 
-    CoreDescriptor cd = new CoreDescriptor(this, coreName, instancePath, parameters);
+    CoreDescriptor cd = new CoreDescriptor(coreName, instancePath, parameters, getContainerProperties(), isZooKeeperAware());
 
     // TODO: There's a race here, isn't there?
-    if (getAllCoreNames().contains(coreName)) {
+    if (getLoadedCoreNames().contains(coreName)) {
       log.warn("Creating a core with existing name is not allowed");
       // TODO: Shouldn't this be a BAD_REQUEST?
       throw new SolrException(ErrorCode.SERVER_ERROR, "Core with name '" + coreName + "' already exists.");
@@ -957,7 +951,7 @@ public class CoreContainer {
 
     SolrCore core = null;
     try {
-      MDCLoggingContext.setCoreDescriptor(dcore);
+      MDCLoggingContext.setCoreDescriptor(this, dcore);
       SolrIdentifierValidator.validateCoreName(dcore.getName());
       if (zkSys.getZkController() != null) {
         zkSys.getZkController().preRegister(dcore);
@@ -967,7 +961,7 @@ public class CoreContainer {
       dcore.setConfigSetTrusted(coreConfig.isTrusted());
       log.info("Creating SolrCore '{}' using configuration from {}, trusted={}", dcore.getName(), coreConfig.getName(), dcore.isConfigSetTrusted());
       try {
-        core = new SolrCore(dcore, coreConfig);
+        core = new SolrCore(this, dcore, coreConfig);
       } catch (SolrException e) {
         core = processCoreCreateException(e, dcore, coreConfig);
       }
@@ -977,7 +971,7 @@ public class CoreContainer {
         core.getUpdateHandler().getUpdateLog().recoverFromLog();
       }
 
-      registerCore(dcore.getName(), core, publishState, newCollection);
+      registerCore(dcore, core, publishState, newCollection);
 
       return core;
     } catch (Exception e) {
@@ -1041,7 +1035,7 @@ public class CoreContainer {
             if (leader != null && leader.getState() == State.ACTIVE) {
               log.info("Found active leader, will attempt to create fresh core and recover.");
               resetIndexDirectory(dcore, coreConfig);
-              return new SolrCore(dcore, coreConfig);
+              return new SolrCore(this, dcore, coreConfig);
             }
           } catch (SolrException se) {
             se.addSuppressed(original);
@@ -1064,7 +1058,7 @@ public class CoreContainer {
   private void resetIndexDirectory(CoreDescriptor dcore, ConfigSet coreConfig) {
     SolrConfig config = coreConfig.getSolrConfig();
 
-    String registryName = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, dcore.getName());
+    String registryName = SolrMetricManager.getRegistryName(SolrInfoBean.Group.core, dcore.getName());
     DirectoryFactory df = DirectoryFactory.loadDirectoryFactory(config, this, registryName);
     String dataDir = SolrCore.findDataDir(df, null, config, dcore);
 
@@ -1097,7 +1091,7 @@ public class CoreContainer {
   /**
    * @return a Collection of the names that loaded cores are mapped to
    */
-  public Collection<String> getCoreNames() {
+  public Collection<String> getLoadedCoreNames() {
     return solrCores.getLoadedCoreNames();
   }
 
@@ -1153,13 +1147,18 @@ public class CoreContainer {
   public void reload(String name) {
     SolrCore core = solrCores.getCoreFromAnyList(name, false);
     if (core != null) {
-      CoreDescriptor cd = core.getCoreDescriptor();
+      // The underlying core properties files may have changed, we don't really know. So we have a (perhaps) stale
+      // CoreDescriptor we need to reload it if it's out there. 
+      CorePropertiesLocator cpl = new CorePropertiesLocator(null);
+      CoreDescriptor cd = cpl.reload(this, core.getCoreDescriptor());
+      if (cd == null) cd = core.getCoreDescriptor();
+      solrCores.addCoreDescriptor(cd);
       try {
         solrCores.waitAddPendingCoreOps(cd.getName());
         ConfigSet coreConfig = coreConfigService.getConfig(cd);
         log.info("Reloading SolrCore '{}' using configuration from {}", cd.getName(), coreConfig.getName());
         SolrCore newCore = core.reload(coreConfig);
-        registerCore(cd.getName(), newCore, false, false);
+        registerCore(cd, newCore, false, false);
         if (getZkController() != null) {
           boolean onlyLeaderIndexes = getZkController().getClusterState().getCollection(cd.getCollectionName()).getRealtimeReplicas() == 1;
           if (onlyLeaderIndexes && !cd.getCloudDescriptor().isLeader()) {
@@ -1238,7 +1237,6 @@ public class CoreContainer {
     boolean close = solrCores.isLoadedNotPendingClose(name);
     SolrCore core = solrCores.remove(name);
     coresLocator.delete(this, cd);
-
     if (core == null) {
       // transient core
       SolrCore.deleteUnloadedCore(cd, deleteDataDir, deleteInstanceDir);
@@ -1253,7 +1251,7 @@ public class CoreContainer {
       core.getSolrCoreState().cancelRecovery();
     }
     
-    core.unloadOnClose(deleteIndexDir, deleteDataDir, deleteInstanceDir);
+    core.unloadOnClose(cd, deleteIndexDir, deleteDataDir, deleteInstanceDir);
     if (close)
       core.closeAndWait();
 
@@ -1267,6 +1265,9 @@ public class CoreContainer {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Error unregistering core [" + name + "] from cloud state", e);
       }
     }
+    if (deleteInstanceDir) { // we aren't going to reload this if we delete the instance dir.
+      solrCores.removeCoreDescriptor(cd);
+    }
   }
 
   public void rename(String name, String toName) {
@@ -1276,8 +1277,15 @@ public class CoreContainer {
         String oldRegistryName = core.getCoreMetricManager().getRegistryName();
         String newRegistryName = SolrCoreMetricManager.createRegistryName(core, toName);
         metricManager.swapRegistries(oldRegistryName, newRegistryName);
-        registerCore(toName, core, true, false);
+        // The old coreDescriptor is obsolete, so remove it. registerCore will put it back.
+        CoreDescriptor cd = core.getCoreDescriptor();
+        solrCores.removeCoreDescriptor(cd);
+        cd.setProperty("name", toName);
+        solrCores.addCoreDescriptor(cd);
+        core.setName(toName);
+        registerCore(cd, core, true, false);
         SolrCore old = solrCores.remove(name);
+
         coresLocator.rename(this, old.getCoreDescriptor(), core.getCoreDescriptor());
       }
     }
@@ -1292,12 +1300,7 @@ public class CoreContainer {
   }
 
   public CoreDescriptor getCoreDescriptor(String coreName) {
-    // TODO make this less hideous!
-    for (CoreDescriptor cd : getCoreDescriptors()) {
-      if (cd.getName().equals(coreName))
-        return cd;
-    }
-    return null;
+    return solrCores.getCoreDescriptor(coreName);
   }
 
   public Path getCoreRootDirectory() {
@@ -1317,29 +1320,32 @@ public class CoreContainer {
     // Do this in two phases since we don't want to lock access to the cores over a load.
     SolrCore core = solrCores.getCoreFromAnyList(name, true);
 
+    // If a core is loaded, we're done just return it.
     if (core != null) {
       return core;
     }
 
-    // OK, it's not presently in any list, is it in the list of dynamic cores but not loaded yet? If so, load it.
-    CoreDescriptor desc = solrCores.getDynamicDescriptor(name);
-    if (desc == null) { //Nope, no transient core with this name
+    // If it's not yet loaded, we can check if it's had a core init failure and "do the right thing"
+    CoreDescriptor desc = solrCores.getCoreDescriptor(name);
 
-      // if there was an error initializing this core, throw a 500
-      // error with the details for clients attempting to access it.
-      CoreLoadFailure loadFailure = getCoreInitFailures().get(name);
-      if (null != loadFailure) {
-        throw new SolrCoreInitializationException(name, loadFailure.exception);
-      }
-      // otherwise the user is simply asking for something that doesn't exist.
-      return null;
+    // if there was an error initializing this core, throw a 500
+    // error with the details for clients attempting to access it.
+    CoreLoadFailure loadFailure = getCoreInitFailures().get(name);
+    if (null != loadFailure) {
+      throw new SolrCoreInitializationException(name, loadFailure.exception);
     }
-
+    // This is a bit of awkwardness where SolrCloud and transient cores don't play nice together. For transient cores,
+    // we have to allow them to be created at any time there hasn't been a core load failure (use reload to cure that).
+    // But for TestConfigSetsAPI.testUploadWithScriptUpdateProcessor, this needs to _not_ try to load the core if
+    // the core is null and there was an error. If you change this, be sure to run both TestConfiSetsAPI and 
+    // TestLazyCores
+    if (desc == null || zkSys.getZkController() != null) return null;
+    
     // This will put an entry in pending core ops if the core isn't loaded
     core = solrCores.waitAddPendingCoreOps(name);
 
     if (isShutDown) return null; // We're quitting, so stop. This needs to be after the wait above since we may come off
-                                 // the wait as a consequence of shutting down.
+    // the wait as a consequence of shutting down.
     try {
       if (core == null) {
         if (zkSys.getZkController() != null) {
@@ -1382,7 +1388,7 @@ public class CoreContainer {
       containerHandlers.put(path, (SolrRequestHandler)handler);
     }
     if (handler instanceof SolrMetricProducer) {
-      ((SolrMetricProducer)handler).initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), path);
+      ((SolrMetricProducer)handler).initializeMetrics(metricManager, SolrInfoBean.Group.node.toString(), path);
     }
     return handler;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
index 0dc2c71..1747fa2 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java
@@ -121,8 +121,6 @@ public class CoreDescriptor {
       CloudDescriptor.NUM_SHARDS
   );
 
-  private final CoreContainer coreContainer;
-
   private final CloudDescriptor cloudDesc;
 
   private final Path instanceDir;
@@ -139,8 +137,9 @@ public class CoreDescriptor {
   /** The properties for this core, substitutable by resource loaders */
   protected final Properties substitutableProperties = new Properties();
 
-  public CoreDescriptor(CoreContainer container, String name, Path instanceDir, String... properties) {
-    this(container, name, instanceDir, toMap(properties));
+  public CoreDescriptor(String name, Path instanceDir, Properties containerProperties,
+                        boolean isZooKeeperAware, String... properties) {
+    this(name, instanceDir, toMap(properties), containerProperties, isZooKeeperAware);
   }
 
   private static Map<String, String> toMap(String... properties) {
@@ -154,12 +153,14 @@ public class CoreDescriptor {
 
   /**
    * Create a new CoreDescriptor with a given name and instancedir
-   * @param container     the CoreDescriptor's container
    * @param name          the CoreDescriptor's name
    * @param instanceDir   the CoreDescriptor's instancedir
+   * @param containerProperties the enclosing container properties for variable resolution
+   * @param isZooKeeperAware whether we are part of SolrCloud or not. 
    */
-  public CoreDescriptor(CoreContainer container, String name, Path instanceDir) {
-    this(container, name, instanceDir, Collections.emptyMap());
+  public CoreDescriptor(String name, Path instanceDir,
+                        Properties containerProperties, boolean isZooKeeperAware) {
+    this(name, instanceDir, Collections.emptyMap(), containerProperties, isZooKeeperAware);
   }
 
   /**
@@ -168,7 +169,6 @@ public class CoreDescriptor {
    * @param other    the CoreDescriptor to copy
    */
   public CoreDescriptor(String coreName, CoreDescriptor other) {
-    this.coreContainer = other.coreContainer;
     this.cloudDesc = other.cloudDesc;
     this.instanceDir = other.instanceDir;
     this.originalExtraProperties.putAll(other.originalExtraProperties);
@@ -183,20 +183,20 @@ public class CoreDescriptor {
 
   /**
    * Create a new CoreDescriptor.
-   * @param container       the CoreDescriptor's container
    * @param name            the CoreDescriptor's name
    * @param instanceDir     a Path resolving to the instanceDir
    * @param coreProps       a Map of the properties for this core
+   * @param containerProperties the properties from the enclosing container.
+   * @param isZooKeeperAware if true, we ar in SolrCloud mode.
    */
-  public CoreDescriptor(CoreContainer container, String name, Path instanceDir,
-                        Map<String, String> coreProps) {
 
-    this.coreContainer = container;
+
+  public CoreDescriptor(String name, Path instanceDir, Map<String, String> coreProps,
+                        Properties containerProperties, boolean isZooKeeperAware) {
     this.instanceDir = instanceDir;
 
     originalCoreProperties.setProperty(CORE_NAME, name);
 
-    Properties containerProperties = container.getContainerProperties();
     name = PropertiesUtil.substituteProperty(checkPropertyIsNotEmpty(name, CORE_NAME),
                                              containerProperties);
 
@@ -221,7 +221,7 @@ public class CoreDescriptor {
     buildSubstitutableProperties();
 
     // TODO maybe make this a CloudCoreDescriptor subclass?
-    if (container.isZooKeeperAware()) {
+    if (isZooKeeperAware) {
       cloudDesc = new CloudDescriptor(name, coreProperties, this);
     }
     else {
@@ -324,15 +324,19 @@ public class CoreDescriptor {
   public String getName() {
     return coreProperties.getProperty(CORE_NAME);
   }
+  
+  public void setProperty(String prop, String val) {
+    if (substitutableProperties.containsKey(prop)) {
+      substitutableProperties.setProperty(prop, val);
+      return;
+    }
+    coreProperties.setProperty(prop, val);
+  }
 
   public String getCollectionName() {
     return cloudDesc == null ? null : cloudDesc.getCollectionName();
   }
 
-  public CoreContainer getCoreContainer() {
-    return coreContainer;
-  }
-
   public CloudDescriptor getCloudDescriptor() {
     return cloudDesc;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java b/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
index b37402b..385d11b 100644
--- a/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
+++ b/solr/core/src/java/org/apache/solr/core/CorePropertiesLocator.java
@@ -163,6 +163,14 @@ public class CorePropertiesLocator implements CoresLocator {
     return cds;
   }
 
+  @Override
+  public CoreDescriptor reload(CoreContainer cc, CoreDescriptor cd) {
+    if (cd == null) return null;
+    
+    Path coreProps = cd.getInstanceDir().resolve(CoreDescriptor.DEFAULT_EXTERNAL_PROPERTIES_FILE);
+    return buildCoreDescriptor(coreProps, cc);
+  }
+
   protected CoreDescriptor buildCoreDescriptor(Path propertiesFile, CoreContainer cc) {
 
     Path instanceDir = propertiesFile.getParent();
@@ -174,7 +182,7 @@ public class CorePropertiesLocator implements CoresLocator {
       for (String key : coreProperties.stringPropertyNames()) {
         propMap.put(key, coreProperties.getProperty(key));
       }
-      return new CoreDescriptor(cc, name, instanceDir, propMap);
+      return new CoreDescriptor(name, instanceDir, propMap, cc.getContainerProperties(), cc.isZooKeeperAware());
     }
     catch (IOException e) {
       logger.error("Couldn't load core descriptor from {}:{}", propertiesFile, e.toString());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/java/org/apache/solr/core/CoresLocator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoresLocator.java b/solr/core/src/java/org/apache/solr/core/CoresLocator.java
index d4f40cd..beaa690 100644
--- a/solr/core/src/java/org/apache/solr/core/CoresLocator.java
+++ b/solr/core/src/java/org/apache/solr/core/CoresLocator.java
@@ -69,4 +69,13 @@ public interface CoresLocator {
    */
   public List<CoreDescriptor> discover(CoreContainer cc);
 
+  /**
+   * reload an existing CoreDescriptor, that is read it from disk.
+   * 
+   * @param cc the CoreContainer
+   * @param cd the old CoreDescriptor. If null, this is a no-op
+   * @return the reloaded coreDescriptor or null          
+   */
+  public CoreDescriptor reload(CoreContainer cc, CoreDescriptor cd);
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index cc24e6c..20824ab 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.NoSuchFileException;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
@@ -321,13 +320,6 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
     return cd.getInstanceDir().resolve(cd.getDataDir()).toAbsolutePath().toString();
   }
 
-  /**
-   * Optionally allow the DirectoryFactory to request registration of some MBeans.
-   */
-  public Collection<SolrInfoMBean> offerMBeans() {
-    return Collections.emptySet();
-  }
-
   public void cleanupOldIndexDirectories(final String dataDirPath, final String currentIndexDirPath, boolean afterCoreReload) {
     File dataDir = new File(dataDirPath);
     if (!dataDir.isDirectory()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
index db953d3..260a991 100644
--- a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.net.URLEncoder;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -51,6 +50,8 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.store.blockcache.BlockCache;
 import org.apache.solr.store.blockcache.BlockDirectory;
 import org.apache.solr.store.blockcache.BlockDirectoryCache;
@@ -70,7 +71,7 @@ import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.RemovalListener;
 import com.google.common.cache.RemovalNotification;
 
-public class HdfsDirectoryFactory extends CachingDirectoryFactory implements SolrCoreAware {
+public class HdfsDirectoryFactory extends CachingDirectoryFactory implements SolrCoreAware, SolrMetricProducer {
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
   public static final String BLOCKCACHE_SLAB_COUNT = "solr.hdfs.blockcache.slab.count";
@@ -493,13 +494,14 @@ public class HdfsDirectoryFactory extends CachingDirectoryFactory implements Sol
   }
 
   @Override
-  public Collection<SolrInfoMBean> offerMBeans() {
-    return Arrays.<SolrInfoMBean>asList(MetricsHolder.metrics, LocalityHolder.reporter);
+  public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
+    MetricsHolder.metrics.initializeMetrics(manager, registry, scope);
+    LocalityHolder.reporter.initializeMetrics(manager, registry, scope);
   }
 
   @Override
   public void inform(SolrCore core) {
-    setHost(core.getCoreDescriptor().getCoreContainer().getHostName());
+    setHost(core.getCoreContainer().getHostName());
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java b/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
deleted file mode 100644
index 4fb0dcd..0000000
--- a/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java
+++ /dev/null
@@ -1,478 +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.core;
-
-import javax.management.Attribute;
-import javax.management.AttributeList;
-import javax.management.AttributeNotFoundException;
-import javax.management.DynamicMBean;
-import javax.management.InstanceNotFoundException;
-import javax.management.InvalidAttributeValueException;
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanException;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServer;
-import javax.management.MBeanServerFactory;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.Query;
-import javax.management.QueryExp;
-import javax.management.ReflectionException;
-import javax.management.openmbean.OpenMBeanAttributeInfoSupport;
-import javax.management.openmbean.OpenType;
-import javax.management.openmbean.SimpleType;
-import javax.management.remote.JMXConnectorServer;
-import javax.management.remote.JMXConnectorServerFactory;
-import javax.management.remote.JMXServiceURL;
-import java.lang.invoke.MethodHandles;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Hashtable;
-import java.util.List;
-import java.util.Locale;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.SolrConfig.JmxConfiguration;
-import org.apache.solr.metrics.reporters.JmxObjectNameFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.common.params.CommonParams.ID;
-import static org.apache.solr.common.params.CommonParams.NAME;
-
-/**
- * <p>
- * Responsible for finding (or creating) a MBeanServer from given configuration
- * and registering all SolrInfoMBean objects with JMX.
- * </p>
- * <p>
- * Please see http://wiki.apache.org/solr/SolrJmx for instructions on usage and configuration
- * </p>
- *
- *
- * @see org.apache.solr.core.SolrConfig.JmxConfiguration
- * @since solr 1.3
- */
-public class JmxMonitoredMap<K, V> extends
-        ConcurrentHashMap<String, SolrInfoMBean> {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  private static final String REPORTER_NAME = "_jmx_";
-
-  // set to true to use cached statistics NamedLists between getMBeanInfo calls to work
-  // around over calling getStatistics on MBeanInfos when iterating over all attributes (SOLR-6586)
-  private final boolean useCachedStatsBetweenGetMBeanInfoCalls = Boolean.getBoolean("useCachedStatsBetweenGetMBeanInfoCalls");
-  
-  private final MBeanServer server;
-
-  private final String jmxRootName;
-
-  private final String coreHashCode;
-
-  private final JmxObjectNameFactory nameFactory;
-
-  private final String registryName;
-
-  public JmxMonitoredMap(String coreName, String coreHashCode, String registryName,
-                         final JmxConfiguration jmxConfig) {
-    this.coreHashCode = coreHashCode;
-    this.registryName = registryName;
-    jmxRootName = (null != jmxConfig.rootName ?
-                   jmxConfig.rootName
-                   : ("solr" + (null != coreName ? "/" + coreName : "")));
-      
-    if (jmxConfig.serviceUrl == null) {
-      List<MBeanServer> servers = null;
-
-      if (jmxConfig.agentId == null) {
-        // Try to find the first MBeanServer
-        servers = MBeanServerFactory.findMBeanServer(null);
-      } else if (jmxConfig.agentId != null) {
-        // Try to find the first MBean server with the given agentId
-        servers = MBeanServerFactory.findMBeanServer(jmxConfig.agentId);
-        // throw Exception if no servers were found with the given agentId
-        if (servers == null || servers.isEmpty())
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-                  "No JMX Servers found with agentId: " + jmxConfig.agentId);
-      }
-
-      if (servers == null || servers.isEmpty()) {
-        server = null;
-        nameFactory = null;
-        log.debug("No JMX servers found, not exposing Solr information with JMX.");
-        return;
-      }
-      server = servers.get(0);
-      log.info("JMX monitoring is enabled. Adding Solr mbeans to JMX Server: "
-               + server);
-    } else {
-      MBeanServer newServer = null;
-      try {
-        // Create a new MBeanServer with the given serviceUrl
-        newServer = MBeanServerFactory.newMBeanServer();
-        JMXConnectorServer connector = JMXConnectorServerFactory
-                .newJMXConnectorServer(new JMXServiceURL(jmxConfig.serviceUrl),
-                        null, newServer);
-        connector.start();
-        log.info("JMX monitoring is enabled at " + jmxConfig.serviceUrl);
-      } catch (Exception e) {
-        // Release the reference
-        throw new RuntimeException("Could not start JMX monitoring ", e);
-      }
-      server = newServer;
-    }
-    nameFactory = new JmxObjectNameFactory(REPORTER_NAME + coreHashCode, registryName);
-  }
-
-  /**
-   * Clears the map and unregisters all SolrInfoMBeans in the map from
-   * MBeanServer
-   */
-  @Override
-  public void clear() {
-    if (server != null) {
-      QueryExp exp = Query.or(Query.eq(Query.attr("coreHashCode"), Query.value(coreHashCode)),
-                            Query.eq(Query.attr("reporter"), Query.value(REPORTER_NAME + coreHashCode)));
-      
-      Set<ObjectName> objectNames = null;
-      try {
-        objectNames = server.queryNames(null, exp);
-      } catch (Exception e) {
-        log.warn("Exception querying for mbeans", e);
-      }
-      
-      if (objectNames != null)  {
-        for (ObjectName name : objectNames) {
-          try {
-            server.unregisterMBean(name);
-          } catch (InstanceNotFoundException ie) {
-            // ignore - someone else already deleted this one
-          } catch (Exception e) {
-            log.warn("Exception un-registering mbean {}", name, e);
-          }
-        }
-      }
-    }
-
-    super.clear();
-  }
-
-  /**
-   * Adds the SolrInfoMBean to the map and registers the given SolrInfoMBean
-   * instance with the MBeanServer defined for this core. If a SolrInfoMBean is
-   * already registered with the MBeanServer then it is unregistered and then
-   * re-registered.
-   *
-   * @param key      the JMX type name for this SolrInfoMBean
-   * @param infoBean the SolrInfoMBean instance to be registered
-   */
-  @Override
-  public SolrInfoMBean put(String key, SolrInfoMBean infoBean) {
-    if (server != null && infoBean != null) {
-      try {
-        // back-compat name
-        ObjectName name = getObjectName(key, infoBean);
-        if (server.isRegistered(name))
-          server.unregisterMBean(name);
-        SolrDynamicMBean mbean = new SolrDynamicMBean(coreHashCode, infoBean, useCachedStatsBetweenGetMBeanInfoCalls);
-        server.registerMBean(mbean, name);
-        // now register it also under new name
-        String beanName = createBeanName(infoBean, key);
-        name = nameFactory.createName(null, registryName, beanName);
-        if (server.isRegistered(name))
-          server.unregisterMBean(name);
-        server.registerMBean(mbean, name);
-      } catch (Exception e) {
-        log.warn( "Failed to register info bean: key=" + key + ", infoBean=" + infoBean, e);
-      }
-    }
-
-    return super.put(key, infoBean);
-  }
-
-  private String createBeanName(SolrInfoMBean infoBean, String key) {
-    if (infoBean.getCategory() == null) {
-      throw new IllegalArgumentException("SolrInfoMBean.category must never be null: " + infoBean);
-    }
-    StringBuilder sb = new StringBuilder();
-    sb.append(infoBean.getCategory().toString());
-    sb.append('.');
-    sb.append(key);
-    sb.append('.');
-    sb.append(infoBean.getName());
-    return sb.toString();
-  }
-
-  /**
-   * Removes the SolrInfoMBean object at the given key and unregisters it from
-   * MBeanServer
-   *
-   * @param key the JMX type name for this SolrInfoMBean
-   */
-  @Override
-  public SolrInfoMBean remove(Object key) {
-    SolrInfoMBean infoBean = get(key);
-    if (infoBean != null) {
-      try {
-        unregister((String) key, infoBean);
-      } catch (RuntimeException e) {
-        log.warn( "Failed to unregister info bean: " + key, e);
-      }
-    }
-    return super.remove(key);
-  }
-
-  private void unregister(String key, SolrInfoMBean infoBean) {
-    if (server == null)
-      return;
-
-    try {
-      // remove legacy name
-      ObjectName name = getObjectName(key, infoBean);
-      if (server.isRegistered(name) && coreHashCode.equals(server.getAttribute(name, "coreHashCode"))) {
-        server.unregisterMBean(name);
-      }
-      // remove new name
-      String beanName = createBeanName(infoBean, key);
-      name = nameFactory.createName(null, registryName, beanName);
-      if (server.isRegistered(name)) {
-        server.unregisterMBean(name);
-      }
-    } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-              "Failed to unregister info bean: " + key, e);
-    }
-  }
-
-  private ObjectName getObjectName(String key, SolrInfoMBean infoBean)
-          throws MalformedObjectNameException {
-    Hashtable<String, String> map = new Hashtable<>();
-    map.put("type", key);
-    if (infoBean.getName() != null && !"".equals(infoBean.getName())) {
-      map.put(ID, infoBean.getName());
-    }
-    return ObjectName.getInstance(jmxRootName, map);
-  }
-
-  /** For test verification */
-  public MBeanServer getServer() {
-    return server;
-  }
-
-  /**
-   * DynamicMBean is used to dynamically expose all SolrInfoMBean
-   * getStatistics() NameList keys as String getters.
-   */
-  static class SolrDynamicMBean implements DynamicMBean {
-    private SolrInfoMBean infoBean;
-
-    private HashSet<String> staticStats;
-
-    private String coreHashCode;
-    
-    private volatile NamedList cachedDynamicStats;
-    
-    private boolean useCachedStatsBetweenGetMBeanInfoCalls;
-    
-    public SolrDynamicMBean(String coreHashCode, SolrInfoMBean managedResource) {
-      this(coreHashCode, managedResource, false);
-    }
-
-    public SolrDynamicMBean(String coreHashCode, SolrInfoMBean managedResource, boolean useCachedStatsBetweenGetMBeanInfoCalls) {
-      this.useCachedStatsBetweenGetMBeanInfoCalls = useCachedStatsBetweenGetMBeanInfoCalls;
-      if (managedResource instanceof JmxAugmentedSolrInfoMBean) {
-        final JmxAugmentedSolrInfoMBean jmxSpecific = (JmxAugmentedSolrInfoMBean)managedResource;
-        this.infoBean = new SolrInfoMBeanWrapper(jmxSpecific) {
-          @Override
-          public NamedList getStatistics() { return jmxSpecific.getStatisticsForJmx(); }
-        };
-      } else {
-        this.infoBean = managedResource;
-      }
-      staticStats = new HashSet<>();
-
-      // For which getters are already available in SolrInfoMBean
-      staticStats.add(NAME);
-      staticStats.add("version");
-      staticStats.add("description");
-      staticStats.add("category");
-      staticStats.add("source");
-      this.coreHashCode = coreHashCode;
-    }
-
-    @Override
-    public MBeanInfo getMBeanInfo() {
-      ArrayList<MBeanAttributeInfo> attrInfoList = new ArrayList<>();
-
-      for (String stat : staticStats) {
-        attrInfoList.add(new MBeanAttributeInfo(stat, String.class.getName(),
-                null, true, false, false));
-      }
-
-      // add core's hashcode
-      attrInfoList.add(new MBeanAttributeInfo("coreHashCode", String.class.getName(),
-                null, true, false, false));
-
-      try {
-        NamedList dynamicStats = infoBean.getStatistics();
-        
-        if (useCachedStatsBetweenGetMBeanInfoCalls) {
-          cachedDynamicStats = dynamicStats;
-        }
-        
-        if (dynamicStats != null) {
-          for (int i = 0; i < dynamicStats.size(); i++) {
-            String name = dynamicStats.getName(i);
-            if (staticStats.contains(name)) {
-              continue;
-            }
-            Class type = dynamicStats.get(name).getClass();
-            OpenType typeBox = determineType(type);
-            if (type.equals(String.class) || typeBox == null) {
-              attrInfoList.add(new MBeanAttributeInfo(dynamicStats.getName(i),
-                  String.class.getName(), null, true, false, false));
-            } else {
-              attrInfoList.add(new OpenMBeanAttributeInfoSupport(
-                  dynamicStats.getName(i), dynamicStats.getName(i), typeBox,
-                  true, false, false));
-            }
-          }
-        }
-      } catch (Exception e) {
-        // don't log issue if the core is closing
-        if (!(SolrException.getRootCause(e) instanceof AlreadyClosedException))
-          log.warn("Could not getStatistics on info bean {}", infoBean.getName(), e);
-      }
-
-      MBeanAttributeInfo[] attrInfoArr = attrInfoList
-              .toArray(new MBeanAttributeInfo[attrInfoList.size()]);
-      return new MBeanInfo(getClass().getName(), infoBean
-              .getDescription(), attrInfoArr, null, null, null);
-    }
-
-    private OpenType determineType(Class type) {
-      try {
-        for (Field field : SimpleType.class.getFields()) {
-          if (field.getType().equals(SimpleType.class)) {
-            SimpleType candidate = (SimpleType) field.get(SimpleType.class);
-            if (candidate.getTypeName().equals(type.getName())) {
-              return candidate;
-            }
-          }
-        }
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-      return null;
-    }
-
-    @Override
-    public Object getAttribute(String attribute)
-            throws AttributeNotFoundException, MBeanException, ReflectionException {
-      Object val;
-      if ("coreHashCode".equals(attribute)) {
-        val = coreHashCode;
-      } else if (staticStats.contains(attribute) && attribute != null
-              && attribute.length() > 0) {
-        try {
-          String getter = "get" + attribute.substring(0, 1).toUpperCase(Locale.ROOT)
-                  + attribute.substring(1);
-          Method meth = infoBean.getClass().getMethod(getter);
-          val = meth.invoke(infoBean);
-        } catch (Exception e) {
-          throw new AttributeNotFoundException(attribute);
-        }
-      } else {
-        NamedList stats = null;
-        if (useCachedStatsBetweenGetMBeanInfoCalls) {
-          NamedList cachedStats = this.cachedDynamicStats;
-          if (cachedStats != null) {
-            stats = cachedStats;
-          }
-        }
-        if (stats == null) {
-          stats = infoBean.getStatistics();
-        }
-        val = stats.get(attribute);
-      }
-
-      if (val != null) {
-        // It's String or one of the simple types, just return it as JMX suggests direct support for such types
-        for (String simpleTypeName : SimpleType.ALLOWED_CLASSNAMES_LIST) {
-          if (val.getClass().getName().equals(simpleTypeName)) {
-            return val;
-          }
-        }
-        // It's an arbitrary object which could be something complex and odd, return its toString, assuming that is
-        // a workable representation of the object
-        return val.toString();
-      }
-      return null;
-    }
-
-    @Override
-    public AttributeList getAttributes(String[] attributes) {
-      AttributeList list = new AttributeList();
-      for (String attribute : attributes) {
-        try {
-          list.add(new Attribute(attribute, getAttribute(attribute)));
-        } catch (Exception e) {
-          log.warn("Could not get attribute " + attribute);
-        }
-      }
-
-      return list;
-    }
-
-    @Override
-    public void setAttribute(Attribute attribute)
-            throws AttributeNotFoundException, InvalidAttributeValueException,
-            MBeanException, ReflectionException {
-      throw new UnsupportedOperationException("Operation not Supported");
-    }
-
-    @Override
-    public AttributeList setAttributes(AttributeList attributes) {
-      throw new UnsupportedOperationException("Operation not Supported");
-    }
-
-    @Override
-    public Object invoke(String actionName, Object[] params, String[] signature)
-            throws MBeanException, ReflectionException {
-      throw new UnsupportedOperationException("Operation not Supported");
-    }
-  }
-
-  /**
-   * SolrInfoMBean that provides JMX-specific statistics.  Used, for example,
-   * if generating full statistics is expensive; the expensive statistics can
-   * be generated normally for use with the web ui, while an abbreviated version
-   * are generated for period jmx use.
-   */
-  public interface JmxAugmentedSolrInfoMBean extends SolrInfoMBean {
-    /**
-     * JMX-specific statistics
-     */
-    public NamedList getStatisticsForJmx();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/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 de2dcea..7a209d0 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -17,7 +17,10 @@
 package org.apache.solr.core;
 
 import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.Properties;
+import java.util.Set;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.logging.LogWatcherConfig;
@@ -64,6 +67,8 @@ public class NodeConfig {
 
   private final PluginInfo[] metricReporterPlugins;
 
+  private final Set<String> hiddenSysProps;
+
   private final PluginInfo transientCacheConfig;
 
   private NodeConfig(String nodeName, Path coreRootDirectory, Path configSetBaseDirectory, String sharedLibDirectory,
@@ -73,7 +78,7 @@ public class NodeConfig {
                      LogWatcherConfig logWatcherConfig, CloudConfig cloudConfig, Integer coreLoadThreads,
                      int transientCacheSize, boolean useSchemaCache, String managementPath, SolrResourceLoader loader,
                      Properties solrProperties, PluginInfo[] backupRepositoryPlugins,
-                     PluginInfo[] metricReporterPlugins, PluginInfo transientCacheConfig) {
+                     PluginInfo[] metricReporterPlugins, Set<String> hiddenSysProps, PluginInfo transientCacheConfig) {
     this.nodeName = nodeName;
     this.coreRootDirectory = coreRootDirectory;
     this.configSetBaseDirectory = configSetBaseDirectory;
@@ -94,6 +99,7 @@ public class NodeConfig {
     this.solrProperties = solrProperties;
     this.backupRepositoryPlugins = backupRepositoryPlugins;
     this.metricReporterPlugins = metricReporterPlugins;
+    this.hiddenSysProps = hiddenSysProps;
     this.transientCacheConfig = transientCacheConfig;
 
     if (this.cloudConfig != null && this.getCoreLoadThreadCount(false) < 2) {
@@ -187,6 +193,10 @@ public class NodeConfig {
     return metricReporterPlugins;
   }
 
+  public Set<String> getHiddenSysProps() {
+    return hiddenSysProps;
+  }
+
   public PluginInfo getTransientCachePluginInfo() { return transientCacheConfig; }
 
   public static class NodeConfigBuilder {
@@ -211,6 +221,7 @@ public class NodeConfig {
     private Properties solrProperties = new Properties();
     private PluginInfo[] backupRepositoryPlugins;
     private PluginInfo[] metricReporterPlugins;
+    private Set<String> hiddenSysProps = new HashSet<>(DEFAULT_HIDDEN_SYS_PROPS);
     private PluginInfo transientCacheConfig;
 
     private final SolrResourceLoader loader;
@@ -227,6 +238,14 @@ public class NodeConfig {
     private static final String DEFAULT_COLLECTIONSHANDLERCLASS = "org.apache.solr.handler.admin.CollectionsHandler";
     private static final String DEFAULT_CONFIGSETSHANDLERCLASS = "org.apache.solr.handler.admin.ConfigSetsHandler";
 
+    public static final Set<String> DEFAULT_HIDDEN_SYS_PROPS = new HashSet<>(Arrays.asList(
+        "javax.net.ssl.keyStorePassword",
+        "javax.net.ssl.trustStorePassword",
+        "basicauth",
+        "zkDigestPassword",
+        "zkDigestReadonlyPassword"
+    ));
+
     public NodeConfigBuilder(String nodeName, SolrResourceLoader loader) {
       this.nodeName = nodeName;
       this.loader = loader;
@@ -331,11 +350,16 @@ public class NodeConfig {
       return this;
     }
 
+    public NodeConfigBuilder setHiddenSysProps(Set<String> hiddenSysProps) {
+      this.hiddenSysProps = hiddenSysProps;
+      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, metricReporterPlugins, transientCacheConfig);
+                            backupRepositoryPlugins, metricReporterPlugins, hiddenSysProps, transientCacheConfig);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/java/org/apache/solr/core/PluginBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/PluginBag.java b/solr/core/src/java/org/apache/solr/core/PluginBag.java
index 65978f3..4c0858e 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -294,8 +294,8 @@ public class PluginBag<T> implements AutoCloseable {
 
   private void registerMBean(Object inst, SolrCore core, String pluginKey) {
     if (core == null) return;
-    if (inst instanceof SolrInfoMBean) {
-      SolrInfoMBean mBean = (SolrInfoMBean) inst;
+    if (inst instanceof SolrInfoBean) {
+      SolrInfoBean mBean = (SolrInfoBean) inst;
       String name = (inst instanceof SolrRequestHandler) ? pluginKey : mBean.getName();
       core.registerInfoBean(name, mBean);
     }
@@ -455,7 +455,7 @@ public class PluginBag<T> implements AutoCloseable {
     }
 
     public RuntimeLib(SolrCore core) {
-      coreContainer = core.getCoreDescriptor().getCoreContainer();
+      coreContainer = core.getCoreContainer();
     }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/solr/core/src/java/org/apache/solr/core/SolrConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index a244420..4e7ab48 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -276,18 +276,12 @@ public class SolrConfig extends Config implements MapSerializable {
     hashSetInverseLoadFactor = 1.0f / getFloat("//HashDocSet/@loadFactor", 0.75f);
     hashDocSetMaxSize = getInt("//HashDocSet/@maxSize", 3000);
 
-    httpCachingConfig = new HttpCachingConfig(this);
+    if (get("jmx", null) != null) {
+      log.warn("solrconfig.xml: <jmx> is no longer supported, use solr.xml:/metrics/reporter section instead");
+    }
 
-    Node jmx = getNode("jmx", false);
-    if (jmx != null) {
-      jmxConfig = new JmxConfiguration(true,
-          get("jmx/@agentId", null),
-          get("jmx/@serviceUrl", null),
-          get("jmx/@rootName", null));
+    httpCachingConfig = new HttpCachingConfig(this);
 
-    } else {
-      jmxConfig = new JmxConfiguration(false, null, null, null);
-    }
     maxWarmingSearchers = getInt("query/maxWarmingSearchers", 1);
     slowQueryThresholdMillis = getInt("query/slowQueryThresholdMillis", -1);
     for (SolrPluginInfo plugin : plugins) loadPluginInfo(plugin);
@@ -510,48 +504,12 @@ public class SolrConfig extends Config implements MapSerializable {
   protected String dataDir;
   public final int slowQueryThresholdMillis;  // threshold above which a query is considered slow
 
-  //JMX configuration
-  public final JmxConfiguration jmxConfig;
-
   private final HttpCachingConfig httpCachingConfig;
 
   public HttpCachingConfig getHttpCachingConfig() {
     return httpCachingConfig;
   }
 
-  public static class JmxConfiguration implements MapSerializable {
-    public boolean enabled = false;
-    public String agentId;
-    public String serviceUrl;
-    public String rootName;
-
-    public JmxConfiguration(boolean enabled,
-                            String agentId,
-                            String serviceUrl,
-                            String rootName) {
-      this.enabled = enabled;
-      this.agentId = agentId;
-      this.serviceUrl = serviceUrl;
-      this.rootName = rootName;
-
-      if (agentId != null && serviceUrl != null) {
-        throw new SolrException
-            (SolrException.ErrorCode.SERVER_ERROR,
-                "Incorrect JMX Configuration in solrconfig.xml, " +
-                    "both agentId and serviceUrl cannot be specified at the same time");
-      }
-
-    }
-
-    @Override
-    public Map<String, Object> toMap(Map<String, Object> map) {
-      map.put("agentId", agentId);
-      map.put("serviceUrl", serviceUrl);
-      map.put("rootName", rootName);
-      return map;
-    }
-  }
-
   public static class HttpCachingConfig implements MapSerializable {
 
     /**
@@ -858,7 +816,6 @@ public class SolrConfig extends Config implements MapSerializable {
     m.put("queryResultMaxDocsCached", queryResultMaxDocsCached);
     m.put("enableLazyFieldLoading", enableLazyFieldLoading);
     m.put("maxBooleanClauses", booleanQueryMaxClauseCount);
-    if (jmxConfig != null) result.put("jmx", jmxConfig);
     for (SolrPluginInfo plugin : plugins) {
       List<PluginInfo> infos = getPluginInfos(plugin.clazz.getName());
       if (infos == null || infos.isEmpty()) continue;
@@ -884,7 +841,6 @@ public class SolrConfig extends Config implements MapSerializable {
 
 
     addCacheConfig(m, filterCacheConfig, queryResultCacheConfig, documentCacheConfig, fieldValueCacheConfig);
-    if (jmxConfig != null) result.put("jmx", jmxConfig);
     m = new LinkedHashMap();
     result.put("requestDispatcher", m);
     m.put("handleSelect", handleSelect);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8df9f8c/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 a6ba2dc..b26fc2f 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -27,7 +27,6 @@ import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.lang.invoke.MethodHandles;
 import java.lang.reflect.Constructor;
-import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
@@ -58,6 +57,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantLock;
 
 import com.codahale.metrics.Counter;
+import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
 import com.google.common.collect.MapMaker;
 import org.apache.commons.io.FileUtils;
@@ -133,7 +133,7 @@ import org.apache.solr.schema.IndexSchemaFactory;
 import org.apache.solr.schema.ManagedIndexSchema;
 import org.apache.solr.schema.SimilarityFactory;
 import org.apache.solr.search.QParserPlugin;
-import org.apache.solr.search.SolrFieldCacheMBean;
+import org.apache.solr.search.SolrFieldCacheBean;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.ValueSourceParser;
 import org.apache.solr.search.stats.LocalStatsCache;
@@ -171,7 +171,7 @@ import static org.apache.solr.common.params.CommonParams.PATH;
 /**
  *
  */
-public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closeable {
+public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeable {
 
   public static final String version="1.0";
 
@@ -180,7 +180,6 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
 
   private String name;
   private String logid; // used to show what name is set
-  private CoreDescriptor coreDescriptor;
 
   private boolean isReloaded = false;
 
@@ -202,7 +201,7 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
   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 Map<String, SolrInfoBean> infoRegistry = new ConcurrentHashMap<>();
   private final IndexDeletionPolicyWrapper solrDelPolicy;
   private final SolrSnapshotMetaDataManager snapshotMgr;
   private final DirectoryFactory directoryFactory;
@@ -221,6 +220,13 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
   private Counter newSearcherCounter;
   private Counter newSearcherMaxReachedCounter;
   private Counter newSearcherOtherErrorsCounter;
+  private final CoreContainer coreContainer;
+
+  private Set<String> metricNames = new HashSet<>();
+
+  public Set<String> getMetricNames() {
+    return metricNames;
+  }
 
   public Date getStartTimeStamp() { return startTime; }
 
@@ -424,10 +430,8 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
   }
 
   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();
     }
@@ -448,14 +452,14 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
   }
 
   /**
-   * Returns a Map of name vs SolrInfoMBean objects. The returned map is an instance of
+   * Returns a Map of name vs SolrInfoBean objects. The returned map is an instance of
    * a ConcurrentHashMap and therefore no synchronization is needed for putting, removing
    * or iterating over it.
    *
-   * @return the Info Registry map which contains SolrInfoMBean objects keyed by name
+   * @return the Info Registry map which contains SolrInfoBean objects keyed by name
    * @since solr 1.3
    */
-  public Map<String, SolrInfoMBean> getInfoRegistry() {
+  public Map<String, SolrInfoBean> getInfoRegistry() {
     return infoRegistry;
   }
 
@@ -636,9 +640,9 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
       boolean success = false;
       SolrCore core = null;
       try {
-        CoreDescriptor cd = new CoreDescriptor(coreDescriptor.getName(), coreDescriptor);
+        CoreDescriptor cd = new CoreDescriptor(name, getCoreDescriptor());
         cd.loadExtraProperties(); //Reload the extra properties
-        core = new SolrCore(getName(), getDataDir(), coreConfig.getSolrConfig(),
+        core = new SolrCore(coreContainer, getName(), getDataDir(), coreConfig.getSolrConfig(),
             coreConfig.getIndexSchema(), coreConfig.getProperties(),
             cd, updateHandler, solrDelPolicy, currentCore, true);
         
@@ -658,7 +662,7 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
   }
 
   private DirectoryFactory initDirectoryFactory() {
-    return DirectoryFactory.loadDirectoryFactory(solrConfig, getCoreDescriptor().getCoreContainer(), coreMetricManager.getRegistryName());
+    return DirectoryFactory.loadDirectoryFactory(solrConfig, coreContainer, coreMetricManager.getRegistryName());
   }
 
   private RecoveryStrategy.Builder initRecoveryStrategyBuilder() {
@@ -845,12 +849,16 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
     return createReloadedUpdateHandler(className, "Update Handler", updateHandler);
   }
 
-  public SolrCore(CoreDescriptor cd, ConfigSet coreConfig) {
-    this(cd.getName(), null, coreConfig.getSolrConfig(), coreConfig.getIndexSchema(), coreConfig.getProperties(),
+  public SolrCore(CoreContainer coreContainer, CoreDescriptor cd, ConfigSet coreConfig) {
+    this(coreContainer, cd.getName(), null, coreConfig.getSolrConfig(), coreConfig.getIndexSchema(), coreConfig.getProperties(),
         cd, null, null, null, false);
   }
 
-  
+  public CoreContainer getCoreContainer() {
+    return coreContainer;
+  }
+
+
   /**
    * Creates a new core and register it in the list of cores. If a core with the
    * same name already exists, it will be stopped and replaced by this one.
@@ -864,14 +872,18 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
    *
    * @since solr 1.3
    */
-  public SolrCore(String name, String dataDir, SolrConfig config,
-      IndexSchema schema, NamedList configSetProperties,
-      CoreDescriptor coreDescriptor, UpdateHandler updateHandler,
-      IndexDeletionPolicyWrapper delPolicy, SolrCore prev, boolean reload) {
+  public SolrCore(CoreContainer coreContainer, String name, String dataDir, SolrConfig config,
+                  IndexSchema schema, NamedList configSetProperties,
+                  CoreDescriptor coreDescriptor, UpdateHandler updateHandler,
+                  IndexDeletionPolicyWrapper delPolicy, SolrCore prev, boolean reload) {
+
+    this.coreContainer = coreContainer;
     
     assert ObjectReleaseTracker.track(searcherExecutor); // ensure that in unclean shutdown tests we still close this
-    
-    this.coreDescriptor = Objects.requireNonNull(coreDescriptor, "coreDescriptor cannot be null");
+
+    CoreDescriptor cd = Objects.requireNonNull(coreDescriptor, "coreDescriptor cannot be null");
+    coreContainer.solrCores.addCoreDescriptor(cd);
+
     setName(name);
     MDCLoggingContext.setCore(this);
     
@@ -900,14 +912,17 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
 
     checkVersionFieldExistsInSchema(schema, coreDescriptor);
 
-    SolrMetricManager metricManager = this.coreDescriptor.getCoreContainer().getMetricManager();
+    SolrMetricManager metricManager = coreContainer.getMetricManager();
 
     // initialize searcher-related metrics
     initializeMetrics(metricManager, coreMetricManager.getRegistryName(), null);
 
-    // Initialize JMX
-    this.infoRegistry = initInfoRegistry(name, config);
-    infoRegistry.put("fieldCache", new SolrFieldCacheMBean());
+    SolrFieldCacheBean solrFieldCacheBean = new SolrFieldCacheBean();
+    // this is registered at the CONTAINER level because it's not core-specific - for now we
+    // also register it here for back-compat
+    solrFieldCacheBean.initializeMetrics(metricManager, coreMetricManager.getRegistryName(), "core");
+    infoRegistry.put("fieldCache", solrFieldCacheBean);
+
 
     initSchema(config, schema);
 
@@ -998,15 +1013,9 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
     // from the core.
     resourceLoader.inform(infoRegistry);
 
-    // Allow the directory factory to register MBeans as well
-    for (SolrInfoMBean bean : directoryFactory.offerMBeans()) {
-      log.debug("Registering JMX bean [{}] from directory factory.", bean.getName());
-      // Not worried about concurrency, so no reason to use putIfAbsent
-      if (infoRegistry.containsKey(bean.getName())){
-        log.debug("Ignoring JMX bean [{}] due to name conflict.", bean.getName());
-      } else {
-        infoRegistry.put(bean.getName(), bean);
-      }
+    // Allow the directory factory to report metrics
+    if (directoryFactory instanceof SolrMetricProducer) {
+      ((SolrMetricProducer)directoryFactory).initializeMetrics(metricManager, coreMetricManager.getRegistryName(), "directoryFactory");
     }
 
     // seed version buckets with max from index during core initialization ... requires a searcher!
@@ -1040,15 +1049,15 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
 
   /** Set UpdateLog to buffer updates if the slice is in construction. */
   private void bufferUpdatesIfConstructing(CoreDescriptor coreDescriptor) {
-    final CoreContainer cc = coreDescriptor.getCoreContainer();
-    if (cc != null && cc.isZooKeeperAware()) {
+    
+    if (coreContainer != null && coreContainer.isZooKeeperAware()) {
       if (reqHandlers.get("/get") == null) {
         log.warn("WARNING: RealTimeGetHandler is not registered at /get. " +
             "SolrCloud will always use full index replication instead of the more efficient PeerSync method.");
       }
 
       // ZK pre-register would have already happened so we read slice properties now
-      final ClusterState clusterState = cc.getZkController().getClusterState();
+      final ClusterState clusterState = coreContainer.getZkController().getClusterState();
       final DocCollection collection = clusterState.getCollection(coreDescriptor.getCloudDescriptor().getCollectionName());
       final Slice slice = collection.getSlice(coreDescriptor.getCloudDescriptor().getShardId());
       if (slice.getState() == Slice.State.CONSTRUCTION) {
@@ -1126,34 +1135,45 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
 
   @Override
   public void initializeMetrics(SolrMetricManager manager, String registry, String scope) {
-    newSearcherCounter = manager.counter(registry, "new", Category.SEARCHER.toString());
-    newSearcherTimer = manager.timer(registry, "time", Category.SEARCHER.toString(), "new");
-    newSearcherWarmupTimer = manager.timer(registry, "warmup", Category.SEARCHER.toString(), "new");
-    newSearcherMaxReachedCounter = manager.counter(registry, "maxReached", Category.SEARCHER.toString(), "new");
-    newSearcherOtherErrorsCounter = manager.counter(registry, "errors", Category.SEARCHER.toString(), "new");
-
-    manager.registerGauge(registry, () -> name == null ? "(null)" : name, true, "coreName", Category.CORE.toString());
-    manager.registerGauge(registry, () -> startTime, true, "startTime", Category.CORE.toString());
-    manager.registerGauge(registry, () -> getOpenCount(), true, "refCount", Category.CORE.toString());
-    manager.registerGauge(registry, () -> resourceLoader.getInstancePath().toString(), true, "instanceDir", Category.CORE.toString());
-    manager.registerGauge(registry, () -> getIndexDir(), true, "indexDir", Category.CORE.toString());
-    manager.registerGauge(registry, () -> getIndexSize(), true, "sizeInBytes", Category.INDEX.toString());
-    manager.registerGauge(registry, () -> NumberUtils.readableSize(getIndexSize()), true, "size", Category.INDEX.toString());
-    manager.registerGauge(registry, () -> coreDescriptor.getCoreContainer().getCoreNames(this), true, "aliases", Category.CORE.toString());
+    newSearcherCounter = manager.counter(this, registry, "new", Category.SEARCHER.toString());
+    newSearcherTimer = manager.timer(this, registry, "time", Category.SEARCHER.toString(), "new");
+    newSearcherWarmupTimer = manager.timer(this, registry, "warmup", Category.SEARCHER.toString(), "new");
+    newSearcherMaxReachedCounter = manager.counter(this, registry, "maxReached", Category.SEARCHER.toString(), "new");
+    newSearcherOtherErrorsCounter = manager.counter(this, registry, "errors", Category.SEARCHER.toString(), "new");
+
+    manager.registerGauge(this, registry, () -> name == null ? "(null)" : name, true, "coreName", Category.CORE.toString());
+    manager.registerGauge(this, registry, () -> startTime, true, "startTime", Category.CORE.toString());
+    manager.registerGauge(this, registry, () -> getOpenCount(), true, "refCount", Category.CORE.toString());
+    manager.registerGauge(this, registry, () -> resourceLoader.getInstancePath().toString(), true, "instanceDir", Category.CORE.toString());
+    manager.registerGauge(this, registry, () -> getIndexDir(), true, "indexDir", Category.CORE.toString());
+    manager.registerGauge(this, registry, () -> getIndexSize(), true, "sizeInBytes", Category.INDEX.toString());
+    manager.registerGauge(this, registry, () -> NumberUtils.readableSize(getIndexSize()), true, "size", Category.INDEX.toString());
+    if (coreContainer != null) {
+      manager.registerGauge(this, registry, () -> coreContainer.getCoreNames(this), true, "aliases", Category.CORE.toString());
+      final CloudDescriptor cd = getCoreDescriptor().getCloudDescriptor();
+      if (cd != null) {
+        manager.registerGauge(this, registry, () -> {
+          if (cd.getCollectionName() != null) {
+            return cd.getCollectionName();
+          } else {
+            return "_notset_";
+          }
+        }, true, "collection", Category.CORE.toString());
+
+        manager.registerGauge(this, registry, () -> {
+          if (cd.getShardId() != null) {
+            return cd.getShardId();
+          } else {
+            return "_auto_";
+          }
+        }, true, "shard", Category.CORE.toString());
+      }
+    }
     // initialize disk total / free metrics
     Path dataDirPath = Paths.get(dataDir);
     File dataDirFile = dataDirPath.toFile();
-    manager.registerGauge(registry, () -> dataDirFile.getTotalSpace(), true, "totalSpace", Category.CORE.toString(), "fs");
-    manager.registerGauge(registry, () -> dataDirFile.getUsableSpace(), true, "usableSpace", Category.CORE.toString(), "fs");
-  }
-
-  private Map<String,SolrInfoMBean> initInfoRegistry(String name, SolrConfig config) {
-    if (config.jmxConfig.enabled) {
-      return new JmxMonitoredMap<String, SolrInfoMBean>(name, coreMetricManager.getRegistryName(), String.valueOf(this.hashCode()), config.jmxConfig);
-    } else  {
-      log.debug("JMX monitoring not detected for core: " + name);
-      return new ConcurrentHashMap<>();
-    }
+    manager.registerGauge(this, registry, () -> dataDirFile.getTotalSpace(), true, "totalSpace", Category.CORE.toString(), "fs");
+    manager.registerGauge(this, registry, () -> dataDirFile.getUsableSpace(), true, "usableSpace", Category.CORE.toString(), "fs");
   }
 
   private void checkVersionFieldExistsInSchema(IndexSchema schema, CoreDescriptor coreDescriptor) {
@@ -2685,6 +2705,9 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
     for (PluginInfo info : pluginInfos) {
       T o = createInitInstance(info,type, type.getSimpleName(), defClassName);
       registry.put(info.name, o);
+      if (o instanceof SolrMetricProducer) {
+        coreMetricManager.registerMetricProducer(type.getSimpleName() + "." + info.name, (SolrMetricProducer)o);
+      }
       if(info.isDefault()){
         def = o;
       }
@@ -2692,6 +2715,12 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
     return def;
   }
 
+  public void initDefaultPlugin(Object plugin, Class type) {
+    if (plugin instanceof SolrMetricProducer) {
+      coreMetricManager.registerMetricProducer(type.getSimpleName() + ".default", (SolrMetricProducer)plugin);
+    }
+  }
+
   /**For a given List of PluginInfo return the instances as a List
    * @param defClassName The default classname if PluginInfo#className == null
    * @return The instances initialized
@@ -2746,7 +2775,7 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
     if (initArgs == null)
       initArgs = new NamedList<>();
 
-    String collection = coreDescriptor.getCollectionName();
+    String collection = getCoreDescriptor().getCollectionName();
     StorageIO storageIO =
         ManagedResourceStorage.newStorageIO(collection, resourceLoader, initArgs);
     mgr.init(resourceLoader, initArgs, storageIO);
@@ -2755,7 +2784,7 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
   }
 
   public CoreDescriptor getCoreDescriptor() {
-    return coreDescriptor;
+    return coreContainer.getCoreDescriptor(name);
   }
 
   public IndexDeletionPolicyWrapper getDeletionPolicy(){
@@ -2775,15 +2804,10 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
   }
 
   /////////////////////////////////////////////////////////////////////
-  // SolrInfoMBean stuff: Statistics and Module Info
+  // SolrInfoBean stuff: Statistics and Module Info
   /////////////////////////////////////////////////////////////////////
 
   @Override
-  public String getVersion() {
-    return SolrCore.version;
-  }
-
-  @Override
   public String getDescription() {
     return "SolrCore";
   }
@@ -2794,55 +2818,15 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
   }
 
   @Override
-  public String getSource() {
-    return null;
-  }
-
-  @Override
-  public URL[] getDocs() {
-    return null;
-  }
-
-  @Override
-  public NamedList getStatistics() {
-    NamedList<Object> lst = new SimpleOrderedMap<>(8);
-    lst.add("coreName", name==null ? "(null)" : name);
-    lst.add("startTime", startTime);
-    lst.add("refCount", getOpenCount());
-    lst.add("instanceDir", resourceLoader.getInstancePath());
-    lst.add("indexDir", getIndexDir());
-    long size = getIndexSize();
-    lst.add("sizeInBytes", size);
-    lst.add("size", NumberUtils.readableSize(size));
-
-    CoreDescriptor cd = getCoreDescriptor();
-    if (cd != null) {
-      if (null != cd && cd.getCoreContainer() != null) {
-        lst.add("aliases", getCoreDescriptor().getCoreContainer().getCoreNames(this));
-      }
-      CloudDescriptor cloudDesc = cd.getCloudDescriptor();
-      if (cloudDesc != null) {
-        String collection = cloudDesc.getCollectionName();
-        if (collection == null) {
-          collection = "_notset_";
-        }
-        lst.add("collection", collection);
-        String shard = cloudDesc.getShardId();
-        if (shard == null) {
-          shard = "_auto_";
-        }
-        lst.add("shard", shard);
-      }
-    }
-
-    return lst;
+  public MetricRegistry getMetricRegistry() {
+    return coreMetricManager.getRegistry();
   }
 
   public Codec getCodec() {
     return codec;
   }
 
-  public void unloadOnClose(boolean deleteIndexDir, boolean deleteDataDir, boolean deleteInstanceDir) {
+  public void unloadOnClose(final CoreDescriptor desc, boolean deleteIndexDir, boolean deleteDataDir, boolean deleteInstanceDir) {
     if (deleteIndexDir) {
       try {
         directoryFactory.remove(getIndexDir());
@@ -2865,13 +2849,12 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
 
         @Override
         public void postClose(SolrCore core) {
-          CoreDescriptor cd = core.getCoreDescriptor();
-          if (cd != null) {
+          if (desc != null) {
             try {
-              FileUtils.deleteDirectory(cd.getInstanceDir().toFile());
+              FileUtils.deleteDirectory(desc.getInstanceDir().toFile());
             } catch (IOException e) {
               SolrException.log(log, "Failed to delete instance dir for core:"
-                  + core.getName() + " dir:" + cd.getInstanceDir());
+                  + core.getName() + " dir:" + desc.getInstanceDir());
             }
           }
         }
@@ -2930,7 +2913,7 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
 
   public static Runnable getConfListener(SolrCore core, ZkSolrResourceLoader zkSolrResourceLoader) {
     final String coreName = core.getName();
-    final CoreContainer cc = core.getCoreDescriptor().getCoreContainer();
+    final CoreContainer cc = core.getCoreContainer();
     final String overlayPath = zkSolrResourceLoader.getConfigSetZkPath() + "/" + ConfigOverlay.RESOURCE_NAME;
     final String solrConfigPath = zkSolrResourceLoader.getConfigSetZkPath() + "/" + core.getSolrConfig().getName();
     String schemaRes = null;
@@ -2983,11 +2966,11 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
     };
   }
 
-  public void registerInfoBean(String name, SolrInfoMBean solrInfoMBean) {
-    infoRegistry.put(name, solrInfoMBean);
+  public void registerInfoBean(String name, SolrInfoBean solrInfoBean) {
+    infoRegistry.put(name, solrInfoBean);
 
-    if (solrInfoMBean instanceof SolrMetricProducer) {
-      SolrMetricProducer producer = (SolrMetricProducer) solrInfoMBean;
+    if (solrInfoBean instanceof SolrMetricProducer) {
+      SolrMetricProducer producer = (SolrMetricProducer) solrInfoBean;
       coreMetricManager.registerMetricProducer(name, producer);
     }
   }
@@ -3064,7 +3047,6 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
     if (!BlobRepository.BLOB_KEY_PATTERN_CHECKER.matcher(key).matches()) {
       throw new IllegalArgumentException("invalid key format, must end in /N where N is the version number");
     }
-    CoreContainer coreContainer = getCoreDescriptor().getCoreContainer();
     // define the blob
     BlobRepository.BlobContentRef blobRef = coreContainer.getBlobRepository().getBlobIncRef(key, decoder);
     addCloseHook(new CloseHook() {
@@ -3074,7 +3056,7 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea
 
       @Override
       public void postClose(SolrCore core) {
-        core.getCoreDescriptor().getCoreContainer().getBlobRepository().decrementBlobRefCount(blobRef);
+        coreContainer.getBlobRepository().decrementBlobRefCount(blobRef);
       }
     });
     return blobRef;