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

[lucene-solr] branch branch_8x updated: SOLR-14456: Fix Content-Type header forwarding on compressed requests (#1480)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 7b32e68  SOLR-14456: Fix Content-Type header forwarding on compressed requests (#1480)
7b32e68 is described below

commit 7b32e68d054adb511999a97da8bcc2ad5a8a2428
Author: Samuel García Martínez <sa...@gmail.com>
AuthorDate: Tue May 12 23:50:07 2020 +0200

    SOLR-14456: Fix Content-Type header forwarding on compressed requests (#1480)
    
    Co-authored-by: Samuel García Martínez <sa...@inditex.com>
---
 solr/CHANGES.txt                                   |  3 +
 .../java/org/apache/solr/servlet/HttpSolrCall.java |  2 +-
 .../solr/client/solrj/impl/Http2SolrClient.java    | 65 ++++++++++------------
 .../solr/client/solrj/impl/HttpSolrClient.java     | 46 ++++++++-------
 .../client/solrj/impl/BasicHttpSolrClientTest.java | 45 ++++++++-------
 .../impl/Http2SolrClientCompatibilityTest.java     |  7 +--
 .../client/solrj/impl/Http2SolrClientTest.java     | 41 +++++++-------
 7 files changed, 100 insertions(+), 109 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4a34de0..0d1952f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -91,6 +91,9 @@ Bug Fixes
 
 * SOLR-14463: Solr Admin ZkStatus page now works with ZK 3.6, without 'For input string: "null"' error (janhoy, Bernd Wahlen)
 
+* SOLR-14456: Fix Content-Type header usage when a request is forwarded from Solr node to Solr
+  node with compression enabled (samuelgmartinez via Houston Putman)
+
 Other Changes
 ---------------------
 * SOLR-14197: SolrResourceLoader: marked many methods as deprecated, and in some cases rerouted exiting logic to avoid
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 d6ab2fd..99cc3b0 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -726,7 +726,7 @@ public class HttpSolrCall {
 
       if (httpEntity != null) {
         if (httpEntity.getContentEncoding() != null)
-          resp.setCharacterEncoding(httpEntity.getContentEncoding().getValue());
+          resp.setHeader(httpEntity.getContentEncoding().getName(), httpEntity.getContentEncoding().getValue());
         if (httpEntity.getContentType() != null) resp.setContentType(httpEntity.getContentType().getValue());
 
         InputStream is = httpEntity.getContent();
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 6a08816..bdb4291 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
@@ -25,6 +25,7 @@ import java.lang.invoke.MethodHandles;
 import java.net.ConnectException;
 import java.net.MalformedURLException;
 import java.net.URL;
+import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Collection;
@@ -96,7 +97,8 @@ import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.client.solrj.impl.BaseHttpSolrClient.*;
+import static org.apache.solr.client.solrj.impl.BaseHttpSolrClient.RemoteExecutionException;
+import static org.apache.solr.client.solrj.impl.BaseHttpSolrClient.RemoteSolrException;
 import static org.apache.solr.common.util.Utils.getObjectByPath;
 
 /**
@@ -116,7 +118,7 @@ public class Http2SolrClient extends SolrClient {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final String AGENT = "Solr[" + Http2SolrClient.class.getName() + "] 2.0";
-  private static final String UTF_8 = StandardCharsets.UTF_8.name();
+  private static final Charset FALLBACK_CHARSET = StandardCharsets.UTF_8;
   private static final String DEFAULT_PATH = "/select";
   private static final List<String> errPath = Arrays.asList("metadata", "error-class");
 
@@ -285,7 +287,7 @@ public class Http2SolrClient extends SolrClient {
     @Override
     public void close() throws IOException {
       if (isXml) {
-        write("</stream>".getBytes(StandardCharsets.UTF_8));
+        write("</stream>".getBytes(FALLBACK_CHARSET));
       }
       this.outProvider.getOutputStream().close();
     }
@@ -329,7 +331,7 @@ public class Http2SolrClient extends SolrClient {
     OutStream outStream = new OutStream(collection, origParams, provider, responseListener,
         isXml);
     if (isXml) {
-      outStream.write("<stream>".getBytes(StandardCharsets.UTF_8));
+      outStream.write("<stream>".getBytes(FALLBACK_CHARSET));
     }
     return outStream;
   }
@@ -351,7 +353,7 @@ public class Http2SolrClient extends SolrClient {
           byte[] content = String.format(Locale.ROOT,
               fmt, params.getBool(UpdateParams.WAIT_SEARCHER, false)
                   + "")
-              .getBytes(StandardCharsets.UTF_8);
+              .getBytes(FALLBACK_CHARSET);
           outStream.write(content);
         }
       }
@@ -383,7 +385,7 @@ public class Http2SolrClient extends SolrClient {
             InputStream is = getContentAsInputStream();
             assert ObjectReleaseTracker.track(is);
             rsp = processErrorsAndResponse(result.getResponse(),
-                parser, is, getEncoding(), isV2ApiRequest(solrRequest));
+                parser, is, getMediaType(), getEncoding(), isV2ApiRequest(solrRequest));
             onComplete.onSuccess(rsp);
           } catch (Exception e) {
             onComplete.onFailure(e);
@@ -398,7 +400,15 @@ public class Http2SolrClient extends SolrClient {
         Response response = listener.get(idleTimeout, TimeUnit.MILLISECONDS);
         InputStream is = listener.getInputStream();
         assert ObjectReleaseTracker.track(is);
-        return processErrorsAndResponse(response, parser, is, getEncoding(response), isV2ApiRequest(solrRequest));
+
+        ContentType contentType = getContentType(response);
+        String mimeType = null;
+        String encoding = null;
+        if (contentType != null) {
+          mimeType = contentType.getMimeType();
+          encoding = contentType.getCharset() != null? contentType.getCharset().name() : null;
+        }
+        return processErrorsAndResponse(response, parser, is, mimeType, encoding, isV2ApiRequest(solrRequest));
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         throw new RuntimeException(e);
@@ -421,31 +431,15 @@ public class Http2SolrClient extends SolrClient {
     }
   }
 
-  private String getEncoding(Response response) {
+  private ContentType getContentType(Response response) {
     String contentType = response.getHeaders().get(HttpHeader.CONTENT_TYPE);
-    if (contentType != null) {
-      String charset = "charset=";
-      int index = contentType.toLowerCase(Locale.ENGLISH).indexOf(charset);
-      if (index > 0) {
-        String encoding = contentType.substring(index + charset.length());
-        // Sometimes charsets arrive with an ending semicolon.
-        int semicolon = encoding.indexOf(';');
-        if (semicolon > 0)
-          encoding = encoding.substring(0, semicolon).trim();
-        // Sometimes charsets are quoted.
-        int lastIndex = encoding.length() - 1;
-        if (encoding.charAt(0) == '"' && encoding.charAt(lastIndex) == '"')
-          encoding = encoding.substring(1, lastIndex).trim();
-        return encoding;
-      }
-    }
-    return null;
+    return StringUtils.isEmpty(contentType)? null : ContentType.parse(contentType);
   }
 
   private void setBasicAuthHeader(SolrRequest solrRequest, Request req) {
     if (solrRequest.getBasicAuthUser() != null && solrRequest.getBasicAuthPassword() != null) {
       String userPass = solrRequest.getBasicAuthUser() + ":" + solrRequest.getBasicAuthPassword();
-      String encoded = Base64.byteArrayToBase64(userPass.getBytes(StandardCharsets.UTF_8));
+      String encoded = Base64.byteArrayToBase64(userPass.getBytes(FALLBACK_CHARSET));
       req.header("Authorization", "Basic " + encoded);
     }
   }
@@ -626,7 +620,7 @@ public class Http2SolrClient extends SolrClient {
           }
         }
       }
-      req.content(new FormContentProvider(fields, StandardCharsets.UTF_8));
+      req.content(new FormContentProvider(fields, FALLBACK_CHARSET));
     }
 
     return req;
@@ -639,6 +633,7 @@ public class Http2SolrClient extends SolrClient {
   private NamedList<Object> processErrorsAndResponse(Response response,
                                                      final ResponseParser processor,
                                                      InputStream is,
+                                                     String mimeType,
                                                      String encoding,
                                                      final boolean isV2Api)
       throws SolrServerException {
@@ -647,10 +642,6 @@ public class Http2SolrClient extends SolrClient {
       // handle some http level checks before trying to parse the response
       int httpStatus = response.getStatus();
 
-      String contentType;
-      contentType = response.getHeaders().get("content-type");
-      if (contentType == null) contentType = "";
-
       switch (httpStatus) {
         case HttpStatus.SC_OK:
         case HttpStatus.SC_BAD_REQUEST:
@@ -664,7 +655,7 @@ public class Http2SolrClient extends SolrClient {
           }
           break;
         default:
-          if (processor == null || "".equals(contentType)) {
+          if (processor == null || mimeType == null) {
             throw new RemoteSolrException(serverBaseUrl, httpStatus, "non ok status: " + httpStatus
                 + ", message:" + response.getReason(),
                 null);
@@ -683,14 +674,14 @@ public class Http2SolrClient extends SolrClient {
       String procCt = processor.getContentType();
       if (procCt != null) {
         String procMimeType = ContentType.parse(procCt).getMimeType().trim().toLowerCase(Locale.ROOT);
-        String mimeType = ContentType.parse(contentType).getMimeType().trim().toLowerCase(Locale.ROOT);
         if (!procMimeType.equals(mimeType)) {
           // unexpected mime type
           String msg = "Expected mime type " + procMimeType + " but got " + mimeType + ".";
+          String exceptionEncoding = encoding != null? encoding : FALLBACK_CHARSET.name();
           try {
-            msg = msg + " " + IOUtils.toString(is, encoding);
+            msg = msg + " " + IOUtils.toString(is, exceptionEncoding);
           } catch (IOException e) {
-            throw new RemoteSolrException(serverBaseUrl, httpStatus, "Could not parse response with encoding " + encoding, e);
+            throw new RemoteSolrException(serverBaseUrl, httpStatus, "Could not parse response with encoding " + exceptionEncoding, e);
           }
           throw new RemoteSolrException(serverBaseUrl, httpStatus, msg, null);
         }
@@ -721,14 +712,16 @@ public class Http2SolrClient extends SolrClient {
           }
         } catch (Exception ex) {}
         if (reason == null) {
+
           StringBuilder msg = new StringBuilder();
           msg.append(response.getReason())
               .append("\n\n")
               .append("request: ")
               .append(response.getRequest().getMethod());
           try {
-            reason = java.net.URLDecoder.decode(msg.toString(), UTF_8);
+            reason = java.net.URLDecoder.decode(msg.toString(), FALLBACK_CHARSET.name());
           } catch (UnsupportedEncodingException e) {
+            // This shouldn't occur because FALLBACK_CHARSET will always be supported.
           }
         }
         RemoteSolrException rss = new RemoteSolrException(serverBaseUrl, httpStatus, reason, null);
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 4dd32c8..c4b68f3 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
@@ -25,6 +25,7 @@ import java.net.ConnectException;
 import java.net.MalformedURLException;
 import java.net.SocketTimeoutException;
 import java.net.URL;
+import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 import java.security.Principal;
 import java.util.Arrays;
@@ -67,7 +68,6 @@ import org.apache.http.entity.mime.content.InputStreamBody;
 import org.apache.http.entity.mime.content.StringBody;
 import org.apache.http.message.BasicHeader;
 import org.apache.http.message.BasicNameValuePair;
-import org.apache.http.util.EntityUtils;
 import org.apache.solr.client.solrj.ResponseParser;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -95,7 +95,7 @@ import static org.apache.solr.common.util.Utils.getObjectByPath;
  */
 public class HttpSolrClient extends BaseHttpSolrClient {
 
-  private static final String UTF_8 = StandardCharsets.UTF_8.name();
+  private static final Charset FALLBACK_CHARSET = StandardCharsets.UTF_8;
   private static final String DEFAULT_PATH = "/select";
   private static final long serialVersionUID = -946812319974801896L;
   
@@ -272,7 +272,7 @@ public class HttpSolrClient extends BaseHttpSolrClient {
   private void setBasicAuthHeader(SolrRequest request, HttpRequestBase method) throws UnsupportedEncodingException {
     if (request.getBasicAuthUser() != null && request.getBasicAuthPassword() != null) {
       String userPass = request.getBasicAuthUser() + ":" + request.getBasicAuthPassword();
-      String encoded = Base64.byteArrayToBase64(userPass.getBytes(UTF_8));
+      String encoded = Base64.byteArrayToBase64(userPass.getBytes(FALLBACK_CHARSET));
       method.setHeader(new BasicHeader("Authorization", "Basic " + encoded));
     }
   }
@@ -568,12 +568,18 @@ public class HttpSolrClient extends BaseHttpSolrClient {
       // Read the contents
       entity = response.getEntity();
       respBody = entity.getContent();
-      Header ctHeader = response.getLastHeader("content-type");
-      String contentType;
-      if (ctHeader != null) {
-        contentType = ctHeader.getValue();
-      } else {
-        contentType = "";
+      String mimeType = null;
+      Charset charset = null;
+      String charsetName = null;
+
+      ContentType contentType = ContentType.get(entity);
+      if (contentType != null) {
+        mimeType = contentType.getMimeType().trim().toLowerCase(Locale.ROOT);
+        charset = contentType.getCharset();
+
+        if (charset != null) {
+          charsetName = charset.name();
+        }
       }
 
       // handle some http level checks before trying to parse the response
@@ -590,7 +596,7 @@ public class HttpSolrClient extends BaseHttpSolrClient {
           }
           break;
         default:
-          if (processor == null || "".equals(contentType)) {
+          if (processor == null || contentType == null) {
             throw new RemoteSolrException(baseUrl, httpStatus, "non ok status: " + httpStatus
                 + ", message:" + response.getStatusLine().getReasonPhrase(),
                 null);
@@ -606,34 +612,26 @@ public class HttpSolrClient extends BaseHttpSolrClient {
         shouldClose = false;
         return rsp;
       }
-      
+
       String procCt = processor.getContentType();
       if (procCt != null) {
         String procMimeType = ContentType.parse(procCt).getMimeType().trim().toLowerCase(Locale.ROOT);
-        String mimeType = ContentType.parse(contentType).getMimeType().trim().toLowerCase(Locale.ROOT);
         if (!procMimeType.equals(mimeType)) {
           // unexpected mime type
           String msg = "Expected mime type " + procMimeType + " but got " + mimeType + ".";
-          Header encodingHeader = response.getEntity().getContentEncoding();
-          String encoding;
-          if (encodingHeader != null) {
-            encoding = encodingHeader.getValue();
-          } else {
-            encoding = "UTF-8"; // try UTF-8
-          }
+          Charset exceptionCharset = charset != null? charset : FALLBACK_CHARSET;
           try {
-            msg = msg + " " + IOUtils.toString(respBody, encoding);
+            msg = msg + " " + IOUtils.toString(respBody, exceptionCharset);
           } catch (IOException e) {
-            throw new RemoteSolrException(baseUrl, httpStatus, "Could not parse response with encoding " + encoding, e);
+            throw new RemoteSolrException(baseUrl, httpStatus, "Could not parse response with encoding " + exceptionCharset, e);
           }
           throw new RemoteSolrException(baseUrl, httpStatus, msg, null);
         }
       }
       
       NamedList<Object> rsp = null;
-      String charset = EntityUtils.getContentCharSet(response.getEntity());
       try {
-        rsp = processor.processResponse(respBody, charset);
+        rsp = processor.processResponse(respBody, charsetName);
       } catch (Exception e) {
         throw new RemoteSolrException(baseUrl, httpStatus, e.getMessage(), e);
       }
@@ -660,7 +658,7 @@ public class HttpSolrClient extends BaseHttpSolrClient {
             .append("\n\n")
             .append("request: ")
             .append(method.getURI());
-          reason = java.net.URLDecoder.decode(msg.toString(), UTF_8);
+          reason = java.net.URLDecoder.decode(msg.toString(), FALLBACK_CHARSET.name());
         }
         RemoteSolrException rss = new RemoteSolrException(baseUrl, httpStatus, reason, null);
         if (metadata != null) rss.setMetadata(metadata);
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 ec77736..22c27eb 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
@@ -41,7 +41,6 @@ import org.apache.http.HttpException;
 import org.apache.http.HttpRequest;
 import org.apache.http.HttpRequestInterceptor;
 import org.apache.http.HttpResponse;
-import org.apache.http.ParseException;
 import org.apache.http.client.CookieStore;
 import org.apache.http.client.config.RequestConfig;
 import org.apache.http.client.methods.HttpGet;
@@ -241,7 +240,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
     try (HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/debug/foo")) {
       SolrQuery q = new SolrQuery("foo");
       q.setParam("a", "\u1234");
-      expectThrows(ParseException.class, () -> client.query(q, METHOD.GET));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.query(q, METHOD.GET));
 
       //default method
       assertEquals("get", DebugServlet.lastMethod);
@@ -265,7 +264,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
 
       //POST
       DebugServlet.clear();
-      expectThrows(ParseException.class, () -> client.query(q, METHOD.POST));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.query(q, METHOD.POST));
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals("Solr[" + HttpSolrClient.class.getName() + "] 1.0", DebugServlet.headers.get("User-Agent"));
@@ -281,7 +280,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
 
       //PUT
       DebugServlet.clear();
-      expectThrows(ParseException.class, () -> client.query(q, METHOD.PUT));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.query(q, METHOD.PUT));
 
       assertEquals("put", DebugServlet.lastMethod);
       assertEquals("Solr[" + HttpSolrClient.class.getName() + "] 1.0", DebugServlet.headers.get("User-Agent"));
@@ -298,7 +297,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       //XML/GET
       client.setParser(new XMLResponseParser());
       DebugServlet.clear();
-      expectThrows(ParseException.class, () -> client.query(q, METHOD.GET));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.query(q, METHOD.GET));
 
       assertEquals("get", DebugServlet.lastMethod);
       assertEquals("Solr[" + HttpSolrClient.class.getName() + "] 1.0", DebugServlet.headers.get("User-Agent"));
@@ -314,7 +313,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       //XML/POST
       client.setParser(new XMLResponseParser());
       DebugServlet.clear();
-      expectThrows(ParseException.class, () -> client.query(q, METHOD.POST));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.query(q, METHOD.POST));
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals("Solr[" + HttpSolrClient.class.getName() + "] 1.0", DebugServlet.headers.get("User-Agent"));
@@ -330,7 +329,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
 
       client.setParser(new XMLResponseParser());
       DebugServlet.clear();
-      expectThrows(ParseException.class, () -> client.query(q, METHOD.PUT));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.query(q, METHOD.PUT));
 
       assertEquals("put", DebugServlet.lastMethod);
       assertEquals("Solr[" + HttpSolrClient.class.getName() + "] 1.0", DebugServlet.headers.get("User-Agent"));
@@ -351,7 +350,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
   public void testDelete() throws Exception {
     DebugServlet.clear();
     try (HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/debug/foo")) {
-      expectThrows(ParseException.class, () -> client.deleteById("id"));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.deleteById("id"));
 
       //default method
       assertEquals("post", DebugServlet.lastMethod);
@@ -370,7 +369,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
 
       //XML
       client.setParser(new XMLResponseParser());
-      expectThrows(ParseException.class, () -> client.deleteByQuery("*:*"));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.deleteByQuery("*:*"));
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals("Solr[" + HttpSolrClient.class.getName() + "] 1.0", DebugServlet.headers.get("User-Agent"));
@@ -389,10 +388,10 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
     DebugServlet.clear();
     try (HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/debug/foo")) {
       Collection<String> ids = Collections.singletonList("a");
-      expectThrows(ParseException.class, () -> client.getById("a"));
-      expectThrows(ParseException.class, () -> client.getById(ids, null));
-      expectThrows(ParseException.class, () -> client.getById("foo", "a"));
-      expectThrows(ParseException.class, () -> client.getById("foo", ids, null));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.getById("a"));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.getById(ids, null));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.getById("foo", "a"));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.getById("foo", ids, null));
     }
   }
 
