You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/10/16 22:50:58 UTC

lucene-solr:jira/http2: Moving setup of Http2SolrClient to authenticationPlugin level so each CoreContainer will have its own instance instead of using an shared from HttpClientUtil.httpClientBuilder

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/http2 b0c932213 -> 6558203fd


Moving setup of Http2SolrClient to authenticationPlugin level so each CoreContainer will have its own instance instead of using an shared from HttpClientUtil.httpClientBuilder


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

Branch: refs/heads/jira/http2
Commit: 6558203fdbcc86d1f62bfd361b278675dbd9e656
Parents: b0c9322
Author: Cao Manh Dat <da...@apache.org>
Authored: Wed Oct 17 05:50:50 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Wed Oct 17 05:50:50 2018 +0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +-
 .../org/apache/solr/core/CoreContainer.java     |   4 +-
 .../component/HttpShardHandlerFactory.java      |   7 +-
 .../handler/component/ShardHandlerFactory.java  |   3 +-
 .../ConfigurableInternodeAuthHadoopPlugin.java  |   6 +
 .../solr/security/HttpClientBuilderPlugin.java  |   5 +
 .../apache/solr/security/KerberosPlugin.java    |   6 +
 .../solr/security/PKIAuthenticationPlugin.java  |  16 +--
 .../apache/solr/update/UpdateShardHandler.java  |   7 +-
 .../solr/security/BasicAuthIntegrationTest.java |   1 -
 .../HttpParamDelegationTokenPlugin.java         |  15 ++-
 .../src/major-changes-in-solr-8.adoc            |   4 +-
 .../solr/client/solrj/impl/Http2SolrClient.java |   5 +-
 .../solrj/impl/HttpClientBuilderFactory.java    |   3 +
 .../solrj/impl/Krb5HttpClientBuilder.java       | 124 +++++++++----------
 ...PreemptiveBasicAuthClientBuilderFactory.java |  23 ++--
 .../solrj/impl/SolrHttpClientBuilder.java       |  15 ---
 .../java/org/apache/solr/SolrTestCaseJ4.java    |  12 +-
 18 files changed, 136 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b5bfd2a..0322db3 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -81,7 +81,7 @@ Upgrade Notes
   are marked as deprecated, uses LBSolrClient.Req and LBSolrClient.Rsp instead.
 
 * SOLR-12644: Custom AuthenticationPlugin must provide its own setup for Http2SolrClient through
-  SolrHttpClientBuilder.setHttp2Configurator, if not internal requests can't be authenticated. (Cao Manh Dat)
+  implementing HttpClientBuilderPlugin.setup, if not internal requests can't be authenticated. (Cao Manh Dat)
 
 New Features
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
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 c1d9c58..20c3e24 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -400,8 +400,8 @@ public class CoreContainer {
       // Setup HttpClient for internode communication
       HttpClientBuilderPlugin builderPlugin = ((HttpClientBuilderPlugin) authcPlugin);
       SolrHttpClientBuilder builder = builderPlugin.getHttpClientBuilder(HttpClientUtil.getHttpClientBuilder());
-      shardHandlerFactory.setSecurityBuilder(builder);
-      updateShardHandler.setSecurityBuilder(builder);
+      shardHandlerFactory.setSecurityBuilder(builderPlugin);
+      updateShardHandler.setSecurityBuilder(builderPlugin);
 
       // The default http client of the core container's shardHandlerFactory has already been created and
       // configured using the default httpclient configurer. We need to reconfigure it using the plugin's

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index 3f0979d..18371f4 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -55,6 +55,7 @@ import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.security.HttpClientBuilderPlugin;
 import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.stats.InstrumentedHttpListenerFactory;
@@ -213,10 +214,8 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
   }
 
   @Override
