You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ge...@apache.org on 2023/01/23 17:05:39 UTC

[solr] branch branch_9x updated: SOLR-16615: Reuse Jersey apps for cores with the same configset (#1286)

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

gerlowskija 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 36345f81165 SOLR-16615: Reuse Jersey apps for cores with the same configset (#1286)
36345f81165 is described below

commit 36345f81165e846f55d3d07970c1770a63c1ac11
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Mon Jan 23 11:25:13 2023 -0500

    SOLR-16615: Reuse Jersey apps for cores with the same configset (#1286)
    
    Prior to this commit our Jersey integration involved creating an 'ApplicationHandler'
    class for each SolrCore. These classes are expensive and their creation can have a
    noticeable cumulative effect at startup time on nodes hosting many many cores.
    
    This PR addresses this problem by allowing ApplicationHandler's to be shared across
    multiple SolrCores, as long as each of those cores use the same configset. To do this
    we introduce JerseyAppHandlerCache, which takes over ownership of all core-level
    ApplicationHandler's. AH's are ref-counted so that they can be removed from the cache
    and closed when no longer needed by any SolrCores.
---
 .../src/java/org/apache/solr/api/V2HttpCall.java   | 31 +++++--
 .../java/org/apache/solr/core/ConfigOverlay.java   | 29 ++++---
 .../java/org/apache/solr/core/CoreContainer.java   |  8 ++
 .../src/java/org/apache/solr/core/PluginBag.java   | 29 +++++--
 .../src/java/org/apache/solr/core/SolrConfig.java  | 25 +++++-
 .../src/java/org/apache/solr/core/SolrCore.java    | 33 ++++++--
 .../org/apache/solr/core/SolrResourceLoader.java   | 46 +++++++++-
 .../org/apache/solr/handler/SolrConfigHandler.java |  7 +-
 .../designer/SchemaDesignerSettingsDAO.java        |  2 +-
 .../org/apache/solr/jersey/InjectionFactories.java | 22 +++++
 .../apache/solr/jersey/JerseyAppHandlerCache.java  | 98 +++++++++++++++++++++
 .../org/apache/solr/jersey/JerseyApplications.java | 21 ++---
 .../org/apache/solr/jersey/MetricBeanFactory.java  | 55 ------------
 .../org/apache/solr/jersey/RequestContextKeys.java |  2 +
 .../apache/solr/jersey/RequestMetricHandling.java  | 34 +++++---
 .../solr/jersey/JerseyApplicationSharingTest.java  | 99 ++++++++++++++++++++++
 16 files changed, 415 insertions(+), 126 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/api/V2HttpCall.java b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
index 41d3c11417e..b2512efcb07 100644
--- a/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
+++ b/solr/core/src/java/org/apache/solr/api/V2HttpCall.java
@@ -337,14 +337,19 @@ public class V2HttpCall extends HttpSolrCall {
   }
 
   private boolean invokeJerseyRequest(
-      CoreContainer cores, SolrCore core, ApplicationHandler primary, SolrQueryResponse rsp) {
-    return invokeJerseyRequest(cores, core, primary, rsp, Map.of());
+      CoreContainer cores,
+      SolrCore core,
+      ApplicationHandler jerseyHandler,
+      PluginBag<SolrRequestHandler> requestHandlers,
+      SolrQueryResponse rsp) {
+    return invokeJerseyRequest(cores, core, jerseyHandler, requestHandlers, rsp, Map.of());
   }
 
   private boolean invokeJerseyRequest(
       CoreContainer cores,
       SolrCore core,
       ApplicationHandler jerseyHandler,
+      PluginBag<SolrRequestHandler> requestHandlers,
       SolrQueryResponse rsp,
       Map<String, String> additionalProperties) {
     final ContainerRequest containerRequest =
@@ -355,6 +360,8 @@ public class V2HttpCall extends HttpSolrCall {
     containerRequest.setProperty(RequestContextKeys.SOLR_QUERY_REQUEST, solrReq);
     containerRequest.setProperty(RequestContextKeys.SOLR_QUERY_RESPONSE, rsp);
     containerRequest.setProperty(RequestContextKeys.CORE_CONTAINER, cores);
+    containerRequest.setProperty(
+        RequestContextKeys.RESOURCE_TO_RH_MAPPING, requestHandlers.getJaxrsRegistry());
     containerRequest.setProperty(RequestContextKeys.HTTP_SERVLET_REQ, req);
     containerRequest.setProperty(RequestContextKeys.REQUEST_TYPE, requestType);
     containerRequest.setProperty(RequestContextKeys.SOLR_PARAMS, queryParams);
@@ -401,7 +408,12 @@ public class V2HttpCall extends HttpSolrCall {
     SolrQueryResponse solrResp = new SolrQueryResponse();
     final boolean jerseyResourceFound =
         invokeJerseyRequest(
-            cores, null, cores.getJerseyApplicationHandler(), solrResp, suppressNotFoundProp);
+            cores,
+            null,
+            cores.getJerseyApplicationHandler(),
+            cores.getRequestHandlers(),
+            solrResp,
+            suppressNotFoundProp);
     if (jerseyResourceFound) {
       logAndFlushAdminRequest(solrResp);
       return;
@@ -415,7 +427,8 @@ public class V2HttpCall extends HttpSolrCall {
   @Override
   protected void handleAdmin(SolrQueryResponse solrResp) {
     if (api == null) {
-      invokeJerseyRequest(cores, null, cores.getJerseyApplicationHandler(), solrResp);
+      invokeJerseyRequest(
+          cores, null, cores.getJerseyApplicationHandler(), cores.getRequestHandlers(), solrResp);
     } else {
       SolrCore.preDecorateResponse(solrReq, solrResp);
       try {
@@ -449,10 +462,16 @@ public class V2HttpCall extends HttpSolrCall {
           Map.of(RequestContextKeys.SUPPRESS_ERROR_ON_NOT_FOUND_EXCEPTION, "true");
       final boolean resourceFound =
           invokeJerseyRequest(
-              cores, core, core.getJerseyApplicationHandler(), rsp, suppressNotFoundProp);
+              cores,
+              core,
+              core.getJerseyApplicationHandler(),
+              core.getRequestHandlers(),
+              rsp,
+              suppressNotFoundProp);
       if (!resourceFound) {
         response.getHeaderNames().stream().forEach(name -> response.setHeader(name, null));
-        invokeJerseyRequest(cores, null, cores.getJerseyApplicationHandler(), rsp);
+        invokeJerseyRequest(
+            cores, null, cores.getJerseyApplicationHandler(), cores.getRequestHandlers(), rsp);
       }
     } else {
       SolrCore.preDecorateResponse(solrReq, rsp);
diff --git a/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java b/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
index acfa64941e1..56a01afec14 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigOverlay.java
@@ -34,15 +34,15 @@ import org.apache.solr.common.util.Utils;
  * performed on tbhis gives a new copy of the object with the changed value
  */
 public class ConfigOverlay implements MapSerializable {
-  private final int znodeVersion;
+  private final int version;
   private final Map<String, Object> data;
   private Map<String, Object> props;
   private Map<String, Object> userProps;
 
   @SuppressWarnings({"unchecked"})
-  public ConfigOverlay(Map<String, Object> jsonObj, int znodeVersion) {
+  public ConfigOverlay(Map<String, Object> jsonObj, int version) {
     if (jsonObj == null) jsonObj = Collections.emptyMap();
-    this.znodeVersion = znodeVersion;
+    this.version = version;
     data = Collections.unmodifiableMap(jsonObj);
     props = (Map<String, Object>) data.get("props");
     if (props == null) props = Collections.emptyMap();
@@ -71,7 +71,7 @@ public class ConfigOverlay implements MapSerializable {
     copy.put(key, val);
     Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
     jsonObj.put("userProps", copy);
-    return new ConfigOverlay(jsonObj, znodeVersion);
+    return new ConfigOverlay(jsonObj, version);
   }
 
   public ConfigOverlay unsetUserProperty(String key) {
@@ -80,7 +80,7 @@ public class ConfigOverlay implements MapSerializable {
     copy.remove(key);
     Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
     jsonObj.put("userProps", copy);
-    return new ConfigOverlay(jsonObj, znodeVersion);
+    return new ConfigOverlay(jsonObj, version);
   }
 
   @SuppressWarnings({"unchecked", "rawtypes"})
@@ -103,7 +103,7 @@ public class ConfigOverlay implements MapSerializable {
     Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
     jsonObj.put("props", deepCopy);
 
-    return new ConfigOverlay(jsonObj, znodeVersion);
+    return new ConfigOverlay(jsonObj, version);
   }
 
   public static final String NOT_EDITABLE = "''{0}'' is not an editable property";
@@ -139,15 +139,15 @@ public class ConfigOverlay implements MapSerializable {
     Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
     jsonObj.put("props", deepCopy);
 
-    return new ConfigOverlay(jsonObj, znodeVersion);
+    return new ConfigOverlay(jsonObj, version);
   }
 
   public byte[] toByteArray() {
     return Utils.toJSON(data);
   }
 
-  public int getZnodeVersion() {
-    return znodeVersion;
+  public int getVersion() {
+    return version;
   }
 
   @Override
@@ -236,7 +236,7 @@ public class ConfigOverlay implements MapSerializable {
 
   @Override
   public Map<String, Object> toMap(Map<String, Object> map) {
-    map.put(ZNODEVER, znodeVersion);
+    map.put(ZNODEVER, version);
     map.putAll(data);
     return map;
   }
@@ -258,7 +258,7 @@ public class ConfigOverlay implements MapSerializable {
     Map<String, Object> existing = (Map<String, Object>) dataCopy.get(typ);
     if (existing == null) dataCopy.put(typ, existing = new LinkedHashMap<>());
     existing.put(info.get(CoreAdminParams.NAME).toString(), info);
-    return new ConfigOverlay(dataCopy, this.znodeVersion);
+    return new ConfigOverlay(dataCopy, this.version);
   }
 
   @SuppressWarnings({"unchecked"})
@@ -267,9 +267,14 @@ public class ConfigOverlay implements MapSerializable {
     Map<?, ?> reqHandler = (Map<?, ?>) dataCopy.get(typ);
     if (reqHandler == null) return this;
     reqHandler.remove(name);
-    return new ConfigOverlay(dataCopy, this.znodeVersion);
+    return new ConfigOverlay(dataCopy, this.version);
   }
 
   public static final String ZNODEVER = "znodeVersion";
   public static final String NAME = "overlay";
+
+  @Override
+  public int hashCode() {
+    return data.hashCode();
+  }
 }
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 666536abc72..ce4b0cc2573 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -129,6 +129,7 @@ import org.apache.solr.handler.api.V2ApiUtils;
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.handler.designer.SchemaDesignerAPI;
 import org.apache.solr.jersey.InjectionFactories;
+import org.apache.solr.jersey.JerseyAppHandlerCache;
 import org.apache.solr.logging.LogWatcher;
 import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.metrics.SolrCoreMetricManager;
@@ -192,11 +193,16 @@ public class CoreContainer {
       new PluginBag<>(SolrRequestHandler.class, null);
 
   private volatile ApplicationHandler jerseyAppHandler;
+  private volatile JerseyAppHandlerCache appHandlersByConfigSetId;
 
   public ApplicationHandler getJerseyApplicationHandler() {
     return jerseyAppHandler;
   }
 
+  public JerseyAppHandlerCache getAppHandlerCache() {
+    return appHandlersByConfigSetId;
+  }
+
   /** Minimize exposure to CoreContainer. Mostly only ZK interface is required */
   public final Supplier<SolrZkClient> zkClientSupplier = () -> getZkController().getZkClient();
 
@@ -1092,6 +1098,8 @@ public class CoreContainer {
       jerseyAppHandler = new ApplicationHandler(containerHandlers.getJerseyEndpoints());
     }
 
+    appHandlersByConfigSetId = new JerseyAppHandlerCache();
+
     // Do Node setup logic after all handlers have been registered.
     if (isZooKeeperAware()) {
       clusterSingletons.setReady();
diff --git a/solr/core/src/java/org/apache/solr/core/PluginBag.java b/solr/core/src/java/org/apache/solr/core/PluginBag.java
index 3711e2c0f17..7e06449ca5a 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -69,26 +69,38 @@ public class PluginBag<T> implements AutoCloseable {
   private final ApiBag apiBag;
   private final ResourceConfig jerseyResources;
 
-  public static class JerseyMetricsLookupRegistry
+  /**
+   * Allows JAX-RS 'filters' to find the requestHandler (if any) associated particular JAX-RS
+   * resource classes
+   *
+   * <p>Used primarily by JAX-RS when recording per-request metrics, which requires a {@link
+   * org.apache.solr.handler.RequestHandlerBase.HandlerMetrics} object from the relevant
+   * requestHandler.
+   */
+  public static class JaxrsResourceToHandlerMappings
       extends HashMap<Class<? extends JerseyResource>, RequestHandlerBase> {}
 
-  private final JerseyMetricsLookupRegistry infoBeanByResource;
+  private final JaxrsResourceToHandlerMappings jaxrsResourceRegistry;
+
+  public JaxrsResourceToHandlerMappings getJaxrsRegistry() {
+    return jaxrsResourceRegistry;
+  }
 
   /** Pass needThreadSafety=true if plugins can be added and removed concurrently with lookups. */
   public PluginBag(Class<T> klass, SolrCore core, boolean needThreadSafety) {
     if (klass == SolrRequestHandler.class && V2ApiUtils.isEnabled()) {
       this.loadV2ApisIfPresent = true;
       this.apiBag = new ApiBag(core != null);
-      this.infoBeanByResource = new JerseyMetricsLookupRegistry();
+      this.jaxrsResourceRegistry = new JaxrsResourceToHandlerMappings();
       this.jerseyResources =
           (core == null)
-              ? new JerseyApplications.CoreContainerApp(infoBeanByResource)
-              : new JerseyApplications.SolrCoreApp(core, infoBeanByResource);
+              ? new JerseyApplications.CoreContainerApp()
+              : new JerseyApplications.SolrCoreApp();
     } else {
       this.loadV2ApisIfPresent = false;
       this.apiBag = null;
       this.jerseyResources = null;
-      this.infoBeanByResource = null;
+      this.jaxrsResourceRegistry = null;
     }
     this.core = core;
     this.klass = klass;
@@ -250,10 +262,11 @@ public class PluginBag<T> implements AutoCloseable {
                   log.debug("Registering jersey resource class: {}", jerseyClazz.getName());
                 }
                 jerseyResources.register(jerseyClazz);
-                // See MetricsBeanFactory javadocs for a better understanding of this resource->RH
+                // See RequestMetricHandling javadocs for a better understanding of this
+                // resource->RH
                 // mapping
                 if (inst instanceof RequestHandlerBase) {
-                  infoBeanByResource.put(jerseyClazz, (RequestHandlerBase) inst);
+                  jaxrsResourceRegistry.put(jerseyClazz, (RequestHandlerBase) inst);
                 }
               }
             }
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 a3be18156c5..522ee33a357 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -108,6 +108,7 @@ public class SolrConfig implements MapSerializable {
 
   private int znodeVersion;
   ConfigNode root;
+  int hashCode;
   private final SolrResourceLoader resourceLoader;
   private Properties substituteProperties;
 
@@ -178,9 +179,14 @@ public class SolrConfig implements MapSerializable {
         ZkSolrResourceLoader.ZkByteArrayInputStream zkin =
             (ZkSolrResourceLoader.ZkByteArrayInputStream) in;
         zkVersion = zkin.getStat().getVersion();
-        hash = Objects.hash(zkin.getStat().getCtime(), zkVersion, overlay.getZnodeVersion());
+        hash = Objects.hash(zkin.getStat().getCtime(), zkVersion, overlay.getVersion());
         this.fileName = zkin.fileName;
       }
+      if (in instanceof SolrResourceLoader.SolrFileInputStream) {
+        SolrResourceLoader.SolrFileInputStream sfin = (SolrResourceLoader.SolrFileInputStream) in;
+        zkVersion = (int) sfin.getLastModified();
+        hash = Objects.hash(sfin.getLastModified(), overlay.getVersion());
+      }
     }
 
     @Override
@@ -235,6 +241,10 @@ public class SolrConfig implements MapSerializable {
           }
         });
     try {
+      // This will hash the solrconfig.xml, user properties and overlay (all possible ways of
+      // modifying the resulting SolrConfig)
+      hashCode = Objects.hash(this.root.txt(), overlay.getVersion());
+
       getRequestParams();
       initLibs(loader, isConfigsetTrusted);
       String val =
@@ -576,11 +586,17 @@ public class SolrConfig implements MapSerializable {
         return new ConfigOverlay(Collections.emptyMap(), -1);
       }
 
-      int version = 0; // will be always 0 for file based resourceLoader
+      int version = 0;
       if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
         version = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
         log.debug("Config overlay loaded. version : {} ", version);
       }
+      if (in instanceof SolrResourceLoader.SolrFileInputStream) {
+        // We should be ok, it is unlikely that a configOverlay is loaded decades apart and has the
+        // same version after casting to an int
+        version = (int) ((SolrResourceLoader.SolrFileInputStream) in).getLastModified();
+        log.debug("Config overlay loaded. version : {} ", version);
+      }
       @SuppressWarnings("unchecked")
       Map<String, Object> m = (Map<String, Object>) Utils.fromJSON(in);
       return new ConfigOverlay(m, version);
@@ -1162,4 +1178,9 @@ public class SolrConfig implements MapSerializable {
   public ConfigNode get(String name, Predicate<ConfigNode> test) {
     return root.get(name, test);
   }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
 }
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 75cb384d4fd..4f06a96c156 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -110,6 +110,7 @@ import org.apache.solr.handler.SolrConfigHandler;
 import org.apache.solr.handler.api.V2ApiUtils;
 import org.apache.solr.handler.component.HighlightComponent;
 import org.apache.solr.handler.component.SearchComponent;
+import org.apache.solr.jersey.JerseyAppHandlerCache;
 import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.metrics.SolrCoreMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
@@ -225,7 +226,7 @@ public class SolrCore implements SolrInfoBean, Closeable {
   private final Date startTime = new Date();
   private final long startNanoTime = System.nanoTime();
   private final RequestHandlers reqHandlers;
-  private final ApplicationHandler jerseyAppHandler;
+  private final RefCounted<ApplicationHandler> appHandlerForConfigSet;
   private final PluginBag<SearchComponent> searchComponents =
       new PluginBag<>(SearchComponent.class, this);
   private final PluginBag<UpdateRequestProcessorFactory> updateProcessors =
@@ -1136,10 +1137,23 @@ public class SolrCore implements SolrInfoBean, Closeable {
       updateProcessorChains = loadUpdateProcessorChains();
       reqHandlers = new RequestHandlers(this);
       reqHandlers.initHandlersFromConfig(solrConfig);
-      jerseyAppHandler =
-          (V2ApiUtils.isEnabled())
-              ? new ApplicationHandler(reqHandlers.getRequestHandlers().getJerseyEndpoints())
-              : null;
+      if (V2ApiUtils.isEnabled()) {
+        final String effectiveConfigsetId = JerseyAppHandlerCache.generateIdForConfigSet(configSet);
+        appHandlerForConfigSet =
+            coreContainer
+                .getAppHandlerCache()
+                .computeIfAbsent(
+                    effectiveConfigsetId,
+                    () -> {
+                      log.debug(
+                          "Creating Jersey ApplicationHandler for 'effective configset' [{}]",
+                          effectiveConfigsetId);
+                      return new ApplicationHandler(
+                          reqHandlers.getRequestHandlers().getJerseyEndpoints());
+                    });
+      } else {
+        appHandlerForConfigSet = null;
+      }
 
       // cause the executor to stall so firstSearcher events won't fire
       // until after inform() has been called for all components.
@@ -1775,6 +1789,9 @@ public class SolrCore implements SolrInfoBean, Closeable {
     }
 
     if (reqHandlers != null) reqHandlers.close();
+    if (V2ApiUtils.isEnabled()) {
+      appHandlerForConfigSet.decref();
+    }
     responseWriters.close();
     searchComponents.close();
     qParserPlugins.close();
@@ -1962,7 +1979,7 @@ public class SolrCore implements SolrInfoBean, Closeable {
   }
 
   public ApplicationHandler getJerseyApplicationHandler() {
-    return jerseyAppHandler;
+    return appHandlerForConfigSet.get();
   }
 
   /**
@@ -3390,8 +3407,8 @@ public class SolrCore implements SolrInfoBean, Closeable {
         if (solrCore == null || solrCore.isClosed() || solrCore.getCoreContainer().isShutDown())
           return;
         cfg = solrCore.getSolrConfig();
-        solrConfigversion = solrCore.getSolrConfig().getOverlay().getZnodeVersion();
-        overlayVersion = solrCore.getSolrConfig().getZnodeVersion();
+        solrConfigversion = solrCore.getSolrConfig().getZnodeVersion();
+        overlayVersion = solrCore.getSolrConfig().getOverlay().getVersion();
         if (managedSchmaResourcePath != null) {
           managedSchemaVersion =
               ((ManagedIndexSchema) solrCore.getLatestSchema()).getSchemaZkVersion();
diff --git a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
index 59641b6b912..f01f74ce580 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
@@ -359,11 +359,14 @@ public class SolrResourceLoader
       // The resource is either inside instance dir or we allow unsafe loading, so allow testing if
       // file exists
       if (Files.exists(inConfigDir) && Files.isReadable(inConfigDir)) {
-        return Files.newInputStream(inConfigDir);
+        return new SolrFileInputStream(
+            Files.newInputStream(inConfigDir), Files.getLastModifiedTime(inConfigDir).toMillis());
       }
 
       if (Files.exists(inInstanceDir) && Files.isReadable(inInstanceDir)) {
-        return Files.newInputStream(inInstanceDir);
+        return new SolrFileInputStream(
+            Files.newInputStream(inInstanceDir),
+            Files.getLastModifiedTime(inInstanceDir).toMillis());
       }
     }
 
@@ -985,4 +988,43 @@ public class SolrResourceLoader
   // This is to verify if this requires to use the schema classloader for classes loaded from
   // packages
   private static final ThreadLocal<ResourceLoaderAware> CURRENT_AWARE = new ThreadLocal<>();
+
+  public static class SolrFileInputStream extends InputStream {
+    private final InputStream delegate;
+    private final long lastModified;
+
+    public SolrFileInputStream(InputStream delegate, long lastModified) {
+      this.delegate = delegate;
+      this.lastModified = lastModified;
+    }
+
+    public long getLastModified() {
+      return lastModified;
+    }
+
+    @Override
+    public synchronized int read() throws IOException {
+      return delegate.read();
+    }
+
+    @Override
+    public synchronized int read(byte[] bs, int off, int len) throws IOException {
+      return delegate.read(bs, off, len);
+    }
+
+    @Override
+    public int available() throws IOException {
+      return delegate.available();
+    }
+
+    @Override
+    public synchronized long skip(long n) throws IOException {
+      return delegate.skip(n);
+    }
+
+    @Override
+    public void close() throws IOException {
+      delegate.close();
+    }
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
index b83fdb57839..aeb17522b8d 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -213,13 +213,12 @@ public class SolrConfigHandler extends RequestHandlerBase
             resp.add(
                 ZNODEVER,
                 Map.of(
-                    ConfigOverlay.NAME,
-                        req.getCore().getSolrConfig().getOverlay().getZnodeVersion(),
+                    ConfigOverlay.NAME, req.getCore().getSolrConfig().getOverlay().getVersion(),
                     RequestParams.NAME,
                         req.getCore().getSolrConfig().getRequestParams().getZnodeVersion()));
             boolean isStale = false;
             int expectedVersion = req.getParams().getInt(ConfigOverlay.NAME, -1);
-            int actualVersion = req.getCore().getSolrConfig().getOverlay().getZnodeVersion();
+            int actualVersion = req.getCore().getSolrConfig().getOverlay().getVersion();
             if (expectedVersion > actualVersion) {
               log.info(
                   "expecting overlay version {} but my version is {}",
@@ -589,7 +588,7 @@ public class SolrConfigHandler extends RequestHandlerBase
         int latestVersion =
             ZkController.persistConfigResourceToZooKeeper(
                 (ZkSolrResourceLoader) loader,
-                overlay.getZnodeVersion(),
+                overlay.getVersion(),
                 ConfigOverlay.RESOURCE_NAME,
                 overlay.toByteArray(),
                 true);
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettingsDAO.java b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettingsDAO.java
index 63e11ea4f5d..eea151dc69d 100644
--- a/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettingsDAO.java
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettingsDAO.java
@@ -99,7 +99,7 @@ class SchemaDesignerSettingsDAO implements SchemaDesignerConstants {
     if (changed) {
       ZkController.persistConfigResourceToZooKeeper(
           zkLoaderForConfigSet(configSet),
-          overlay.getZnodeVersion(),
+          overlay.getVersion(),
           ConfigOverlay.RESOURCE_NAME,
           overlay.toByteArray(),
           true);
diff --git a/solr/core/src/java/org/apache/solr/jersey/InjectionFactories.java b/solr/core/src/java/org/apache/solr/jersey/InjectionFactories.java
index ee430c86258..e794ea30237 100644
--- a/solr/core/src/java/org/apache/solr/jersey/InjectionFactories.java
+++ b/solr/core/src/java/org/apache/solr/jersey/InjectionFactories.java
@@ -17,8 +17,11 @@
 
 package org.apache.solr.jersey;
 
+import static org.apache.solr.jersey.RequestContextKeys.SOLR_CORE;
+
 import javax.inject.Inject;
 import javax.ws.rs.container.ContainerRequestContext;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.glassfish.hk2.api.Factory;
@@ -62,6 +65,25 @@ public class InjectionFactories {
     public void dispose(SolrQueryResponse instance) {}
   }
 
+  /** Fetch the (existing) SolrCore from the request context */
+  public static class SolrCoreFactory implements Factory<SolrCore> {
+
+    private final ContainerRequestContext containerRequestContext;
+
+    @Inject
+    public SolrCoreFactory(ContainerRequestContext containerRequestContext) {
+      this.containerRequestContext = containerRequestContext;
+    }
+
+    @Override
+    public SolrCore provide() {
+      return (SolrCore) containerRequestContext.getProperty(SOLR_CORE);
+    }
+
+    @Override
+    public void dispose(SolrCore instance) {}
+  }
+
   public static class SingletonFactory<T> implements Factory<T> {
 
     private final T singletonVal;
diff --git a/solr/core/src/java/org/apache/solr/jersey/JerseyAppHandlerCache.java b/solr/core/src/java/org/apache/solr/jersey/JerseyAppHandlerCache.java
new file mode 100644
index 00000000000..49b317ef185
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/jersey/JerseyAppHandlerCache.java
@@ -0,0 +1,98 @@
+/*
+ * 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.jersey;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.solr.core.ConfigSet;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.util.RefCounted;
+import org.glassfish.jersey.server.ApplicationHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Stores Jersey 'ApplicationHandler' instances by an ID or hash derived from their {@link
+ * ConfigSet}.
+ *
+ * <p>ApplicationHandler creation is expensive; caching these objects allows them to be shared by
+ * multiple cores with the same configuration.
+ */
+public class JerseyAppHandlerCache {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final Map<String, RefCounted<ApplicationHandler>> applicationByConfigSetId;
+
+  public JerseyAppHandlerCache() {
+    this.applicationByConfigSetId = new ConcurrentHashMap<>();
+  }
+
+  /**
+   * Return the 'ApplicationHandler' associated with the provided ID, creating it first if
+   * necessary.
+   *
+   * <p>This method is thread-safe by virtue of its delegation to {@link
+   * ConcurrentHashMap#computeIfAbsent(Object, Function)} internally.
+   *
+   * @param effectiveConfigSetId an ID to associate the ApplicationHandler with. Usually created via
+   *     {@link #generateIdForConfigSet(ConfigSet)}.
+   * @param createApplicationHandler a Supplier producing an ApplicationHandler
+   */
+  public RefCounted<ApplicationHandler> computeIfAbsent(
+      String effectiveConfigSetId, Supplier<ApplicationHandler> createApplicationHandler) {
+    final Function<String, RefCounted<ApplicationHandler>> wrapper =
+        s -> {
+          return new RefCounted<>(createApplicationHandler.get()) {
+            @Override
+            public void close() {
+              log.info(
+                  "Removing AppHandler from cache for 'effective configset' [{}]",
+                  effectiveConfigSetId);
+              applicationByConfigSetId.remove(effectiveConfigSetId);
+            }
+          };
+        };
+
+    final RefCounted<ApplicationHandler> fetched =
+        applicationByConfigSetId.computeIfAbsent(effectiveConfigSetId, wrapper);
+    fetched.incref();
+    return fetched;
+  }
+
+  public int size() {
+    return applicationByConfigSetId.size();
+  }
+
+  /**
+   * Generates a String ID to represent the provided {@link ConfigSet}
+   *
+   * <p>Relies on {@link SolrConfig#hashCode()} to generate a different ID for each "unique"
+   * configset (where "uniqueness" considers various overlays that get applied to the {@link
+   * ConfigSet})
+   *
+   * @see SolrCore#hashCode()
+   */
+  public static String generateIdForConfigSet(ConfigSet configSet) {
+    return configSet.getName() + "-" + configSet.getSolrConfig().hashCode();
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java b/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java
index 7a4bb484e4b..d3670accfce 100644
--- a/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java
+++ b/solr/core/src/java/org/apache/solr/jersey/JerseyApplications.java
@@ -20,8 +20,6 @@ package org.apache.solr.jersey;
 import io.swagger.v3.oas.annotations.OpenAPIDefinition;
 import io.swagger.v3.oas.annotations.info.Info;
 import io.swagger.v3.oas.annotations.info.License;
-import javax.inject.Singleton;
-import org.apache.solr.core.PluginBag;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
@@ -44,7 +42,7 @@ import org.glassfish.jersey.server.ResourceConfig;
 public class JerseyApplications {
 
   public static class CoreContainerApp extends ResourceConfig {
-    public CoreContainerApp(PluginBag.JerseyMetricsLookupRegistry beanRegistry) {
+    public CoreContainerApp() {
       super();
 
       // Authentication and authorization
@@ -63,15 +61,6 @@ public class JerseyApplications {
       register(RequestMetricHandling.PreRequestMetricsFilter.class);
       register(RequestMetricHandling.PostRequestMetricsFilter.class);
       register(PostRequestDecorationFilter.class);
-      register(
-          new AbstractBinder() {
-            @Override
-            protected void configure() {
-              bindFactory(new MetricBeanFactory(beanRegistry))
-                  .to(PluginBag.JerseyMetricsLookupRegistry.class)
-                  .in(Singleton.class);
-            }
-          });
       register(
           new AbstractBinder() {
             @Override
@@ -102,17 +91,17 @@ public class JerseyApplications {
 
   public static class SolrCoreApp extends CoreContainerApp {
 
-    public SolrCoreApp(SolrCore solrCore, PluginBag.JerseyMetricsLookupRegistry beanRegistry) {
-      super(beanRegistry);
+    public SolrCoreApp() {
+      super();
 
       // Dependency Injection for Jersey resources
       register(
           new AbstractBinder() {
             @Override
             protected void configure() {
-              bindFactory(new InjectionFactories.SingletonFactory<>(solrCore))
+              bindFactory(InjectionFactories.SolrCoreFactory.class)
                   .to(SolrCore.class)
-                  .in(Singleton.class);
+                  .in(RequestScoped.class);
             }
           });
     }
diff --git a/solr/core/src/java/org/apache/solr/jersey/MetricBeanFactory.java b/solr/core/src/java/org/apache/solr/jersey/MetricBeanFactory.java
deleted file mode 100644
index c23851359d9..00000000000
--- a/solr/core/src/java/org/apache/solr/jersey/MetricBeanFactory.java
+++ /dev/null
@@ -1,55 +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.jersey;
-
-import org.apache.solr.core.PluginBag;
-import org.glassfish.hk2.api.Factory;
-
-/**
- * Factory to inject JerseyMetricsLookupRegistry instances into Jersey resources and filters.
- *
- * <p>Currently, Jersey resources that have a corresponding v1 API produce the same metrics as their
- * v1 equivalent and rely on the v1 requestHandler instance to do so. Solr facilitates this by
- * building a map of the Jersey resource to requestHandler mapping (a {@link
- * org.apache.solr.core.PluginBag.JerseyMetricsLookupRegistry}), and injecting it into the pre- and
- * post- Jersey filters that handle metrics.
- *
- * <p>This isn't ideal, as requestHandler's don't really "fit" conceptually here. But it's
- * unavoidable while we want our v2 APIs to exactly match the metrics produced by v1 calls.
- *
- * @see RequestMetricHandling.PreRequestMetricsFilter
- * @see RequestMetricHandling.PostRequestMetricsFilter
- */
-public class MetricBeanFactory implements Factory<PluginBag.JerseyMetricsLookupRegistry> {
-
-  private final PluginBag.JerseyMetricsLookupRegistry metricsLookupRegistry;
-
-  public MetricBeanFactory(PluginBag.JerseyMetricsLookupRegistry metricsLookupRegistry) {
-    this.metricsLookupRegistry = metricsLookupRegistry;
-  }
-
-  @Override
-  public PluginBag.JerseyMetricsLookupRegistry provide() {
-    return metricsLookupRegistry;
-  }
-
-  @Override
-  public void dispose(PluginBag.JerseyMetricsLookupRegistry instance) {
-    /* No-op */
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/jersey/RequestContextKeys.java b/solr/core/src/java/org/apache/solr/jersey/RequestContextKeys.java
index 68b37aa8758..f300ffa4bbb 100644
--- a/solr/core/src/java/org/apache/solr/jersey/RequestContextKeys.java
+++ b/solr/core/src/java/org/apache/solr/jersey/RequestContextKeys.java
@@ -23,6 +23,7 @@ import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.container.ContainerRequestContext;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.PluginBag;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.SolrQueryRequest;
@@ -42,6 +43,7 @@ public interface RequestContextKeys {
   String SOLR_QUERY_REQUEST = SolrQueryRequest.class.getName();
   String SOLR_QUERY_RESPONSE = SolrQueryResponse.class.getName();
   String CORE_CONTAINER = CoreContainer.class.getName();
+  String RESOURCE_TO_RH_MAPPING = PluginBag.JaxrsResourceToHandlerMappings.class.getName();
   String SOLR_CORE = SolrCore.class.getName();
   String REQUEST_TYPE = AuthorizationContext.RequestType.class.getName();
   String SOLR_PARAMS = SolrParams.class.getName();
diff --git a/solr/core/src/java/org/apache/solr/jersey/RequestMetricHandling.java b/solr/core/src/java/org/apache/solr/jersey/RequestMetricHandling.java
index 595027005ad..a6e8b03919e 100644
--- a/solr/core/src/java/org/apache/solr/jersey/RequestMetricHandling.java
+++ b/solr/core/src/java/org/apache/solr/jersey/RequestMetricHandling.java
@@ -24,7 +24,6 @@ import static org.apache.solr.jersey.RequestContextKeys.TIMER;
 import com.codahale.metrics.Timer;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import javax.inject.Inject;
 import javax.ws.rs.container.ContainerRequestContext;
 import javax.ws.rs.container.ContainerRequestFilter;
 import javax.ws.rs.container.ContainerResponseContext;
@@ -40,9 +39,18 @@ import org.slf4j.LoggerFactory;
 /**
  * A request and response filter used to initialize and report per-request metrics.
  *
- * <p>Currently, JAX-RS v2 APIs rely on a {@link
- * org.apache.solr.handler.RequestHandlerBase.HandlerMetrics} instance from an associated request
- * handler.
+ * <p>Currently, Jersey resources that have a corresponding v1 API produce the same metrics as their
+ * v1 equivalent and rely on the v1 requestHandler instance to do so. Solr facilitates this by
+ * building a map of the JAX-RS resources to requestHandler mapping (a {@link
+ * org.apache.solr.core.PluginBag.JaxrsResourceToHandlerMappings}), and using that to look up the
+ * associated request handler (if one exists) in pre- and post- filters
+ *
+ * <p>This isn't ideal, as requestHandler's don't really "fit" conceptually here. But it's
+ * unavoidable while we want our v2 APIs to exactly match the metrics produced by v1 calls, and
+ * while metrics are bundled in with requestHandlers as they are currently.
+ *
+ * @see RequestMetricHandling.PreRequestMetricsFilter
+ * @see RequestMetricHandling.PostRequestMetricsFilter
  */
 public class RequestMetricHandling {
 
@@ -58,16 +66,18 @@ public class RequestMetricHandling {
 
     @Context private ResourceInfo resourceInfo;
 
-    private PluginBag.JerseyMetricsLookupRegistry beanRegistry;
-
-    @Inject
-    public PreRequestMetricsFilter(PluginBag.JerseyMetricsLookupRegistry beanRegistry) {
-      this.beanRegistry = beanRegistry;
-    }
-
     @Override
     public void filter(ContainerRequestContext requestContext) throws IOException {
-      final RequestHandlerBase handlerBase = beanRegistry.get(resourceInfo.getResourceClass());
+      final PluginBag.JaxrsResourceToHandlerMappings requestHandlerByJerseyResource =
+          (PluginBag.JaxrsResourceToHandlerMappings)
+              requestContext.getProperty(RequestContextKeys.RESOURCE_TO_RH_MAPPING);
+      if (requestHandlerByJerseyResource == null) {
+        log.debug("No jax-rs registry found for request {}", requestContext);
+        return;
+      }
+
+      final RequestHandlerBase handlerBase =
+          requestHandlerByJerseyResource.get(resourceInfo.getResourceClass());
       if (handlerBase == null) {
         log.debug("No handler found for request {}", requestContext);
         return;
diff --git a/solr/core/src/test/org/apache/solr/jersey/JerseyApplicationSharingTest.java b/solr/core/src/test/org/apache/solr/jersey/JerseyApplicationSharingTest.java
new file mode 100644
index 00000000000..6319ecb0bb2
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/jersey/JerseyApplicationSharingTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.jersey;
+
+import java.util.Map;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests ensuring that Jersey apps are shared between cores as expected.
+ *
+ * <p>Jersey applications should be shared by any cores on the same node that have the same
+ * "effective configset" (i.e. the same configset content and any overlays or relevant configuration
+ * properties)
+ */
+public class JerseyApplicationSharingTest extends SolrCloudTestCase {
+
+  private static final String collection = "collection1";
+  private static final String confDir = collection + "/conf";
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf1", configset("cloud-minimal"))
+        .addConfig("conf2", configset("cloud-minimal"))
+        .configure();
+  }
+
+  @Test
+  public void testMultipleCoresWithSameConfigsetShareApplication() throws Exception {
+    final SolrClient solrClient = cluster.getSolrClient();
+
+    // No applications should be in the cache to start
+    assertJerseyAppCacheHasSize(0);
+
+    // All replicas for the created collection should share a single Jersey ApplicationHandler entry
+    // in the cache
+    final CollectionAdminRequest.Create coll1Create =
+        CollectionAdminRequest.createCollection("coll1", "conf1", 2, 2);
+    assertEquals(0, coll1Create.process(solrClient).getStatus());
+    assertJerseyAppCacheHasSize(1);
+
+    // A new collection using the same configset will also share the existing cached Jersey
+    // ApplicationHandler
+    final CollectionAdminRequest.Create coll2Create =
+        CollectionAdminRequest.createCollection("coll2", "conf1", 2, 2);
+    assertEquals(0, coll2Create.process(solrClient).getStatus());
+    assertJerseyAppCacheHasSize(1);
+
+    // Using a different configset WILL cause a new Jersey ApplicationHandler to be used (total
+    // cache-count = 2)
+    final CollectionAdminRequest.Create coll3Create =
+        CollectionAdminRequest.createCollection("coll3", "conf2", 2, 2);
+    assertEquals(0, coll3Create.process(solrClient).getStatus());
+    assertJerseyAppCacheHasSize(2);
+
+    // Modifying properties that affect a configset will also cause a new Jersey ApplicationHandler
+    // to be created (total cache-count = 3)
+    final CollectionAdminRequest.Create coll4Create =
+        CollectionAdminRequest.createCollection("coll4", "conf1", 2, 2);
+    coll4Create.setProperties(
+        Map.of(
+            "solr.commitwithin.softcommit",
+            "false")); // Set any collection property used in the cloud-minimal configset
+    assertEquals(0, coll4Create.process(solrClient).getStatus());
+    assertJerseyAppCacheHasSize(3);
+
+    // Deleting the only cores that used given ApplicationHandler will remove it from the cache
+    // (total cache-count = 2)
+    final CollectionAdminRequest.Delete deleteColl3 =
+        CollectionAdminRequest.deleteCollection("coll3");
+    assertEquals(0, deleteColl3.process(solrClient).getStatus());
+    assertJerseyAppCacheHasSize(2);
+  }
+
+  private void assertJerseyAppCacheHasSize(int expectedSize) {
+    assertEquals(
+        expectedSize,
+        cluster.getJettySolrRunners().get(0).getCoreContainer().getAppHandlerCache().size());
+  }
+}