@@ -403,7 +402,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       UpdateRequest req = new UpdateRequest();
       req.add(new SolrInputDocument());
       req.setParam("a", "\u1234");
-      expectThrows(ParseException.class, () -> client.request(req));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.request(req));
 
       //default method
       assertEquals("post", DebugServlet.lastMethod);
@@ -424,7 +423,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       //XML response and writer
       client.setParser(new XMLResponseParser());
       client.setRequestWriter(new RequestWriter());
-      expectThrows(ParseException.class, () -> client.request(req));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.request(req));
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals("Solr[" + HttpSolrClient.class.getName() + "] 1.0", DebugServlet.headers.get("User-Agent"));
@@ -440,7 +439,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       client.setParser(new BinaryResponseParser());
       client.setRequestWriter(new BinaryRequestWriter());
       DebugServlet.clear();
-      expectThrows(ParseException.class, () -> client.request(req));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.request(req));
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals("Solr[" + HttpSolrClient.class.getName() + "] 1.0", DebugServlet.headers.get("User-Agent"));
@@ -483,21 +482,21 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
     try (HttpSolrClient client = getHttpSolrClient(clientUrl)) {
       // verify request header gets set
       DebugServlet.clear();
-      expectThrows(ParseException.class, () -> client.query(q));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.query(q));
       assertNull(DebugServlet.headers.toString(), DebugServlet.headers.get("Accept-Encoding"));
     }
     
     try (HttpSolrClient client = getHttpSolrClient(clientUrl, null, null, true)) {
       try {
         client.query(q);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
       assertNotNull(DebugServlet.headers.get("Accept-Encoding"));
     }
     
     try (HttpSolrClient client = getHttpSolrClient(clientUrl, null, null, false)) {
       try {
         client.query(q);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
     }
     assertNull(DebugServlet.headers.get("Accept-Encoding"));
     
@@ -705,7 +704,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       client.setQueryParams(setOf("serverOnly"));
       UpdateRequest req = new UpdateRequest();
       setReqParamsOf(req, "serverOnly", "notServer");
-      expectThrows(ParseException.class, () -> client.request(req));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.request(req));
       verifyServletState(client, req);
   
       // test without server query params
@@ -714,7 +713,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       UpdateRequest req2 = new UpdateRequest();
       req2.setQueryParams(setOf("requestOnly"));
       setReqParamsOf(req2, "requestOnly", "notRequest");
-      expectThrows(ParseException.class, () -> client.request(req2));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.request(req2));
       verifyServletState(client, req2);
   
       // test with both request and server query params
