You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2020/05/12 19:44:22 UTC

[lucene-solr] branch master updated: SOLR-14423: Move static SolrClientCache from StreamHandler to CoreContainer for wider reuse and better life-cycle management.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 4680e92  SOLR-14423: Move static SolrClientCache from StreamHandler to CoreContainer for wider reuse and better life-cycle management.
4680e92 is described below

commit 4680e9245f26e8ba99400dfbbdc0002c6b2886fe
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue May 12 21:44:00 2020 +0200

    SOLR-14423: Move static SolrClientCache from StreamHandler to CoreContainer for wider reuse and better life-cycle management.
---
 solr/CHANGES.txt                                   |  2 +
 .../org/apache/solr/handler/AnalyticsHandler.java  |  6 --
 .../java/org/apache/solr/cloud/ZkController.java   |  5 +-
 .../api/collections/ReindexCollectionCmd.java      |  6 +-
 .../java/org/apache/solr/core/CoreContainer.java   | 24 ++++++++
 .../java/org/apache/solr/handler/GraphHandler.java |  5 +-
 .../org/apache/solr/handler/StreamHandler.java     | 28 +++------
 .../org/apache/solr/handler/admin/ColStatus.java   |  5 +-
 .../solr/handler/admin/CollectionsHandler.java     |  2 +-
 .../apache/solr/handler/sql/CalciteSolrDriver.java | 16 ++++-
 .../org/apache/solr/handler/sql/SolrSchema.java    | 62 +++++++++++--------
 .../org/apache/solr/handler/sql/SolrTable.java     |  3 +-
 .../solr/metrics/reporters/solr/SolrReporter.java  | 70 +++++++++++++++++++++-
 .../metrics/reporters/solr/SolrShardReporter.java  |  2 +-
 .../org/apache/solr/search/join/XCJFQuery.java     |  3 +-
 .../client/solrj/impl/SolrClientCloudManager.java  | 18 +++++-
 .../apache/solr/common/util/MapBackedCache.java    |  4 ++
 17 files changed, 187 insertions(+), 74 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 620ef35..a4f2684 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -108,6 +108,8 @@ Improvements
 
 * SOLR-14433: Metrics: SolrShardReporter's default metrics list now includes TLOG and UPDATE./update (David Smiley)
 
+* SOLR-14423: Move static SolrClientCache from StreamHandler to CoreContainer for wider reuse and better life-cycle management. (ab)
+
 Optimizations
 ---------------------
 * SOLR-8306: Do not collect expand documents when expand.rows=0 (Marshall Sanders, Amelia Henderson)
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/handler/AnalyticsHandler.java b/solr/contrib/analytics/src/java/org/apache/solr/handler/AnalyticsHandler.java
index b289d1c..9ffbaf4 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/handler/AnalyticsHandler.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/handler/AnalyticsHandler.java
@@ -28,8 +28,6 @@ import org.apache.solr.analytics.AnalyticsRequestParser;
 import org.apache.solr.analytics.ExpressionFactory;
 import org.apache.solr.analytics.TimeExceededStubException;
 import org.apache.solr.analytics.stream.AnalyticsShardResponseParser;
-import org.apache.solr.client.solrj.io.ModelCache;
-import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
@@ -61,9 +59,6 @@ public class AnalyticsHandler extends RequestHandlerBase implements SolrCoreAwar
   public static final String NAME = "/analytics";
   private IndexSchema indexSchema;
 
