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:38:40 UTC

[lucene-solr] 02/36: Refactoring. Plugins for management of speficic types of resources.

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 51da363d37807666f593472220cf75fc1ecdd089
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Wed Jun 19 13:02:36 2019 +0200

    Refactoring. Plugins for management of speficic types of resources.
---
 .../managed/AbstractResourceManagerPlugin.java     |   7 ++
 ...rceManager.java => DefaultResourceManager.java} | 132 ++++-----------------
 .../DefaultResourceManagerPluginFactory.java       |  36 ++++++
 .../src/java/org/apache/solr/managed/Limit.java    |  35 ------
 .../src/java/org/apache/solr/managed/Limits.java   |  75 ------------
 .../org/apache/solr/managed/ManagedResource.java   |  63 ++++++++++
 .../solr/managed/ProportionalResourceManager.java  |  48 --------
 .../org/apache/solr/managed/ResourceManaged.java   |  23 ----
 .../org/apache/solr/managed/ResourceManager.java   |  10 +-
 .../apache/solr/managed/ResourceManagerPlugin.java |  20 ++++
 .../solr/managed/ResourceManagerPluginFactory.java |  11 ++
 .../apache/solr/managed/ResourceManagerPool.java   |  95 +++++++++++++++
 .../solr/managed/plugins/CacheManagerPlugin.java   |  95 +++++++++++++++
 .../src/java/org/apache/solr/search/LRUCache.java  |   8 +-
 14 files changed, 356 insertions(+), 302 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/managed/AbstractResourceManagerPlugin.java b/solr/core/src/java/org/apache/solr/managed/AbstractResourceManagerPlugin.java
new file mode 100644
index 0000000..f75d104
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/managed/AbstractResourceManagerPlugin.java
@@ -0,0 +1,7 @@
+package org.apache.solr.managed;
+
+/**
+ *
+ */
+public abstract class AbstractResourceManagerPlugin implements ResourceManagerPlugin {
+}
diff --git a/solr/core/src/java/org/apache/solr/managed/AbstractResourceManager.java b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManager.java
similarity index 52%
rename from solr/core/src/java/org/apache/solr/managed/AbstractResourceManager.java
rename to solr/core/src/java/org/apache/solr/managed/DefaultResourceManager.java
index 41cb185..c509e2b 100644
--- a/solr/core/src/java/org/apache/solr/managed/AbstractResourceManager.java
+++ b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManager.java
@@ -1,15 +1,11 @@
 package org.apache.solr.managed;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
@@ -17,6 +13,7 @@ import org.apache.solr.common.util.ExecutorUtil;
 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.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.SolrPluginUtils;
 import org.slf4j.Logger;
@@ -25,7 +22,7 @@ import org.slf4j.LoggerFactory;
 /**
  *
  */
