You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mk...@apache.org on 2016/10/07 09:08:29 UTC

lucene-solr:branch_6x: SOLR-9604, SOLR-9608: Ensure SSL connections are re-used.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x b1a5c5f92 -> b58ccc390


SOLR-9604,SOLR-9608: Ensure SSL connections are re-used. 

Fix ConnectionReuseTest. Add coverage for all SolrClients.
Remove explicit cacheKey for HttpRequestContext, make it singleton.   
  


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/b58ccc39
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/b58ccc39
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/b58ccc39

Branch: refs/heads/branch_6x
Commit: b58ccc3906014fb13ecffe17ae989ea7d07b814e
Parents: b1a5c5f
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Oct 5 10:22:22 2016 +0100
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Fri Oct 7 12:03:07 2016 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../solr/security/PKIAuthenticationPlugin.java  |   5 +-
 .../org/apache/solr/servlet/HttpSolrCall.java   |   4 +-
 .../src/java/org/apache/solr/util/SolrCLI.java  |   3 +-
 .../solr/client/solrj/ConnectionReuseTest.java  | 216 ------------------
 .../client/solrj/impl/ConnectionReuseTest.java  | 224 +++++++++++++++++++
 .../security/TestAuthorizationFramework.java    |   3 +-
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |   3 +-
 .../solr/client/solrj/impl/HttpClientUtil.java  |  28 ++-
 .../solr/client/solrj/impl/HttpSolrClient.java  |   7 +-
 .../client/solrj/SolrSchemalessExampleTest.java |  21 +-
 .../client/solrj/embedded/JettyWebappTest.java  |   3 +-
 .../solrj/embedded/SolrExampleJettyTest.java    |   9 +-
 .../solrj/impl/BasicHttpSolrClientTest.java     |   2 +-
 .../solrj/impl/HttpSolrClientConPoolTest.java   | 181 +++++++++++++++
 .../impl/HttpSolrClientSSLAuthConPoolTest.java  |  39 ++++
 .../org/apache/solr/util/RestTestHarness.java   |  16 +-
 17 files changed, 518 insertions(+), 249 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9c132c4..be9d2ba 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -129,6 +129,9 @@ Bug Fixes
 
 * SOLR-9278: Index replication interactions with IndexWriter can cause deadlock. (Xunlong via Mark Miller)
 
+* SOLR-9604: Pooled SSL connections were not being re-used (Alan Woodward,
+  Mikhail Khludnev, hossman)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
index 077a21c..c010350 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -40,6 +40,7 @@ import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.http.protocol.HttpContext;
 import org.apache.http.util.EntityUtils;
 import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.ExecutorUtil;
