You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2017/02/24 23:50:36 UTC

lucene-solr:branch_6x: Revert "SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json"

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 6c63df0b1 -> dbcbdeb07


Revert "SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json"

This reverts commit 024a39399dbb77678d06f70029575e0e66ded4b4.


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

Branch: refs/heads/branch_6x
Commit: dbcbdeb07f1090bfae99e2cde21df684b7f20a26
Parents: 6c63df0
Author: Jan H�ydahl <ja...@apache.org>
Authored: Sat Feb 25 00:45:19 2017 +0100
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Sat Feb 25 00:50:16 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 -
 .../org/apache/solr/core/CoreContainer.java     |   9 +-
 .../solr/security/PKIAuthenticationPlugin.java  |  42 +-----
 .../org/apache/solr/servlet/HttpSolrCall.java   |   4 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |  11 +-
 .../solr/security/BasicAuthDistributedTest.java | 136 -------------------
 .../security/TestPKIAuthenticationPlugin.java   |  38 +-----
 .../solr/BaseDistributedSearchTestCase.java     |  37 +----
 8 files changed, 19 insertions(+), 260 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dbcbdeb0/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index bb9ab46..b727bf7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -78,8 +78,6 @@ New Features
   SOLR_HOME on every node. Editing config through API is supported but affects only that one node.
   This feature should be considered experimental for this release and may not work with SSL yet. (janhoy)
 
-* SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json (janhoy)
-
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dbcbdeb0/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 0c55a3a..8698713 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -476,9 +476,7 @@ public class CoreContainer {
     hostName = cfg.getNodeName();
 
     zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
-    pkiAuthenticationPlugin = isZooKeeperAware() ?
-        new PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName()) :
-        new PKIAuthenticationPlugin(this, getNodeNameLocal());
+    if(isZooKeeperAware())  pkiAuthenticationPlugin = new PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName());
 
     MDCLoggingContext.setNode(this);
 
@@ -600,11 +598,6 @@ public class CoreContainer {
     }
   }
 
-  // Builds a node name to be used with PKIAuth.
-  private String getNodeNameLocal() {
-    return getConfig().getCloudConfig().getHost()+":"+getConfig().getCloudConfig().getSolrHostPort()+"_solr";
-  }
-
   public void securityNodeChanged() {
     log.info("Security node changed, reloading security.json");
     reloadSecurityProperties();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dbcbdeb0/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 7cccc1f..c1501d3 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -22,9 +22,7 @@ import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequestWrapper;
 import java.io.IOException;
-import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
-import java.net.URLDecoder;
 import java.nio.ByteBuffer;
 import java.security.Principal;
 import java.security.PublicKey;
@@ -196,14 +194,9 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
   }
 
   PublicKey getRemotePublicKey(String nodename) {
-    String url, uri = null;
-    if (cores.isZooKeeperAware()) {
-      url = cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodename);
-    } else {
-      url = getBaseUrlForNodeNameLocal(nodename);
-    }
+    String url = cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodename);
     try {
-      uri += PATH + "?wt=json&omitHeader=true";
+      String uri = url + PATH + "?wt=json&omitHeader=true";
       log.debug("Fetching fresh public key from : {}",uri);
       HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient()
           .execute(new HttpGet(uri), HttpClientUtil.createNewHttpClientRequestContext());
@@ -220,7 +213,7 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
       keyCache.put(nodename, pubKey);
       return pubKey;
     } catch (Exception e) {
-      log.error("Exception trying to get public key from : " + uri, e);
+      log.error("Exception trying to get public key from : " + url, e);
       return null;
     }
 
@@ -228,35 +221,6 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
 
   private HttpHeaderClientConfigurer clientConfigurer = new HttpHeaderClientConfigurer();
 