@@ -723,7 +722,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       client.setQueryParams(setOf("serverOnly", "both"));
       req3.setQueryParams(setOf("requestOnly", "both"));
       setReqParamsOf(req3, "serverOnly", "requestOnly", "both", "neither");
-      expectThrows(ParseException.class, () -> client.request(req3));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.request(req3));
       verifyServletState(client, req3);
   
       // test with both request and server query params with single stream
@@ -733,7 +732,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
       client.setQueryParams(setOf("serverOnly", "both"));
       req4.setQueryParams(setOf("requestOnly", "both"));
       setReqParamsOf(req4, "serverOnly", "requestOnly", "both", "neither");
-      expectThrows(ParseException.class, () -> client.request(req4));
+      expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> client.request(req4));
       // NOTE: single stream requests send all the params
       // as part of the query string.  So add "neither" to the request
       // so it passes the verification step.
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientCompatibilityTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientCompatibilityTest.java
index 81f64b6..16f6875 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientCompatibilityTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/Http2SolrClientCompatibilityTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.solr.client.solrj.impl;
 
-import org.apache.http.ParseException;
 import org.apache.solr.SolrJettyTestBase;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrQuery;
@@ -60,7 +59,7 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
       assertTrue(client.getHttpClient().getTransport() instanceof HttpClientTransportOverHTTP);
       try {
         client.query(new SolrQuery("*:*"), SolrRequest.METHOD.GET);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
     } finally {
       afterSolrJettyTestBase();
     }
