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 2019/12/18 16:39:01 UTC

[lucene-solr] 23/36: First round of refactoring for better type safety.

This is an automated email from the ASF dual-hosted git repository.

ab pushed a commit to branch jira/solr-13579
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit cd040a178a0495bf5478562624f61ba965dee94c
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Aug 1 13:37:59 2019 +0200

    First round of refactoring for better type safety.
---
 .../java/org/apache/solr/core/CoreContainer.java   |   3 +-
 .../solr/handler/admin/ResourceManagerHandler.java |  69 +++++++----
 .../managed/AbstractResourceManagerPlugin.java     |  23 ----
 .../solr/managed/DefaultResourceManager.java       |   2 +-
 .../DefaultResourceManagerPluginFactory.java       |  25 +++-
 .../solr/managed/DefaultResourceManagerPool.java   |  12 +-
 .../org/apache/solr/managed/ManagedComponent.java  |  53 --------
 .../apache/solr/managed/ManagedMetricProducer.java |  47 -------
 .../apache/solr/managed/NoOpResourceManager.java   |  61 +++++++--
 .../org/apache/solr/managed/ResourceManager.java   |  19 ++-
 .../apache/solr/managed/ResourceManagerPlugin.java |  28 +++--
 .../solr/managed/ResourceManagerPluginFactory.java |  18 ++-
 .../apache/solr/managed/ResourceManagerPool.java   |   2 +
 .../{plugins => types}/CacheManagerPlugin.java     |  54 ++++++--
 .../solr/managed/types/ManagedCacheComponent.java  |  22 ++++
 .../managed/{plugins => types}/package-info.java   |   2 +-
 .../java/org/apache/solr/search/FastLRUCache.java  | 138 ++++++++-------------
 .../src/java/org/apache/solr/search/LFUCache.java  | 125 ++++++-------------
 .../src/java/org/apache/solr/search/LRUCache.java  |  87 ++++++-------
 .../src/java/org/apache/solr/search/SolrCache.java |   5 +-
 .../org/apache/solr/search/SolrCacheHolder.java    |  58 +++++++--
 .../managed/TestDefaultResourceManagerPool.java    |   2 +-
 .../org/apache/solr/search/TestFastLRUCache.java   |   8 +-
 .../test/org/apache/solr/search/TestLFUCache.java  |   4 +-
 .../test/org/apache/solr/search/TestLRUCache.java  |   8 +-
 25 files changed, 425 insertions(+), 450 deletions(-)

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 5da961b..daff00f 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -101,7 +101,6 @@ import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.managed.DefaultResourceManager;
 import org.apache.solr.managed.NoOpResourceManager;
 import org.apache.solr.managed.ResourceManager;
-import org.apache.solr.managed.plugins.CacheManagerPlugin;
 import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