@@ -58,7 +59,6 @@ import org.slf4j.LoggerFactory;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-
 public class PKIAuthenticationPlugin extends AuthenticationPlugin implements HttpClientInterceptorPlugin {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final Map<String, PublicKey> keyCache = new ConcurrentHashMap<>();
@@ -198,7 +198,8 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     try {
       String uri = url + PATH + "?wt=json&omitHeader=true";
       log.debug("Fetching fresh public key from : {}",uri);
-      HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient().execute(new HttpGet(uri));
+      HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient()
+          .execute(new HttpGet(uri), HttpClientUtil.createNewHttpClientRequestContext());
       byte[] bytes = EntityUtils.toByteArray(rsp.getEntity());
       Map m = (Map) Utils.fromJSON(bytes);
       String key = (String) m.get("key");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 9189e62..c41595e 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -57,6 +57,7 @@ import org.apache.http.client.methods.HttpPut;
 import org.apache.http.client.methods.HttpRequestBase;
 import org.apache.http.entity.InputStreamEntity;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Aliases;
@@ -557,7 +558,8 @@ public class HttpSolrCall {
         method.removeHeaders(CONTENT_LENGTH_HEADER);
       }
 
-      final HttpResponse response = solrDispatchFilter.httpClient.execute(method);
+      final HttpResponse response
+          = solrDispatchFilter.httpClient.execute(method, HttpClientUtil.createNewHttpClientRequestContext());
       int httpStatus = response.getStatusLine().getStatusCode();
       httpEntity = response.getEntity();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/core/src/java/org/apache/solr/util/SolrCLI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
index ff5d483..574911c 100644
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -649,7 +649,8 @@ public class SolrCLI {
     // ensure we're requesting JSON back from Solr
     HttpGet httpGet = new HttpGet(new URIBuilder(getUrl).setParameter(CommonParams.WT, CommonParams.JSON).build());
     // make the request and get back a parsed JSON object
-    Map<String,Object> json = httpClient.execute(httpGet, new SolrResponseHandler());
+    Map<String,Object> json = httpClient.execute(httpGet, new SolrResponseHandler(), 
+                                                HttpClientUtil.createNewHttpClientRequestContext());
     // check the response JSON from Solr to see if it is an error
     Long statusCode = asLong("/responseHeader/status", json);
     if (statusCode == -1) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/core/src/test/org/apache/solr/client/solrj/ConnectionReuseTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/client/solrj/ConnectionReuseTest.java b/solr/core/src/test/org/apache/solr/client/solrj/ConnectionReuseTest.java
deleted file mode 100644
index 77d4990..0000000
--- a/solr/core/src/test/org/apache/solr/client/solrj/ConnectionReuseTest.java
+++ /dev/null
@@ -1,216 +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.client.solrj;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.net.URL;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.http.HttpConnectionMetrics;
-import org.apache.http.HttpException;
-import org.apache.http.HttpHost;
-import org.apache.http.HttpRequest;
-import org.apache.http.HttpVersion;
-import org.apache.http.client.HttpClient;
-import org.apache.http.conn.ClientConnectionRequest;
-import org.apache.http.conn.ConnectionPoolTimeoutException;
-import org.apache.http.conn.ManagedClientConnection;
-import org.apache.http.conn.routing.HttpRoute;
-import org.apache.http.impl.conn.PoolingClientConnectionManager;
-import org.apache.http.message.BasicHttpRequest;
-import org.apache.http.params.BasicHttpParams;
-import org.apache.http.params.HttpProtocolParams;
-import org.apache.http.protocol.BasicHttpContext;
-import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
-import org.apache.solr.client.solrj.impl.HttpClientUtil;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.cloud.SolrCloudTestCase;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.update.AddUpdateCommand;
-import org.apache.solr.util.TestInjection;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@SuppressSSL
-public class ConnectionReuseTest extends SolrCloudTestCase {
-  
-  private AtomicInteger id = new AtomicInteger();
-
-  private static final String COLLECTION = "collection1";
-
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    TestInjection.failUpdateRequests = "true:100";
-    configureCluster(1)
-        .addConfig("config", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
-        .configure();
-
-    CollectionAdminRequest.createCollection(COLLECTION, "config", 1, 1)
-        .processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
-
-    cluster.getSolrClient().waitForState(COLLECTION, DEFAULT_TIMEOUT, TimeUnit.SECONDS,
-        (n, c) -> DocCollection.isFullyActive(n, c, 1, 1));
-  }
-
-  private SolrClient buildClient(HttpClient httpClient, URL url) {
-    switch (random().nextInt(3)) {
-      case 0:
-        // currently only testing with 1 thread
-        return new ConcurrentUpdateSolrClient(url.toString() + "/" + COLLECTION, httpClient, 6, 1) {
-          @Override
-          public void handleError(Throwable ex) {
-            // we're expecting random errors here, don't spam the logs
-          }
-        };
-      case 1:
-        return getHttpSolrClient(url.toString() + "/" + COLLECTION, httpClient);
-      case 2:
-        CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress(), random().nextBoolean(), httpClient);
-        client.setParallelUpdates(random().nextBoolean());
-        client.setDefaultCollection(COLLECTION);
-        client.getLbClient().setConnectionTimeout(30000);
-        client.getLbClient().setSoTimeout(60000);
-        return client;
-    }
-    throw new RuntimeException("impossible");
-  }
-  
-  @Test
-  public void testConnectionReuse() throws Exception {
-
-    URL url = cluster.getJettySolrRunners().get(0).getBaseUrl();
-
-    HttpClient httpClient = HttpClientUtil.createClient(null);
-    PoolingClientConnectionManager cm = (PoolingClientConnectionManager) httpClient.getConnectionManager();
-
-    try (SolrClient client = buildClient(httpClient, url)) {
-
-      log.info("Using client of type {}", client.getClass());
-
-      HttpHost target = new HttpHost(url.getHost(), url.getPort(), isSSLMode() ? "https" : "http");
-      HttpRoute route = new HttpRoute(target);
-
-      ClientConnectionRequest mConn = getClientConnectionRequest(httpClient, route);
-
-      ManagedClientConnection conn1 = getConn(mConn);
-      headerRequest(target, route, conn1);
-      conn1.releaseConnection();
-      cm.releaseConnection(conn1, -1, TimeUnit.MILLISECONDS);
-
-      int queueBreaks = 0;
-      int cnt1 = atLeast(3);
-      int cnt2 = atLeast(30);
-      for (int j = 0; j < cnt1; j++) {
-        boolean done = false;
-        for (int i = 0; i < cnt2; i++) {
-          AddUpdateCommand c = new AddUpdateCommand(null);
-          c.solrDoc = sdoc("id", id.incrementAndGet());
-          try {
-            client.add(c.solrDoc);
-            log.info("Added document");
-          } catch (Exception e) {
-            log.info("Error adding document: {}", e.getMessage());
-          }
-          if (!done && i > 0 && i < cnt2 - 1 && client instanceof ConcurrentUpdateSolrClient
-              && random().nextInt(10) > 8) {
-            log.info("Pausing - should start a new request");
-            queueBreaks++;
-            done = true;
-            Thread.sleep(350); // wait past streaming client poll time of 250ms
-          }
-        }
-      }
-      if (client instanceof ConcurrentUpdateSolrClient) {
-        ((ConcurrentUpdateSolrClient) client).blockUntilFinished();
-      }
-
-
-      route = new HttpRoute(new HttpHost(url.getHost(), url.getPort(), isSSLMode() ? "https" : "http"));
-
-      mConn = cm.requestConnection(route, null);
-
-      ManagedClientConnection conn2 = getConn(mConn);
-
-      HttpConnectionMetrics metrics = conn2.getMetrics();
-      headerRequest(target, route, conn2);
-      conn2.releaseConnection();
-      cm.releaseConnection(conn2, -1, TimeUnit.MILLISECONDS);
-
-
-      assertNotNull("No connection metrics found - is the connection getting aborted? server closing the connection? " + client.getClass().getSimpleName(), metrics);
-
-      // we try and make sure the connection we get has handled all of the requests in this test
-      if (client instanceof ConcurrentUpdateSolrClient) {
-        // we can't fully control queue polling breaking up requests - allow a bit of leeway
-        log.info("Outer loop count: {}", cnt1);
-        log.info("Queue breaks: {}", queueBreaks);
-        int exp = queueBreaks + 3;
-        log.info("Expected: {}", exp);
-        log.info("Requests: {}", metrics.getRequestCount());
-        assertTrue(
-            "We expected all communication via streaming client to use one connection! expected=" + exp + " got="
-                + metrics.getRequestCount(),
-            Math.max(exp, metrics.getRequestCount()) - Math.min(exp, metrics.getRequestCount()) < 3);
-      } else {
-        log.info("Outer loop count: {}", cnt1);
-        log.info("Inner loop count: {}", cnt2);
-        assertTrue("We expected all communication to use one connection! " + client.getClass().getSimpleName(),
-            cnt1 * cnt2 + 2 <= metrics.getRequestCount());
-      }
-    }
-    finally {
-      HttpClientUtil.close(httpClient);
-    }
-
-  }
-
-  public ManagedClientConnection getConn(ClientConnectionRequest mConn)
-      throws InterruptedException, ConnectionPoolTimeoutException {
-    ManagedClientConnection conn = mConn.getConnection(30, TimeUnit.SECONDS);
-    conn.setIdleDuration(-1, TimeUnit.MILLISECONDS);
-    conn.markReusable();
-    return conn;
-  }
-
-  public void headerRequest(HttpHost target, HttpRoute route, ManagedClientConnection conn)
-      throws IOException, HttpException {
-    HttpRequest req = new BasicHttpRequest("OPTIONS", "*", HttpVersion.HTTP_1_1);
-
-    req.addHeader("Host", target.getHostName());
-    BasicHttpParams p = new BasicHttpParams();
-    HttpProtocolParams.setVersion(p, HttpVersion.HTTP_1_1);
-    if (!conn.isOpen()) conn.open(route, new BasicHttpContext(null), p);
-    conn.sendRequestHeader(req);
-    conn.flush();
-    conn.receiveResponseHeader();
-  }
-
-  public ClientConnectionRequest getClientConnectionRequest(HttpClient httpClient, HttpRoute route) {
-    ClientConnectionRequest mConn = ((PoolingClientConnectionManager) httpClient.getConnectionManager()).requestConnection(route, null);
-    return mConn;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java b/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
new file mode 100644
index 0000000..f08f86f
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/client/solrj/impl/ConnectionReuseTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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.client.solrj.impl;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.http.HttpConnectionMetrics;
+import org.apache.http.HttpException;
+import org.apache.http.HttpHost;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpVersion;
+import org.apache.http.client.HttpClient;
+import org.apache.http.conn.ClientConnectionRequest;
+import org.apache.http.conn.ConnectionPoolTimeoutException;
+import org.apache.http.conn.ManagedClientConnection;
+import org.apache.http.conn.routing.HttpRoute;
+import org.apache.http.impl.conn.PoolingClientConnectionManager;
+import org.apache.http.message.BasicHttpRequest;
+import org.apache.http.params.BasicHttpParams;
+import org.apache.http.params.HttpProtocolParams;
+import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.util.TestInjection;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressSSL
+public class ConnectionReuseTest extends SolrCloudTestCase {
+  
+  private AtomicInteger id = new AtomicInteger();
+
+  private static final String COLLECTION = "collection1";
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    TestInjection.failUpdateRequests = "true:100";
+    configureCluster(1)
+        .addConfig("config", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+
+    CollectionAdminRequest.createCollection(COLLECTION, "config", 1, 1)
+        .processAndWait(cluster.getSolrClient(), DEFAULT_TIMEOUT);
+
+    cluster.getSolrClient().waitForState(COLLECTION, DEFAULT_TIMEOUT, TimeUnit.SECONDS,
+        (n, c) -> DocCollection.isFullyActive(n, c, 1, 1));
+  }
+
+  private SolrClient buildClient(HttpClient httpClient, URL url) {
+    switch (random().nextInt(3)) {
+      case 0:
+        // currently only testing with 1 thread
+        return new ConcurrentUpdateSolrClient(url.toString() + "/" + COLLECTION, httpClient, 6, 1) {
+          @Override
+          public void handleError(Throwable ex) {
+            // we're expecting random errors here, don't spam the logs
+          }
+        };
+      case 1:
+        return getHttpSolrClient(url.toString() + "/" + COLLECTION, httpClient);
+      case 2:
+        CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress(), random().nextBoolean(), httpClient);
+        client.setParallelUpdates(random().nextBoolean());
+        client.setDefaultCollection(COLLECTION);
+        client.getLbClient().setConnectionTimeout(30000);
+        client.getLbClient().setSoTimeout(60000);
+        return client;
+    }
+    throw new RuntimeException("impossible");
+  }
+  
+  @Test
+  public void testConnectionReuse() throws Exception {
+
+    URL url = cluster.getJettySolrRunners().get(0).getBaseUrl();
+
+    HttpClient httpClient = HttpClientUtil.createClient(null);
+    PoolingClientConnectionManager cm = (PoolingClientConnectionManager) httpClient.getConnectionManager();
+
+    try (SolrClient client = buildClient(httpClient, url)) {
+
+      log.info("Using client of type {}", client.getClass());
+
+      HttpHost target = new HttpHost(url.getHost(), url.getPort(), isSSLMode() ? "https" : "http");
+      HttpRoute route = new HttpRoute(target);
+
+      ClientConnectionRequest mConn = getClientConnectionRequest(httpClient, route);
+
+      ManagedClientConnection conn1 = getConn(mConn);
+      headerRequest(target, route, conn1);
+      conn1.releaseConnection();
+      cm.releaseConnection(conn1, -1, TimeUnit.MILLISECONDS);
+
+      int queueBreaks = 0;
+      int cnt1 = atLeast(3);
+      int cnt2 = atLeast(30);
+      for (int j = 0; j < cnt1; j++) {
+        boolean done = false;
+        for (int i = 0; i < cnt2; i++) {
+          AddUpdateCommand c = new AddUpdateCommand(null);
+          c.solrDoc = sdoc("id", id.incrementAndGet());
+          try {
+            client.add(c.solrDoc);
+            log.info("Added document");
+          } catch (Exception e) {
+            log.info("Error adding document: {}", e.getMessage());
+          }
+          if (!done && i > 0 && i < cnt2 - 1 && client instanceof ConcurrentUpdateSolrClient
+              && random().nextInt(10) > 8) {
+            log.info("Pausing - should start a new request");
+            queueBreaks++;
+            done = true;
+            Thread.sleep(350); // wait past streaming client poll time of 250ms
+          }
+        }
+      }
+      if (client instanceof ConcurrentUpdateSolrClient) {
+        ((ConcurrentUpdateSolrClient) client).blockUntilFinished();
+      }
+
+
+      route = new HttpRoute(new HttpHost(url.getHost(), url.getPort(), isSSLMode() ? "https" : "http"));
+
+      mConn = cm.requestConnection(route, HttpSolrClient.cacheKey 
+          );
+
+      ManagedClientConnection conn2 = getConn(mConn);
+
+      headerRequest(target, route, conn2);
+      HttpConnectionMetrics metrics = conn2.getMetrics();
+      conn2.releaseConnection();
+      cm.releaseConnection(conn2, -1, TimeUnit.MILLISECONDS);
+
+
+      assertNotNull("No connection metrics found - is the connection getting aborted? server closing the connection? " +
+         client.getClass().getSimpleName(), metrics);
+
+      // we try and make sure the connection we get has handled all of the requests in this test
+      final long requestCount = metrics.getRequestCount();
+     // System.out.println(cm.getTotalStats() + " " + cm );
+      if (client instanceof ConcurrentUpdateSolrClient) {
+        // we can't fully control queue polling breaking up requests - allow a bit of leeway
+        log.info("Outer loop count: {}", cnt1);
+        log.info("Queue breaks: {}", queueBreaks);
+        int exp = queueBreaks + 3;
+        log.info("Expected: {}", exp);
+        log.info("Requests: {}", requestCount);
+        assertTrue(
+            "We expected all communication via streaming client to use one connection! expected=" + exp + " got="
+                + requestCount + " "+ client.getClass().getSimpleName(), 
+            Math.max(exp, requestCount) - Math.min(exp, requestCount) < 3);
+      } else {
+        log.info("Outer loop count: {}", cnt1);
+        log.info("Inner loop count: {}", cnt2);
+        assertTrue("We expected all communication to use one connection! " + client.getClass().getSimpleName()
+            +" "+cnt1+" "+ cnt2+ " "+ requestCount,
+            cnt1 * cnt2 + 2 <= requestCount);
+      }
+    }
+    finally {
+      HttpClientUtil.close(httpClient);
+    }
+
+  }
+
+  public ManagedClientConnection getConn(ClientConnectionRequest mConn)
+      throws InterruptedException, ConnectionPoolTimeoutException {
+    ManagedClientConnection conn = mConn.getConnection(30, TimeUnit.SECONDS);
+    conn.setIdleDuration(-1, TimeUnit.MILLISECONDS);
+    conn.markReusable();
+    return conn;
+  }
+
+  public void headerRequest(HttpHost target, HttpRoute route, ManagedClientConnection conn)
+      throws IOException, HttpException {
+    HttpRequest req = new BasicHttpRequest("OPTIONS", "*", HttpVersion.HTTP_1_1);
+
+    req.addHeader("Host", target.getHostName());
+    BasicHttpParams p = new BasicHttpParams();
+    HttpProtocolParams.setVersion(p, HttpVersion.HTTP_1_1);
+    if (!conn.isOpen()) {
+      conn.open(route, HttpClientUtil.createNewHttpClientRequestContext(), p);
+    }
+    conn.sendRequestHeader(req);
+    conn.flush();
+    conn.receiveResponseHeader();
+  }
+
+  public ClientConnectionRequest getClientConnectionRequest(HttpClient httpClient, HttpRoute route) { // passing second argument to mimic what happens in HttpSolrClient.execute()
+    ClientConnectionRequest mConn = ((PoolingClientConnectionManager) httpClient.getConnectionManager()).requestConnection(route, HttpSolrClient.cacheKey);
+    return mConn;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java b/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java
index 3f57376..4c4b52e 100644
--- a/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java
+++ b/solr/core/src/test/org/apache/solr/security/TestAuthorizationFramework.java
@@ -28,6 +28,7 @@ import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.util.EntityUtils;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -89,7 +90,7 @@ public class TestAuthorizationFramework extends AbstractFullDistribZkTestBase {
     List<String> hierarchy = StrUtils.splitSmart(objPath, '/');
     for (int i = 0; i < count; i++) {
       HttpGet get = new HttpGet(url);
-      s = EntityUtils.toString(cl.execute(get).getEntity());
+      s = EntityUtils.toString(cl.execute(get, HttpClientUtil.createNewHttpClientRequestContext()).getEntity());
       Map m = (Map) Utils.fromJSONString(s);
 
       Object actual = Utils.getObjectByPath(m, true, hierarchy);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
index 0cae1ef..def6572 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
@@ -308,7 +308,8 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
           method.addHeader("User-Agent", HttpSolrClient.AGENT);
           method.addHeader("Content-Type", contentType);
 
-          response = client.getHttpClient().execute(method);
+          response = client.getHttpClient()
+              .execute(method, HttpClientUtil.createNewHttpClientRequestContext());
           rspBody = response.getEntity().getContent();
           int statusCode = response.getStatusLine().getStatusCode();
           if (statusCode != HttpStatus.SC_OK) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
index b8a6cd8..d4fe61c 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientUtil.java
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.client.solrj.impl;
 
+import static java.util.Collections.singletonList;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
@@ -39,6 +41,7 @@ import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.UsernamePasswordCredentials;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.params.ClientParamBean;
+import org.apache.http.client.protocol.HttpClientContext;
 import org.apache.http.conn.ClientConnectionManager;
 import org.apache.http.conn.scheme.Scheme;
 import org.apache.http.conn.ssl.SSLSocketFactory;
@@ -49,6 +52,7 @@ import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
 import org.apache.http.impl.client.SystemDefaultHttpClient;
 import org.apache.http.impl.conn.PoolingClientConnectionManager;
+import org.apache.http.impl.conn.SchemeRegistryFactory;
 import org.apache.http.impl.conn.tsccm.ThreadSafeClientConnManager;
 import org.apache.http.params.HttpConnectionParams;
 import org.apache.http.protocol.HttpContext;
@@ -59,8 +63,6 @@ import org.apache.solr.common.params.SolrParams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.util.Collections.singletonList;
-
 /**
  * Utility class for creating/configuring httpclient instances. 
  */
@@ -149,6 +151,11 @@ public class HttpClientUtil {
     return httpClient;
   }
   
+  /** test usage. subject to change @lucene.experimental */ 
+  static PoolingClientConnectionManager createPoolingConnectionManager() {
+    return new PoolingClientConnectionManager(SchemeRegistryFactory.createSystemDefault());
+  }
+  
   /**
    * Creates new http client by using the provided configuration.
    * 
@@ -432,4 +439,21 @@ public class HttpClientUtil {
     }
   }
 
+  /**
+   * Create a HttpClientContext object and {@link HttpClientContext#setUserToken(Object)}
+   * to an internal singleton. It allows to reuse underneath {@link HttpClient} 
+   * in connection pools 
+   *
+   * If the client is going to be re-used, then you should pass in an object that
+   * can be used by internal connection pools as a cache key.  This is particularly
+   * important if client authentication is enabled, as SSL connections will not
+   * be re-used if no cache key is provided.
+   *
+   */
+  public static HttpClientContext createNewHttpClientRequestContext() {
+    HttpClientContext context = new HttpClientContext();
+
+    context.setUserToken(HttpSolrClient.cacheKey);
+    return context;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
index 6e25968..cd37e95 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
@@ -47,6 +47,7 @@ import org.apache.http.client.methods.HttpPost;
 import org.apache.http.client.methods.HttpPut;
 import org.apache.http.client.methods.HttpRequestBase;
 import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.client.protocol.HttpClientContext;
 import org.apache.http.conn.ClientConnectionManager;
 import org.apache.http.entity.ContentType;
 import org.apache.http.entity.InputStreamEntity;
@@ -150,6 +151,8 @@ public class HttpSolrClient extends SolrClient {
   private final boolean internalClient;
 
   private volatile Set<String> queryParams = Collections.emptySet();
+  
+  static final Class<HttpSolrClient> cacheKey = HttpSolrClient.class;
 
   /**
    * @param baseURL
@@ -491,8 +494,8 @@ public class HttpSolrClient extends SolrClient {
     InputStream respBody = null;
     boolean shouldClose = true;
     try {
-      // Execute the method.
-      final HttpResponse response = httpClient.execute(method);
+      HttpClientContext httpClientRequestContext = HttpClientUtil.createNewHttpClientRequestContext();
+      final HttpResponse response = httpClient.execute(method, httpClientRequestContext);
       int httpStatus = response.getStatusLine().getStatusCode();
       
       // Read the contents

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
index 0d970ad..e577921 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrSchemalessExampleTest.java
@@ -16,6 +16,14 @@
  */
 package org.apache.solr.client.solrj;
 
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.OutputStreamWriter;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
@@ -23,21 +31,15 @@ import org.apache.http.client.methods.HttpPost;
 import org.apache.http.entity.InputStreamEntity;
 import org.apache.solr.client.solrj.impl.BinaryRequestWriter;
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.ExternalPaths;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.OutputStreamWriter;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Properties;
-
 public class SolrSchemalessExampleTest extends SolrExampleTestsBase {
 
   @BeforeClass
@@ -78,7 +80,8 @@ public class SolrSchemalessExampleTest extends SolrExampleTestsBase {
     HttpPost post = new HttpPost(client.getBaseURL() + "/update/json/docs");
     post.setHeader("Content-Type", "application/json");
     post.setEntity(new InputStreamEntity(new ByteArrayInputStream(json.getBytes("UTF-8")), -1));
-    HttpResponse response = httpClient.execute(post);
+    HttpResponse response = httpClient.execute(post, HttpClientUtil.createNewHttpClientRequestContext());
+    Utils.consumeFully(response.getEntity());
     assertEquals(200, response.getStatusLine().getStatusCode());
     client.commit();
     assertNumFound("*:*", 2);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
index ca37114..6c13e40 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/JettyWebappTest.java
@@ -31,6 +31,7 @@ import org.apache.http.client.methods.HttpRequestBase;
 import org.apache.http.impl.client.HttpClients;
 import org.apache.solr.SolrJettyTestBase;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.util.ExternalPaths;
 import org.eclipse.jetty.server.Connector;
 import org.eclipse.jetty.server.HttpConnectionFactory;
@@ -110,7 +111,7 @@ public class JettyWebappTest extends SolrTestCaseJ4
 
     HttpClient client = HttpClients.createDefault();
     HttpRequestBase m = new HttpGet(adminPath);
-    HttpResponse response = client.execute(m);
+    HttpResponse response = client.execute(m, HttpClientUtil.createNewHttpClientRequestContext());
     assertEquals(200, response.getStatusLine().getStatusCode());
     Header header = response.getFirstHeader("X-Frame-Options");
     assertEquals("DENY", header.getValue().toUpperCase(Locale.ROOT));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
index 6022b9a..2cf8e04 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleJettyTest.java
@@ -16,6 +16,9 @@
  */
 package org.apache.solr.client.solrj.embedded;
 
+import java.io.ByteArrayInputStream;
+import java.util.Map;
+
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpPost;
@@ -23,6 +26,7 @@ import org.apache.http.entity.InputStreamEntity;
 import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
 import org.apache.solr.client.solrj.SolrExampleTests;
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrDocument;
@@ -31,9 +35,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.noggit.ObjectBuilder;
 
-import java.io.ByteArrayInputStream;
-import java.util.Map;
-
 /**
  * TODO? perhaps use:
  *  http://docs.codehaus.org/display/JETTY/ServletTester
@@ -75,7 +76,7 @@ public class SolrExampleJettyTest extends SolrExampleTests {
     HttpPost post = new HttpPost(client.getBaseURL() + "/update/json/docs");
     post.setHeader("Content-Type", "application/json");
     post.setEntity(new InputStreamEntity(new ByteArrayInputStream(json.getBytes("UTF-8")), -1));
-    HttpResponse response = httpClient.execute(post);
+    HttpResponse response = httpClient.execute(post, HttpClientUtil.createNewHttpClientRequestContext());
     assertEquals(200, response.getStatusLine().getStatusCode());
     client.commit();
     QueryResponse rsp = getSolrClient().query(new SolrQuery("*:*"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
index 23b73db..fddc332 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
@@ -553,7 +553,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
     CloseableHttpClient httpclient = HttpClientUtil.createClient(null);
     HttpEntity entity = null;
     try {
-      HttpResponse response = httpclient.execute(get);
+      HttpResponse response = httpclient.execute(get, HttpClientUtil.createNewHttpClientRequestContext());
       entity = response.getEntity();
       Header ceheader = entity.getContentEncoding();
       assertEquals("gzip", ceheader.getValue());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
new file mode 100644
index 0000000..e6f63ae
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientConPoolTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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.client.solrj.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.conn.PoolingClientConnectionManager;
+import org.apache.http.pool.PoolStats;
+import org.apache.solr.SolrJettyTestBase;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrjNamedThreadFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class HttpSolrClientConPoolTest extends SolrJettyTestBase {
+
+  protected static JettySolrRunner yetty;
+  private static String fooUrl;
+  private static String barUrl;
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    createJetty(legacyExampleCollection1SolrHome());
+    // stealing the first made jetty
+    yetty = jetty;
+    barUrl = yetty.getBaseUrl().toString() + "/" + "collection1";
+
+    createJetty(legacyExampleCollection1SolrHome());
+    fooUrl = jetty.getBaseUrl().toString() + "/" + "collection1";
+  }
+  
+  @AfterClass
+  public static void stopYetty() throws Exception {
+    yetty.stop();
+    yetty = null;
+  }
+  
+  public void testPoolSize() throws SolrServerException, IOException {
+    PoolingClientConnectionManager pool = HttpClientUtil.createPoolingConnectionManager();
+    final HttpSolrClient client1 ;
+    {
+      CloseableHttpClient httpClient = HttpClientUtil.createClient(new ModifiableSolrParams(), pool);
+      client1 = getHttpSolrClient(fooUrl, httpClient);
+      client1.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
+    }
+    
+    {
+      for (String url : new String[]{fooUrl, barUrl}) {
+        if (!client1.getBaseURL().equals(url)) {
+          client1.setBaseURL(url);
+        }
+        client1.deleteByQuery("*:*");
+      }
+    }
+    
+    List<String> urls = new ArrayList<>();
+    for(int i=0; i<17; i++) {
+      urls.add(fooUrl);
+    }
+    for(int i=0; i<31; i++) {
+      urls.add(barUrl);
+    }
+    
+    Collections.shuffle(urls, random());
+    
+    try {
+      int i=0;
+      for (String url : urls) {
+        if (!client1.getBaseURL().equals(url)) {
+          client1.setBaseURL(url);
+        }
+        client1.add(new SolrInputDocument("id", ""+(i++)));
+      }
+      client1.setBaseURL(fooUrl);
+      client1.commit();
+      assertEquals(17, client1.query(new SolrQuery("*:*")).getResults().getNumFound());
+      
+      client1.setBaseURL(barUrl);
+      client1.commit();
+      assertEquals(31, client1.query(new SolrQuery("*:*")).getResults().getNumFound());
+      
+      PoolStats stats = pool.getTotalStats();
+      assertEquals("oh "+stats, 2, stats.getAvailable());
+    } finally {
+        HttpClientUtil.close(client1.getHttpClient());
+        client1.close();
+    }
+  }
+  public void testLBClient() throws IOException, SolrServerException {
+    
+    PoolingClientConnectionManager pool = HttpClientUtil.createPoolingConnectionManager();
+    final HttpSolrClient client1 ;
+    int threadCount = atLeast(2);
+    final ExecutorService threads = ExecutorUtil.newMDCAwareFixedThreadPool(threadCount,
+        new SolrjNamedThreadFactory(getClass().getSimpleName()+"TestScheduler"));
+    try(  CloseableHttpClient httpClient = HttpClientUtil.createClient(new ModifiableSolrParams(), pool)){
+      final LBHttpSolrClient roundRobin = new LBHttpSolrClient.Builder().
+                withBaseSolrUrl(fooUrl).
+                withBaseSolrUrl(barUrl).
+                withHttpClient(httpClient)
+                .build();
+      
+      List<ConcurrentUpdateSolrClient> concurrentClients = Arrays.asList(
+          new ConcurrentUpdateSolrClient.Builder(fooUrl)
+          .withHttpClient(httpClient).withThreadCount(threadCount)
+          .withQueueSize(10)
+         .withExecutorService(threads).build(),
+           new ConcurrentUpdateSolrClient.Builder(barUrl)
+          .withHttpClient(httpClient).withThreadCount(threadCount)
+          .withQueueSize(10)
+         .withExecutorService(threads).build()); 
+      
+      for (int i=0; i<2; i++) {
+        roundRobin.deleteByQuery("*:*");
+      }
+      
+      for (int i=0; i<57; i++) {
+        final SolrInputDocument doc = new SolrInputDocument("id", ""+i);
+        if (random().nextBoolean()) {
+          final ConcurrentUpdateSolrClient concurrentClient = concurrentClients.get(random().nextInt(concurrentClients.size()));
+          concurrentClient.add(doc); // here we are testing that CUSC and plain clients reuse pool 
+          concurrentClient.blockUntilFinished();
+        } else {
+          if (random().nextBoolean()) {
+            roundRobin.add(doc);
+          } else {
+            final UpdateRequest updateRequest = new UpdateRequest();
+            updateRequest.add(doc); // here we mimic CloudSolrClient impl
+            final List<String> urls = Arrays.asList(fooUrl, barUrl);
+            Collections.shuffle(urls, random());
+            LBHttpSolrClient.Req req = new LBHttpSolrClient.Req(updateRequest, 
+                    urls);
+             roundRobin.request(req);
+          }
+        }
+      }
+      
+      for (int i=0; i<2; i++) {
+        roundRobin.commit();
+      }
+      int total=0;
+      for (int i=0; i<2; i++) {
+        total += roundRobin.query(new SolrQuery("*:*")).getResults().getNumFound();
+      }
+      assertEquals(57, total);
+      PoolStats stats = pool.getTotalStats();
+      //System.out.println("\n"+stats);
+      assertEquals("expected number of connections shouldn't exceed number of endpoints" + stats, 
+          2, stats.getAvailable());
+    }finally {
+      threads.shutdown();
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientSSLAuthConPoolTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientSSLAuthConPoolTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientSSLAuthConPoolTest.java
new file mode 100644
index 0000000..c3fbe5f
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/HttpSolrClientSSLAuthConPoolTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.client.solrj.impl;
+
+import java.net.URL;
+import java.util.Arrays;
+
+import org.apache.solr.util.RandomizeSSL;
+import org.junit.BeforeClass;
+
+@RandomizeSSL(1.0)
+public class HttpSolrClientSSLAuthConPoolTest extends HttpSolrClientConPoolTest {
+
+  @BeforeClass
+  public static void checkUrls() throws Exception {
+    URL[] urls = new URL[] {
+        jetty.getBaseUrl(),yetty.getBaseUrl()
+    };
+    for (URL u : urls) {
+      assertEquals("expect https urls ","https", u.getProtocol());
+    }
+    assertFalse("expect different urls "+Arrays.toString(urls),
+            urls[0].equals(urls[1]));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b58ccc39/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java b/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
index de5f662..934384f 100644
--- a/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
+++ b/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
@@ -15,6 +15,13 @@
  * limitations under the License.
  */
 package org.apache.solr.util;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpressionException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+
 import org.apache.http.HttpEntity;
 import org.apache.http.client.methods.HttpDelete;
 import org.apache.http.client.methods.HttpGet;
@@ -28,13 +35,6 @@ import org.apache.http.util.EntityUtils;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.common.params.ModifiableSolrParams;
 
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPathExpressionException;
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.URLEncoder;
-import java.nio.charset.StandardCharsets;
-
 /**
  * Facilitates testing Solr's REST API via a provided embedded Jetty
  */
@@ -204,7 +204,7 @@ public class RestTestHarness extends BaseTestHarness implements Closeable {
   private String getResponse(HttpUriRequest request) throws IOException {
     HttpEntity entity = null;
     try {
-      entity = httpClient.execute(request).getEntity();
+      entity = httpClient.execute(request, HttpClientUtil.createNewHttpClientRequestContext()).getEntity();
       return EntityUtils.toString(entity, StandardCharsets.UTF_8);
     } finally {
       EntityUtils.consumeQuietly(entity);