@@ -80,7 +79,7 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
       assertTrue(client.getHttpClient().getTransport() instanceof HttpClientTransportOverHTTP);
       try {
         client.query(new SolrQuery("*:*"), SolrRequest.METHOD.GET);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
     } finally {
       afterSolrJettyTestBase();
     }
@@ -102,7 +101,7 @@ public class Http2SolrClientCompatibilityTest extends SolrJettyTestBase {
       try {
         client.query(new SolrQuery("*:*"), SolrRequest.METHOD.GET);
         fail("Jetty client with HTTP2 transport should not be able to connect to HTTP1 only nodes");
-      } catch (ParseException ignored) {
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {
         fail("Jetty client with HTTP2 transport should not be able to connect to HTTP1 only nodes");
       } catch (SolrServerException e) {
         // expected
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 6462f2c..53df0ca 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
@@ -32,7 +32,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.http.ParseException;
 import org.apache.solr.SolrJettyTestBase;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
@@ -189,7 +188,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
     try(Http2SolrClient client = getHttp2SolrClient(jetty.getBaseUrl().toString() + "/debug/foo", DEFAULT_CONNECTION_TIMEOUT, 0)) {
       try {
         client.query(q, SolrRequest.METHOD.GET);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
     }
 
   }
@@ -227,7 +226,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       q.setParam("a", "\u1234");
       try {
         client.query(q, SolrRequest.METHOD.GET);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       //default method
       assertEquals("get", DebugServlet.lastMethod);
@@ -251,7 +250,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       DebugServlet.clear();
       try {
         client.query(q, SolrRequest.METHOD.POST);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
@@ -268,7 +267,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       DebugServlet.clear();
       try {
         client.query(q, SolrRequest.METHOD.PUT);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       assertEquals("put", DebugServlet.lastMethod);
       assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
@@ -286,7 +285,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       DebugServlet.clear();
       try {
         client.query(q, SolrRequest.METHOD.GET);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       assertEquals("get", DebugServlet.lastMethod);
       assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
@@ -303,7 +302,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       DebugServlet.clear();
       try {
         client.query(q, SolrRequest.METHOD.POST);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
@@ -320,7 +319,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       DebugServlet.clear();
       try {
         client.query(q, SolrRequest.METHOD.PUT);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       assertEquals("put", DebugServlet.lastMethod);
       assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
@@ -342,7 +341,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
     try (Http2SolrClient client = getHttp2SolrClient(jetty.getBaseUrl().toString() + "/debug/foo")) {
       try {
         client.deleteById("id");
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       //default method
       assertEquals("post", DebugServlet.lastMethod);
@@ -361,7 +360,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       client.setParser(new XMLResponseParser());
       try {
         client.deleteByQuery("*:*");
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
@@ -381,19 +380,19 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       Collection<String> ids = Collections.singletonList("a");
       try {
         client.getById("a");
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       try {
         client.getById(ids, null);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       try {
         client.getById("foo", "a");
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       try {
         client.getById("foo", ids, null);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
     }
   }
 
@@ -406,7 +405,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       req.setParam("a", "\u1234");
       try {
         client.request(req);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       //default method
       assertEquals("post", DebugServlet.lastMethod);
@@ -429,7 +428,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       client.setRequestWriter(new RequestWriter());
       try {
         client.request(req);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
@@ -447,7 +446,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       DebugServlet.clear();
       try {
         client.request(req);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
 
       assertEquals("post", DebugServlet.lastMethod);
       assertEquals(EXPECTED_USER_AGENT, DebugServlet.headers.get("user-agent"));
@@ -557,7 +556,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       setReqParamsOf(req, "serverOnly", "notServer");
       try {
         client.request(req);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
       verifyServletState(client, req);
 
       // test without server query params
@@ -568,7 +567,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       setReqParamsOf(req, "requestOnly", "notRequest");
       try {
         client.request(req);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
       verifyServletState(client, req);
 
       // test with both request and server query params
@@ -579,7 +578,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       setReqParamsOf(req, "serverOnly", "requestOnly", "both", "neither");
       try {
         client.request(req);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
       verifyServletState(client, req);
 
       // test with both request and server query params with single stream
@@ -591,7 +590,7 @@ public class Http2SolrClientTest extends SolrJettyTestBase {
       setReqParamsOf(req, "serverOnly", "requestOnly", "both", "neither");
       try {
         client.request(req);
-      } catch (ParseException ignored) {}
+      } catch (BaseHttpSolrClient.RemoteSolrException ignored) {}
       // NOTE: single stream requests send all the params
       // as part of the query string.  So add "neither" to the request
       // so it passes the verification step.