You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2021/03/05 18:51:36 UTC

[lucene-solr] branch master updated: SOLR-15154: Let Http2SolrClient pass Basic Auth credentials to all requests (#2445)

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

tflobbe 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 fe33a43  SOLR-15154: Let Http2SolrClient pass Basic Auth credentials to all requests (#2445)
fe33a43 is described below

commit fe33a436a084a43b3fdceb7beea662730c535337
Author: Tomas Fernandez Lobbe <tf...@apache.org>
AuthorDate: Fri Mar 5 10:51:22 2021 -0800

    SOLR-15154: Let Http2SolrClient pass Basic Auth credentials to all requests (#2445)
    
    Credentials can now be set explicitly at the client level, or can be read from System properties like in the previous version of the client when using PreemptiveBasicAuthClientBuilderFactory. Other implementations of HttpClientBuilderFactory can now also be used.
---
 solr/CHANGES.txt                                   |   2 +
 .../solr/client/solrj/impl/Http2SolrClient.java    |  61 +++++++-
 .../PreemptiveBasicAuthClientBuilderFactory.java   |  91 ++++++-----
 .../client/solrj/impl/Http2SolrClientTest.java     | 168 +++++++++++++++++++--
 ...reemptiveBasicAuthClientBuilderFactoryTest.java |  78 ++++++++++
 5 files changed, 341 insertions(+), 59 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index ef474f9..b864471 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -266,6 +266,8 @@ Improvements
 
 * SOLR-15194: Allow Solr to make outbound non SSL calls to a JWT IDP via -Dsolr.auth.jwt.allowOutboundHttp=true property. (Eric Pugh)  
 
+* SOLR-15154: Let Http2SolrClient pass Basic Auth credentials to all requests (Tomás Fernández Löbbe)
+
 Optimizations
 ---------------------
 * SOLR-15079: Block Collapse - Faster collapse code when groups are co-located via Block Join style nested doc indexing.
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 1543fbe..1ddf4ef 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
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.invoke.MethodHandles;
+import java.lang.reflect.InvocationTargetException;
 import java.net.ConnectException;
 import java.net.MalformedURLException;
 import java.net.URL;
@@ -137,6 +138,8 @@ public class Http2SolrClient extends SolrClient {
   private ExecutorService executor;
   private boolean shutdownExecutor;
 
+  private final String basicAuthAuthorizationStr;
+
   protected Http2SolrClient(String serverBaseUrl, Builder builder) {
     if (serverBaseUrl != null)  {
       if (!serverBaseUrl.equals("/") && serverBaseUrl.endsWith("/")) {
@@ -158,6 +161,11 @@ public class Http2SolrClient extends SolrClient {
     } else {
       httpClient = builder.http2SolrClient.httpClient;
     }
+    if (builder.basicAuthUser != null && builder.basicAuthPassword != null) {
+      basicAuthAuthorizationStr = basicAuthCredentialsToAuthorizationString(builder.basicAuthUser, builder.basicAuthPassword);
+    } else {
+      basicAuthAuthorizationStr = null;
+    }
     assert ObjectReleaseTracker.track(this);
   }
 
@@ -231,6 +239,7 @@ public class Http2SolrClient extends SolrClient {
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
+
     return httpClient;
   }
 
@@ -470,12 +479,18 @@ public class Http2SolrClient extends SolrClient {
 
   private void setBasicAuthHeader(@SuppressWarnings({"rawtypes"})SolrRequest solrRequest, Request req) {
     if (solrRequest.getBasicAuthUser() != null && solrRequest.getBasicAuthPassword() != null) {
-      String userPass = solrRequest.getBasicAuthUser() + ":" + solrRequest.getBasicAuthPassword();
-      String encoded = Base64.byteArrayToBase64(userPass.getBytes(FALLBACK_CHARSET));
-      req.header("Authorization", "Basic " + encoded);
+      String encoded = basicAuthCredentialsToAuthorizationString(solrRequest.getBasicAuthUser(), solrRequest.getBasicAuthPassword());
+      req.header("Authorization", encoded);
+    } else if (basicAuthAuthorizationStr != null) {
+      req.header("Authorization", basicAuthAuthorizationStr);
     }
   }
 
+  private String basicAuthCredentialsToAuthorizationString(String user, String pass) {
+    String userPass = user + ":" + pass;
+    return "Basic " + Base64.byteArrayToBase64(userPass.getBytes(FALLBACK_CHARSET));
+  }
+
   private Request makeRequest(@SuppressWarnings({"rawtypes"})SolrRequest solrRequest, String collection)
       throws SolrServerException, IOException {
     Request req = createRequest(solrRequest, collection);
@@ -841,6 +856,8 @@ public class Http2SolrClient extends SolrClient {
     private Integer idleTimeout;
     private Integer connectionTimeout;
     private Integer maxConnectionsPerHost;
+    private String basicAuthUser;
+    private String basicAuthPassword;
     private boolean useHttp1_1 = Boolean.getBoolean("solr.http1");
     protected String baseSolrUrl;
     private ExecutorService executor;
@@ -854,7 +871,32 @@ public class Http2SolrClient extends SolrClient {
     }
 
     public Http2SolrClient build() {
-      return new Http2SolrClient(baseSolrUrl, this);
+      Http2SolrClient client = new Http2SolrClient(baseSolrUrl, this);
+      try {
+        httpClientBuilderSetup(client);
+      } catch (RuntimeException e) {
+        try {
+          client.close();
+        } catch (Exception exceptionOnClose) {
+          e.addSuppressed(exceptionOnClose);
+        }
+        throw e;
+      }
+      return client;
+    }
+
+    private void httpClientBuilderSetup(Http2SolrClient client) {
+      String factoryClassName = System.getProperty(HttpClientUtil.SYS_PROP_HTTP_CLIENT_BUILDER_FACTORY);
+      if (factoryClassName != null) {
+        log.debug ("Using Http Builder Factory: {}", factoryClassName);
+        HttpClientBuilderFactory factory;
+        try {
+          factory = (HttpClientBuilderFactory)Class.forName(factoryClassName).getConstructor().newInstance();
+        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException | InvocationTargetException | NoSuchMethodException e) {
+          throw new RuntimeException("Unable to instantiate " + Http2SolrClient.class.getName(), e);
+        }
+        factory.setup(client);
+      }
     }
 
     /**
@@ -875,6 +917,17 @@ public class Http2SolrClient extends SolrClient {
       return this;
     }
 
+    public Builder withBasicAuthCredentials(String user, String pass) {
+      if (user != null || pass != null) {
+        if (user == null || pass == null) {
+          throw new IllegalStateException("Invalid Authentication credentials. Either both username and password or none must be provided");
+        }
+      }
+      this.basicAuthUser = user;
+      this.basicAuthPassword = pass;
+      return this;
+    }
+
     /**
      * Set maxConnectionsPerHost for http1 connections, maximum number http2 connections is limited by 4
      */
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 8df7248..2b4f6c2 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
@@ -23,6 +23,7 @@ import java.io.InputStreamReader;
 import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.http.auth.AuthScope;
@@ -30,6 +31,7 @@ import org.apache.http.auth.UsernamePasswordCredentials;
 import org.apache.http.client.CredentialsProvider;
 import org.apache.http.impl.auth.BasicScheme;
 import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.solr.common.StringUtils;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.StrUtils;
@@ -42,7 +44,6 @@ import org.eclipse.jetty.client.WWWAuthenticationProtocolHandler;
  * HttpClientConfigurer implementation providing support for preemptive Http Basic authentication
  * scheme.
  */
-@SuppressWarnings({"rawtypes", "unchecked"})
 public class PreemptiveBasicAuthClientBuilderFactory implements HttpClientBuilderFactory {
   /**
    * A system property used to specify a properties file containing default parameters used for
@@ -59,45 +60,15 @@ public class PreemptiveBasicAuthClientBuilderFactory implements HttpClientBuilde
    */
   public static final String SYS_PROP_BASIC_AUTH_CREDENTIALS = "basicauth";
 
-  private static SolrParams defaultParams;
   private static PreemptiveAuth requestInterceptor = new PreemptiveAuth(new BasicScheme());
 
-  static {
-    String credentials = System.getProperty(SYS_PROP_BASIC_AUTH_CREDENTIALS);
-    String configFile = System.getProperty(SYS_PROP_HTTP_CLIENT_CONFIG);
-
-    if (credentials != null && configFile != null) {
-      throw new RuntimeException("Basic authentication credentials passed via a configuration file"
-          + " as well as java system property. Please choose one mechanism!");
-    }
-
-    if (credentials != null) {
-      List<String> ss = StrUtils.splitSmart(credentials, ':');
-      if (ss.size() != 2) {
-        throw new RuntimeException("Please provide 'basicauth' in the 'user:password' format");
-      }
-      Properties defaultProps = new Properties();
-      defaultProps.setProperty(HttpClientUtil.PROP_BASIC_AUTH_USER, ss.get(0));
-      defaultProps.setProperty(HttpClientUtil.PROP_BASIC_AUTH_PASS, ss.get(1));
-      defaultParams = new MapSolrParams(new HashMap(defaultProps));
-    }
-
-    if(configFile != null) {
-      try {
-        Properties defaultProps = new Properties();
-        defaultProps.load(new InputStreamReader(new FileInputStream(configFile), StandardCharsets.UTF_8));
-        defaultParams = new MapSolrParams(new HashMap(defaultProps));
-      } catch (IOException e) {
-        throw new IllegalArgumentException("Unable to read the Http client config file", e);
-      }
-    }
-  }
+  private static CredentialsResolver CREDENTIAL_RESOLVER = new CredentialsResolver();
 
   /**
    * This method enables configuring system wide defaults (apart from using a config file based approach).
    */
   public static void setDefaultSolrParams(SolrParams params) {
-    defaultParams = params;
+    CREDENTIAL_RESOLVER.defaultParams = params;
   }
 
   @Override
@@ -107,9 +78,13 @@ 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) {
+    final String basicAuthUser = CREDENTIAL_RESOLVER.defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_USER);
+    final String basicAuthPass = CREDENTIAL_RESOLVER.defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_PASS);
+    this.setup(client, basicAuthUser, basicAuthPass);
+  }
+
+  public void setup(Http2SolrClient client, String basicAuthUser, String basicAuthPass) {
+    if (basicAuthUser == null || basicAuthPass == null) {
       throw new IllegalArgumentException("username & password must be specified with " + getClass().getName());
     }
 
@@ -120,11 +95,12 @@ public class PreemptiveBasicAuthClientBuilderFactory implements HttpClientBuilde
     client.getProtocolHandlers().put(new ProxyAuthenticationProtocolHandler(client.getHttpClient()));
   }
 
+
   @Override
   public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
-    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) {
+    final String basicAuthUser = CREDENTIAL_RESOLVER.defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_USER);
+    final String basicAuthPass = CREDENTIAL_RESOLVER.defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_PASS);
+    if (basicAuthUser == null || basicAuthPass == null) {
       throw new IllegalArgumentException("username & password must be specified with " + getClass().getName());
     }
 
@@ -141,4 +117,41 @@ public class PreemptiveBasicAuthClientBuilderFactory implements HttpClientBuilde
     HttpClientUtil.addRequestInterceptor(requestInterceptor);
     return builder;
   }
+
+  static class CredentialsResolver {
+
+    public volatile SolrParams defaultParams;
+
+    public CredentialsResolver() {
+      String credentials = System.getProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_BASIC_AUTH_CREDENTIALS);
+      String configFile = System.getProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_HTTP_CLIENT_CONFIG);
+
+      if (credentials != null && configFile != null) {
+        throw new IllegalArgumentException("Basic authentication credentials passed via a configuration file"
+                + " as well as java system property. Please choose one mechanism!");
+      }
+
+      if (credentials != null) {
+        List<String> ss = StrUtils.splitSmart(credentials, ':');
+        if (ss.size() != 2 || StringUtils.isEmpty(ss.get(0)) || StringUtils.isEmpty(ss.get(1))) {
+          throw new IllegalArgumentException("Invalid Authentication credentials: Please provide 'basicauth' in the 'user:password' format");
+        }
+        defaultParams = new MapSolrParams(Map.of(HttpClientUtil.PROP_BASIC_AUTH_USER, ss.get(0), HttpClientUtil.PROP_BASIC_AUTH_PASS, ss.get(1)));
+      } else if (configFile != null) {
+        Properties defaultProps = new Properties();
+        try {
+          defaultProps.load(new InputStreamReader(new FileInputStream(configFile), StandardCharsets.UTF_8));
+        } catch (IOException e) {
+          throw new IllegalArgumentException("Unable to read credentials file at " + configFile, e);
+        }
+        Map<String, String> map = new HashMap<>();
+        defaultProps.forEach((k, v) -> map.put((String) k, (String) v));
+        defaultParams = new MapSolrParams(map);
+      } else {
+        defaultParams = null;
+      }
+
+    }
+  }
+
 }
\ No newline at end of file
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientTest.java
index 4bc1b6c..6d728b6 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientTest.java
@@ -17,42 +17,49 @@
 
 package org.apache.solr.client.solrj.impl;
 
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
 import org.apache.solr.SolrJettyTestBase;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettyConfig;
+import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.SuppressForbidden;
+import org.eclipse.jetty.client.WWWAuthenticationProtocolHandler;
+import org.eclipse.jetty.http.HttpStatus;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
 public class Http2SolrClientTest extends SolrJettyTestBase {
 
   private static final String EXPECTED_USER_AGENT = "Solr[" + Http2SolrClient.class.getName() + "] 2.0";
 
-
   public static class DebugServlet extends HttpServlet {
     public static void clear() {
       lastMethod = null;
@@ -61,6 +68,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       errorCode = null;
       queryString = null;
       cookies = null;
+      responseHeaders = null;
     }
 
     public static Integer errorCode = null;
@@ -69,11 +77,19 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
     public static Map<String,String[]> parameters = null;
     public static String queryString = null;
     public static javax.servlet.http.Cookie[] cookies = null;
+    public static List<String[]> responseHeaders = null;
 
     public static void setErrorCode(Integer code) {
       errorCode = code;
     }
 
+    public static void addResponseHeader(String headerName, String headerValue) {
+      if (responseHeaders == null) {
+        responseHeaders = new ArrayList<>();
+      }
+      responseHeaders.add(new String[]{headerName, headerValue});
+    }
+
     @Override
     protected void doDelete(HttpServletRequest req, HttpServletResponse resp)
         throws ServletException, IOException {
@@ -137,6 +153,11 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       setParameters(req);
       setQueryString(req);
       setCookies(req);
+      if (responseHeaders != null) {
+        for (String[] h : responseHeaders) {
+          resp.addHeader(h[0], h[1]);
+        }
+      }
       if (null != errorCode) {
         try {
           resp.sendError(errorCode);
@@ -158,6 +179,14 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
     createAndStartJetty(legacyExampleCollection1SolrHome(), jettyConfig);
   }
 
+  @Override
+  public void tearDown() throws Exception {
+    System.clearProperty("basicauth");
+    System.clearProperty(HttpClientUtil.SYS_PROP_HTTP_CLIENT_BUILDER_FACTORY);
+    DebugServlet.clear();
+    super.tearDown();
+  }
+
   private Http2SolrClient getHttp2SolrClient(String url, int connectionTimeOut, int socketTimeout) {
     return new Http2SolrClient.Builder(url)
         .connectionTimeout(connectionTimeOut)
@@ -616,6 +645,113 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
     System.clearProperty("javax.net.ssl.trustStoreType");
   }
 
+  protected void expectThrowsAndMessage(Class<? extends Exception> expectedType, ThrowingRunnable executable, String expectedMessage) {
+    Exception e = expectThrows(expectedType, executable);
+    assertTrue("Expecting message to contain \"" + expectedMessage + "\" but was: " + e.getMessage(), e.getMessage().contains(expectedMessage));
+  }
+
+  @Test
+  public void testBadExplicitCredentials() {
+    expectThrowsAndMessage(IllegalStateException.class, () -> new Http2SolrClient.Builder()
+            .withBasicAuthCredentials("foo", null), "Invalid Authentication credentials");
+    expectThrowsAndMessage(IllegalStateException.class, () -> new Http2SolrClient.Builder()
+            .withBasicAuthCredentials(null, "foo"), "Invalid Authentication credentials");
+  }
+
+  @Test
+  public void testSetCredentialsExplicitly() {
+    try (Http2SolrClient client = new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo")
+            .withBasicAuthCredentials("foo", "explicit")
+            .build();) {
+      QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
+      try {
+        ignoreException("Error from server");
+        client.request(r);
+      } catch (Exception e) {
+        // expected
+      }
+      unIgnoreException("Error from server");
+      assertTrue(DebugServlet.headers.size() > 0);
+      String authorizationHeader = DebugServlet.headers.get("authorization");
+      assertNotNull("No authorization information in headers found. Headers: " + DebugServlet.headers, authorizationHeader);
+      assertEquals("Basic " + Base64.byteArrayToBase64("foo:explicit".getBytes(StandardCharsets.UTF_8)),  authorizationHeader);
+    }
+  }
+
+  @Test
+  public void testSetCredentialsWithSysProps() throws IOException, SolrServerException {
+    System.setProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_BASIC_AUTH_CREDENTIALS, "foo:bar");
+    System.setProperty(HttpClientUtil.SYS_PROP_HTTP_CLIENT_BUILDER_FACTORY, PreemptiveBasicAuthClientBuilderFactory.class.getName());
+    // Hack to ensure we get a new set of parameters for this test
+    PreemptiveBasicAuthClientBuilderFactory.setDefaultSolrParams(new PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver().defaultParams);
+    try (Http2SolrClient client = new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo").build();) {
+      QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
+      DebugServlet.addResponseHeader(WWWAuthenticationProtocolHandler.NAME, "Basic realm=\"Debug Servlet\"");
+      DebugServlet.setErrorCode(HttpStatus.UNAUTHORIZED_401);
+      try {
+        client.request(r);
+      } catch (Exception e) {
+        // expected
+      }
+      assertTrue(DebugServlet.headers.size() > 0);
+      String authorizationHeader = DebugServlet.headers.get("authorization");
+      assertNotNull("No authorization information in headers found. Headers: " + DebugServlet.headers, authorizationHeader);
+      assertEquals("Basic " + Base64.byteArrayToBase64("foo:bar".getBytes(StandardCharsets.UTF_8)),  authorizationHeader);
+    } finally {
+      System.clearProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_BASIC_AUTH_CREDENTIALS);
+      System.clearProperty(HttpClientUtil.SYS_PROP_HTTP_CLIENT_BUILDER_FACTORY);
+      PreemptiveBasicAuthClientBuilderFactory.setDefaultSolrParams(new MapSolrParams(new HashMap<>()));
+    }
+  }
+
+  @Test
+  public void testPerRequestCredentialsWin() {
+    try (Http2SolrClient client = new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo")
+            .withBasicAuthCredentials("foo2", "explicit").build();) {
+      QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
+      r.setBasicAuthCredentials("foo3", "per-request");
+      try {
+        ignoreException("Error from server");
+        client.request(r);
+      } catch (Exception e) {
+        // expected
+      }
+      unIgnoreException("Error from server");
+      assertTrue(DebugServlet.headers.size() > 0);
+      String authorizationHeader = DebugServlet.headers.get("authorization");
+      assertNotNull("No authorization information in headers found. Headers: " + DebugServlet.headers, authorizationHeader);
+      assertEquals("Basic " + Base64.byteArrayToBase64("foo3:per-request".getBytes(StandardCharsets.UTF_8)),  authorizationHeader);
+    } finally {
+      System.clearProperty("basicauth");
+    }
+  }
+
+  @Test
+  public void testNoCredentials() {
+    try (Http2SolrClient client = new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo").build();) {
+      QueryRequest r = new QueryRequest(new SolrQuery("quick brown fox"));
+      try {
+        ignoreException("Error from server");
+        client.request(r);
+      } catch (Exception e) {
+        // expected
+      }
+      unIgnoreException("Error from server");
+      assertFalse("Expecting no authorization header but got: " + DebugServlet.headers, DebugServlet.headers.containsKey("authorization"));
+    }
+  }
+
+  @Test
+  public void testBadHttpFactory() {
+    System.setProperty(HttpClientUtil.SYS_PROP_HTTP_CLIENT_BUILDER_FACTORY, "FakeClassName");
+    try {
+      client = new Http2SolrClient.Builder(jetty.getBaseUrl().toString() + "/debug/foo").build();
+      fail("Expecting exception");
+    } catch (RuntimeException e ) {
+      assertTrue(e.getMessage().contains("Unable to instantiate"));
+    }
+  }
+
   /**
    * Missed tests :
    * - set cookies via interceptor
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/PreemptiveBasicAuthClientBuilderFactoryTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/PreemptiveBasicAuthClientBuilderFactoryTest.java
new file mode 100644
index 0000000..2da0def
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/PreemptiveBasicAuthClientBuilderFactoryTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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 org.apache.solr.SolrTestCase;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Properties;
+
+public class PreemptiveBasicAuthClientBuilderFactoryTest extends SolrTestCase {
+
+    private void assertIllegalArgumentException(ThrowingRunnable executable, String expectedMessage) {
+        Exception e = expectThrows(IllegalArgumentException.class, executable);
+        assertTrue("Expecting message to contain \"" + expectedMessage + "\" but was: " + e.getMessage(), e.getMessage().contains(expectedMessage));
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        System.clearProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_BASIC_AUTH_CREDENTIALS);
+        System.clearProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_HTTP_CLIENT_CONFIG);
+        super.tearDown();
+    }
+
+    public void testBadSysPropsCredentials() {
+        System.setProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_BASIC_AUTH_CREDENTIALS, "foo");
+        assertIllegalArgumentException(PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver::new, "Invalid Authentication credentials");
+        System.setProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_BASIC_AUTH_CREDENTIALS, "foo:");
+        assertIllegalArgumentException(PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver::new, "Invalid Authentication credentials");
+        System.setProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_BASIC_AUTH_CREDENTIALS, ":foo");
+        assertIllegalArgumentException(PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver::new, "Invalid Authentication credentials");
+    }
+
+    public void testSysPropsAndPropsFile() {
+        System.setProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_BASIC_AUTH_CREDENTIALS, "foo");
+        System.setProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_HTTP_CLIENT_CONFIG, "foo");
+        assertIllegalArgumentException(PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver::new, "Basic authentication credentials passed");
+
+    }
+
+    public void testCredentialsFromSystemProperties() {
+        System.setProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_BASIC_AUTH_CREDENTIALS, "foo:bar");
+        PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver credentialsResolver = new PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver();
+        assertEquals("foo", credentialsResolver.defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_USER));
+        assertEquals("bar", credentialsResolver.defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_PASS));
+    }
+
+    public void testCredentialsFromConfigFile() throws IOException {
+        Properties p = new Properties();
+        p.setProperty("httpBasicAuthUser", "foo");
+        p.setProperty("httpBasicAuthPassword", "bar");
+        File f = createTempFile().toFile();
+        try (FileWriter fw = new FileWriter(f, StandardCharsets.UTF_8)) {
+            p.store(fw, "tmp properties file for PreemptiveBasicAuthClientBuilderFactoryTest.testCredentialsFromConfigFile");
+        }
+        System.setProperty(PreemptiveBasicAuthClientBuilderFactory.SYS_PROP_HTTP_CLIENT_CONFIG, f.getAbsolutePath());
+        PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver credentialsResolver = new PreemptiveBasicAuthClientBuilderFactory.CredentialsResolver();
+        assertEquals("foo", credentialsResolver.defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_USER));
+        assertEquals("bar", credentialsResolver.defaultParams.get(HttpClientUtil.PROP_BASIC_AUTH_PASS));
+    }
+}