-  public void setSecurityBuilder(SolrHttpClientBuilder securityBuilder) {
-    if (securityBuilder != null) {
-      securityBuilder.applyHttp2Configurator(defaultClient);
-    }
+  public void setSecurityBuilder(HttpClientBuilderPlugin clientBuilderPlugin) {
+    clientBuilderPlugin.setup(defaultClient);
   }
 
   protected <T> T getParameter(NamedList initArgs, String configKey, T defaultValue, StringBuilder sb) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/core/src/java/org/apache/solr/handler/component/ShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/ShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/ShardHandlerFactory.java
index 72dd6e4..81d8261 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/ShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/ShardHandlerFactory.java
@@ -21,6 +21,7 @@ import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.security.HttpClientBuilderPlugin;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 
 import java.util.Collections;
@@ -32,7 +33,7 @@ public abstract class ShardHandlerFactory {
 
   public abstract void close();
 
-  public void setSecurityBuilder(SolrHttpClientBuilder securityBuilder){};
+  public void setSecurityBuilder(HttpClientBuilderPlugin clientBuilderPlugin){};
 
   /**
    * Create a new ShardHandlerFactory instance

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java b/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
index f3bb70f..8fe0070 100644
--- a/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/ConfigurableInternodeAuthHadoopPlugin.java
@@ -21,6 +21,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 
+import org.apache.solr.client.solrj.impl.Http2SolrClient;
 import org.apache.solr.client.solrj.impl.HttpClientBuilderFactory;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
 import org.apache.solr.core.CoreContainer;
@@ -53,6 +54,11 @@ public class ConfigurableInternodeAuthHadoopPlugin extends HadoopAuthPlugin impl
   }
 
   @Override
+  public void setup(Http2SolrClient client) {
+    factory.setup(client);
+  }
+
+  @Override
   public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
     return factory.getHttpClientBuilder(Optional.ofNullable(builder));
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/core/src/java/org/apache/solr/security/HttpClientBuilderPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/HttpClientBuilderPlugin.java b/solr/core/src/java/org/apache/solr/security/HttpClientBuilderPlugin.java
index 8b7e80b..c5bcc36 100644
--- a/solr/core/src/java/org/apache/solr/security/HttpClientBuilderPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/HttpClientBuilderPlugin.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.security;
 
+import org.apache.solr.client.solrj.impl.Http2SolrClient;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
 
 /**
@@ -34,4 +35,8 @@ public interface HttpClientBuilderPlugin {
    * @lucene.experimental
    */
   public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder);
+
+  public default void setup(Http2SolrClient client) {
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java b/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
index 42d22ca..87f37a8 100644
--- a/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/KerberosPlugin.java
@@ -35,6 +35,7 @@ import javax.servlet.http.HttpServletResponseWrapper;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.collections.iterators.IteratorEnumeration;
+import org.apache.solr.client.solrj.impl.Http2SolrClient;
 import org.apache.solr.client.solrj.impl.Krb5HttpClientBuilder;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
 import org.apache.solr.cloud.ZkController;
@@ -256,6 +257,11 @@ public class KerberosPlugin extends AuthenticationPlugin implements HttpClientBu
   }
 
   @Override
+  public void setup(Http2SolrClient client) {
+    kerberosBuilder.setup(client);
+  }
+
+  @Override
   public void close() {
     kerberosFilter.destroy();
     kerberosBuilder.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/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 68b823c..84388b6 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.auth.BasicUserPrincipal;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.protocol.HttpContext;
 import org.apache.http.util.EntityUtils;
+import org.apache.solr.client.solrj.impl.Http2SolrClient;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpListenerFactory;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
@@ -226,20 +227,19 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
   }
 
   @Override
-  public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
-    HttpClientUtil.addRequestInterceptor(interceptor);
+  public void setup(Http2SolrClient client) {
     final HttpListenerFactory.RequestResponseListener listener = new HttpListenerFactory.RequestResponseListener() {
       @Override
       public void onQueued(Request request) {
         generateToken().ifPresent(s -> request.header(HEADER, myNodeName + " " + s));
-        if (log.isDebugEnabled()) {
-          log.debug("Add token {} for request {}", request.getHeaders().get(HEADER), request);
-        }
       }
     };
-    builder.setHttp2Configurator(client -> {
-      client.addListenerFactory(() -> listener);
-    });
+    client.addListenerFactory(() -> listener);
+  }
+
+  @Override
+  public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
+    HttpClientUtil.addRequestInterceptor(interceptor);
     return builder;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
index a885ec7..4e9fa21 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -40,6 +40,7 @@ import org.apache.solr.common.util.SolrjNamedThreadFactory;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.security.HttpClientBuilderPlugin;
 import org.apache.solr.util.stats.HttpClientMetricNameStrategy;
 import org.apache.solr.util.stats.InstrumentedHttpListenerFactory;
 import org.apache.solr.util.stats.InstrumentedHttpRequestExecutor;
@@ -247,9 +248,7 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
     return connectionTimeout;
   }
 