-public abstract class AbstractResourceManager implements ResourceManager {
+public class DefaultResourceManager implements ResourceManager {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public static final String SCHEDULE_DELAY_SECONDS_PARAM = "scheduleDelaySeconds";
@@ -33,99 +30,8 @@ public abstract class AbstractResourceManager implements ResourceManager {
 
   public static final int DEFAULT_MAX_POOLS = 20;
 
-  public static class Pool implements Runnable, Closeable {
-    private final AbstractResourceManager resourceManager;
-    private final Map<String, ResourceManaged> resources = new ConcurrentHashMap<>();
-    private Limits limits;
-    private final Map<String, Object> params;
-    private final Map<String, Float> totalCosts = new ConcurrentHashMap<>();
-    private Map<String, Map<String, Float>> currentValues = null;
-    private Map<String, Float> totalValues = null;
-    int scheduleDelaySeconds;
-    ScheduledFuture<?> scheduledFuture;
 
-    public Pool(AbstractResourceManager resourceManager, Limits limits, Map<String, Object> params) {
-      this.resourceManager = resourceManager;
-      this.limits = limits.copy();
-      this.params = new HashMap<>(params);
-    }
-
-    public synchronized void addResource(ResourceManaged resourceManaged) {
-      if (resources.containsKey(resourceManaged.getName())) {
-        throw new IllegalArgumentException("Pool already has resource '" + resourceManaged.getName() + "'.");
-      }
-      resources.put(resourceManaged.getName(), resourceManaged);
-      Limits managedLimits = resourceManaged.getManagedLimits();
-      managedLimits.forEach(entry -> {
-        Float total = totalCosts.get(entry.getKey());
-        if (total == null) {
-          totalCosts.put(entry.getKey(), entry.getValue().cost);
-        } else {
-          totalCosts.put(entry.getKey(), entry.getValue().cost + total);
-        }
-      });
-    }
-
-    public Map<String, ResourceManaged> getResources() {
-      return Collections.unmodifiableMap(resources);
-    }
-
-    public Map<String, Map<String, Float>> getCurrentValues() {
-      // collect current values
-      currentValues = new HashMap<>();
-      for (ResourceManaged resource : resources.values()) {
-        currentValues.put(resource.getName(), resource.getManagedValues());
-      }
-      // calculate totals
-      totalValues = new HashMap<>();
-      currentValues.values().forEach(map -> map.forEach((k, v) -> {
-        Float total = totalValues.get(k);
-        if (total == null) {
-          totalValues.put(k, v);
-        } else {
-          totalValues.put(k, total + v);
-        }
-      }));
-      return Collections.unmodifiableMap(currentValues);
-    }
-
-    /**
-     * This returns cumulative values of all resources. NOTE:
-     * you must call {@link #getCurrentValues()} first!
-     * @return
-     */
-    public Map<String, Float> getTotalValues() {
-      return Collections.unmodifiableMap(totalValues);
-    }
-
-    public Map<String, Float> getTotalCosts() {
-      return Collections.unmodifiableMap(totalCosts);
-    }
-
-    public Limits getLimits() {
-      return limits;
-    }
-
-    public void setLimits(Limits limits) {
-      this.limits = limits.copy();
-    }
-
-    @Override
-    public void run() {
-      resourceManager.managePool(this);
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (scheduledFuture != null) {
-        scheduledFuture.cancel(true);
-        scheduledFuture = null;
-      }
-    }
-  }
-
-
-  private Map<String, Pool> resourcePools = new ConcurrentHashMap<>();
+  private Map<String, ResourceManagerPool> resourcePools = new ConcurrentHashMap<>();
   private PluginInfo pluginInfo;
   private int maxNumPools = DEFAULT_MAX_POOLS;
   private TimeSource timeSource;
@@ -138,7 +44,11 @@ public abstract class AbstractResourceManager implements ResourceManager {
   protected boolean isClosed = false;
   protected boolean enabled = true;
 
-  public AbstractResourceManager(TimeSource timeSource) {
+  protected ResourceManagerPluginFactory resourceManagerPluginFactory;
+  protected SolrResourceLoader loader;
+
+  public DefaultResourceManager(SolrResourceLoader loader, TimeSource timeSource) {
+    this.loader = loader;
     this.timeSource = timeSource;
   }
 
@@ -158,6 +68,8 @@ public abstract class AbstractResourceManager implements ResourceManager {
         new DefaultSolrThreadFactory(getClass().getSimpleName()));
     scheduledThreadPoolExecutor.setRemoveOnCancelPolicy(true);
     scheduledThreadPoolExecutor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
+    // TODO: make configurable
+    resourceManagerPluginFactory = new DefaultResourceManagerPluginFactory(loader);
   }
 
   public void setMaxNumPools(Integer maxNumPools) {
@@ -180,10 +92,8 @@ public abstract class AbstractResourceManager implements ResourceManager {
     return pluginInfo;
   }
 
-  protected abstract void managePool(Pool pool);
-
   @Override
-  public void createPool(String name, Limits limits, Map<String, Object> params) throws Exception {
+  public void createPool(String name, String type, Map<String, Float> limits, Map<String, Object> params) throws Exception {
     ensureNotClosed();
     if (resourcePools.containsKey(name)) {
       throw new IllegalArgumentException("Pool '" + name + "' already exists.");
@@ -191,7 +101,7 @@ public abstract class AbstractResourceManager implements ResourceManager {
     if (resourcePools.size() >= maxNumPools) {
       throw new IllegalArgumentException("Maximum number of pools (" + maxNumPools + ") reached.");
     }
-    Pool newPool = new Pool(this, limits, params);
+    ResourceManagerPool newPool = new ResourceManagerPool(resourceManagerPluginFactory, type, limits, params);
     newPool.scheduleDelaySeconds = Integer.parseInt(String.valueOf(params.getOrDefault(SCHEDULE_DELAY_SECONDS_PARAM, 10)));
     resourcePools.putIfAbsent(name, newPool);
     newPool.scheduledFuture = scheduledThreadPoolExecutor.scheduleWithFixedDelay(newPool, 0,
@@ -200,9 +110,9 @@ public abstract class AbstractResourceManager implements ResourceManager {
   }
 
   @Override
-  public void modifyPoolLimits(String name, Limits limits) throws Exception {
+  public void modifyPoolLimits(String name, Map<String, Float> limits) throws Exception {
     ensureNotClosed();
-    Pool pool = resourcePools.get(name);
+    ResourceManagerPool pool = resourcePools.get(name);
     if (pool == null) {
       throw new IllegalArgumentException("Pool '" + name + "' doesn't exist.");
     }
@@ -212,7 +122,7 @@ public abstract class AbstractResourceManager implements ResourceManager {
   @Override
   public void removePool(String name) throws Exception {
     ensureNotClosed();
-    Pool pool = resourcePools.remove(name);
+    ResourceManagerPool pool = resourcePools.remove(name);
     if (pool == null) {
       throw new IllegalArgumentException("Pool '" + name + "' doesn't exist.");
     }
@@ -222,21 +132,21 @@ public abstract class AbstractResourceManager implements ResourceManager {
   }
 
   @Override
-  public void addResources(String name, Collection<ResourceManaged> resourceManaged) {
+  public void addResources(String name, Collection<ManagedResource> managedResource) {
     ensureNotClosed();
-    for (ResourceManaged resource : resourceManaged) {
+    for (ManagedResource resource : managedResource) {
       addResource(name, resource);
     }
   }
 
   @Override
-  public void addResource(String name, ResourceManaged resourceManaged) {
+  public void addResource(String name, ManagedResource managedResource) {
     ensureNotClosed();
-    Pool pool = resourcePools.get(name);
+    ResourceManagerPool pool = resourcePools.get(name);
     if (pool == null) {
       throw new IllegalArgumentException("Pool '" + name + "' doesn't exist.");
     }
-    pool.addResource(resourceManaged);
+    pool.addResource(managedResource);
   }
 
   @Override
@@ -244,7 +154,7 @@ public abstract class AbstractResourceManager implements ResourceManager {
     synchronized (this) {
       isClosed = true;
       log.debug("Closing all pools.");
-      for (Pool pool : resourcePools.values()) {
+      for (ResourceManagerPool pool : resourcePools.values()) {
         IOUtils.closeQuietly(pool);
       }
       resourcePools.clear();
diff --git a/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPluginFactory.java b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPluginFactory.java
new file mode 100644
index 0000000..3967c87
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/managed/DefaultResourceManagerPluginFactory.java
@@ -0,0 +1,36 @@
+package org.apache.solr.managed;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.managed.plugins.CacheManagerPlugin;
+
+/**
+ *
+ */
+public class DefaultResourceManagerPluginFactory implements ResourceManagerPluginFactory {
+
+  private static final Map<String, String> typeToClass = new HashMap<>();
+
+  static {
+    typeToClass.put(CacheManagerPlugin.TYPE, CacheManagerPlugin.class.getName());
+  }
+
+  private final SolrResourceLoader loader;
+
+  public DefaultResourceManagerPluginFactory(SolrResourceLoader loader) {
+    this.loader = loader;
+  }
+
+  @Override
+  public ResourceManagerPlugin create(String type, Map<String, Object> params) throws Exception {
+    String pluginClazz = typeToClass.get(type);
+    if (pluginClazz == null) {
+      throw new IllegalArgumentException("Unsupported plugin type '" + type + "'");
+    }
+    ResourceManagerPlugin resourceManagerPlugin = loader.newInstance(pluginClazz, ResourceManagerPlugin.class);
+    resourceManagerPlugin.init(params);
+    return resourceManagerPlugin;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/managed/Limit.java b/solr/core/src/java/org/apache/solr/managed/Limit.java
deleted file mode 100644
index 0eee67d..0000000
--- a/solr/core/src/java/org/apache/solr/managed/Limit.java
+++ /dev/null
@@ -1,35 +0,0 @@
-package org.apache.solr.managed;
-
-/**
- *
- */
-public class Limit {
-  public final float min, max, cost;
-
-  public Limit() {
-    this(Float.MIN_VALUE, Float.MAX_VALUE, 1.0f);
-  }
-
-  public Limit(float min, float max) {
-    this(min, max, 1.0f);
-  }
-
-  public Limit(float min, float max, float cost) {
-    if (cost <= 0.0f) {
-      throw new IllegalArgumentException("cost must be > 0.0f");
-    }
-    this.min = min;
-    this.max = max;
-    this.cost = cost;
-  }
-
-  public float deltaOutsideLimit(float currentValue) {
-    if (currentValue < min) {
-      return currentValue - min;
-    } else if (currentValue > max) {
-      return currentValue - max;
-    } else {
-      return 0;
-    }
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/managed/Limits.java b/solr/core/src/java/org/apache/solr/managed/Limits.java
deleted file mode 100644
index d817be9..0000000
--- a/solr/core/src/java/org/apache/solr/managed/Limits.java
+++ /dev/null
@@ -1,75 +0,0 @@
-package org.apache.solr.managed;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-/**
- *
- */
-public class Limits implements Iterable<Map.Entry<String, Limit>> {
-
-  public static final Limit UNLIMITED = new Limit(Float.MIN_VALUE, Float.MAX_VALUE);
-
-  private Map<String, Limit> values = new HashMap<>();
-
-  public void setLimit(String key, Limit value) {
-    if (value != null) {
-      values.put(key, value);
-    } else {
-      values.remove(key);
-    }
-  }
-
-  public void setLimitMax(String key, float max) {
-    Limit limit = values.computeIfAbsent(key, k -> new Limit(Float.MIN_VALUE, max));
-    if (limit.max == max) {
-      return;
-    } else {
-      values.put(key, new Limit(limit.min, max));
-    }
-  }
-
-  public void setLimitMin(String key, float min) {
-    Limit limit = values.computeIfAbsent(key, k -> new Limit(min, Float.MAX_VALUE));
-    if (limit.min == min) {
-      return;
-    } else {
-      values.put(key, new Limit(min, limit.max));
-    }
-  }
-
-  public Limit getLimit(String key) {
-    return getLimit(key, UNLIMITED);
-  }
-
-  public Limit getLimit(String key, Limit defValue) {
-    Limit value = values.get(key);
-    if (value != null) {
-      return value;
-    } else {
-      return defValue;
-    }
-  }
-
-  public Set<String> getKeys() {
-    return Collections.unmodifiableSet(values.keySet());
-  }
-
-  public void removeLimit(String key) {
-    values.remove(key);
-  }
-
-  public Limits copy() {
-    Limits cloned = new Limits();
-    cloned.values.putAll(values);
-    return cloned;
-  }
-
-  @Override
-  public Iterator<Map.Entry<String, Limit>> iterator() {
-    return Collections.unmodifiableMap(values).entrySet().iterator();
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/managed/ManagedResource.java b/solr/core/src/java/org/apache/solr/managed/ManagedResource.java
new file mode 100644
index 0000000..04d5207
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/managed/ManagedResource.java
@@ -0,0 +1,63 @@
+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;
+
+/**
+ *
+ */
+public interface ManagedResource {
+  Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * Unique name of this resource.
+   */
+  String getName();
+
+  /**
+   * Returns types of management schemes supported by this resource. This always
+   * returns a non-null collection with at least one entry.
+   */
+  Collection<String> getManagedResourceTypes();
+
+  /**
+   * Set current managed limits.
+   * @param limits map of limit names and values
+   */
+  default void setManagedLimits(Map<String, Float> limits) {
+    if (limits == null) {
+      return;
+    }
+    limits.forEach((key, value) -> {
+      try {
+        setManagedLimit(key, value);
+      } catch (Exception e) {
+        log.warn("Exception setting managed limit on {}: key={}, value={}, exception={}",
+            getName(), key, value, e);
+      }
+    });
+  }
+
+  /**
+   * Set a managed limit.
+   * @param key limit name
+   * @param value limit value
+   */
+  void setManagedLimit(String key, float value) throws Exception;
+
+  /**
+   * Returns current managed limits.
+   */
+  Map<String, Float> getManagedLimits();
+
+  /**
+   * Returns monitored values that are used for calculating optimal setting of managed limits.
+   * @param tags value names
+   * @return map of names to current values.
+   */
+  Map<String, Float> getManagedValues(Collection<String> tags);
+}
diff --git a/solr/core/src/java/org/apache/solr/managed/ProportionalResourceManager.java b/solr/core/src/java/org/apache/solr/managed/ProportionalResourceManager.java
deleted file mode 100644
index f2edf65..0000000
--- a/solr/core/src/java/org/apache/solr/managed/ProportionalResourceManager.java
+++ /dev/null
@@ -1,48 +0,0 @@
-package org.apache.solr.managed;
-
-import java.util.Map;
-
-import org.apache.solr.common.util.TimeSource;
-
-/**
- *
- */
-public class ProportionalResourceManager extends AbstractResourceManager {
-  public ProportionalResourceManager(TimeSource timeSource) {
-    super(timeSource);
-  }
-
-  @Override
-  protected void managePool(Pool pool) {
-    Map<String, Map<String, Float>> currentValues = pool.getCurrentValues();
-    Map<String, Float> totalValues = pool.getTotalValues();
-    Map<String, Float> totalCosts = pool.getTotalCosts();
-    pool.getLimits().forEach(entry -> {
-      Limit poolLimit = entry.getValue();
-      Float totalValue = totalValues.get(entry.getKey());
-      if (totalValue == null) {
-        return;
-      }
-      float delta = poolLimit.deltaOutsideLimit(totalValue);
-      Float totalCost = totalCosts.get(entry.getKey());
-      if (totalCost == null || totalCost == 0) {
-        return;
-      }
-      // re-adjust the limits based on relative costs
-      pool.getResources().forEach((name, resource) -> {
-        Map<String, Float> current = currentValues.get(name);
-        if (current == null) {
-          return;
-        }
-        Limits limits = resource.getManagedLimits();
-        Limit limit = limits.getLimit(entry.getKey());
-        if (limit == null) {
-          return;
-        }
-        float newMax = limit.max - delta * limit.cost / totalCost;
-        resource.setManagedLimitMax(entry.getKey(), newMax);
-      });
-    });
-  }
-
-}
diff --git a/solr/core/src/java/org/apache/solr/managed/ResourceManaged.java b/solr/core/src/java/org/apache/solr/managed/ResourceManaged.java
deleted file mode 100644
index 889bc85..0000000
--- a/solr/core/src/java/org/apache/solr/managed/ResourceManaged.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package org.apache.solr.managed;
-
-import java.util.Map;
-
-/**
- *
- */
-public interface ResourceManaged {
-
-  String getName();
-
-  void setManagedLimits(Limits limits);
-
-  void setManagedLimit(String key, Limit limit);
-
-  void setManagedLimitMax(String key, float max);
-
-  void setManagedLimitMin(String key, float min);
-
-  Limits getManagedLimits();
-
-  Map<String, Float> getManagedValues();
-}
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 cd9e87d..a2e3fe3 100644
--- a/solr/core/src/java/org/apache/solr/managed/ResourceManager.java
+++ b/solr/core/src/java/org/apache/solr/managed/ResourceManager.java
@@ -16,20 +16,20 @@ public interface ResourceManager extends SolrCloseable, PluginInfoInitialized {
 
   PluginInfo getPluginInfo();
 
-  void createPool(String name, Limits limits, Map<String, Object> params) throws Exception;
+  void createPool(String name, String type, Map<String, Float> limits, Map<String, Object> params) throws Exception;
 
-  void modifyPoolLimits(String name, Limits limits) throws Exception;
+  void modifyPoolLimits(String name, Map<String, Float> limits) throws Exception;
 
   void removePool(String name) throws Exception;
 
-  default void addResources(String pool, Collection<ResourceManaged> resourceManaged) {
+  default void addResources(String pool, Collection<ManagedResource> managedResource) {
     ensureNotClosed();
-    for (ResourceManaged resource : resourceManaged) {
+    for (ManagedResource resource : managedResource) {
       addResource(pool, resource);
     }
   }
 
-  void addResource(String pool, ResourceManaged resourceManaged);
+  void addResource(String pool, ManagedResource managedResource);
 
   default void ensureNotClosed() {
     if (isClosed()) {
diff --git a/solr/core/src/java/org/apache/solr/managed/ResourceManagerPlugin.java b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPlugin.java
new file mode 100644
index 0000000..a73cdcb
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPlugin.java
@@ -0,0 +1,20 @@
+package org.apache.solr.managed;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ *
+ */
+public interface ResourceManagerPlugin {
+
+  String getType();
+
+  void init(Map<String, Object> params);
+
+  Collection<String> getMonitoredTags();
+  Collection<String> getControlledTags();
+
+  void manage(ResourceManagerPool pool);
+
+}
diff --git a/solr/core/src/java/org/apache/solr/managed/ResourceManagerPluginFactory.java b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPluginFactory.java
new file mode 100644
index 0000000..b4e1a54
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPluginFactory.java
@@ -0,0 +1,11 @@
+package org.apache.solr.managed;
+
+import java.util.Map;
+
+/**
+ *
+ */
+public interface ResourceManagerPluginFactory {
+
+  ResourceManagerPlugin create(String type, Map<String, Object> params) throws Exception;
+}
diff --git a/solr/core/src/java/org/apache/solr/managed/ResourceManagerPool.java b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPool.java
new file mode 100644
index 0000000..88b4c06
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/managed/ResourceManagerPool.java
@@ -0,0 +1,95 @@
+package org.apache.solr.managed;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledFuture;
+
+/**
+ *
+ */
+public class ResourceManagerPool implements Runnable, Closeable {
+  private final Map<String, ManagedResource> resources = new ConcurrentHashMap<>();
+  private Map<String, Float> limits;
+  private final String type;
+  private final ResourceManagerPlugin resourceManagerPlugin;
+  private final Map<String, Object> params;
+  private Map<String, Map<String, Float>> currentValues = null;
+  private Map<String, Float> totalValues = null;
+  int scheduleDelaySeconds;
+  ScheduledFuture<?> scheduledFuture;
+
+  public ResourceManagerPool(ResourceManagerPluginFactory factory, String type, Map<String, Float> limits, Map<String, Object> params) throws Exception {
+    this.type = type;
+    this.resourceManagerPlugin = factory.create(type, params);
+    this.limits = new HashMap<>(limits);
+    this.params = new HashMap<>(params);
+  }
+
+  public synchronized void addResource(ManagedResource managedResource) {
+    if (resources.containsKey(managedResource.getName())) {
+      throw new IllegalArgumentException("Pool already has resource '" + managedResource.getName() + "'.");
+    }
+    Collection<String> types = managedResource.getManagedResourceTypes();
+    if (!types.contains(type)) {
+      throw new IllegalArgumentException("Pool type '" + type + "' is not supported by the resource " + managedResource.getName());
+    }
+    resources.put(managedResource.getName(), managedResource);
+  }
+
+  public Map<String, ManagedResource> getResources() {
+    return Collections.unmodifiableMap(resources);
+  }
+
+  public Map<String, Map<String, Float>> getCurrentValues() {
+    // collect current values
+    currentValues = new HashMap<>();
+    for (ManagedResource resource : resources.values()) {
+      currentValues.put(resource.getName(), resource.getManagedValues(resourceManagerPlugin.getMonitoredTags()));
+    }
+    // calculate totals
+    totalValues = new HashMap<>();
+    currentValues.values().forEach(map -> map.forEach((k, v) -> {
+      Float total = totalValues.get(k);
+      if (total == null) {
+        totalValues.put(k, v);
+      } else {
+        totalValues.put(k, total + v);
+      }
+    }));
+    return Collections.unmodifiableMap(currentValues);
+  }
+
+  /**
+   * This returns cumulative values of all resources. NOTE:
+   * you must call {@link #getCurrentValues()} first!
+   */
+  public Map<String, Float> getTotalValues() {
+    return Collections.unmodifiableMap(totalValues);
+  }
+
+  public Map<String, Float> getLimits() {
+    return limits;
+  }
+
+  public void setLimits(Map<String, Float> limits) {
+    this.limits = new HashMap(limits);
+  }
+
+  @Override
+  public void run() {
+    resourceManagerPlugin.manage(this);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (scheduledFuture != null) {
+      scheduledFuture.cancel(true);
+      scheduledFuture = null;
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/managed/plugins/CacheManagerPlugin.java b/solr/core/src/java/org/apache/solr/managed/plugins/CacheManagerPlugin.java
new file mode 100644
index 0000000..7f2f5cd
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/managed/plugins/CacheManagerPlugin.java
@@ -0,0 +1,95 @@
+package org.apache.solr.managed.plugins;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.managed.AbstractResourceManagerPlugin;
+import org.apache.solr.managed.ResourceManagerPool;
+
+/**
+ *
+ */
+public class CacheManagerPlugin extends AbstractResourceManagerPlugin {
+  public static String TYPE = "cache";
+
+  public static final String SIZE_TAG = "size";
+  public static final String HIT_RATIO_TAG = "hitratio";
+  public static final String RAM_BYTES_USED_TAG = "ramBytesUsed";
+  public static final String MAX_RAM_MB_TAG = "maxRamMB";
+
+  private static final Map<String, String> controlledToMonitored = new HashMap<>();
+
+  static {
+    controlledToMonitored.put(MAX_RAM_MB_TAG, RAM_BYTES_USED_TAG);
+    controlledToMonitored.put(SIZE_TAG, SIZE_TAG);
+  }
+
+  private static final Collection<String> MONITORED_TAGS = Arrays.asList(
+      SIZE_TAG,
+      HIT_RATIO_TAG,
+      RAM_BYTES_USED_TAG
+  );
+
+  @Override
+  public Collection<String> getMonitoredTags() {
+    return MONITORED_TAGS;
+  }
+
+  private static final Collection<String> CONTROLLED_TAGS = Arrays.asList(
+      MAX_RAM_MB_TAG,
+      SIZE_TAG
+  );
+
+  @Override
+  public Collection<String> getControlledTags() {
+    return CONTROLLED_TAGS;
+  }
+
+  @Override
+  public String getType() {
+    return TYPE;
+  }
+
+  @Override
+  public void init(Map<String, Object> params) {
+
+  }
+
+  @Override
+  public void manage(ResourceManagerPool pool) {
+    Map<String, Map<String, Float>> currentValues = pool.getCurrentValues();
+    Map<String, Float> totalValues = pool.getTotalValues();
+    pool.getLimits().forEach((poolLimitName, poolLimitValue) -> {
+      String monitoredTag = controlledToMonitored.get(poolLimitName);
+      if (monitoredTag == null) {
+        return;
+      }
+      Float totalValue = totalValues.get(monitoredTag);
+      if (totalValue == null) {
+        return;
+      }
+      float totalDelta = poolLimitValue - totalValue;
+      pool.getResources().forEach((name, resource) -> {
+        Map<String, Float> current = currentValues.get(name);
+        if (current == null) {
+          return;
+        }
+        Map<String, Float> limits = resource.getManagedLimits();
+        Float managedSize = limits.get(SIZE_TAG);
+        Float resMaxRamMB = limits.get(MAX_RAM_MB_TAG);
+        Float currentSize = current.get(SIZE_TAG);
+        Float currentHitratio = current.get(HIT_RATIO_TAG);
+        Float ramBytesUsed = current.get(RAM_BYTES_USED_TAG);
+
+        // logic to adjust per-resource controlled limits
+        if (poolLimitName.equals(MAX_RAM_MB_TAG)) {
+          // adjust per-resource maxRamMB
+        } else if (poolLimitName.equals(SIZE_TAG)) {
+          // adjust per-resource size
+        }
+      });
+    });
+  }
+}
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 8f53b45..0e0ff6c 100644
--- a/solr/core/src/java/org/apache/solr/search/LRUCache.java
+++ b/solr/core/src/java/org/apache/solr/search/LRUCache.java
@@ -345,11 +345,9 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
         res.put("inserts", inserts);
         res.put("evictions", evictions);
         res.put("size", map.size());
-        if (maxRamBytes != Long.MAX_VALUE)  {
-          res.put("maxRamMB", maxRamBytes / 1024L / 1024L);
-          res.put("ramBytesUsed", ramBytesUsed());
-          res.put("evictionsRamUsage", evictionsRamUsage);
-        }
+        res.put("maxRamMB", maxRamBytes / 1024L / 1024L);
+        res.put("ramBytesUsed", ramBytesUsed());
+        res.put("evictionsRamUsage", evictionsRamUsage);
       }
       res.put("warmupTime", warmupTime);