You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ma...@apache.org on 2023/08/30 20:35:34 UTC

[solr] branch branch_9x updated: SOLR-16654: Add support for node-level caches (#1351)

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

magibney pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 58c8dfc94a9 SOLR-16654: Add support for node-level caches (#1351)
58c8dfc94a9 is described below

commit 58c8dfc94a974664d6a80a3410a5ebaef6f1201b
Author: Michael Gibney <mi...@michaelgibney.net>
AuthorDate: Wed Aug 30 16:08:03 2023 -0400

    SOLR-16654: Add support for node-level caches (#1351)
    
    (cherry picked from commit 22ac9105ea0382215c8fe558fc55977dc98aa14c)
---
 solr/CHANGES.txt                                   |   2 +-
 .../java/org/apache/solr/core/CoreContainer.java   |  35 ++
 .../src/java/org/apache/solr/core/NodeConfig.java  |  16 +
 .../src/java/org/apache/solr/core/SolrConfig.java  |   3 +-
 .../src/java/org/apache/solr/core/SolrCore.java    |   1 +
 .../java/org/apache/solr/core/SolrXmlConfig.java   |  16 +
 .../java/org/apache/solr/search/CacheConfig.java   |  14 +-
 .../java/org/apache/solr/search/CaffeineCache.java |   6 +
 .../src/java/org/apache/solr/search/SolrCache.java |  10 +
 .../org/apache/solr/search/SolrIndexSearcher.java  |  12 +
 .../conf/solrconfig-nodelevelcaches.xml            |  34 ++
 .../src/test-files/solr/solr-nodelevelcaches.xml   |  30 ++
 .../src/test/org/apache/solr/core/TestSolrXml.java |  16 +
 .../test/org/apache/solr/search/TestThinCache.java | 160 +++++++
 .../src/test/org/apache/solr/search/ThinCache.java | 472 +++++++++++++++++++++
 .../pages/configuring-solr-xml.adoc                |  20 +
 .../src/java/org/apache/solr/util/TestHarness.java |   4 +
 17 files changed, 846 insertions(+), 5 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b6926905aef..f81bfc4aeac 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -8,7 +8,7 @@ https://github.com/apache/solr/blob/main/solr/solr-ref-guide/modules/upgrade-not
 ==================  9.4.0 ==================
 New Features
 ---------------------
-(No changes)
+* SOLR-16654: Add support for node-level caches (Michael Gibney)
 
 Improvements
 ---------------------
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 f223f41e36f..2e5c788e81f 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -41,6 +41,7 @@ import java.nio.file.Paths;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
@@ -96,6 +97,7 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.State;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.CollectionUtil;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.ObjectCache;
@@ -138,6 +140,8 @@ import org.apache.solr.metrics.SolrMetricsContext;
 import org.apache.solr.pkg.SolrPackageLoader;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.search.CacheConfig;
+import org.apache.solr.search.SolrCache;
 import org.apache.solr.search.SolrFieldCacheBean;
 import org.apache.solr.security.AllowListUrlChecker;
 import org.apache.solr.security.AuditLoggerPlugin;
@@ -274,6 +278,8 @@ public class CoreContainer {
 
   private volatile SolrClientCache solrClientCache;
 
+  private volatile Map<String, SolrCache<?, ?>> caches;
+
   private final ObjectCache objectCache = new ObjectCache();
 
   public final NodeRoles nodeRoles = new NodeRoles(System.getProperty(NodeRoles.NODE_ROLES_PROP));
@@ -712,6 +718,10 @@ public class CoreContainer {
     return packageStoreAPI;
   }
 
+  public SolrCache<?, ?> getCache(String name) {
+    return caches.get(name);
+  }
+
   public SolrClientCache getSolrClientCache() {
     return solrClientCache;
   }
@@ -798,6 +808,20 @@ public class CoreContainer {
 
     solrClientCache = new SolrClientCache(updateShardHandler.getDefaultHttpClient());
 
+    Map<String, CacheConfig> cachesConfig = cfg.getCachesConfig();
+    if (cachesConfig.isEmpty()) {
+      this.caches = Collections.emptyMap();
+    } else {
+      Map<String, SolrCache<?, ?>> m = CollectionUtil.newHashMap(cachesConfig.size());
+      for (Map.Entry<String, CacheConfig> e : cachesConfig.entrySet()) {
+        SolrCache<?, ?> c = e.getValue().newInstance();
+        String cacheName = e.getKey();
+        c.initializeMetrics(solrMetricsContext, "nodeLevelCache/" + cacheName);
+        m.put(cacheName, c);
+      }
+      this.caches = Collections.unmodifiableMap(m);
+    }
+
     StartupLoggingUtils.checkRequestLogging();
 
     hostName = cfg.getNodeName();
@@ -1265,6 +1289,17 @@ public class CoreContainer {
       // Now clear all the cores that are being operated upon.
       solrCores.close();
 
+      final Map<String, SolrCache<?, ?>> closeCaches = caches;
+      if (closeCaches != null) {
+        for (Map.Entry<String, SolrCache<?, ?>> e : caches.entrySet()) {
+          try {
+            e.getValue().close();
+          } catch (Exception ex) {
+            log.warn("error closing node-level cache: {}", e.getKey(), ex);
+          }
+        }
+      }
+
       objectCache.clear();
 
       // It's still possible that one of the pending dynamic load operation is waiting, so wake it
diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
index 8117aad4523..581a2d07c0a 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -41,6 +41,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.logging.DeprecationLog;
 import org.apache.solr.logging.LogWatcherConfig;
+import org.apache.solr.search.CacheConfig;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.apache.solr.util.ModuleUtils;
@@ -109,6 +110,8 @@ public class NodeConfig {
 
   private final MetricsConfig metricsConfig;
 
+  private final Map<String, CacheConfig> cachesConfig;
+
   private final PluginInfo tracerConfig;
 
   // Track if this config was loaded from zookeeper so that we can skip validating the zookeeper
@@ -144,6 +147,7 @@ public class NodeConfig {
       Properties solrProperties,
       PluginInfo[] backupRepositoryPlugins,
       MetricsConfig metricsConfig,
+      Map<String, CacheConfig> cachesConfig,
       PluginInfo tracerConfig,
       boolean fromZookeeper,
       String defaultZkHost,
@@ -179,6 +183,7 @@ public class NodeConfig {
     this.solrProperties = solrProperties;
     this.backupRepositoryPlugins = backupRepositoryPlugins;
     this.metricsConfig = metricsConfig;
+    this.cachesConfig = cachesConfig == null ? Collections.emptyMap() : cachesConfig;
     this.tracerConfig = tracerConfig;
     this.fromZookeeper = fromZookeeper;
     this.defaultZkHost = defaultZkHost;
@@ -395,6 +400,10 @@ public class NodeConfig {
     return metricsConfig;
   }
 
+  public Map<String, CacheConfig> getCachesConfig() {
+    return cachesConfig;
+  }
+
   public PluginInfo getTracerConfiguratorPluginInfo() {
     return tracerConfig;
   }
@@ -600,6 +609,7 @@ public class NodeConfig {
     private Properties solrProperties = new Properties();
     private PluginInfo[] backupRepositoryPlugins;
     private MetricsConfig metricsConfig;
+    private Map<String, CacheConfig> cachesConfig;
     private PluginInfo tracerConfig;
     private boolean fromZookeeper = false;
     private String defaultZkHost;
@@ -769,6 +779,11 @@ public class NodeConfig {
       return this;
     }
 
+    public NodeConfigBuilder setCachesConfig(Map<String, CacheConfig> cachesConfig) {
+      this.cachesConfig = cachesConfig;
+      return this;
+    }
+
     public NodeConfigBuilder setTracerConfig(PluginInfo tracerConfig) {
       this.tracerConfig = tracerConfig;
       return this;
@@ -881,6 +896,7 @@ public class NodeConfig {
           solrProperties,
           backupRepositoryPlugins,
           metricsConfig,
+          cachesConfig,
           tracerConfig,
           fromZookeeper,
           defaultZkHost,
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index 8ad87ff98f0..563d817803e 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -346,7 +346,8 @@ public class SolrConfig implements MapSerializable {
       for (SolrPluginInfo plugin : plugins) loadPluginInfo(plugin);
 
       Map<String, CacheConfig> userCacheConfigs =
-          CacheConfig.getMultipleConfigs(this, "query/cache", get("query").getAll("cache"));
+          CacheConfig.getMultipleConfigs(
+              getResourceLoader(), this, "query/cache", get("query").getAll("cache"));
       List<PluginInfo> caches = getPluginInfos(SolrCache.class.getName());
       if (!caches.isEmpty()) {
         for (PluginInfo c : caches) {
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index b0b1dd2e784..b1c723aa7a5 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -2639,6 +2639,7 @@ public class SolrCore implements SolrInfoBean, Closeable {
           future =
               searcherExecutor.submit(
                   () -> {
+                    newSearcher.bootstrapFirstSearcher();
                     for (SolrEventListener listener : firstSearcherListeners) {
                       try {
                         listener.newSearcher(newSearcher, null);
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index 676026c82ed..0996341a79d 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -46,6 +46,7 @@ import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.logging.LogWatcherConfig;
 import org.apache.solr.metrics.reporters.SolrJmxReporter;
+import org.apache.solr.search.CacheConfig;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.apache.solr.util.DOMConfigNode;
@@ -177,6 +178,7 @@ public class SolrXmlConfig {
     // Remove this line in 10.0
     configBuilder.setHiddenSysProps(getHiddenSysProps(root.get("metrics")));
     configBuilder.setMetricsConfig(getMetricsConfig(root.get("metrics")));
+    configBuilder.setCachesConfig(getCachesConfig(loader, root.get("caches")));
     configBuilder.setFromZookeeper(fromZookeeper);
     configBuilder.setDefaultZkHost(defaultZkHost);
     configBuilder.setCoreAdminHandlerActions(coreAdminHandlerActions);
@@ -690,6 +692,20 @@ public class SolrXmlConfig {
     return builder.setMetricReporterPlugins(reporterPlugins).build();
   }
 
+  private static Map<String, CacheConfig> getCachesConfig(
+      SolrResourceLoader loader, ConfigNode caches) {
+    Map<String, CacheConfig> ret =
+        CacheConfig.getMultipleConfigs(loader, null, null, caches.getAll("cache"));
+    for (CacheConfig c : ret.values()) {
+      if (c.getRegenerator() != null) {
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "node-level caches should not be configured with a regenerator!");
+      }
+    }
+    return Collections.unmodifiableMap(ret);
+  }
+
   private static Object decodeNullValue(Object o) {
     if (o instanceof String) { // check if it's a JSON object
       String str = (String) o;
diff --git a/solr/core/src/java/org/apache/solr/search/CacheConfig.java b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
index e7e2063f031..aa4d61a2204 100644
--- a/solr/core/src/java/org/apache/solr/search/CacheConfig.java
+++ b/solr/core/src/java/org/apache/solr/search/CacheConfig.java
@@ -80,13 +80,13 @@ public class CacheConfig implements MapSerializable {
   }
 
   public static Map<String, CacheConfig> getMultipleConfigs(
-      SolrConfig solrConfig, String configPath, List<ConfigNode> nodes) {
+      SolrResourceLoader loader, SolrConfig solrConfig, String configPath, List<ConfigNode> nodes) {
     if (nodes == null || nodes.size() == 0) return new LinkedHashMap<>();
     Map<String, CacheConfig> result = CollectionUtil.newHashMap(nodes.size());
     for (ConfigNode node : nodes) {
       if (node.boolAttr("enabled", true)) {
         CacheConfig config =
-            getConfig(solrConfig, node.name(), node.attributes().asMap(), configPath);
+            getConfig(loader, solrConfig, node.name(), node.attributes().asMap(), configPath);
         result.put(config.args.get(NAME), config);
       }
     }
@@ -105,6 +105,15 @@ public class CacheConfig implements MapSerializable {
 
   public static CacheConfig getConfig(
       SolrConfig solrConfig, String nodeName, Map<String, String> attrs, String xpath) {
+    return getConfig(solrConfig.getResourceLoader(), solrConfig, nodeName, attrs, xpath);
+  }
+
+  public static CacheConfig getConfig(
+      SolrResourceLoader loader,
+      SolrConfig solrConfig,
+      String nodeName,
+      Map<String, String> attrs,
+      String xpath) {
     CacheConfig config = new CacheConfig();
     config.nodeName = nodeName;
     Map<String, String> attrsCopy = CollectionUtil.newLinkedHashMap(attrs.size());
@@ -128,7 +137,6 @@ public class CacheConfig implements MapSerializable {
       config.args.put(NAME, config.nodeName);
     }
 
-    SolrResourceLoader loader = solrConfig.getResourceLoader();
     config.cacheImpl = config.args.get("class");
     if (config.cacheImpl == null) config.cacheImpl = "solr.CaffeineCache";
     config.clazz =
diff --git a/solr/core/src/java/org/apache/solr/search/CaffeineCache.java b/solr/core/src/java/org/apache/solr/search/CaffeineCache.java
index 146a6c37326..1d921030ff7 100644
--- a/solr/core/src/java/org/apache/solr/search/CaffeineCache.java
+++ b/solr/core/src/java/org/apache/solr/search/CaffeineCache.java
@@ -379,6 +379,12 @@ public class CaffeineCache<K, V> extends SolrCacheBase
     }
   }
 
+  protected void adjustMetrics(long hitsAdjust, long insertsAdjust, long lookupsAdjust) {
+    hits.add(-hitsAdjust);
+    inserts.add(-insertsAdjust);
+    lookups.add(-lookupsAdjust);
+  }
+
   @Override
   public void warm(SolrIndexSearcher searcher, SolrCache<K, V> old) {
     if (regenerator == null) {
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 870d18ea2c9..5b09399f295 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrCache.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrCache.java
@@ -134,6 +134,16 @@ public interface SolrCache<K, V> extends SolrInfoBean {
    */
   public State getState();
 
+  /**
+   * A hook for caches that would like to perform some initialization for the first registered
+   * searcher. This method is analogous to {@link #warm(SolrIndexSearcher, SolrCache)}. The default
+   * implementation is a no-op. Implementers should not retain object references to the specified
+   * searcher.
+   */
+  default void initialSearcher(SolrIndexSearcher initialSearcher) {
+    // no-op
+  }
+
   /**
    * Warm this cache associated with <code>searcher</code> using the <code>old</code> cache object.
    * <code>this</code> and <code>old</code> will have the same concrete type.
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 70a5292ce75..d8a04645dbc 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -2428,6 +2428,18 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     return a.intersects(getDocSet(deState));
   }
 
+  /**
+   * Called on the initial searcher for each core, immediately before <code>firstSearcherListeners
+   * </code> are called for the searcher. This provides the opportunity to perform initialization on
+   * the first registered searcher before the searcher begins to see any <code>firstSearcher</code>
+   * -triggered events.
+   */
+  public void bootstrapFirstSearcher() {
+    for (SolrCache<?, ?> solrCache : cacheList) {
+      solrCache.initialSearcher(this);
+    }
+  }
+
   /** Warm this searcher based on an old one (primarily for auto-cache warming). */
   @SuppressWarnings({"unchecked"})
   public void warm(SolrIndexSearcher old) {
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-nodelevelcaches.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-nodelevelcaches.xml
new file mode 100644
index 00000000000..d6d33fe7bc6
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-nodelevelcaches.xml
@@ -0,0 +1,34 @@
+<?xml version="1.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.
+-->
+
+<config>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+  <dataDir>${solr.data.dir:}</dataDir>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+  <requestHandler name="/select" class="solr.SearchHandler" />
+  <query>
+    <filterCache
+      class="solr.ThinCache"
+      parentCacheName="myNodeLevelCacheThin"
+      size="5"
+      initialSize="5"/>
+  </query>
+</config>
diff --git a/solr/core/src/test-files/solr/solr-nodelevelcaches.xml b/solr/core/src/test-files/solr/solr-nodelevelcaches.xml
new file mode 100644
index 00000000000..cc99abb7a5a
--- /dev/null
+++ b/solr/core/src/test-files/solr/solr-nodelevelcaches.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<solr>
+  <caches>
+    <cache name="myNodeLevelCache"
+      size="10"
+      initialSize="10"
+      />
+    <cache name="myNodeLevelCacheThin"
+      class="solr.ThinCache$NodeLevelCache"
+      size="10"
+      initialSize="10"
+      />
+  </caches>
+</solr>
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrXml.java b/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
index 9cdc5b1a253..bbe4af69094 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
@@ -31,6 +31,10 @@ import org.apache.commons.exec.OS;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.search.CacheConfig;
+import org.apache.solr.search.CaffeineCache;
+import org.apache.solr.search.SolrCache;
+import org.apache.solr.search.ThinCache;
 import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.hamcrest.MatcherAssert;
 import org.junit.Before;
@@ -152,6 +156,18 @@ public class TestSolrXml extends SolrTestCaseJ4 {
     assertFalse("schema cache", cfg.hasSchemaCache());
   }
 
+  public void testNodeLevelCache() throws IOException {
+    Path testSrcRoot = TEST_PATH();
+    Files.copy(testSrcRoot.resolve("solr-nodelevelcaches.xml"), solrHome.resolve("solr.xml"));
+
+    NodeConfig cfg = SolrXmlConfig.fromSolrHome(solrHome, new Properties());
+    Map<String, CacheConfig> cachesConfig = cfg.getCachesConfig();
+    SolrCache<?, ?> nodeLevelCache = cachesConfig.get("myNodeLevelCache").newInstance();
+    assertTrue(nodeLevelCache instanceof CaffeineCache);
+    SolrCache<?, ?> nodeLevelCacheThin = cachesConfig.get("myNodeLevelCacheThin").newInstance();
+    assertTrue(nodeLevelCacheThin instanceof ThinCache.NodeLevelCache);
+  }
+
   public void testExplicitNullGivesDefaults() {
     System.setProperty("jetty.port", "8000");
     String solrXml =
diff --git a/solr/core/src/test/org/apache/solr/search/TestThinCache.java b/solr/core/src/test/org/apache/solr/search/TestThinCache.java
new file mode 100644
index 00000000000..95f795054ad
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/TestThinCache.java
@@ -0,0 +1,160 @@
+/*
+ * 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.search;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricsContext;
+import org.apache.solr.util.stats.MetricUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/** Test for {@link ThinCache}. */
+public class TestThinCache extends SolrTestCaseJ4 {
+
+  SolrMetricManager metricManager = new SolrMetricManager();
+  String registry = TestUtil.randomSimpleString(random(), 2, 10);
+  String scope = TestUtil.randomSimpleString(random(), 2, 10);
+
+  @Test
+  public void testSimple() {
+    Object cacheScope = new Object();
+    ThinCache.NodeLevelCache<Object, Integer, String> backing = new ThinCache.NodeLevelCache<>();
+    ThinCache<Object, Integer, String> lfuCache = new ThinCache<>();
+    lfuCache.setBacking(cacheScope, backing);
+    SolrMetricsContext solrMetricsContext = new SolrMetricsContext(metricManager, registry, "foo");
+    lfuCache.initializeMetrics(solrMetricsContext, scope + "-1");
+
+    Object cacheScope2 = new Object();
+    ThinCache<Object, Integer, String> newLFUCache = new ThinCache<>();
+    newLFUCache.setBacking(cacheScope2, backing);
+    newLFUCache.initializeMetrics(solrMetricsContext, scope + "-2");
+
+    Map<String, String> params = new HashMap<>();
+    params.put("size", "100");
+    params.put("initialSize", "10");
+
+    NoOpRegenerator regenerator = new NoOpRegenerator();
+    backing.init(params, null, null);
+    Object initObj =
+        lfuCache.init(Collections.singletonMap("autowarmCount", "25"), null, regenerator);
+    lfuCache.setState(SolrCache.State.LIVE);
+    for (int i = 0; i < 101; i++) {
+      lfuCache.put(i + 1, Integer.toString(i + 1));
+    }
+    assertEquals("15", lfuCache.get(15));
+    assertEquals("75", lfuCache.get(75));
+    assertNull(lfuCache.get(110));
+    Map<String, Object> nl = lfuCache.getMetricsMap().getValue();
+    assertEquals(3L, nl.get("lookups"));
+    assertEquals(2L, nl.get("hits"));
+    assertEquals(101L, nl.get("inserts"));
+
+    assertNull(lfuCache.get(1)); // first item put in should be the first out
+
+    // Test autowarming
+    newLFUCache.init(Collections.singletonMap("autowarmCount", "25"), initObj, regenerator);
+    newLFUCache.warm(null, lfuCache);
+    newLFUCache.setState(SolrCache.State.LIVE);
+
+    newLFUCache.put(103, "103");
+    assertEquals("15", newLFUCache.get(15));
+    assertEquals("75", newLFUCache.get(75));
+    assertNull(newLFUCache.get(50));
+    nl = newLFUCache.getMetricsMap().getValue();
+    assertEquals(3L, nl.get("lookups"));
+    assertEquals(2L, nl.get("hits"));
+    assertEquals(1L, nl.get("inserts"));
+    assertEquals(0L, nl.get("evictions"));
+
+    assertEquals(7L, nl.get("cumulative_lookups"));
+    assertEquals(4L, nl.get("cumulative_hits"));
+    assertEquals(102L, nl.get("cumulative_inserts"));
+  }
+
+  @Test
+  public void testInitCore() throws Exception {
+    for (int i = 0; i < 20; i++) {
+      assertU(adoc("id", Integer.toString(i)));
+    }
+    assertU(commit());
+    assertQ(req("q", "*:*", "fq", "id:0"));
+    assertQ(req("q", "*:*", "fq", "id:0"));
+    assertQ(req("q", "*:*", "fq", "id:1"));
+    Map<String, Object> nodeMetricsSnapshot =
+        MetricUtils.convertMetrics(
+            h.getCoreContainer().getMetricManager().registry("solr.node"),
+            List.of(
+                "CACHE.nodeLevelCache/myNodeLevelCacheThin",
+                "CACHE.nodeLevelCache/myNodeLevelCache"));
+    Map<String, Object> coreMetricsSnapshot =
+        MetricUtils.convertMetrics(
+            h.getCore().getCoreMetricManager().getRegistry(),
+            List.of("CACHE.searcher.filterCache"));
+
+    // check that metrics are accessible, and the core cache writes through to the node-level cache
+    Map<String, Number> assertions = Map.of("lookups", 3L, "hits", 1L, "inserts", 2L, "size", 2);
+    for (Map.Entry<String, Number> e : assertions.entrySet()) {
+      String key = e.getKey();
+      Number val = e.getValue();
+      assertEquals(
+          val, nodeMetricsSnapshot.get("CACHE.nodeLevelCache/myNodeLevelCacheThin.".concat(key)));
+      assertEquals(val, coreMetricsSnapshot.get("CACHE.searcher.filterCache.".concat(key)));
+    }
+
+    // for the other node-level cache, simply check that metrics are accessible
+    assertEquals(0, nodeMetricsSnapshot.get("CACHE.nodeLevelCache/myNodeLevelCache.size"));
+  }
+
+  @BeforeClass
+  public static void setupSolrHome() throws Exception {
+    // make a solr home underneath the test's TEMP_DIR, else we don't have write access to copy in
+    // `solr.xml`
+    Path tmpFile = createTempDir();
+
+    // make data and conf dirs
+    Files.createDirectories(tmpFile.resolve("data"));
+    Path confDir = tmpFile.resolve("collection1").resolve("conf");
+    Files.createDirectories(confDir);
+
+    // copy over configuration files
+    copyXmlToHome(
+        tmpFile.toFile(),
+        TEST_PATH().resolve("solr-nodelevelcaches.xml").toAbsolutePath().toString());
+    Files.copy(
+        getFile("solr/collection1/conf/solrconfig-nodelevelcaches.xml").toPath(),
+        confDir.resolve("solrconfig.xml"));
+    Files.copy(
+        getFile("solr/collection1/conf/solrconfig.snippet.randomindexconfig.xml").toPath(),
+        confDir.resolve("solrconfig.snippet.randomindexconfig.xml"));
+    Files.copy(
+        getFile("solr/collection1/conf/schema-minimal.xml").toPath(),
+        confDir.resolve("schema.xml"));
+
+    // we want the actual `solr.xml` file to be read, instead of the normal operation, which creates
+    // a synthetic "test NodeConfig"
+    System.setProperty("solr.tests.loadSolrXml", "true");
+    initCore("solrconfig.xml", "schema.xml", tmpFile.toAbsolutePath().toString());
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/search/ThinCache.java b/solr/core/src/test/org/apache/solr/search/ThinCache.java
new file mode 100644
index 00000000000..75910b94bb9
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/ThinCache.java
@@ -0,0 +1,472 @@
+/*
+ * 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.search;
+
+import com.github.benmanes.caffeine.cache.RemovalCause;
+import com.github.benmanes.caffeine.cache.RemovalListener;
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.ref.WeakReference;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import org.apache.lucene.util.Accountable;
+import org.apache.solr.metrics.MetricsMap;
+import org.apache.solr.metrics.SolrMetricsContext;
+import org.apache.solr.util.IOFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DISCLAIMER: This class is initially included in the test codebase as a proof-of-concept for
+ * demonstrating/validating node-level cache configuration. Although the implementation is
+ * relatively naive, it should be usable as-is as a plugin, or as a template for developing more
+ * robust implementations.
+ *
+ * <p>A "thin" cache that does not hold strong direct references to the values that it stores and
+ * supplies. Strong references to values are held by a backing {@link NodeLevelCache}. Local
+ * references to keys (and weak references to values) are held by this ThinCache only as an
+ * approximation of the contents of the cache.
+ *
+ * <p>There are no strong guarantees regarding the consistency of local bookkeeping in the ThinCache
+ * (wrt the "source of truth" backing cache). Such guarantees are not necessary, because the local
+ * bookkeeping only exists to support functionality (such as auto-warming and metrics reporting)
+ * where strict correctness is not essential.
+ *
+ * <p>There <i>is</i> however a guarantee that any inconsistency will only be in a safe direction --
+ * i.e., that although there may be entries in the backing cache that are not represented locally in
+ * the ThinCache bookkeeping, the reverse is not true (to protect against memory leak resulting from
+ * the accumulation of stale local references with no corresponding entry in the backing cache).
+ *
+ * <p>NOTE REGARDING AUTOWARMING: because both the warming cache and the cache associated with the
+ * active searcher are backed by the same underlying node-level cache, some extra consideration must
+ * be taken in configuring autowarming. Crosstalk between thin caches is an unavoidable consequence
+ * of the node-level cache approach. Indeed, in the sense that "dynamic resource allocation" is a
+ * type of crosstalk, crosstalk could be considered to be the distinguishing <i>feature</i> of the
+ * node-level cache approach! But in order to prevent competition between active searchers and
+ * corresponding warming searchers, it is advisable to autowarm by percentage -- generally &lt;= 50%
+ * -- or set a relatively low autowarm count (wrt the anticipated overall size of the backing
+ * cache).
+ */
+public class ThinCache<S, K, V> extends SolrCacheBase
+    implements SolrCache<K, V>, Accountable, RemovalListener<K, V> {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final class ScopedKey<S, K> {
+    public final S scope;
+    public final K key;
+
+    private ScopedKey(S scope, K key) {
+      this.scope = scope;
+      this.key = key;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      ScopedKey<?, ?> scopedKey = (ScopedKey<?, ?>) o;
+      return scope.equals(scopedKey.scope) && key.equals(scopedKey.key);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(scope, key);
+    }
+  }
+
+  private interface RemovalListenerRegistry<S, K, V> extends RemovalListener<ScopedKey<S, K>, V> {
+
+    void register(S scope, RemovalListener<K, V> listener);
+
+    void unregister(S scope);
+  }
+
+  private static class RemovalListenerRegistryImpl<S, K, V>
+      implements RemovalListenerRegistry<S, K, V> {
+
+    private final Map<S, RemovalListener<K, V>> listeners = new ConcurrentHashMap<>();
+
+    @Override
+    public void register(S scope, RemovalListener<K, V> listener) {
+      if (listeners.put(scope, listener) != null) {
+        throw new IllegalStateException("removal listener already added for scope " + scope);
+      }
+    }
+
+    @Override
+    public void unregister(S scope) {
+      if (listeners.remove(scope) == null) {
+        log.warn("no removal listener found for scope {}", scope);
+      }
+    }
+
+    @Override
+    public void onRemoval(ScopedKey<S, K> key, V value, RemovalCause cause) {
+      RemovalListener<K, V> listener;
+      if (key != null && (listener = listeners.get(key.scope)) != null) {
+        listener.onRemoval(key.key, value, cause);
+      }
+    }
+  }
+
+  public static final class NodeLevelCache<S, K, V> extends CaffeineCache<ScopedKey<S, K>, V>
+      implements RemovalListenerRegistry<S, K, V> {
+
+    private final RemovalListenerRegistry<S, K, V> removalListenerRegistry =
+        new RemovalListenerRegistryImpl<>();
+
+    @Override
+    public void onRemoval(ScopedKey<S, K> key, V value, RemovalCause cause) {
+      super.onRemoval(key, value, cause);
+      removalListenerRegistry.onRemoval(key, value, cause);
+    }
+
+    @Override
+    public void register(S scope, RemovalListener<K, V> listener) {
+      removalListenerRegistry.register(scope, listener);
+    }
+
+    @Override
+    public void unregister(S scope) {
+      removalListenerRegistry.unregister(scope);
+    }
+
+    @Override
+    public String getName() {
+      return NodeLevelCache.class.getName();
+    }
+
+    @Override
+    public String getDescription() {
+      return String.format(Locale.ROOT, "Node Level Cache(impl=%s)", super.getDescription());
+    }
+  }
+
+  private String description = "Thin Cache";
+
+  private NodeLevelCache<S, K, V> backing;
+
+  private S scope;
+
+  private String parentCacheName;
+
+  private final ConcurrentMap<K, ValEntry<V>> local = new ConcurrentHashMap<>();
+
+  private static final class ValEntry<V> {
+    private final LongAdder ct = new LongAdder();
+    private final WeakReference<V> ref;
+
+    private ValEntry(V val) {
+      this.ref = new WeakReference<>(val);
+    }
+  }
+
+  private static final class HitCountEntry<K, V> {
+    private final long ct;
+    private final K key;
+    private final V val;
+
+    private HitCountEntry(long ct, K key, V val) {
+      this.ct = ct;
+      this.key = key;
+      this.val = val;
+    }
+  }
+
+  @VisibleForTesting
+  void setBacking(S scope, NodeLevelCache<S, K, V> backing) {
+    this.scope = scope;
+    this.backing = backing;
+  }
+
+  private void initForSearcher(SolrIndexSearcher searcher) {
+    if (searcher != null) {
+      // `searcher` may be null for tests, in which case we assume that `this.backing` will
+      // have been set manually via `setBacking()`. In normal use, we expect `searcher != null`.
+      @SuppressWarnings("unchecked")
+      S scope = (S) searcher.getTopReaderContext().reader().getReaderCacheHelper().getKey();
+      this.scope = scope;
+      @SuppressWarnings("unchecked")
+      NodeLevelCache<S, K, V> backing =
+          (NodeLevelCache<S, K, V>) searcher.getCore().getCoreContainer().getCache(parentCacheName);
+      this.backing = backing;
+    }
+    description = generateDescription();
+    backing.register(scope, this);
+  }
+
+  @Override
+  public void initialSearcher(SolrIndexSearcher initialSearcher) {
+    initForSearcher(initialSearcher);
+  }
+
+  @Override
+  public void warm(SolrIndexSearcher searcher, SolrCache<K, V> old) {
+    initForSearcher(searcher);
+    @SuppressWarnings("unchecked")
+    ThinCache<S, K, V> other = (ThinCache<S, K, V>) old;
+    long warmingStartTimeNanos = System.nanoTime();
+    List<HitCountEntry<K, V>> orderedEntries = Collections.emptyList();
+    // warm entries
+    if (isAutowarmingOn()) {
+      orderedEntries = new ArrayList<>(other.local.size() << 1); // oversize
+      for (Entry<K, ValEntry<V>> e : other.local.entrySet()) {
+        ValEntry<V> valEntry = e.getValue();
+        V val = valEntry.ref.get();
+        if (val != null) {
+          orderedEntries.add(new HitCountEntry<>(valEntry.ct.sum(), e.getKey(), val));
+        }
+      }
+      orderedEntries.sort((a, b) -> Long.compare(b.ct, a.ct));
+    }
+
+    int size = autowarm.getWarmCount(orderedEntries.size());
+    int ct = 0;
+    for (HitCountEntry<K, V> entry : orderedEntries) {
+      try {
+        boolean continueRegen =
+            regenerator.regenerateItem(searcher, this, old, entry.key, entry.val);
+        if (!continueRegen || ++ct >= size) {
+          break;
+        }
+      } catch (Exception e) {
+        log.error("Error during auto-warming of key: {}", entry.key, e);
+      }
+    }
+
+    backing.adjustMetrics(hits.sumThenReset(), inserts.sumThenReset(), lookups.sumThenReset());
+    evictions.reset();
+    priorHits = other.hits.sum() + other.priorHits;
+    priorInserts = other.inserts.sum() + other.priorInserts;
+    priorLookups = other.lookups.sum() + other.priorLookups;
+    priorEvictions = other.evictions.sum() + other.priorEvictions;
+    warmupTimeMillis =
+        TimeUnit.MILLISECONDS.convert(
+            System.nanoTime() - warmingStartTimeNanos, TimeUnit.NANOSECONDS);
+  }
+
+  @Override
+  public Object init(Map<String, String> args, Object persistence, CacheRegenerator regenerator) {
+    super.init(args, regenerator);
+    parentCacheName = args.get("parentCacheName");
+    return persistence;
+  }
+
+  private MetricsMap cacheMap;
+  private SolrMetricsContext solrMetricsContext;
+
+  private final LongAdder hits = new LongAdder();
+  private final LongAdder inserts = new LongAdder();
+  private final LongAdder lookups = new LongAdder();
+  private final LongAdder evictions = new LongAdder();
+  private long warmupTimeMillis;
+
+  private long priorHits;
+  private long priorInserts;
+  private long priorLookups;
+  private long priorEvictions;
+
+  @Override
+  public void initializeMetrics(SolrMetricsContext parentContext, String scope) {
+    solrMetricsContext = parentContext.getChildContext(this);
+    cacheMap =
+        new MetricsMap(
+            map -> {
+              long hitCount = hits.sum();
+              long insertCount = inserts.sum();
+              long lookupCount = lookups.sum();
+              long evictionCount = evictions.sum();
+
+              map.put(LOOKUPS_PARAM, lookupCount);
+              map.put(HITS_PARAM, hitCount);
+              map.put(HIT_RATIO_PARAM, hitRate(hitCount, lookupCount));
+              map.put(INSERTS_PARAM, insertCount);
+              map.put(EVICTIONS_PARAM, evictionCount);
+              map.put(SIZE_PARAM, local.size());
+              map.put("warmupTime", warmupTimeMillis);
+              map.put(RAM_BYTES_USED_PARAM, ramBytesUsed());
+              map.put(MAX_RAM_MB_PARAM, getMaxRamMB());
+
+              long cumLookups = priorLookups + lookupCount;
+              long cumHits = priorHits + hitCount;
+              map.put("cumulative_lookups", cumLookups);
+              map.put("cumulative_hits", cumHits);
+              map.put("cumulative_hitratio", hitRate(cumHits, cumLookups));
+              map.put("cumulative_inserts", priorInserts + insertCount);
+              map.put("cumulative_evictions", priorEvictions + evictionCount);
+            });
+    solrMetricsContext.gauge(cacheMap, true, scope, getCategory().toString());
+  }
+
+  @VisibleForTesting
+  MetricsMap getMetricsMap() {
+    return cacheMap;
+  }
+
+  // TODO: refactor this common method out of here and `CaffeineCache`
+  private static double hitRate(long hitCount, long lookupCount) {
+    return lookupCount == 0 ? 1.0 : (double) hitCount / lookupCount;
+  }
+
+  @Override
+  public SolrMetricsContext getSolrMetricsContext() {
+    return solrMetricsContext;
+  }
+
+  @Override
+  public void close() throws IOException {
+    backing.unregister(scope);
+    SolrCache.super.close();
+  }
+
+  @Override
+  public void onRemoval(K key, V value, RemovalCause cause) {
+    if (cause.wasEvicted()) {
+      evictions.increment();
+    }
+    local.remove(key);
+  }
+
+  @Override
+  public String toString() {
+    return name() + (cacheMap != null ? cacheMap.getValue().toString() : "");
+  }
+
+  @Override
+  public int size() {
+    return local.size();
+  }
+
+  @Override
+  public V put(K key, V value) {
+    inserts.increment();
+    ValEntry<V> valEntry = new ValEntry<>(value);
+    valEntry.ct.increment();
+    local.put(key, valEntry);
+    return backing.put(new ScopedKey<>(scope, key), value);
+  }
+
+  @Override
+  public V get(K key) {
+    lookups.increment();
+    V ret = backing.get(new ScopedKey<>(scope, key));
+    if (ret != null) {
+      hits.increment();
+      ValEntry<V> valEntry = local.get(key);
+      if (valEntry != null) {
+        valEntry.ct.increment();
+      }
+    }
+    return ret;
+  }
+
+  @Override
+  public V remove(K key) {
+    // NOTE: rely on `onRemoval()` to remove entry from `local`
+    return backing.remove(new ScopedKey<>(scope, key));
+  }
+
+  @Override
+  public V computeIfAbsent(K key, IOFunction<? super K, ? extends V> mappingFunction)
+      throws IOException {
+    lookups.increment();
+    boolean[] hit = new boolean[] {true};
+    V ret =
+        backing.computeIfAbsent(
+            new ScopedKey<>(scope, key),
+            (k) -> {
+              hit[0] = false;
+              inserts.increment();
+              V innerRet = mappingFunction.apply(k.key);
+              ValEntry<V> valEntry = new ValEntry<>(innerRet);
+              valEntry.ct.increment();
+              local.put(key, valEntry);
+              return innerRet;
+            });
+    if (hit[0]) {
+      hits.increment();
+    }
+    return ret;
+  }
+
+  @Override
+  public void clear() {
+    for (K key : local.keySet()) {
+      backing.remove(new ScopedKey<>(scope, key));
+    }
+    // NOTE: rely on `onRemoval()` to remove entries from `local`
+  }
+
+  @Override
+  public int getMaxSize() {
+    return backing == null ? 0 : backing.getMaxSize();
+  }
+
+  @Override
+  public void setMaxSize(int maxSize) {
+    throw new UnsupportedOperationException(
+        "limits cannot be configured directly on " + getClass());
+  }
+
+  @Override
+  public int getMaxRamMB() {
+    return backing == null ? 0 : backing.getMaxRamMB();
+  }
+
+  @Override
+  public void setMaxRamMB(int maxRamMB) {
+    throw new UnsupportedOperationException(
+        "limits cannot be configured directly on " + getClass());
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    // TODO: this has yet to be implemented.
+    //  the actual implementation should be straightforward, but there are questions about what's
+    //  in and out of scope for calculating ramBytesUsed.
+    return 0;
+  }
+
+  @Override
+  public String getName() {
+    return ThinCache.class.getName();
+  }
+
+  /** Returns the description of this cache. */
+  private String generateDescription() {
+    return String.format(
+        Locale.ROOT,
+        "Thin Cache(backing=%s%s)",
+        backing.getDescription(),
+        isAutowarmingOn() ? (", " + getAutowarmDescription()) : "");
+  }
+
+  @Override
+  public String getDescription() {
+    return description;
+  }
+}
diff --git a/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc b/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc
index 26743e7d9af..523e7bc0c8c 100644
--- a/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc
+++ b/solr/solr-ref-guide/modules/configuration-guide/pages/configuring-solr-xml.adoc
@@ -638,6 +638,26 @@ You can define system properties that should not be returned, or define custom s
 
 If you would like to customize the metrics for your installation, see the xref:deployment-guide:metrics-reporting.adoc#metrics-configuration[Metrics Configuration] section.
 
+=== The <caches> Element
+
+The `<caches>` element in `solr.xml` supports defining and configuring named node-level caches.
+
+These caches are analogous to xref:caches-warming.adoc#user-defined-caches[user-defined caches in `solrconfig.xml`], except that each named cache exists as a long-lived singleton at the node level. These node-level caches are accessible from application code via `CoreContainer.getCache(String cacheName)`.
+
+Note that because node-level caches exist above the context of an individual core, config parameters that hook into the lifecycle of a core/searcher (such as `autowarmCount` and `regenerator`) are irrelevant/ignored for node-level caches.
+
+[source,xml]
+----
+<solr>
+  <caches>
+    <cache name="myNodeLevelUserCache"
+           class="solr.CaffeineCache"
+           size="4096"
+           initialSize="1024" />
+  </caches>
+</solr>
+----
+
 == Substituting JVM System Properties in solr.xml
 
 Solr supports variable substitution of JVM system property values in `solr.xml`, which allows runtime specification of various configuration options.
diff --git a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
index 13b5baa54af..fd344922967 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/TestHarness.java
@@ -25,6 +25,7 @@ import java.nio.file.Paths;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.impl.SolrZkClientTimeout;
 import org.apache.solr.common.SolrException;
@@ -186,6 +187,9 @@ public class TestHarness extends BaseTestHarness {
   }
 
   public static NodeConfig buildTestNodeConfig(Path solrHome) {
+    if ("true".equals(System.getProperty("solr.tests.loadSolrXml"))) {
+      return SolrXmlConfig.fromSolrHome(solrHome, new Properties());
+    }
     CloudConfig cloudConfig =
         (null == System.getProperty("zkHost"))
             ? null