@@ -676,7 +675,7 @@ public class CoreContainer {
     }
     try {
       resourceManager = ResourceManager.load(loader, TimeSource.NANO_TIME, DefaultResourceManager.class,
-          new PluginInfo("resourceManager", Collections.emptyMap()), poolConfigs);
+          new PluginInfo("resourceManager", Collections.emptyMap()), resManConfig);
     } catch (Exception e) {
       log.warn("Resource manager initialization error - disabling!", e);
       resourceManager = NoOpResourceManager.INSTANCE;
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ResourceManagerHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ResourceManagerHandler.java
index 76b3aa7..8c58dea 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ResourceManagerHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ResourceManagerHandler.java
@@ -237,24 +237,32 @@ public class ResourceManagerHandler extends RequestHandlerBase implements Permis
     NamedList<Object> result = new SimpleOrderedMap<>();
     switch (op) {
       case LIST:
-        pool.getComponents().forEach((n, resource) -> {
+        pool.getComponents().forEach((n, component) -> {
           NamedList<Object> perRes = new SimpleOrderedMap<>();
           result.add(n, perRes);
-          perRes.add("class", resource.getClass().getName());
-          perRes.add("types", resource.getManagedResourceTypes());
-          perRes.add("resourceLimits", resource.getResourceLimits());
+          perRes.add("class", component.getClass().getName());
+          try {
+            perRes.add("resourceLimits", pool.getResourceManagerPlugin().getResourceLimits(component));
+          } catch (Exception e) {
+            log.warn("Error getting resourceLimits of " + component.getManagedComponentId(), e);
+            result.add("error", "Error getting resource limits of " + resName + ": " + e.toString());
+          }
         });
         break;
       case STATUS:
-        ManagedComponent managedComponent = pool.getComponents().get(resName);
-        if (managedComponent == null) {
-          throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "Resource '" + resName + " not found in pool '" + poolName + "'.");
+        ManagedComponent component = pool.getComponents().get(resName);
+        if (component == null) {
+          throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "Component '" + resName + " not found in pool '" + poolName + "'.");
+        }
+        result.add("class", component.getClass().getName());
+        try {
+          result.add("resourceLimits", pool.getResourceManagerPlugin().getResourceLimits(component));
+        } catch (Exception e) {
+          log.warn("Error getting resource limits of " + resName + "/" + poolName + " : " + e.toString(), e);
+          result.add("error", "Error getting resource limits of " + resName + ": " + e.toString());
         }
-        result.add("class", managedComponent.getClass().getName());
-        result.add("types", managedComponent.getManagedResourceTypes());
-        result.add("resourceLimits", managedComponent.getResourceLimits());
         try {
-          result.add("monitoredValues", managedComponent.getMonitoredValues(Collections.emptySet()));
+          result.add("monitoredValues", pool.getResourceManagerPlugin().getMonitoredValues(component));
         } catch (Exception e) {
           log.warn("Error getting monitored values of " + resName + "/" + poolName + " : " + e.toString(), e);
           result.add("error", "Error getting monitored values of " + resName + ": " + e.toString());
@@ -263,26 +271,41 @@ public class ResourceManagerHandler extends RequestHandlerBase implements Permis
       case GETLIMITS:
         ManagedComponent managedComponent1 = pool.getComponents().get(resName);
         if (managedComponent1 == null) {
-          throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "Resource '" + resName + " not found in pool '" + poolName + "'.");
+          throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "Component '" + resName + " not found in pool '" + poolName + "'.");
+        }
+        try {
+          result.add("resourceLimits", pool.getResourceManagerPlugin().getResourceLimits(managedComponent1));
+        } catch (Exception e) {
+          log.warn("Error getting resource limits of " + resName + "/" + poolName + " : " + e.toString(), e);
+          result.add("error", "Error getting resource limits of " + resName + ": " + e.toString());
         }
-        result.add("resourceLimits", managedComponent1.getResourceLimits());
         break;
       case SETLIMITS:
         ManagedComponent managedComponent2 = pool.getComponents().get(resName);
         if (managedComponent2 == null) {
           throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "Resource '" + resName + " not found in pool '" + poolName + "'.");
         }
-        Map<String, Object> currentLimits = new HashMap<>(managedComponent2.getResourceLimits());
-        Map<String, Object> newLimits = getMap(params, LIMIT_PREFIX_PARAM);
-        newLimits.forEach((k, v) -> {
-          if (v == null) {
-            currentLimits.remove(k);
-          } else {
-            currentLimits.put(k, v);
+        try {
+          Map<String, Object> currentLimits = new HashMap<>(pool.getResourceManagerPlugin().getResourceLimits(managedComponent2));
+          Map<String, Object> newLimits = getMap(params, LIMIT_PREFIX_PARAM);
+          newLimits.forEach((k, v) -> {
+            if (v == null) {
+              currentLimits.remove(k);
+            } else {
+              currentLimits.put(k, v);
+            }
+          });
+          try {
+            pool.getResourceManagerPlugin().setResourceLimits(managedComponent2, newLimits);
+            result.add("success", newLimits);
+          } catch (Exception e) {
+            log.warn("Error setting resource limits of " + resName + "/" + poolName + " : " + e.toString(), e);
+            result.add("error", "Error setting resource limits of " + resName + ": " + e.toString());
           }
-        });
-        managedComponent2.setResourceLimits(newLimits);
-        result.add("success", newLimits);
+        } catch (Exception e) {
+          log.warn("Error getting resource limits of " + resName + "/" + poolName + " : " + e.toString(), e);
+          result.add("error", "Error getting resource limits of " + resName + ": " + e.toString());
+        }
         break;
       case DELETE:
         result.add("success", pool.unregisterComponent(resName) ? "removed" : "not found");
diff --git a/solr/core/src/java/org/apache/solr/managed/AbstractResourceManagerPlugin.java b/solr/core/src/java/org/apache/solr/managed/AbstractResourceManagerPlugin.java
deleted file mode 100644
index 1e630b1..0000000
--- a/solr/core/src/java/org/apache/solr/managed/AbstractResourceManagerPlugin.java
+++ /dev/null
@@ -1,23 +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.managed;
-
-/**
- *
- */
-public abstract class AbstractResourceManagerPlugin implements ResourceManagerPlugin {
-}
diff --git a/solr/core/src/java/org/apache/solr/managed/DefaultResourceManager.java b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManager.java
index 93dfb2d2..d485685 100644
--- a/solr/core/src/java/org/apache/solr/managed/DefaultResourceManager.java
+++ b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManager.java
@@ -33,7 +33,7 @@ import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.managed.plugins.CacheManagerPlugin;
+import org.apache.solr.managed.types.CacheManagerPlugin;
 import org.apache.solr.search.SolrCache;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.slf4j.Logger;
diff --git a/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPluginFactory.java b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPluginFactory.java
index ab6be81..0b00e74 100644
--- a/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPluginFactory.java
+++ b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPluginFactory.java
@@ -20,17 +20,20 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.managed.plugins.CacheManagerPlugin;
+import org.apache.solr.managed.types.CacheManagerPlugin;
+import org.apache.solr.managed.types.ManagedCacheComponent;
 
 /**
  * Default implementation of {@link ResourceManagerPluginFactory}.
  */
 public class DefaultResourceManagerPluginFactory implements ResourceManagerPluginFactory {
 
-  private static final Map<String, String> typeToClass = new HashMap<>();
+  private static final Map<String, Class<? extends ResourceManagerPlugin>> typeToPluginClass = new HashMap<>();
+  private static final Map<String, Class<? extends ManagedComponent>> typeToComponentClass = new HashMap<>();
 
   static {
-    typeToClass.put(CacheManagerPlugin.TYPE, CacheManagerPlugin.class.getName());
+    typeToPluginClass.put(CacheManagerPlugin.TYPE, CacheManagerPlugin.class);
+    typeToComponentClass.put(CacheManagerPlugin.TYPE, ManagedCacheComponent.class);
   }
 
   private final SolrResourceLoader loader;
@@ -40,13 +43,23 @@ public class DefaultResourceManagerPluginFactory implements ResourceManagerPlugi
   }
 
   @Override
-  public ResourceManagerPlugin create(String type, Map<String, Object> params) throws Exception {
-    String pluginClazz = typeToClass.get(type);
+  public <T extends ManagedComponent> ResourceManagerPlugin<T> create(String type, Map<String, Object> params) throws Exception {
+    Class<? extends ResourceManagerPlugin> pluginClazz = typeToPluginClass.get(type);
     if (pluginClazz == null) {
       throw new IllegalArgumentException("Unsupported plugin type '" + type + "'");
     }
-    ResourceManagerPlugin resourceManagerPlugin = loader.newInstance(pluginClazz, ResourceManagerPlugin.class);
+    ResourceManagerPlugin<T> resourceManagerPlugin = loader.newInstance(pluginClazz.getName(), ResourceManagerPlugin.class);
     resourceManagerPlugin.init(params);
     return resourceManagerPlugin;
   }
+
+  @Override
+  public Class<? extends ManagedComponent> getComponentClassByType(String type) {
+    return typeToComponentClass.get(type);
+  }
+
+  @Override
+  public Class<? extends ResourceManagerPlugin> getPluginClassByType(String type) {
+    return typeToPluginClass.get(type);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPool.java b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPool.java
index 9f87f42..1729789 100644
--- a/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPool.java
+++ b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPool.java
@@ -40,6 +40,7 @@ public class DefaultResourceManagerPool implements ResourceManagerPool {
   private final Map<String, ManagedComponent> resources = new ConcurrentHashMap<>();
   private Map<String, Object> poolLimits;
   private final String type;
+  private final Class<? extends ManagedComponent> componentClass;
   private final String name;
   private final ResourceManagerPlugin resourceManagerPlugin;
   private final Map<String, Object> args;
@@ -62,6 +63,7 @@ public class DefaultResourceManagerPool implements ResourceManagerPool {
     this.name = name;
     this.type = type;
     this.resourceManagerPlugin = factory.create(type, args);
+    this.componentClass = factory.getComponentClassByType(type);
     this.poolLimits = new TreeMap<>(poolLimits);
     this.args = new HashMap<>(args);
   }
@@ -82,9 +84,13 @@ public class DefaultResourceManagerPool implements ResourceManagerPool {
   }
 
   @Override
+  public ResourceManagerPlugin getResourceManagerPlugin() {
+    return resourceManagerPlugin;
+  }
+
+  @Override
   public void registerComponent(ManagedComponent managedComponent) {
-    Collection<String> types = managedComponent.getManagedResourceTypes();
-    if (!types.contains(type)) {
+    if (!componentClass.isAssignableFrom(managedComponent.getClass())) {
       log.debug("Pool type '" + type + "' is not supported by the resource " + managedComponent.getManagedComponentId());
       return;
     }
@@ -112,7 +118,7 @@ public class DefaultResourceManagerPool implements ResourceManagerPool {
       Map<String, Map<String, Object>> currentValues = new HashMap<>();
       for (ManagedComponent managedComponent : resources.values()) {
         try {
-          currentValues.put(managedComponent.getManagedComponentId().toString(), managedComponent.getMonitoredValues(resourceManagerPlugin.getMonitoredParams()));
+          currentValues.put(managedComponent.getManagedComponentId().toString(), resourceManagerPlugin.getMonitoredValues(managedComponent));
         } catch (Exception e) {
           log.warn("Error getting managed values from " + managedComponent.getManagedComponentId(), e);
         }
diff --git a/solr/core/src/java/org/apache/solr/managed/ManagedComponent.java b/solr/core/src/java/org/apache/solr/managed/ManagedComponent.java
index 2b9e6b8..cf75af3 100644
--- a/solr/core/src/java/org/apache/solr/managed/ManagedComponent.java
+++ b/solr/core/src/java/org/apache/solr/managed/ManagedComponent.java
@@ -16,19 +16,10 @@
  */
 package org.apache.solr.managed;
 
-import java.lang.invoke.MethodHandles;
-import java.util.Collection;
-import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * A managed component.
  */
 public interface ManagedComponent {
-  Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
   /**
    * Unique name of this component. By convention id-s form a dot-separated hierarchy that
    * follows the naming of metric registries and metric names.
@@ -36,50 +27,6 @@ public interface ManagedComponent {
   ManagedComponentId getManagedComponentId();
 
   /**
-   * Returns types of management plugins supported by this component. This must always
-   * return a non-null collection with at least one entry.
-   */
-  Collection<String> getManagedResourceTypes();
-
-  /**
-   * Set values of managed resource limits.
-   * @param limits map of limit names and values
-   */
-  default void setResourceLimits(Map<String, Object> limits) {
-    if (limits == null) {
-      return;
-    }
-    limits.forEach((key, value) -> {
-      try {
-        setResourceLimit(key, value);
-      } catch (Exception e) {
-        log.warn("Exception setting resource limit on {}: key={}, value={}, exception={}",
-            getManagedComponentId(), key, value, e);
-      }
-    });
-  }
-
-  /**
-   * Set value of a managed resource limit.
-   * @param key limit name
-   * @param value limit value
-   */
-  void setResourceLimit(String key, Object value) throws Exception;
-
-  /**
-   * Returns current values of managed resource limits.
-   * @return map where keys are controlled parameters and values are current values of limits
-   */
-  Map<String, Object> getResourceLimits();
-
-  /**
-   * Returns monitored values that are used for calculating optimal settings of managed resource limits.
-   * @param params selected monitored parameters, empty collection to return all monitored values
-   * @return map of parameter names to current values.
-   */
-  Map<String, Object> getMonitoredValues(Collection<String> params) throws Exception;
-
-  /**
    * Component context used for managing additional component state.
    * @return component's context
    */
diff --git a/solr/core/src/java/org/apache/solr/managed/ManagedMetricProducer.java b/solr/core/src/java/org/apache/solr/managed/ManagedMetricProducer.java
deleted file mode 100644
index 1cb9a04..0000000
--- a/solr/core/src/java/org/apache/solr/managed/ManagedMetricProducer.java
+++ /dev/null
@@ -1,47 +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.managed;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.solr.core.SolrInfoBean;
-
-/**
- * Convenience interface for {@link SolrInfoBean}-s that need to be managed.
- */
-public interface ManagedMetricProducer extends SolrInfoBean, ManagedComponent {
-
-  @Override
-  default Map<String, Object> getMonitoredValues(Collection<String> params) {
-    Map<String, Object> metrics = getMetricsSnapshot();
-    if (metrics == null) {
-      return Collections.emptyMap();
-    }
-    Map<String, Object> result = new HashMap<>();
-    params.forEach(tag -> {
-      Object value = metrics.get(tag);
-      if (value == null) {
-        return;
-      }
-      result.put(tag, value);
-    });
-    return result;
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/managed/NoOpResourceManager.java b/solr/core/src/java/org/apache/solr/managed/NoOpResourceManager.java
index e2e12eb..282fd11 100644
--- a/solr/core/src/java/org/apache/solr/managed/NoOpResourceManager.java
+++ b/solr/core/src/java/org/apache/solr/managed/NoOpResourceManager.java
@@ -30,8 +30,42 @@ public class NoOpResourceManager extends ResourceManager {
 
   public static final NoOpResourceManager INSTANCE = new NoOpResourceManager();
 
+  private static final class NoOpResourceManagerPlugin implements ResourceManagerPlugin {
+    static final NoOpResourceManagerPlugin INSTANCE = new NoOpResourceManagerPlugin();
+
+    @Override
+    public String getType() {
+      return NOOP;
+    }
+
+    @Override
+    public Map<String, Object> getMonitoredValues(ManagedComponent component) throws Exception {
+      return Collections.emptyMap();
+    }
+
+    @Override
+    public void setResourceLimit(ManagedComponent component, String limitName, Object value) throws Exception {
+      // no-op
+    }
+
+    @Override
+    public Map<String, Object> getResourceLimits(ManagedComponent component) throws Exception {
+      return Collections.emptyMap();
+    }
+
+    @Override
+    public void manage(ResourceManagerPool pool) throws Exception {
+      // no-op
+    }
+
+    @Override
+    public void init(Map params) {
+      // no-op
+    }
+  }
+
   private static final class NoOpResourcePool implements ResourceManagerPool {
-    static NoOpResourcePool INSTANCE = new NoOpResourcePool();
+    static final NoOpResourcePool INSTANCE = new NoOpResourcePool();
 
     @Override
     public String getName() {
@@ -44,8 +78,13 @@ public class NoOpResourceManager extends ResourceManager {
     }
 
     @Override
-    public void registerComponent(ManagedComponent managedComponent) {
+    public ResourceManagerPlugin getResourceManagerPlugin() {
+      return NoOpResourceManagerPlugin.INSTANCE;
+    }
 
+    @Override
+    public void registerComponent(ManagedComponent managedComponent) {
+      // no-op
     }
 
     @Override
@@ -80,7 +119,7 @@ public class NoOpResourceManager extends ResourceManager {
 
     @Override
     public void setPoolLimits(Map<String, Object> poolLimits) {
-
+      // no-op
     }
 
     @Override
@@ -90,23 +129,23 @@ public class NoOpResourceManager extends ResourceManager {
 
     @Override
     public void close() throws IOException {
-
+      // no-op
     }
 
     @Override
     public void run() {
-
+      // no-op
     }
   }
 
   @Override
   protected void doInit() throws Exception {
-
+    // no-op
   }
 
   @Override
   public void createPool(String name, String type, Map<String, Object> poolLimits, Map<String, Object> args) throws Exception {
-
+    // no-op
   }
 
   @Override
@@ -121,17 +160,17 @@ public class NoOpResourceManager extends ResourceManager {
 
   @Override
   public void setPoolLimits(String name, Map<String, Object> poolLimits) throws Exception {
-
+    // no-op
   }
 
   @Override
   public void removePool(String name) throws Exception {
-
+    // no-op
   }
 
   @Override
   public void registerComponent(String pool, ManagedComponent managedComponent) throws Exception {
-
+    // no-op
   }
 
   @Override
@@ -141,6 +180,6 @@ public class NoOpResourceManager extends ResourceManager {
 
   @Override
   public void close() throws IOException {
-
+    // no-op
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/managed/ResourceManager.java b/solr/core/src/java/org/apache/solr/managed/ResourceManager.java
index 48c2273..d75cb37 100644
--- a/solr/core/src/java/org/apache/solr/managed/ResourceManager.java
+++ b/solr/core/src/java/org/apache/solr/managed/ResourceManager.java
@@ -36,9 +36,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Base class for a resource management. It uses a flat model where there are named
- * resource pools of a given type, each pool with its own defined resource limits. Resources can be added
- * to a pool for the management of a specific aspect of that resource using {@link ResourceManagerPlugin}.
+ * Base class for resource management. It uses a flat model where there are named
+ * resource pools of a given type, each pool with its own defined resource limits. Components can be added
+ * to a pool for the management of a specific aspect of that component using {@link ResourceManagerPlugin}.
  */
 public abstract class ResourceManager implements SolrCloseable, PluginInfoInitialized {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -212,8 +212,7 @@ public abstract class ResourceManager implements SolrCloseable, PluginInfoInitia
    * Add a managed component to a pool.
    * @param pool existing pool name.
    * @param managedComponent managed component. The component must support the management type
-   *                        (in its {@link ManagedComponent#getManagedResourceTypes()}) used
-   *                        in the selected pool. The component must not be already managed by
+   *                        used in the selected pool. The component must not be already managed by
    *                        another pool of the same type.
    */
   public abstract void registerComponent(String pool, ManagedComponent managedComponent) throws Exception;
@@ -224,6 +223,16 @@ public abstract class ResourceManager implements SolrCloseable, PluginInfoInitia
    * @param componentId component id to remove
    * @return true if a component was actually registered and has been removed
    */
+  public boolean unregisterComponent(String pool, ManagedComponentId componentId) {
+    return unregisterComponent(pool, componentId.toString());
+  }
+
+  /**
+   * Remove a managed component from a pool.
+   * @param pool existing pool name.
+   * @param componentId component id to remove
+   * @return true if a component was actually registered and has been removed
+   */
   public abstract boolean unregisterComponent(String pool, String componentId);
 
   protected void ensureActive() {
diff --git a/solr/core/src/java/org/apache/solr/managed/ResourceManagerPlugin.java b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPlugin.java
index 01f8363..61ec2e8 100644
--- a/solr/core/src/java/org/apache/solr/managed/ResourceManagerPlugin.java
+++ b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPlugin.java
@@ -22,23 +22,27 @@ import java.util.Map;
 /**
  * A plugin that implements an algorithm for managing a pool of resources of a given type.
  */
-public interface ResourceManagerPlugin {
+public interface ResourceManagerPlugin<T extends ManagedComponent> {
 
-  /** Plugin type. */
+  /** Plugin symbolic type. */
   String getType();
 
   void init(Map<String, Object> params);
 
-  /**
-   * Name of monitored parameters that {@link ManagedComponent}-s managed by this plugin
-   * are expected to support.
-   */
-  Collection<String> getMonitoredParams();
-  /**
-   * Name of controlled parameters that {@link ManagedComponent}-s managed by this plugin
-   * are expected to support.
-   */
-  Collection<String> getControlledParams();
+  Map<String, Object> getMonitoredValues(T component) throws Exception;
+
+  default void setResourceLimits(T component, Map<String, Object> limits) throws Exception {
+    if (limits == null || limits.isEmpty()) {
+      return;
+    }
+    for (Map.Entry<String, Object> entry : limits.entrySet()) {
+      setResourceLimit(component, entry.getKey(), entry.getValue());
+    }
+  }
+
+  void setResourceLimit(T component, String limitName, Object value) throws Exception;
+
+  Map<String, Object> getResourceLimits(T component) throws Exception;
 
   /**
    * Manage resources in a pool. This method is called periodically by {@link ResourceManager},
diff --git a/solr/core/src/java/org/apache/solr/managed/ResourceManagerPluginFactory.java b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPluginFactory.java
index 6c8578a..508b5e9 100644
--- a/solr/core/src/java/org/apache/solr/managed/ResourceManagerPluginFactory.java
+++ b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPluginFactory.java
@@ -24,9 +24,21 @@ import java.util.Map;
 public interface ResourceManagerPluginFactory {
 
   /**
-   * Create a plugin of a given type.
-   * @param type plugin type
+   * Create a plugin of a given symbolic type.
+   * @param type plugin symbolic type
    * @param params plugin parameters
    */
-  ResourceManagerPlugin create(String type, Map<String, Object> params) throws Exception;
+  <T extends ManagedComponent> ResourceManagerPlugin<T> create(String type, Map<String, Object> params) throws Exception;
+
+  /**
+   * Get the implementation class for a component of a given symbolic type.
+   * @param type symbolic type
+   */
+  Class<? extends ManagedComponent> getComponentClassByType(String type);
+
+  /**
+   * Get the implementation class for a plugin of a given symbolic type.
+   * @param type symbolic type
+   */
+  Class<? extends ResourceManagerPlugin> getPluginClassByType(String type);
 }
diff --git a/solr/core/src/java/org/apache/solr/managed/ResourceManagerPool.java b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPool.java
index 783dc99..dc2ddac 100644
--- a/solr/core/src/java/org/apache/solr/managed/ResourceManagerPool.java
+++ b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPool.java
@@ -14,6 +14,8 @@ public interface ResourceManagerPool extends Runnable, Closeable {
   /** Pool type. */
   String getType();
 
+  ResourceManagerPlugin getResourceManagerPlugin();
+
   /** Add component to this pool. */
   void registerComponent(ManagedComponent managedComponent);
 
diff --git a/solr/core/src/java/org/apache/solr/managed/plugins/CacheManagerPlugin.java b/solr/core/src/java/org/apache/solr/managed/types/CacheManagerPlugin.java
similarity index 70%
rename from solr/core/src/java/org/apache/solr/managed/plugins/CacheManagerPlugin.java
rename to solr/core/src/java/org/apache/solr/managed/types/CacheManagerPlugin.java
index ad1ef3e..0ee888b 100644
--- a/solr/core/src/java/org/apache/solr/managed/plugins/CacheManagerPlugin.java
+++ b/solr/core/src/java/org/apache/solr/managed/types/CacheManagerPlugin.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.solr.managed.plugins;
+package org.apache.solr.managed.types;
 
 import java.lang.invoke.MethodHandles;
 import java.util.Arrays;
@@ -22,7 +22,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.solr.managed.AbstractResourceManagerPlugin;
+import org.apache.solr.managed.ResourceManagerPlugin;
 import org.apache.solr.managed.ResourceManagerPool;
 import org.apache.solr.search.SolrCache;
 import org.slf4j.Logger;
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
  * which can be adjusted using configuration parameter {@link #DEAD_BAND}. If monitored values don't
  * exceed the limits +/- the dead band then no action is taken.</p>
  */
-public class CacheManagerPlugin extends AbstractResourceManagerPlugin {
+public class CacheManagerPlugin implements ResourceManagerPlugin<ManagedCacheComponent> {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public static String TYPE = "cache";
@@ -66,13 +66,45 @@ public class CacheManagerPlugin extends AbstractResourceManagerPlugin {
   protected float deadBand = DEFAULT_DEAD_BAND;
 
   @Override
-  public Collection<String> getMonitoredParams() {
-    return MONITORED_PARAMS;
+  public void setResourceLimit(ManagedCacheComponent component, String limitName, Object val) {
+    if (!(val instanceof Number)) {
+      try {
+        val = Long.parseLong(String.valueOf(val));
+      } catch (Exception e) {
+        throw new IllegalArgumentException("Unsupported value type (not a number) for limit '" + limitName + "': " + val + " (" + val.getClass().getName() + ")");
+      }
+    }
+    Number value = (Number)val;
+    if (value.longValue() > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("Invalid new value for limit '" + limitName +"': " + value);
+    }
+    switch (limitName) {
+      case SolrCache.MAX_SIZE_PARAM:
+        component.setMaxSize(value.intValue());
+        break;
+      case SolrCache.MAX_RAM_MB_PARAM:
+        component.setMaxRamMB(value.intValue());
+        break;
+      default:
+        throw new IllegalArgumentException("Unsupported limit name '" + limitName + "'");
+    }
+  }
+
+  @Override
+  public Map<String, Object> getResourceLimits(ManagedCacheComponent component) {
+    Map<String, Object> limits = new HashMap<>();
+    limits.put(SolrCache.MAX_SIZE_PARAM, component.getMaxSize());
+    limits.put(SolrCache.MAX_RAM_MB_PARAM, component.getMaxRamMB());
+    return limits;
   }
 
   @Override
-  public Collection<String> getControlledParams() {
-    return CONTROLLED_PARAMS;
+  public Map<String, Object> getMonitoredValues(ManagedCacheComponent component) throws Exception {
+    Map<String, Object> values = new HashMap<>();
+    values.put(SolrCache.HIT_RATIO_PARAM, component.getHitRatio());
+    values.put(SolrCache.RAM_BYTES_USED_PARAM, component.ramBytesUsed());
+    values.put(SolrCache.SIZE_PARAM, component.getSize());
+    return values;
   }
 
   @Override
@@ -121,8 +153,8 @@ public class CacheManagerPlugin extends AbstractResourceManagerPlugin {
 
       float changeRatio = poolLimitValue / totalValue.floatValue();
       // modify current limits by the changeRatio
-      pool.getComponents().forEach((name, resource) -> {
-        Map<String, Object> resourceLimits = resource.getResourceLimits();
+      pool.getComponents().forEach((name, component) -> {
+        Map<String, Object> resourceLimits = getResourceLimits((ManagedCacheComponent)component);
         Object limit = resourceLimits.get(poolLimitName);
         // XXX we could attempt here to control eg. ramBytesUsed by adjusting maxSize limit
         // XXX and vice versa if the current limit is undefined or unsupported
@@ -135,10 +167,10 @@ public class CacheManagerPlugin extends AbstractResourceManagerPlugin {
         }
         float newLimit = currentResourceLimit * changeRatio;
         try {
-          resource.setResourceLimit(poolLimitName, newLimit);
+          setResourceLimit((ManagedCacheComponent)component, poolLimitName, newLimit);
         } catch (Exception e) {
           log.warn("Failed to set managed limit " + poolLimitName +
-              " from " + currentResourceLimit + " to " + newLimit + " on " + resource.getManagedComponentId(), e);
+              " from " + currentResourceLimit + " to " + newLimit + " on " + component.getManagedComponentId(), e);
         }
       });
     });
diff --git a/solr/core/src/java/org/apache/solr/managed/types/ManagedCacheComponent.java b/solr/core/src/java/org/apache/solr/managed/types/ManagedCacheComponent.java
new file mode 100644
index 0000000..d36a842
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/managed/types/ManagedCacheComponent.java
@@ -0,0 +1,22 @@
+package org.apache.solr.managed.types;
+
+import org.apache.lucene.util.Accountable;
+import org.apache.solr.managed.ManagedComponent;
+
+/**
+ *
+ */
+public interface ManagedCacheComponent extends ManagedComponent, Accountable {
+  /** Set maximum cache size limit (number of items). */
+  void setMaxSize(int size);
+  /** Set maximum cache size limit (in MB of RAM). */
+  void setMaxRamMB(int maxRamMB);
+  /** Get the configured maximum size limit (number of items). */
+  int getMaxSize();
+  /** Get the configured maximym size limit (in MB of RAM). */
+  int getMaxRamMB();
+  /** Get the current number of items in cache. */
+  int getSize();
+  /** Get the ratio of hits to lookups. */
+  float getHitRatio();
+}
diff --git a/solr/core/src/java/org/apache/solr/managed/plugins/package-info.java b/solr/core/src/java/org/apache/solr/managed/types/package-info.java
similarity index 95%
rename from solr/core/src/java/org/apache/solr/managed/plugins/package-info.java
rename to solr/core/src/java/org/apache/solr/managed/types/package-info.java
index 5b4ff61..5dd0a71 100644
--- a/solr/core/src/java/org/apache/solr/managed/plugins/package-info.java
+++ b/solr/core/src/java/org/apache/solr/managed/types/package-info.java
@@ -18,4 +18,4 @@
 /**
  * Implementations of {@link org.apache.solr.managed.ResourceManagerPlugin}.
  */
-package org.apache.solr.managed.plugins;
+package org.apache.solr.managed.types;
diff --git a/solr/core/src/java/org/apache/solr/search/FastLRUCache.java b/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
index 2fb9ada..ad5d82a 100644
--- a/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
@@ -22,7 +22,7 @@ import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.managed.ManagedComponentId;
 import org.apache.solr.managed.ManagedContext;
-import org.apache.solr.managed.plugins.CacheManagerPlugin;
+import org.apache.solr.managed.types.CacheManagerPlugin;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.util.ConcurrentLRUCache;
@@ -53,7 +53,7 @@ import java.util.concurrent.TimeUnit;
  * @see org.apache.solr.search.SolrCache
  * @since solr 1.4
  */
-public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K,V>, Accountable {
+public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K,V> {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FastLRUCache.class);
@@ -339,23 +339,6 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K,V>,
   }
 
   @Override
-  public Map<String, Object> getResourceLimits() {
-    Map<String, Object> limits = new HashMap<>();
-    limits.put(MAX_SIZE_PARAM, maxSize);
-    limits.put(MIN_SIZE_PARAM, minSize);
-    limits.put(ACCEPTABLE_SIZE_PARAM, acceptableSize);
-    limits.put(CLEANUP_THREAD_PARAM, cleanupThread);
-    limits.put(SHOW_ITEMS_PARAM, showItems);
-    limits.put(MAX_RAM_MB_PARAM, maxRamBytes != Long.MAX_VALUE ? maxRamBytes / 1024L / 1024L : -1L);
-    return limits;
-  }
-
-  @Override
-  public Map<String, Object> getMonitoredValues(Collection<String> params) throws Exception {
-    return cacheMap.getValue();
-  }
-
-  @Override
   public ManagedContext getManagedContext() {
     return managedContext;
   }
@@ -365,75 +348,6 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K,V>,
     return managedComponentId;
   }
 
-  @Override
-  public Collection<String> getManagedResourceTypes() {
-    return Collections.singleton(CacheManagerPlugin.TYPE);
-  }
-
-  @Override
-  public void setResourceLimit(String limitName, Object val) {
-    if (CLEANUP_THREAD_PARAM.equals(limitName)) {
-      Boolean value;
-      try {
-        value = Boolean.parseBoolean(val.toString());
-        cleanupThread = value;
-        cache.setRunCleanupThread(cleanupThread);
-      } catch (Exception e) {
-        throw new IllegalArgumentException("Invalid new value for boolean limit '" + limitName + "': " + val);
-      }
-    }
-    Number value;
-    try {
-      value = Long.parseLong(String.valueOf(val));
-    } catch (Exception e) {
-      throw new IllegalArgumentException("Invalid new value for numeric limit '" + limitName +"': " + val);
-    }
-    if (!limitName.equals(MAX_RAM_MB_PARAM)) {
-      if (value.intValue() <= 1) {
-        throw new IllegalArgumentException("Invalid new value for numeric limit '" + limitName +"': " + value);
-      }
-    }
-    if (value.longValue() > Integer.MAX_VALUE) {
-      throw new IllegalArgumentException("Invalid new value for numeric limit '" + limitName +"': " + value);
-    }
-    switch (limitName) {
-      case MAX_SIZE_PARAM:
-        maxSize = value.intValue();
-        checkAndAdjustLimits();
-        cache.setUpperWaterMark(maxSize);
-        cache.setLowerWaterMark(minSize);
-        break;
-      case MIN_SIZE_PARAM:
-        minSize = value.intValue();
-        checkAndAdjustLimits();
-        cache.setUpperWaterMark(maxSize);
-        cache.setLowerWaterMark(minSize);
-        break;
-      case ACCEPTABLE_SIZE_PARAM:
-        acceptableSize = value.intValue();
-        acceptableSize = Math.max(minSize, acceptableSize);
-        cache.setAcceptableWaterMark(acceptableSize);
-        break;
-      case MAX_RAM_MB_PARAM:
-        long maxRamMB = value.intValue();
-        maxRamBytes = maxRamMB < 0 ? Long.MAX_VALUE : maxRamMB * 1024L * 1024L;
-        if (maxRamMB < 0) {
-          ramLowerWatermark = Long.MIN_VALUE;
-        } else {
-          ramLowerWatermark = Math.round(maxRamBytes * 0.8);
-        }
-        cache.setRamUpperWatermark(maxRamBytes);
-        cache.setRamLowerWatermark(ramLowerWatermark);
-        break;
-      case SHOW_ITEMS_PARAM:
-        showItems = value.intValue();
-        break;
-      default:
-        throw new IllegalArgumentException("Unsupported limit '" + limitName + "'");
-    }
-    description = generateDescription();
-  }
-
   private void checkAndAdjustLimits() {
     if (minSize <= 0) minSize = 1;
     if (maxSize <= minSize) {
@@ -444,6 +358,54 @@ public class FastLRUCache<K, V> extends SolrCacheBase implements SolrCache<K,V>,
       }
     }
   }
+
+  @Override
+  public void setMaxSize(int size) {
+    if (size < 1) {
+      throw new IllegalArgumentException("Invalid new value for maxSize: " + size);
+    }
+    maxSize = size;
+    checkAndAdjustLimits();
+    cache.setUpperWaterMark(maxSize);
+    cache.setLowerWaterMark(minSize);
+    generateDescription();
+  }
+
+  @Override
+  public void setMaxRamMB(int maxRamMB) {
+    maxRamBytes = maxRamMB < 0 ? Long.MAX_VALUE : maxRamMB * MB;
+    if (maxRamMB < 0) {
+      ramLowerWatermark = Long.MIN_VALUE;
+    } else {
+      ramLowerWatermark = Math.round(maxRamBytes * 0.8);
+    }
+    cache.setRamUpperWatermark(maxRamBytes);
+    cache.setRamLowerWatermark(ramLowerWatermark);
+    generateDescription();
+  }
+
+  @Override
+  public int getMaxSize() {
+    return maxSize;
+  }
+
+  @Override
+  public int getMaxRamMB() {
+    return maxRamBytes == Long.MAX_VALUE ? -1 : (int)(maxRamBytes / MB);
+  }
+
+  @Override
+  public int getSize() {
+    return cache.size();
+  }
+
+  @Override
+  public float getHitRatio() {
+    ConcurrentLRUCache.Stats stats = cache.getStats();
+    long lookups = stats.getCumulativeLookups();
+    long hits = stats.getCumulativeHits();
+    return calcHitRatio(lookups, hits);
+  }
 }
 
 
diff --git a/solr/core/src/java/org/apache/solr/search/LFUCache.java b/solr/core/src/java/org/apache/solr/search/LFUCache.java
index 28decac..7bff8de 100644
--- a/solr/core/src/java/org/apache/solr/search/LFUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LFUCache.java
@@ -28,12 +28,11 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 
 import com.codahale.metrics.MetricRegistry;
-import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.managed.ManagedComponentId;
 import org.apache.solr.managed.ManagedContext;
-import org.apache.solr.managed.plugins.CacheManagerPlugin;
+import org.apache.solr.managed.types.CacheManagerPlugin;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.util.ConcurrentLFUCache;
@@ -56,7 +55,7 @@ import static org.apache.solr.common.params.CommonParams.NAME;
  * @see org.apache.solr.search.SolrCache
  * @since solr 3.6
  */
-public class LFUCache<K, V> implements SolrCache<K, V>, Accountable {
+public class LFUCache<K, V> implements SolrCache<K, V> {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(LFUCache.class);
@@ -252,7 +251,7 @@ public class LFUCache<K, V> implements SolrCache<K, V>, Accountable {
   }
 
   // returns a ratio, not a percent.
-  private static String calcHitRatio(long lookups, long hits) {
+  private static String calcHitRatioStr(long lookups, long hits) {
     if (lookups == 0) return "0.00";
     if (lookups == hits) return "1.00";
     int hundredths = (int) (hits * 100 / lookups);   // rounded down
@@ -274,13 +273,14 @@ public class LFUCache<K, V> implements SolrCache<K, V>, Accountable {
 
         map.put("lookups", lookups);
         map.put("hits", hits);
-        map.put(HIT_RATIO_PARAM, calcHitRatio(lookups, hits));
+        map.put(HIT_RATIO_PARAM, calcHitRatioStr(lookups, hits));
         map.put("inserts", inserts);
         map.put("evictions", evictions);
         map.put(SIZE_PARAM, size);
         map.put(MAX_SIZE_PARAM, maxSize);
         map.put(MIN_SIZE_PARAM, minSize);
         map.put(RAM_BYTES_USED_PARAM, ramBytesUsed());
+        map.put(MAX_RAM_MB_PARAM, getMaxRamMB());
 
         map.put("warmupTime", warmupTime);
         map.put("timeDecay", timeDecay);
@@ -300,7 +300,7 @@ public class LFUCache<K, V> implements SolrCache<K, V>, Accountable {
         }
         map.put("cumulative_lookups", clookups);
         map.put("cumulative_hits", chits);
-        map.put("cumulative_hitratio", calcHitRatio(clookups, chits));
+        map.put("cumulative_hitratio", calcHitRatioStr(clookups, chits));
         map.put("cumulative_inserts", cinserts);
         map.put("cumulative_evictions", cevictions);
 
@@ -364,98 +364,51 @@ public class LFUCache<K, V> implements SolrCache<K, V>, Accountable {
     return managedComponentId;
   }
 
+  private void checkAndAdjustLimits() {
+    if (minSize <= 0) minSize = 1;
+    if (maxSize <= minSize) {
+      if (maxSize > 1) {
+        minSize = maxSize - 1;
+      } else {
+        maxSize = minSize + 1;
+      }
+    }
+  }
+
   @Override
-  public Map<String, Object> getResourceLimits() {
-    Map<String, Object> limits = new HashMap<>();
-    limits.put(MAX_SIZE_PARAM, maxSize);
-    limits.put(MIN_SIZE_PARAM, minSize);
-    limits.put(ACCEPTABLE_SIZE_PARAM, acceptableSize);
-    limits.put(AUTOWARM_COUNT_PARAM, autowarmCount);
-    limits.put(CLEANUP_THREAD_PARAM, cleanupThread);
-    limits.put(SHOW_ITEMS_PARAM, showItems);
-    limits.put(TIME_DECAY_PARAM, timeDecay);
-    return limits;
+  public void setMaxSize(int size) {
+    maxSize = size;
+    checkAndAdjustLimits();
+    cache.setUpperWaterMark(maxSize);
+    cache.setLowerWaterMark(minSize);
+    generateDescription();
   }
 
   @Override
-  public Map<String, Object> getMonitoredValues(Collection<String> params) throws Exception {
-    return cacheMap.getValue();
+  public void setMaxRamMB(int maxRamMB) {
+    // no-op
   }
 
   @Override
-  public Collection<String> getManagedResourceTypes() {
-    return Collections.singleton(CacheManagerPlugin.TYPE);
+  public int getMaxSize() {
+    return maxSize;
   }
 
   @Override
-  public synchronized void setResourceLimit(String limitName, Object val) {
-    if (TIME_DECAY_PARAM.equals(limitName) || CLEANUP_THREAD_PARAM.equals(limitName)) {
-      Boolean value;
-      try {
-        value = Boolean.parseBoolean(String.valueOf(val));
-      } catch (Exception e) {
-        throw new IllegalArgumentException("Invalid value of boolean limit '" + limitName + "': " + val);
-      }
-      switch (limitName) {
-        case TIME_DECAY_PARAM:
-          timeDecay = value;
-          cache.setTimeDecay(timeDecay);
-          break;
-        case CLEANUP_THREAD_PARAM:
-          cleanupThread = value;
-          cache.setRunCleanupThread(cleanupThread);
-          break;
-      }
-    } else {
-      Number value;
-      try {
-        value = Long.parseLong(String.valueOf(val));
-      } catch (Exception e) {
-        throw new IllegalArgumentException("Invalid new value for numeric limit '" + limitName +"': " + val);
-      }
-      if (value.intValue() <= 1 || value.longValue() > Integer.MAX_VALUE) {
-        throw new IllegalArgumentException("Out of range new value for numeric limit '" + limitName +"': " + value);
-      }
-      switch (limitName) {
-        case MAX_SIZE_PARAM:
-          maxSize = value.intValue();
-          checkAndAdjustLimits();
-          cache.setUpperWaterMark(maxSize);
-          cache.setLowerWaterMark(minSize);
-          break;
-        case MIN_SIZE_PARAM:
-          minSize = value.intValue();
-          checkAndAdjustLimits();
-          cache.setUpperWaterMark(maxSize);
-          cache.setLowerWaterMark(minSize);
-          break;
-        case ACCEPTABLE_SIZE_PARAM:
-          acceptableSize = value.intValue();
-          acceptableSize = Math.max(minSize, acceptableSize);
-          cache.setAcceptableWaterMark(acceptableSize);
-          break;
-        case AUTOWARM_COUNT_PARAM:
-          autowarmCount = value.intValue();
-          break;
-        case SHOW_ITEMS_PARAM:
-          showItems = value.intValue();
-          break;
-        default:
-          throw new IllegalArgumentException("Unsupported numeric limit '" + limitName + "'");
-      }
-    }
-    description = generateDescription();
+  public int getMaxRamMB() {
+    return -1;
   }
 
-  private void checkAndAdjustLimits() {
-    if (minSize <= 0) minSize = 1;
-    if (maxSize <= minSize) {
-      if (maxSize > 1) {
-        minSize = maxSize - 1;
-      } else {
-        maxSize = minSize + 1;
-      }
-    }
+  @Override
+  public int getSize() {
+    return cache.getStats().getCurrentSize();
   }
 
+  @Override
+  public float getHitRatio() {
+    ConcurrentLFUCache.Stats stats = cache.getStats();
+    long lookups = stats.getCumulativeLookups();
+    long hits = stats.getCumulativeHits();
+    return SolrCacheBase.calcHitRatio(lookups, hits);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/search/LRUCache.java b/solr/core/src/java/org/apache/solr/search/LRUCache.java
index b4669bc..e9f5768 100644
--- a/solr/core/src/java/org/apache/solr/search/LRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LRUCache.java
@@ -35,7 +35,7 @@ import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.managed.ManagedComponentId;
 import org.apache.solr.managed.ManagedContext;
-import org.apache.solr.managed.plugins.CacheManagerPlugin;
+import org.apache.solr.managed.types.CacheManagerPlugin;
 import org.apache.solr.metrics.MetricsMap;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.slf4j.Logger;
@@ -47,7 +47,7 @@ import static org.apache.lucene.util.RamUsageEstimator.QUERY_DEFAULT_RAM_BYTES_U
 /**
  *
  */
-public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Accountable {
+public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V> {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(LRUCache.class);
@@ -163,6 +163,24 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
     return maxRamBytes;
   }
 
+  public int getMaxRamMB() {
+    if (maxRamBytes != Long.MAX_VALUE) {
+      return (int)(maxRamBytes / MB);
+    } else {
+      return -1;
+    }
+  }
+
+  @Override
+  public int getSize() {
+    return map.size();
+  }
+
+  @Override
+  public float getHitRatio() {
+    return calcHitRatio(lookups, hits);
+  }
+
   /**
    * 
    * @return Returns the description of this cache. 
@@ -172,9 +190,7 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
     if (isAutowarmingOn()) {
       description += ", " + getAutowarmDescription();
     }
-    if (getMaxRamBytes() != Long.MAX_VALUE)  {
-      description += ", maxRamMB=" + (getMaxRamBytes() / 1024L / 1024L);
-    }
+    description += ", maxRamMB=" + getMaxRamMB();
     description += ')';
     return description;
   }
@@ -327,7 +343,7 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
         res.put(SIZE_PARAM, map.size());
         res.put(MAX_SIZE_PARAM, maxSize);
         res.put(RAM_BYTES_USED_PARAM, ramBytesUsed());
-        res.put(MAX_RAM_MB_PARAM, maxRamBytes != Long.MAX_VALUE ? maxRamBytes / 1024L / 1024L : -1L);
+        res.put(MAX_RAM_MB_PARAM, getMaxRamMB());
         res.put("evictionsRamUsage", evictionsRamUsage);
       }
       res.put("warmupTime", warmupTime);
@@ -375,19 +391,6 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
   }
 
   @Override
-  public Map<String, Object> getResourceLimits() {
-    Map<String, Object> limits = new HashMap<>();
-    limits.put(MAX_SIZE_PARAM, maxSize);
-    limits.put(MAX_RAM_MB_PARAM, maxRamBytes != Long.MAX_VALUE ? maxRamBytes / 1024L / 1024L : -1L);
-    return limits;
-  }
-
-  @Override
-  public Map<String, Object> getMonitoredValues(Collection<String> params) throws Exception {
-    return cacheMap.getValue();
-  }
-
-  @Override
   public ManagedContext getManagedContext() {
     return managedContext;
   }
@@ -398,41 +401,23 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
   }
 
   @Override
-  public Collection<String> getManagedResourceTypes() {
-    return Collections.singleton(CacheManagerPlugin.TYPE);
+  public void setMaxSize(int newMaxSize) {
+    if (newMaxSize > 0) {
+      maxSize = newMaxSize;
+    } else {
+      maxSize = Integer.MAX_VALUE;
+    }
+    generateDescription();
   }
 
   @Override
-  public void setResourceLimit(String limitName, Object val) {
-    if (!(val instanceof Number)) {
-      try {
-        val = Long.parseLong(String.valueOf(val));
-      } catch (Exception e) {
-        throw new IllegalArgumentException("Unsupported value type (not a number) for limit '" + limitName + "': " + val + " (" + val.getClass().getName() + ")");
-      }
+  public void setMaxRamMB(int newMaxRamMB) {
+    if (newMaxRamMB > 0) {
+      maxRamBytes = newMaxRamMB * MB;
+    } else {
+      maxRamBytes = Long.MAX_VALUE;
     }
-    Number value = (Number)val;
-    if (value.longValue() > Integer.MAX_VALUE) {
-      throw new IllegalArgumentException("Invalid new value for limit '" + limitName +"': " + value);
-    }
-    switch (limitName) {
-      case MAX_SIZE_PARAM:
-        if (value.intValue() > 0) {
-          maxSize = value.intValue();
-        } else {
-          maxSize = Integer.MAX_VALUE;
-        }
-        break;
-      case MAX_RAM_MB_PARAM:
-        if (value.intValue() > 0) {
-          maxRamBytes = value.intValue() * 1024L * 1024L;
-        } else {
-          maxRamBytes = Long.MAX_VALUE;
-        }
-        break;
-      default:
-        throw new IllegalArgumentException("Unsupported limit name '" + limitName + "'");
-    }
-    description = generateDescription();
+    generateDescription();
   }
+
 }
diff --git a/solr/core/src/java/org/apache/solr/search/SolrCache.java b/solr/core/src/java/org/apache/solr/search/SolrCache.java
index 7814384..b1d090d 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrCache.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrCache.java
@@ -18,6 +18,7 @@ package org.apache.solr.search;
 
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.managed.ManagedComponent;
+import org.apache.solr.managed.types.ManagedCacheComponent;
 import org.apache.solr.metrics.SolrMetricProducer;
 
 import java.util.Map;
@@ -26,7 +27,7 @@ import java.util.Map;
 /**
  * Primary API for dealing with Solr's internal caches.
  */
-public interface SolrCache<K,V> extends SolrInfoBean, SolrMetricProducer, ManagedComponent {
+public interface SolrCache<K,V> extends SolrInfoBean, SolrMetricProducer, ManagedCacheComponent {
 
   /** Current size of the cache. */
   String SIZE_PARAM = "size";
@@ -42,6 +43,8 @@ public interface SolrCache<K,V> extends SolrInfoBean, SolrMetricProducer, Manage
   /** Cache hit ratio. */
   String HIT_RATIO_PARAM = "hitratio";
 
+  long MB = 1024L * 1024L;
+
   /**
    * The initialization routine. Instance specific arguments are passed in
    * the <code>args</code> map.
diff --git a/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java b/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
index 29dab92..fb562a2 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrCacheHolder.java
@@ -22,6 +22,8 @@ import java.util.Map;
 import java.util.Set;
 
 import com.codahale.metrics.MetricRegistry;
+import org.apache.solr.managed.ManagedComponentId;
+import org.apache.solr.managed.ManagedContext;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -80,18 +82,6 @@ public class SolrCacheHolder<K, V> implements SolrCache<K,V> {
     delegate.close();
   }
 
-  @Override
-  public Map<String, Object> getResourceLimits() {
-    return delegate.getResourceLimits();
-  }
-
-  @Override
-  public void setResourceLimit(String limitName, Object value) throws Exception {
-    delegate.setResourceLimit(limitName, value);
-
-  }
-
-
   public void warm(SolrIndexSearcher searcher, SolrCacheHolder src) {
     delegate.warm(searcher, src.get());
   }
@@ -139,4 +129,48 @@ public class SolrCacheHolder<K, V> implements SolrCache<K,V> {
 
   }
 
+  @Override
+  public void setMaxSize(int size) {
+    delegate.setMaxSize(size);
+  }
+
+  @Override
+  public void setMaxRamMB(int maxRamMB) {
+    delegate.setMaxRamMB(maxRamMB);
+  }
+
+  @Override
+  public int getMaxSize() {
+    return delegate.getMaxSize();
+  }
+
+  @Override
+  public int getMaxRamMB() {
+    return delegate.getMaxRamMB();
+  }
+
+  @Override
+  public int getSize() {
+    return delegate.getSize();
+  }
+
+  @Override
+  public float getHitRatio() {
+    return delegate.getHitRatio();
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return delegate.ramBytesUsed();
+  }
+
+  @Override
+  public ManagedComponentId getManagedComponentId() {
+    return delegate.getManagedComponentId();
+  }
+
+  @Override
+  public ManagedContext getManagedContext() {
+    return delegate.getManagedContext();
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/managed/TestDefaultResourceManagerPool.java b/solr/core/src/test/org/apache/solr/managed/TestDefaultResourceManagerPool.java
index 3b83093..9f65ec2 100644
--- a/solr/core/src/test/org/apache/solr/managed/TestDefaultResourceManagerPool.java
+++ b/solr/core/src/test/org/apache/solr/managed/TestDefaultResourceManagerPool.java
@@ -30,7 +30,7 @@ public class TestDefaultResourceManagerPool extends SolrTestCaseJ4 {
   }
 
   @Test
-  public void testBasic() throws Exception {
+  public void testBasicRegistration() throws Exception {
 
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/TestFastLRUCache.java b/solr/core/src/test/org/apache/solr/search/TestFastLRUCache.java
index 3159e3e..0fb1765 100644
--- a/solr/core/src/test/org/apache/solr/search/TestFastLRUCache.java
+++ b/solr/core/src/test/org/apache/solr/search/TestFastLRUCache.java
@@ -346,7 +346,7 @@ public class TestFastLRUCache extends SolrTestCase {
     // no evictions yet
     assertEquals(6, cache.size());
     // this also sets minLimit = 4
-    cache.setResourceLimit(SolrCache.SIZE_PARAM, 5);
+    cache.setMaxSize(5);
     // should not happen yet - evictions are triggered by put
     assertEquals(6, cache.size());
     cache.put("6", new Accountable() {
@@ -359,7 +359,7 @@ public class TestFastLRUCache extends SolrTestCase {
     assertEquals(4, cache.size());
 
     // modify ram limit
-    cache.setResourceLimit(SolrCache.MAX_RAM_MB_PARAM, 3);
+    cache.setMaxRamMB(3);
     // should not happen yet - evictions are triggered by put
     assertEquals(4, cache.size());
     // this evicts down to 3MB * 0.8, ie. ramLowerWaterMark
@@ -376,7 +376,7 @@ public class TestFastLRUCache extends SolrTestCase {
 
     // scale up
 
-    cache.setResourceLimit(SolrCache.MAX_RAM_MB_PARAM, 4);
+    cache.setMaxRamMB(4);
     cache.put("8", new Accountable() {
       @Override
       public long ramBytesUsed() {
@@ -385,7 +385,7 @@ public class TestFastLRUCache extends SolrTestCase {
     });
     assertEquals(4, cache.size());
 
-    cache.setResourceLimit(SolrCache.SIZE_PARAM, 10);
+    cache.setMaxSize(10);
     for (int i = 0; i < 6; i++) {
       cache.put("new" + i, new Accountable() {
         @Override
diff --git a/solr/core/src/test/org/apache/solr/search/TestLFUCache.java b/solr/core/src/test/org/apache/solr/search/TestLFUCache.java
index 324d7f4..beac366 100644
--- a/solr/core/src/test/org/apache/solr/search/TestLFUCache.java
+++ b/solr/core/src/test/org/apache/solr/search/TestLFUCache.java
@@ -472,7 +472,7 @@ public class TestLFUCache extends SolrTestCaseJ4 {
     // no evictions yet
     assertEquals(6, cache.size());
     // this sets minSize = 4, evictions will target minSize
-    cache.setResourceLimit(SolrCache.SIZE_PARAM, 5);
+    cache.setMaxSize(5);
     // should not happen yet - evictions are triggered by put
     assertEquals(6, cache.size());
     cache.put("6", "foo 6");
@@ -487,7 +487,7 @@ public class TestLFUCache extends SolrTestCaseJ4 {
 
     // scale up
 
-    cache.setResourceLimit(SolrCache.SIZE_PARAM, 10);
+    cache.setMaxSize(10);
     for (int i = 0; i < 6; i++) {
       cache.put("new" + i, "bar " + i);
     }
diff --git a/solr/core/src/test/org/apache/solr/search/TestLRUCache.java b/solr/core/src/test/org/apache/solr/search/TestLRUCache.java
index 3c33e27..f9cd0db 100644
--- a/solr/core/src/test/org/apache/solr/search/TestLRUCache.java
+++ b/solr/core/src/test/org/apache/solr/search/TestLRUCache.java
@@ -206,7 +206,7 @@ public class TestLRUCache extends SolrTestCase {
     }
     // no evictions yet
     assertEquals(6, cache.size());
-    cache.setResourceLimit(SolrCache.SIZE_PARAM, 5);
+    cache.setMaxSize(5);
     // should not happen yet - evictions are triggered by put
     assertEquals(6, cache.size());
     cache.put("6", new Accountable() {
@@ -219,7 +219,7 @@ public class TestLRUCache extends SolrTestCase {
     assertEquals(5, cache.size());
 
     // modify ram limit
-    cache.setResourceLimit(SolrCache.MAX_RAM_MB_PARAM, 3);
+    cache.setMaxRamMB(3);
     // should not happen yet - evictions are triggered by put
     assertEquals(5, cache.size());
     cache.put("7", new Accountable() {
@@ -235,7 +235,7 @@ public class TestLRUCache extends SolrTestCase {
 
     // scale up
 
-    cache.setResourceLimit(SolrCache.MAX_RAM_MB_PARAM, 4);
+    cache.setMaxRamMB(4);
     cache.put("8", new Accountable() {
       @Override
       public long ramBytesUsed() {
@@ -244,7 +244,7 @@ public class TestLRUCache extends SolrTestCase {
     });
     assertEquals(4, cache.size());
 
-    cache.setResourceLimit(SolrCache.SIZE_PARAM, 10);
+    cache.setMaxSize(10);
     for (int i = 0; i < 6; i++) {
       cache.put("new" + i, new Accountable() {
         @Override