-  protected String getBaseUrlForNodeNameLocal(String nodeName) {
-    final int _offset = nodeName.indexOf("_");
-    if (_offset < 0) {
-      throw new IllegalArgumentException("nodeName does not contain expected '_' seperator: " + nodeName);
-    }
-    final String hostAndPort = nodeName.substring(0,_offset);
-    try {
-      final String path = URLDecoder.decode(nodeName.substring(1+_offset), "UTF-8");
-      // TODO: Find a better way of resolving urlScheme when not using ZK?
-      String urlScheme = resolveUrlScheme();
-      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" + path));
-    } catch (UnsupportedEncodingException e) {
-      throw new IllegalStateException("JVM Does not seem to support UTF-8", e);
-    }
-  }
-
-  /**
-   * Resolve urlScheme first from sysProp "urlScheme", if not set or invalid value, peek at ssl sysProps
-   * @return "https" if SSL is enabled, else "http"
-   */
-  protected static String resolveUrlScheme() {
-    String urlScheme = System.getProperty("urlScheme");
-    if (urlScheme != null && urlScheme.matches("https?")) {
-      return urlScheme;
-    } else {
-      return System.getProperty("solr.jetty.keystore") == null ? "http" : "https";
-    }
-  }
-
   @Override
   public HttpClientConfigurer getClientConfigurer() {
     return clientConfigurer;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dbcbdeb0/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 1314482..f806a4f 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -561,7 +561,7 @@ public class HttpSolrCall {
   }
 
   private boolean shouldAuthorize() {
-    if(path != null && path.endsWith(PKIAuthenticationPlugin.PATH)) return false;
+    if(PKIAuthenticationPlugin.PATH.equals(path)) return false;
     //admin/info/key is the path where public key is exposed . it is always unsecured
     if (cores.getPkiAuthenticationPlugin() != null && req.getUserPrincipal() != null) {
       boolean b = cores.getPkiAuthenticationPlugin().needsAuthorization(req);
@@ -1082,7 +1082,7 @@ public class HttpSolrCall {
           response.delete(response.length() - 1, response.length());
         
         response.append("], Path: [").append(resource).append("]");
-        response.append(" path : ").append(path).append(" params :").append(solrReq == null ? null : solrReq.getParams());
+        response.append(" path : ").append(path).append(" params :").append(solrReq.getParams());
         return response.toString();
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dbcbdeb0/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 4ce57b0..ce65069 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -402,11 +402,11 @@ public class SolrDispatchFilter extends BaseSolrFilter {
     if (authenticationPlugin == null) {
       return true;
     } else {
-      String requestUri = ((HttpServletRequest) request).getRequestURI();
-      if (requestUri != null && requestUri.endsWith(PKIAuthenticationPlugin.PATH)) {
-        log.debug("Passthrough of pki URL " + requestUri);
-        return true;
-      }
+      // /admin/info/key must be always open. see SOLR-9188
+      // tests work only w/ getPathInfo
+      //otherwise it's just enough to have getServletPath()
+      if (PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest) request).getServletPath()) ||
+          PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest) request).getPathInfo())) return true;
       String header = ((HttpServletRequest) request).getHeader(PKIAuthenticationPlugin.HEADER);
       if (header != null && cores.getPkiAuthenticationPlugin() != null)
         authenticationPlugin = cores.getPkiAuthenticationPlugin();