-  static SolrClientCache clientCache = new SolrClientCache();
-  static ModelCache modelCache = null;
-
   @Override
   public PermissionNameProvider.Name getPermissionName(AuthorizationContext request) {
     return PermissionNameProvider.Name.READ_PERM;
@@ -72,7 +67,6 @@ public class AnalyticsHandler extends RequestHandlerBase implements SolrCoreAwar
   @Override
   public void inform(SolrCore core) {
     core.registerResponseWriter(AnalyticsShardResponseWriter.NAME, new AnalyticsShardResponseWriter());
-
     indexSchema = core.getLatestSchema();
     AnalyticsRequestParser.init();
   }
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index e60f2e4..ecbb781 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -755,7 +755,10 @@ public class ZkController implements Closeable {
       cloudSolrClient = new CloudSolrClient.Builder(new ZkClientClusterStateProvider(zkStateReader)).withSocketTimeout(30000).withConnectionTimeout(15000)
           .withHttpClient(cc.getUpdateShardHandler().getDefaultHttpClient())
           .withConnectionTimeout(15000).withSocketTimeout(30000).build();
-      cloudManager = new SolrClientCloudManager(new ZkDistributedQueueFactory(zkClient), cloudSolrClient);
+      cloudManager = new SolrClientCloudManager(
+          new ZkDistributedQueueFactory(zkClient),
+          cloudSolrClient,
+          cc.getObjectCache());
       cloudManager.getClusterStateProvider().connect();
     }
     return cloudManager;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
index c362dd3..f70fc5c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
@@ -38,7 +38,6 @@ import org.apache.solr.client.solrj.cloud.DistribStateManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
@@ -163,7 +162,6 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
         Stream.of(Cmd.values()).collect(Collectors.toMap(Cmd::toLower, Function.identity())));
   }
 