-  public void setSecurityBuilder(SolrHttpClientBuilder builder) {
-    if (builder != null) {
-      builder.applyHttp2Configurator(updateOnlyClient);
-    }
+  public void setSecurityBuilder(HttpClientBuilderPlugin builder) {
+    builder.setup(updateOnlyClient);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index 0c3b33c..06215d8 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -68,7 +68,6 @@ import org.slf4j.LoggerFactory;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Collections.singletonMap;
 
-@LogLevel("org.apache.solr.security=DEBUG;org.eclipse.jetty=DEBUG")
 public class BasicAuthIntegrationTest extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/core/src/test/org/apache/solr/security/HttpParamDelegationTokenPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/HttpParamDelegationTokenPlugin.java b/solr/core/src/test/org/apache/solr/security/HttpParamDelegationTokenPlugin.java
index a69c7d3..a8f0355 100644
--- a/solr/core/src/test/org/apache/solr/security/HttpParamDelegationTokenPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/HttpParamDelegationTokenPlugin.java
@@ -50,6 +50,7 @@ import org.apache.http.HttpRequestInterceptor;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
 import org.apache.http.protocol.HttpContext;
+import org.apache.solr.client.solrj.impl.Http2SolrClient;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpListenerFactory;
 import org.apache.solr.client.solrj.impl.SolrHttpClientBuilder;
@@ -147,18 +148,20 @@ public class HttpParamDelegationTokenPlugin extends KerberosPlugin {
   }
 
   @Override
-  public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
-    HttpClientUtil.addRequestInterceptor(interceptor);
-    builder = super.getHttpClientBuilder(builder);
+  public void setup(Http2SolrClient client) {
     final HttpListenerFactory.RequestResponseListener listener = new HttpListenerFactory.RequestResponseListener() {
       @Override
       public void onQueued(Request request) {
         getPrincipal().ifPresent(usr -> request.header(INTERNAL_REQUEST_HEADER, usr));
       }
     };
-    builder.setHttp2Configurator(client -> {
-      client.addListenerFactory(() -> listener);
-    });
+    client.addListenerFactory(() -> listener);
+  }
+
+  @Override
+  public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
+    HttpClientUtil.addRequestInterceptor(interceptor);
+    builder = super.getHttpClientBuilder(builder);
     return builder;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/solr-ref-guide/src/major-changes-in-solr-8.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/major-changes-in-solr-8.adoc b/solr/solr-ref-guide/src/major-changes-in-solr-8.adoc
index c8b6262..501612a 100644
--- a/solr/solr-ref-guide/src/major-changes-in-solr-8.adoc
+++ b/solr/solr-ref-guide/src/major-changes-in-solr-8.adoc
@@ -56,7 +56,7 @@ However this leads to several changes in configuration and authentication setup
 * {solr-javadocs}/solr-core/org/apache/solr/update/UpdateShardHandler.html[`UpdateShardHandler.maxConnections`] parameter is no longer being used
 * {solr-javadocs}/solr-core/org/apache/solr/handler/component/HttpShardHandler.html[`HttpShardHandlerFactory.maxConnections`] parameter is no longer being used
 *  Custom {solr-javadocs}/solr-core/org/apache/solr/security/AuthenticationPlugin.html[`AuthenticationPlugin`] must provide its own setup for
-   `Http2SolrClient` through
-   {solr-javadocs}/solr-solrj/org/apache/solr/client/solrj/impl/SolrHttpClientBuilder.html[`SolrHttpClientBuilder.setHttp2Configurator`],
+   `Http2SolrClient` through implementing
+   {solr-javadocs}/solr-core/org/apache/solr/security/HttpClientBuilderPlugin.html[`HttpClientBuilderPlugin.setup`],
    if not internal requests can't be authenticated
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
index 8ad3b0d..e3fd2c0 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Http2SolrClient.java
@@ -228,10 +228,9 @@ public class Http2SolrClient extends SolrClient {
     setBasicAuthHeader(solrRequest, req);
     for (HttpListenerFactory factory : listenerFactory) {
       HttpListenerFactory.RequestResponseListener listener = factory.get();
-      //By calling onBegin here, we will make sure that SolrRequestInfo can be get the request from the same thread
-      listener.onBegin(req);
-      req.onComplete(listener);
       req.onRequestQueued(listener);
+      req.onRequestBegin(listener);
+      req.onComplete(listener);
     }
 
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFactory.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFactory.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFactory.java
index 77c4a94..b552f66 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFactory.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFactory.java
@@ -38,4 +38,7 @@ public interface HttpClientBuilderFactory extends Closeable {
    */
   public SolrHttpClientBuilder getHttpClientBuilder(Optional<SolrHttpClientBuilder> builder);
 
+  public default void setup(Http2SolrClient client) {
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
index 7cbe1f7..fd1a7fb 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder.java
@@ -96,6 +96,68 @@ public class Krb5HttpClientBuilder implements HttpClientBuilderFactory {
     return builder.isPresent() ? getBuilder(builder.get()) : getBuilder();
   }
 
+  @Override
+  public void setup(Http2SolrClient http2Client) {
+    HttpAuthenticationStore authenticationStore = new HttpAuthenticationStore();
+    authenticationStore.addAuthentication(new Authentication() {
+      @Override
+      public boolean matches(String type, URI uri, String realm) {
+        return "Negotiate".equals(type);
+      }
+
+      @Override
+      public Result authenticate(Request request, ContentResponse response, HeaderInfo headerInfo, Attributes context) {
+        String challenge = headerInfo.getBase64();
+        if (challenge == null) challenge = "";
+        byte[] input = java.util.Base64.getDecoder().decode(challenge);
+        byte[] token;
+        String authServer = request.getHost();
+        final GSSManager manager = GSSManager.getInstance();
+        try {
+          GSSName serverName = manager.createName("HTTP@" + authServer, GSSName.NT_HOSTBASED_SERVICE);
+          final GSSContext gssContext = createGSSContext(manager, new Oid(SPNEGO_OID), serverName, null);
+          if (input != null) {
+            token = gssContext.initSecContext(input, 0, input.length);
+          } else {
+            token = gssContext.initSecContext(new byte[] {}, 0, 0);
+          }
+        } catch (GSSException e) {
+          throw new IllegalArgumentException("Unable to init GSSContext", e);
+        }
+        return new Result() {
+          AtomicBoolean sentToken = new AtomicBoolean(false);
+          @Override
+          public URI getURI() {
+            // Since Kerberos is connection based authentication, sub-sequence requests won't need to resend the token in header
+            // by return null, the ProtocolHandler won't try to apply this result on sequence requests
+            return null;
+          }
+
+          @Override
+          public void apply(Request request) {
+            if (sentToken.get()) return;
+
+            final String tokenstr = java.util.Base64.getEncoder().encodeToString(token);
+            if (log.isDebugEnabled()) {
+              log.info("Sending response '" + tokenstr + "' back to the auth server");
+            }
+            request.header(headerInfo.getHeader().asString(), "Negotiate "+tokenstr);
+          }
+        };
+      }
+
+      private GSSContext createGSSContext(GSSManager manager, Oid oid, GSSName serverName, final GSSCredential gssCredential) throws GSSException {
+        // Get the credentials from the JAAS configuration rather than here
+        final GSSContext gssContext = manager.createContext(serverName.canonicalize(oid), oid, gssCredential,
+            GSSContext.DEFAULT_LIFETIME);
+        gssContext.requestMutualAuth(true);
+        return gssContext;
+      }
+    });
+    http2Client.getHttpClient().setAuthenticationStore(authenticationStore);
+    http2Client.getProtocolHandlers().put(new SolrWWWAuthenticationProtocolHandler(http2Client.getHttpClient()));
+  }
+
   public SolrHttpClientBuilder getBuilder(SolrHttpClientBuilder builder) {
     if (System.getProperty(LOGIN_CONFIG_PROP) != null) {
       String configValue = System.getProperty(LOGIN_CONFIG_PROP);
@@ -152,68 +214,6 @@ public class Krb5HttpClientBuilder implements HttpClientBuilderFactory {
           return credentialsProvider;
         });
         HttpClientUtil.addRequestInterceptor(bufferedEntityInterceptor);
-
-        //setup for http2
-        builder.setHttp2Configurator(http2Client -> {
-          HttpAuthenticationStore authenticationStore = new HttpAuthenticationStore();
-          authenticationStore.addAuthentication(new Authentication() {
-            @Override
-            public boolean matches(String type, URI uri, String realm) {
-              return "Negotiate".equals(type);
-            }
-
-            @Override
-            public Result authenticate(Request request, ContentResponse response, HeaderInfo headerInfo, Attributes context) {
-              String challenge = headerInfo.getBase64();
-              if (challenge == null) challenge = "";
-              byte[] input = java.util.Base64.getDecoder().decode(challenge);
-              byte[] token;
-              String authServer = request.getHost();
-              final GSSManager manager = GSSManager.getInstance();
-              try {
-                GSSName serverName = manager.createName("HTTP@" + authServer, GSSName.NT_HOSTBASED_SERVICE);
-                final GSSContext gssContext = createGSSContext(manager, new Oid(SPNEGO_OID), serverName, null);
-                if (input != null) {
-                  token = gssContext.initSecContext(input, 0, input.length);
-                } else {
-                  token = gssContext.initSecContext(new byte[] {}, 0, 0);
-                }
-              } catch (GSSException e) {
-                throw new IllegalArgumentException("Unable to init GSSContext", e);
-              }
-              return new Result() {
-                AtomicBoolean sentToken = new AtomicBoolean(false);
-                @Override
-                public URI getURI() {
-                  // Since Kerberos is connection based authentication, sub-sequence requests won't need to resend the token in header
-                  // by return null, the ProtocolHandler won't try to apply this result on sequence requests
-                  return null;
-                }
-
-                @Override
-                public void apply(Request request) {
-                  if (sentToken.get()) return;
-
-                  final String tokenstr = java.util.Base64.getEncoder().encodeToString(token);
-                  if (log.isDebugEnabled()) {
-                    log.info("Sending response '" + tokenstr + "' back to the auth server");
-                  }
-                  request.header(headerInfo.getHeader().asString(), "Negotiate "+tokenstr);
-                }
-              };
-            }
-
-            private GSSContext createGSSContext(GSSManager manager, Oid oid, GSSName serverName, final GSSCredential gssCredential) throws GSSException {
-              // Get the credentials from the JAAS configuration rather than here
-              final GSSContext gssContext = manager.createContext(serverName.canonicalize(oid), oid, gssCredential,
-                  GSSContext.DEFAULT_LIFETIME);
-              gssContext.requestMutualAuth(true);
-              return gssContext;
-            }
-          });
-          http2Client.getHttpClient().setAuthenticationStore(authenticationStore);
-          http2Client.getProtocolHandlers().put(new SolrWWWAuthenticationProtocolHandler(http2Client.getHttpClient()));
-        });
       }
     } else {
       log.warn("{} is configured without specifying system property '{}'",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/solrj/src/java/org/apache/solr/client/solrj/impl/PreemptiveBasicAuthClientBuilderFactory.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/PreemptiveBasicAuthClientBuilderFactory.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/PreemptiveBasicAuthClientBuilderFactory.java
index 5a21a95..e446695 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/PreemptiveBasicAuthClientBuilderFactory.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/PreemptiveBasicAuthClientBuilderFactory.java
@@ -107,6 +107,21 @@ public class PreemptiveBasicAuthClientBuilderFactory implements HttpClientBuilde
   }
 
   @Override
+  public void setup(Http2SolrClient client) {
+    final String basicAuthUser = defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_USER);
+    final String basicAuthPass = defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_PASS);
+    if(basicAuthUser == null || basicAuthPass == null) {
+      throw new IllegalArgumentException("username & password must be specified with " + getClass().getName());
+    }
+
+    HttpAuthenticationStore authenticationStore = new HttpAuthenticationStore();
+    authenticationStore.addAuthentication(new SolrBasicAuthentication(basicAuthUser, basicAuthPass));
+    client.getHttpClient().setAuthenticationStore(authenticationStore);
+    client.getProtocolHandlers().put(new WWWAuthenticationProtocolHandler(client.getHttpClient()));
+    client.getProtocolHandlers().put(new ProxyAuthenticationProtocolHandler(client.getHttpClient()));
+  }
+
+  @Override
   public SolrHttpClientBuilder getHttpClientBuilder(Optional<SolrHttpClientBuilder> optionalBuilder) {
     final String basicAuthUser = defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_USER);
     final String basicAuthPass = defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_PASS);
@@ -117,14 +132,6 @@ public class PreemptiveBasicAuthClientBuilderFactory implements HttpClientBuilde
     SolrHttpClientBuilder builder = optionalBuilder.isPresent() ?
         initHttpClientBuilder(optionalBuilder.get(), basicAuthUser, basicAuthPass)
         : initHttpClientBuilder(SolrHttpClientBuilder.create(), basicAuthUser, basicAuthPass);
-
-    HttpAuthenticationStore authenticationStore = new HttpAuthenticationStore();
-    authenticationStore.addAuthentication(new SolrBasicAuthentication(basicAuthUser, basicAuthPass));
-    builder.setHttp2Configurator(http2Client -> {
-      http2Client.getHttpClient().setAuthenticationStore(authenticationStore);
-      http2Client.getProtocolHandlers().put(new WWWAuthenticationProtocolHandler(http2Client.getHttpClient()));
-      http2Client.getProtocolHandlers().put(new ProxyAuthenticationProtocolHandler(http2Client.getHttpClient()));
-    });
     return builder;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientBuilder.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientBuilder.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientBuilder.java
index ab16d8a..4830290 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientBuilder.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientBuilder.java
@@ -50,14 +50,9 @@ public class SolrHttpClientBuilder {
     Lookup<CookieSpecProvider> getCookieSpecRegistry();
   }
 
-  public interface Http2Configurator {
-    void setup(Http2SolrClient http2SolrClient);
-  }
-  
   private CookieSpecRegistryProvider cookieSpecRegistryProvider;
   private AuthSchemeRegistryProvider authSchemeRegistryProvider;
   private CredentialsProviderProvider credentialsProviderProvider;
-  private Http2Configurator configurator;
 
   protected SolrHttpClientBuilder() {
     super();
@@ -81,12 +76,6 @@ public class SolrHttpClientBuilder {
     return this;
   }
 
-  public final SolrHttpClientBuilder setHttp2Configurator (
-      final Http2Configurator configurator) {
-    this.configurator = configurator;
-    return this;
-  }
-
   public AuthSchemeRegistryProvider getAuthSchemeRegistryProvider() {
     return authSchemeRegistryProvider;
   }
@@ -99,8 +88,4 @@ public class SolrHttpClientBuilder {
     return credentialsProviderProvider;
   }
 
-  public void applyHttp2Configurator(Http2SolrClient client) {
-    if (configurator != null) configurator.setup(client);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6558203f/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index fa5ff65..059b76b 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -195,6 +195,13 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
   
   private static String initialRootLogLevel;
 
+  static {
+    // Set Conscrypt as default OpenSSLProvider for all clients
+    if (Security.getProvider("Conscrypt") == null) {
+      Security.insertProviderAt(new OpenSSLProvider(), 1);
+    }
+  }
+
   protected void writeCoreProperties(Path coreDirectory, String corename) throws IOException {
     Properties props = new Properties();
     props.setProperty("name", corename);
@@ -285,11 +292,6 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     ignoreException("ignore_exception");
     newRandomConfig();
 
-    // Set Conscrypt as default OpenSSLProvider for all clients
-    if (Security.getProvider("Conscrypt") == null) {
-      Security.insertProviderAt(new OpenSSLProvider(), 1);
-    }
-
     sslConfig = buildSSLConfig();
     // based on randomized SSL config, set SchemaRegistryProvider appropriately
     HttpClientUtil.setSchemaRegistryProvider(sslConfig.buildClientSchemaRegistryProvider());