@@ -418,6 +418,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
           wrappedRequest.set(req);
         });
       } catch (Exception e) {
+        log.info("Error authenticating", e);
         throw new SolrException(ErrorCode.SERVER_ERROR, "Error during request authentication, ", e);
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dbcbdeb0/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
deleted file mode 100644
index e35e369..0000000
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.security;
-
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-
-import org.apache.lucene.util.LuceneTestCase.Slow;
-import org.apache.solr.BaseDistributedSearchTestCase;
-import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.handler.admin.SecurityConfHandler;
-import org.apache.solr.handler.admin.SecurityConfHandlerLocalForTesting;
-import org.apache.solr.util.LogLevel;
-import org.junit.Test;
-
-/**
- * Tests basicAuth in a multi shard env
- */
-@Slow
-public class BasicAuthDistributedTest extends BaseDistributedSearchTestCase {
-  public BasicAuthDistributedTest() {
-    super();
-    schemaString = "schema.xml";
-  }
-
-  private SecurityConfHandlerLocalForTesting securityConfHandler;
-
-  @Test
-  @LogLevel("org.apache.solr=DEBUG")
-  public void test() throws Exception {
-    index();
-    testAuth();
-  }
-
-  private void index() throws Exception {
-    del("*:*");
-    indexr(id, "1", "text", "doc one");
-    indexr(id, "2", "text", "doc two");
-    indexr(id, "3", "text", "doc three");
-    indexr(id, "4", "text", "doc four");
-    indexr(id, "5", "text", "doc five");
-
-    commit();  // try to ensure there's more than one segment
-
-    indexr(id, "6", "text", "doc six");
-    indexr(id, "7", "text", "doc seven");
-    indexr(id, "8", "text", "doc eight");
-    indexr(id, "9", "text", "doc nine");
-    indexr(id, "10", "text", "doc ten");
-
-    commit();
-
-    handle.clear();
-    handle.put("QTime", SKIPVAL);
-    handle.put("timestamp", SKIPVAL);
-    handle.put("maxScore", SKIPVAL);
-    handle.put("_version_", SKIPVAL);
-  }
-
-  private void testAuth() throws Exception {
-    QueryResponse rsp = query("q","text:doc", "fl", "id,text", "sort", "id asc");
-    assertEquals(10, rsp.getResults().getNumFound());
-
-    // Enable authentication
-    for (JettySolrRunner j : jettys) {
-      writeSecurityJson(j.getCoreContainer());
-    }
-
-    HttpSolrClient.RemoteSolrException expected = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
-      query("q","text:doc-fail", "fl", "id,text", "sort", "id asc");
-    });
-    assertEquals(401, expected.code());
-
-    // Add auth
-    ModifiableSolrParams params = new ModifiableSolrParams();
-    params.add("q", "text:doc").add("fl", "id,text").add("sort", "id asc");
-    QueryRequest req = new QueryRequest(params);
-    req.setBasicAuthCredentials("solr", "SolrRocks");
-    rsp = req.process(clients.get(0), null);
-    if (jettys.size() > 1) {
-      assertTrue(rsp.getResults().getNumFound() < 10);
-      rsp = query(true, params, "solr", "SolrRocks");
-    }
-    assertEquals(10, rsp.getResults().getNumFound());
-
-    // Disable auth
-    for (JettySolrRunner j : jettys) {
-      deleteSecurityJson(j.getCoreContainer());
-    }
-
-  }
-
-  private void deleteSecurityJson(CoreContainer coreContainer) throws IOException {
-    securityConfHandler = new SecurityConfHandlerLocalForTesting(coreContainer);
-    Files.delete(Paths.get(coreContainer.getSolrHome()).resolve("security.json"));
-    coreContainer.securityNodeChanged();
-  }
-
-  private void writeSecurityJson(CoreContainer coreContainer) throws IOException {
-    securityConfHandler = new SecurityConfHandlerLocalForTesting(coreContainer);
-    securityConfHandler.persistConf(new SecurityConfHandler.SecurityConfig()
-        .setData(Utils.fromJSONString(ALL_CONF.replaceAll("'", "\""))));
-    coreContainer.securityNodeChanged();
-  }
-
-  protected static final String ALL_CONF = "{\n" +
-      "  'authentication':{\n" +
-      "    'blockUnknown':true,\n" +
-      "    'class':'solr.BasicAuthPlugin',\n" +
-      "    'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y= Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}},\n" +
-      "  'authorization':{\n" +
-      "    'class':'solr.RuleBasedAuthorizationPlugin',\n" +
-      "    'user-role':{'solr':'admin'},\n" +
-      "    'permissions':[{'name':'all','role':'admin'}]}}";
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dbcbdeb0/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
index 90c5bd2..a5a279f 100644
--- a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
@@ -35,11 +35,7 @@ import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.util.CryptoKeys;
-import org.junit.Test;
-
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
 
@@ -145,38 +141,10 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     mock1.doAuthenticate(mockReq, null,filterChain );
     assertNotNull(wrappedRequestByFilter.get());
     assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
-  }
 