-  private SolrClientCache solrClientCache;
   private String zkHost;
 
   public ReindexCollectionCmd(OverseerCollectionMessageHandler ocmh) {
@@ -268,7 +266,6 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
     Exception exc = null;
     boolean createdTarget = false;
     try {
-      solrClientCache = new SolrClientCache(ocmh.overseer.getCoreContainer().getUpdateShardHandler().getDefaultHttpClient());
       zkHost = ocmh.zkStateReader.getZkClient().getZkServerAddress();
       // set the running flag
       reindexingState.clear();
@@ -504,7 +501,6 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
       exc = e;
       aborted = true;
     } finally {
-      solrClientCache.close();
       if (aborted) {
         cleanup(collection, targetCollection, chkCollection, daemonUrl, targetCollection, createdTarget);
         if (exc != null) {
@@ -550,7 +546,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
   }
 
   private long getNumberOfDocs(String collection) {
-    CloudSolrClient solrClient = solrClientCache.getCloudSolrClient(zkHost);
+    CloudSolrClient solrClient = ocmh.overseer.getCoreContainer().getSolrClientCache().getCloudSolrClient(zkHost);
     try {
       ModifiableSolrParams params = new ModifiableSolrParams();
       params.add(CommonParams.Q, "*:*");
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 f332837..308fbf3 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -61,6 +61,7 @@ import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.AuthSchemeRegistryProvider;
 import org.apache.solr.client.solrj.impl.SolrHttpClientContextBuilder.CredentialsProviderProvider;
+import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.Overseer;
@@ -76,6 +77,7 @@ import org.apache.solr.common.cloud.Replica.State;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.common.util.ObjectCache;
 import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.DirectoryFactory.DirContext;
@@ -100,6 +102,7 @@ import org.apache.solr.handler.admin.ZookeeperInfoHandler;
 import org.apache.solr.handler.admin.ZookeeperReadAPI;
 import org.apache.solr.handler.admin.ZookeeperStatusHandler;
 import org.apache.solr.handler.component.ShardHandlerFactory;
+import org.apache.solr.handler.sql.CalciteSolrDriver;
 import org.apache.solr.logging.LogWatcher;
 import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.metrics.SolrCoreMetricManager;
@@ -228,6 +231,10 @@ public class CoreContainer {
 
   protected volatile AutoscalingHistoryHandler autoscalingHistoryHandler;
 
+  private volatile SolrClientCache solrClientCache;
+
+  private volatile ObjectCache objectCache = new ObjectCache();
+
   private PackageStoreAPI packageStoreAPI;
   private PackageLoader packageLoader;
 
@@ -576,6 +583,15 @@ public class CoreContainer {
   public PackageStoreAPI getPackageStoreAPI() {
     return packageStoreAPI;
   }
+
+  public SolrClientCache getSolrClientCache() {
+    return solrClientCache;
+  }
+
+  public ObjectCache getObjectCache() {
+    return objectCache;
+  }
+
   //-------------------------------------------------------------------
   // Initialization / Cleanup
   //-------------------------------------------------------------------
@@ -636,6 +652,11 @@ public class CoreContainer {
     updateShardHandler = new UpdateShardHandler(cfg.getUpdateShardHandlerConfig());
     updateShardHandler.initializeMetrics(solrMetricsContext, "updateShardHandler");
 
+    solrClientCache = new SolrClientCache(updateShardHandler.getDefaultHttpClient());
+
+    // initialize CalciteSolrDriver instance to use this solrClientCache
+    CalciteSolrDriver.INSTANCE.setSolrClientCache(solrClientCache);
+
     solrCores.load(loader);
 
 
@@ -1017,6 +1038,9 @@ public class CoreContainer {
       } catch (Exception e) {
         log.warn("Error shutting down CoreAdminHandler. Continuing to close CoreContainer.", e);
       }
+      if (solrClientCache != null) {
+        solrClientCache.close();
+      }
 
     } finally {
       try {
diff --git a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
index bed4086..22c082f 100644
--- a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
 import org.apache.solr.client.solrj.io.graph.Traversal;
@@ -83,6 +84,7 @@ public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, P
   private StreamFactory streamFactory = new DefaultStreamFactory();
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private String coreName;
+  private SolrClientCache solrClientCache;
 
   @Override
   public PermissionNameProvider.Name getPermissionName(AuthorizationContext request) {
@@ -94,6 +96,7 @@ public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, P
     String defaultZkhost;
     CoreContainer coreContainer = core.getCoreContainer();
     this.coreName = core.getName();
+    this.solrClientCache = coreContainer.getSolrClientCache();
 
     if(coreContainer.isZooKeeperAware()) {
       defaultCollection = core.getCoreDescriptor().getCollectionName();
@@ -147,7 +150,7 @@ public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, P
     }
 
     StreamContext context = new StreamContext();
-    context.setSolrClientCache(StreamHandler.clientCache);
+    context.setSolrClientCache(solrClientCache);
     context.put("core", this.coreName);
     Traversal traversal = new Traversal();
     context.put("traversal", traversal);
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index bd76ae9..8c6af43 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -53,7 +53,6 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.core.CloseHook;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrConfig;
@@ -88,12 +87,12 @@ import static org.apache.solr.common.params.CommonParams.ID;
  */
 public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, PermissionNameProvider {
 
-  static SolrClientCache clientCache = new SolrClientCache();
-  static ModelCache modelCache = null;
-  static ConcurrentMap objectCache = new ConcurrentHashMap();
+  private ModelCache modelCache = null;
+  private ConcurrentMap objectCache = new ConcurrentHashMap();
   private SolrDefaultStreamFactory streamFactory = new SolrDefaultStreamFactory();
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private String coreName;
+  private SolrClientCache solrClientCache;
   private Map<String, DaemonStream> daemons = Collections.synchronizedMap(new HashMap());
 
   @Override
@@ -101,14 +100,15 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
     return PermissionNameProvider.Name.READ_PERM;
   }
 
-  public static SolrClientCache getClientCache() {
-    return clientCache;
+  public SolrClientCache getClientCache() {
+    return solrClientCache;
   }
 
   public void inform(SolrCore core) {
     String defaultCollection;
     String defaultZkhost;
     CoreContainer coreContainer = core.getCoreContainer();
+    this.solrClientCache = coreContainer.getSolrClientCache();
     this.coreName = core.getName();
 
     if (coreContainer.isZooKeeperAware()) {
@@ -118,24 +118,12 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       streamFactory.withDefaultZkHost(defaultZkhost);
       modelCache = new ModelCache(250,
           defaultZkhost,
-          clientCache);
+          solrClientCache);
     }
     streamFactory.withSolrResourceLoader(core.getResourceLoader());
 
     // This pulls all the overrides and additions from the config
     addExpressiblePlugins(streamFactory, core);
-
-    core.addCloseHook(new CloseHook() {
-      @Override
-      public void preClose(SolrCore core) {
-        // To change body of implemented methods use File | Settings | File Templates.
-      }
-
-      @Override
-      public void postClose(SolrCore core) {
-        clientCache.close();
-      }
-    });
   }
 
   public static void addExpressiblePlugins(StreamFactory streamFactory, SolrCore core) {
@@ -226,7 +214,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
     context.put("shards", getCollectionShards(params));
     context.workerID = worker;
     context.numWorkers = numWorkers;
-    context.setSolrClientCache(clientCache);
+    context.setSolrClientCache(solrClientCache);
     context.setModelCache(modelCache);
     context.setObjectCache(objectCache);
     context.put("core", this.coreName);
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ColStatus.java b/solr/core/src/java/org/apache/solr/handler/admin/ColStatus.java
index 609b39b..aee0097 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ColStatus.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ColStatus.java
@@ -27,7 +27,6 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
-import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.io.SolrClientCache;
@@ -66,9 +65,9 @@ public class ColStatus {
   public static final String RAW_SIZE_SAMPLING_PERCENT_PROP = SegmentsInfoRequestHandler.RAW_SIZE_SAMPLING_PERCENT_PARAM;
   public static final String SEGMENTS_PROP = "segments";
 
-  public ColStatus(HttpClient httpClient, ClusterState clusterState, ZkNodeProps props) {
+  public ColStatus(SolrClientCache solrClientCache, ClusterState clusterState, ZkNodeProps props) {
     this.props = props;
-    this.solrClientCache = new SolrClientCache(httpClient);
+    this.solrClientCache = solrClientCache;
     this.clusterState = clusterState;
   }
 
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index d63b48f..203efe5 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -529,7 +529,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       if (props.containsKey(CoreAdminParams.NAME) && !props.containsKey(COLLECTION_PROP)) {
         props.put(COLLECTION_PROP, props.get(CoreAdminParams.NAME));
       }
-      new ColStatus(h.coreContainer.getUpdateShardHandler().getDefaultHttpClient(),
+      new ColStatus(h.coreContainer.getSolrClientCache(),
           h.coreContainer.getZkController().getZkStateReader().getClusterState(), new ZkNodeProps(props))
           .getColStatus(rsp.getValues());
       return null;
diff --git a/solr/core/src/java/org/apache/solr/handler/sql/CalciteSolrDriver.java b/solr/core/src/java/org/apache/solr/handler/sql/CalciteSolrDriver.java
index 3a7640d..664cb8a 100644
--- a/solr/core/src/java/org/apache/solr/handler/sql/CalciteSolrDriver.java
+++ b/solr/core/src/java/org/apache/solr/handler/sql/CalciteSolrDriver.java
@@ -19,6 +19,7 @@ package org.apache.solr.handler.sql;
 import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.jdbc.Driver;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.solr.client.solrj.io.SolrClientCache;
 
 import java.sql.Connection;
 import java.sql.SQLException;
@@ -32,12 +33,17 @@ import java.util.Properties;
 public class CalciteSolrDriver extends Driver {
   public final static String CONNECT_STRING_PREFIX = "jdbc:calcitesolr:";
 
+  public static CalciteSolrDriver INSTANCE = new CalciteSolrDriver();
+
+  private SolrClientCache solrClientCache;
+
+
   private CalciteSolrDriver() {
     super();
   }
 
   static {
-    new CalciteSolrDriver().register();
+    INSTANCE.register();
   }
 
   @Override
@@ -59,11 +65,15 @@ public class CalciteSolrDriver extends Driver {
     if(schemaName == null) {
       throw new SQLException("zk must be set");
     }
-    rootSchema.add(schemaName, new SolrSchema(info));
+    final SolrSchema solrSchema = new SolrSchema(info, solrClientCache);
+    rootSchema.add(schemaName, solrSchema);
 
     // Set the default schema
     calciteConnection.setSchema(schemaName);
+    return calciteConnection;
+  }
 
-    return connection;
+  public void setSolrClientCache(SolrClientCache solrClientCache) {
+    this.solrClientCache = solrClientCache;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
index b608442..3bf5bd4 100644
--- a/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
+++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java
@@ -16,10 +16,9 @@
  */
 package org.apache.solr.handler.sql;
 
+import java.io.Closeable;
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
 
@@ -33,6 +32,7 @@ import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.request.LukeRequest;
 import org.apache.solr.client.solrj.response.LukeResponse;
 import org.apache.solr.common.cloud.Aliases;
@@ -41,47 +41,59 @@ import org.apache.solr.common.cloud.ZkStateReader;
 
 import com.google.common.collect.ImmutableMap;
 
-class SolrSchema extends AbstractSchema {
+class SolrSchema extends AbstractSchema implements Closeable {
   final Properties properties;
+  final SolrClientCache solrClientCache;
+  private volatile boolean isClosed = false;
 
-  SolrSchema(Properties properties) {
+  SolrSchema(Properties properties, SolrClientCache solrClientCache) {
     super();
     this.properties = properties;
+    this.solrClientCache = solrClientCache;
+  }
+
+  public SolrClientCache getSolrClientCache() {
+    return solrClientCache;
+  }
+
+  @Override
+  public void close() {
+    isClosed = true;
+  }
+
+  public boolean isClosed() {
+    return isClosed;
   }
 
   @Override
   protected Map<String, Table> getTableMap() {
     String zk = this.properties.getProperty("zk");
-    try(CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder(Collections.singletonList(zk), Optional.empty()).withSocketTimeout(30000).withConnectionTimeout(15000).build()) {
-      cloudSolrClient.connect();
-      ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
-      ClusterState clusterState = zkStateReader.getClusterState();
+    CloudSolrClient cloudSolrClient = solrClientCache.getCloudSolrClient(zk);
+    ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
+    ClusterState clusterState = zkStateReader.getClusterState();
 
-      final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
+    final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
 
-      Set<String> collections = clusterState.getCollectionsMap().keySet();
-      for (String collection : collections) {
-        builder.put(collection, new SolrTable(this, collection));
-      }
+    Set<String> collections = clusterState.getCollectionsMap().keySet();
+    for (String collection : collections) {
+      builder.put(collection, new SolrTable(this, collection));
+    }
 
-      Aliases aliases = zkStateReader.getAliases();
-      for (String alias : aliases.getCollectionAliasListMap().keySet()) {
-        // don't create duplicate entries
-        if (!collections.contains(alias)) {
-          builder.put(alias, new SolrTable(this, alias));
-        }
+    Aliases aliases = zkStateReader.getAliases();
+    for (String alias : aliases.getCollectionAliasListMap().keySet()) {
+      // don't create duplicate entries
+      if (!collections.contains(alias)) {
+        builder.put(alias, new SolrTable(this, alias));
       }
-
-      return builder.build();
-    } catch (IOException e) {
-      throw new RuntimeException(e);
     }
+
+    return builder.build();
   }
 
   private Map<String, LukeResponse.FieldInfo> getFieldInfo(String collection) {
     String zk = this.properties.getProperty("zk");
-    try(CloudSolrClient cloudSolrClient = new CloudSolrClient.Builder(Collections.singletonList(zk), Optional.empty()).withSocketTimeout(30000).withConnectionTimeout(15000).build()) {
-      cloudSolrClient.connect();
+    CloudSolrClient cloudSolrClient = solrClientCache.getCloudSolrClient(zk);
+    try {
       LukeRequest lukeRequest = new LukeRequest();
       lukeRequest.setNumTerms(0);
       LukeResponse lukeResponse = lukeRequest.process(cloudSolrClient, collection);
diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java
index 46b09d2..66ee312 100644
--- a/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java
+++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java
@@ -51,7 +51,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.client.solrj.io.stream.metrics.*;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.handler.StreamHandler;
 
 import java.io.IOException;
 import java.util.*;
@@ -160,7 +159,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
     }
 
     StreamContext streamContext = new StreamContext();
-    streamContext.setSolrClientCache(StreamHandler.getClientCache());
+    streamContext.setSolrClientCache(schema.getSolrClientCache());
     tupleStream.setStreamContext(streamContext);
 
     final TupleStream finalStream = tupleStream;
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
index 8df6817..c126e73 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrReporter.java
@@ -257,18 +257,33 @@ public class SolrReporter extends ScheduledReporter {
      *                    null to indicate that reporting should be skipped. Note: this
      *                    function will be called every time just before report is sent.
      * @return configured instance of reporter
+     * @deprecated use {@link #build(SolrClientCache, Supplier)} instead.
      */
     public SolrReporter build(HttpClient client, Supplier<String> urlProvider) {
       return new SolrReporter(client, urlProvider, metricManager, reports, handler, reporterId, rateUnit, durationUnit,
           params, skipHistograms, skipAggregateValues, cloudClient, compact);
     }
 
+    /**
+     * Build it.
+     * @param solrClientCache an instance of {@link SolrClientCache} to be used for making calls.
+     * @param urlProvider function that returns the base URL of Solr instance to target. May return
+     *                    null to indicate that reporting should be skipped. Note: this
+     *                    function will be called every time just before report is sent.
+     * @return configured instance of reporter
+     */
+    public SolrReporter build(SolrClientCache solrClientCache, Supplier<String> urlProvider) {
+      return new SolrReporter(solrClientCache, false, urlProvider, metricManager, reports, handler, reporterId, rateUnit, durationUnit,
+          params, skipHistograms, skipAggregateValues, cloudClient, compact);
+    }
+
   }
 
   private String reporterId;
   private String handler;
   private Supplier<String> urlProvider;
   private SolrClientCache clientCache;
+  private boolean closeClientCache;
   private List<CompiledReport> compiledReports;
   private SolrMetricManager metricManager;
   private boolean skipHistograms;
@@ -306,11 +321,59 @@ public class SolrReporter extends ScheduledReporter {
   // We delegate to registries anyway, so having a dummy registry is harmless.
   private static final MetricRegistry dummyRegistry = new MetricRegistry();
 
+  // back-compat constructor
+
+  /**
+   * Create a SolrReporter instance.
+   * @param httpClient HttpClient to use for constructing SolrClient instances.
+   * @param urlProvider what URL to send to.
+   * @param metricManager metric manager
+   * @param metrics metric specifications to report
+   * @param handler handler name to report to
+   * @param reporterId my reporter id
+   * @param rateUnit rate unit
+   * @param durationUnit duration unit
+   * @param params request parameters
+   * @param skipHistograms if true then don't send histogram metrics
+   * @param skipAggregateValues if true then don't send aggregate metrics' individual values
+   * @param cloudClient if true then use CloudSolrClient, plain HttpSolrClient otherwise.
+   * @param compact if true then use compact representation.
+   *
+   * @deprecated use {@link SolrReporter#SolrReporter(SolrClientCache, boolean, Supplier, SolrMetricManager, List, String, String, TimeUnit, TimeUnit, SolrParams, boolean, boolean, boolean, boolean)} instead.
+   */
+  @Deprecated(since = "8.6.0")
   public SolrReporter(HttpClient httpClient, Supplier<String> urlProvider, SolrMetricManager metricManager,
                       List<Report> metrics, String handler,
                       String reporterId, TimeUnit rateUnit, TimeUnit durationUnit,
                       SolrParams params, boolean skipHistograms, boolean skipAggregateValues,
                       boolean cloudClient, boolean compact) {
+    this (new SolrClientCache(httpClient), true, urlProvider, metricManager,
+        metrics, handler, reporterId, rateUnit, durationUnit,
+        params, skipHistograms, skipAggregateValues, cloudClient, compact);
+  }
+
+  /**
+   * Create a SolrReporter instance.
+   * @param solrClientCache client cache to use for constructing SolrClient instances.
+   * @param urlProvider what URL to send to.
+   * @param metricManager metric manager
+   * @param metrics metric specifications to report
+   * @param handler handler name to report to
+   * @param reporterId my reporter id
+   * @param rateUnit rate unit
+   * @param durationUnit duration unit
+   * @param params request parameters
+   * @param skipHistograms if true then don't send histogram metrics
+   * @param skipAggregateValues if true then don't send aggregate metrics' individual values
+   * @param cloudClient if true then use CloudSolrClient, plain HttpSolrClient otherwise.
+   * @param compact if true then use compact representation.
+   */
+  public SolrReporter(SolrClientCache solrClientCache, boolean closeClientCache,
+                      Supplier<String> urlProvider, SolrMetricManager metricManager,
+                      List<Report> metrics, String handler,
+                      String reporterId, TimeUnit rateUnit, TimeUnit durationUnit,
+                      SolrParams params, boolean skipHistograms, boolean skipAggregateValues,
+                      boolean cloudClient, boolean compact) {
     super(dummyRegistry, "solr-reporter", MetricFilter.ALL, rateUnit, durationUnit, null, true);
 
     this.metricManager = metricManager;
@@ -320,7 +383,8 @@ public class SolrReporter extends ScheduledReporter {
       handler = MetricsCollectorHandler.HANDLER_PATH;
     }
     this.handler = handler;
-    this.clientCache = new SolrClientCache(httpClient);
+    this.clientCache = solrClientCache;
+    this.closeClientCache = closeClientCache;
     this.compiledReports = new ArrayList<>();
     metrics.forEach(report -> {
       MetricFilter filter = new SolrMetricManager.RegexFilter(report.metricFilters);
@@ -347,7 +411,9 @@ public class SolrReporter extends ScheduledReporter {
 
   @Override
   public void close() {
-    clientCache.close();
+    if (closeClientCache) {
+      clientCache.close();
+    }
     super.close();
   }
 
diff --git a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
index 8609a23..7472af9 100644
--- a/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
+++ b/solr/core/src/java/org/apache/solr/metrics/reporters/solr/SolrShardReporter.java
@@ -154,7 +154,7 @@ public class SolrShardReporter extends SolrCoreReporter {
         .cloudClient(false) // we want to send reports specifically to a selected leader instance
         .skipAggregateValues(true) // we don't want to transport details of aggregates
         .skipHistograms(true) // we don't want to transport histograms
-        .build(core.getCoreContainer().getUpdateShardHandler().getDefaultHttpClient(), new LeaderUrlSupplier(core));
+        .build(core.getCoreContainer().getSolrClientCache(), new LeaderUrlSupplier(core));
 
     reporter.start(period, TimeUnit.SECONDS);
   }
diff --git a/solr/core/src/java/org/apache/solr/search/join/XCJFQuery.java b/solr/core/src/java/org/apache/solr/search/join/XCJFQuery.java
index f5c464b..fbe12e8 100644
--- a/solr/core/src/java/org/apache/solr/search/join/XCJFQuery.java
+++ b/solr/core/src/java/org/apache/solr/search/join/XCJFQuery.java
@@ -261,7 +261,7 @@ public class XCJFQuery extends Query {
     }
 
     private DocSet getDocSet() throws IOException {
-      SolrClientCache solrClientCache = new SolrClientCache();
+      SolrClientCache solrClientCache = searcher.getCore().getCoreContainer().getSolrClientCache();
       TupleStream solrStream;
       if (zkHost != null || solrUrl == null) {
         solrStream = createCloudSolrStream(solrClientCache);
@@ -299,7 +299,6 @@ public class XCJFQuery extends Query {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
       } finally {
         solrStream.close();
-        solrClientCache.close();
       }
 
       return collector.getDocSet();
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
index fcefc2f..7aacd42 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientCloudManager.java
@@ -61,22 +61,36 @@ public class SolrClientCloudManager implements SolrCloudManager {
   private final ZkStateReader zkStateReader;
   private final SolrZkClient zkClient;
   private final ObjectCache objectCache;
+  private final boolean closeObjectCache;
   private volatile boolean isClosed;
 
   public SolrClientCloudManager(DistributedQueueFactory queueFactory, CloudSolrClient solrClient) {
+    this(queueFactory, solrClient, null);
+  }
+
+  public SolrClientCloudManager(DistributedQueueFactory queueFactory, CloudSolrClient solrClient,
+                                ObjectCache objectCache) {
     this.queueFactory = queueFactory;
     this.solrClient = solrClient;
     this.zkStateReader = solrClient.getZkStateReader();
     this.zkClient = zkStateReader.getZkClient();
     this.stateManager = new ZkDistribStateManager(zkClient);
     this.isClosed = false;
-    this.objectCache = new ObjectCache();
+    if (objectCache == null) {
+      this.objectCache = new ObjectCache();
+      closeObjectCache = true;
+    } else {
+      this.objectCache = objectCache;
+      this.closeObjectCache = false;
+    }
   }
 
   @Override
   public void close() {
     isClosed = true;
-    IOUtils.closeQuietly(objectCache);
+    if (closeObjectCache) {
+      IOUtils.closeQuietly(objectCache);
+    }
   }
 
   @Override
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/MapBackedCache.java b/solr/solrj/src/java/org/apache/solr/common/util/MapBackedCache.java
index 552dba6..3efdfe3 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/MapBackedCache.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/MapBackedCache.java
@@ -30,6 +30,10 @@ public class MapBackedCache<K, V> implements Cache<K, V> {
     this.map = map;
   }
 
+  public Map<K, V> asMap() {
+    return map;
+  }
+
   @Override
   public V put(K key, V val) {
     return map.put(key, val);