-  @Test
-  public void testGetBaseUrlForNodeNameLocal() {
-    synchronized (this) {
-      final MockPKIAuthenticationPlugin mock = new MockPKIAuthenticationPlugin(null, "myName");
-      System.clearProperty("solr.jetty.keystore");
-      assertEquals("http://my.host:9876/solr2", mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
-      System.setProperty("solr.jetty.keystore", "foo");
-      assertEquals("https://my.host:9876/solr2", mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
-      System.clearProperty("solr.jetty.keystore");
-    }
-  }
 
-  @Test
-  public void testResolveUrlScheme() {
-    synchronized (this) {
-      System.clearProperty("urlScheme");
-      System.clearProperty("solr.jetty.keystore");
-      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
-      System.setProperty("urlScheme", "http");
-      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
-      System.setProperty("urlScheme", "https");
-      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
-      System.setProperty("urlScheme", "ftp");
-      System.clearProperty("solr.jetty.keystore");
-      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
-      System.setProperty("solr.jetty.keystore", "foo");
-      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
-      System.clearProperty("urlScheme");
-      System.clearProperty("solr.jetty.keystore");
-    }
+
+
   }
 
   private HttpServletRequest createMockRequest(final AtomicReference<Header> header) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dbcbdeb0/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
index b1cea0a..00b7577 100644
--- a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
@@ -50,7 +50,6 @@ import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettyConfig;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
@@ -566,12 +565,6 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
     return rsp;
   }
 
-  protected QueryResponse queryServer(QueryRequest req) throws IOException, SolrServerException {
-    int which = r.nextInt(clients.size());
-    SolrClient client = clients.get(which);
-    return req.process(client, null);
-  }
-
   /**
    * Sets distributed params.
    * Returns the QueryResponse from {@link #queryServer},
@@ -605,31 +598,18 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
    * Returns the QueryResponse from {@link #queryServer}  
    */
   protected QueryResponse query(boolean setDistribParams, SolrParams p) throws Exception {
-    return query(setDistribParams, p, null, null);
-  }
-
-  /**
-   * Returns the QueryResponse from {@link #queryServer}
-   * @param setDistribParams whether to do a distributed request
-   * @param user basic auth username (set to null if not in use)
-   * @param pass basic auth password (set to null if not in use)
-   * @return the query response
-   */
-  protected QueryResponse query(boolean setDistribParams, SolrParams p, String user, String pass) throws Exception {
     
     final ModifiableSolrParams params = new ModifiableSolrParams(p);
 
     // TODO: look into why passing true causes fails
     params.set("distrib", "false");
-    QueryRequest req = generateQueryRequest(params, user, pass);
-    final QueryResponse controlRsp = req.process(controlClient, null);
+    final QueryResponse controlRsp = controlClient.query(params);
     validateControlData(controlRsp);
 
     params.remove("distrib");
     if (setDistribParams) setDistributedParams(params);
-    req = generateQueryRequest(params, user, pass);
 
-    QueryResponse rsp = queryServer(req);
+    QueryResponse rsp = queryServer(params);
 
     compareResponses(rsp, controlRsp);
 
@@ -644,8 +624,7 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
               int which = r.nextInt(clients.size());
               SolrClient client = clients.get(which);
               try {
-                QueryRequest qreq = generateQueryRequest(new ModifiableSolrParams(params), user, pass);
-                QueryResponse rsp = qreq.process(client, null);
+                QueryResponse rsp = client.query(new ModifiableSolrParams(params));
                 if (verifyStress) {
                   compareResponses(rsp, controlRsp);
                 }
@@ -664,15 +643,7 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
     }
     return rsp;
   }
-
-  private QueryRequest generateQueryRequest(ModifiableSolrParams params, String user, String pass) {
-    QueryRequest req = new QueryRequest(params);
-    if (user != null && pass != null) {
-      req.setBasicAuthCredentials(user, pass);
-    }
-    return req;
-  }
-
+  
   public QueryResponse queryAndCompare(SolrParams params, SolrClient... clients) throws SolrServerException, IOException {
     return queryAndCompare(params, Arrays.<SolrClient>asList(clients));
   }