You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2017/02/28 17:26:54 UTC

[27/50] [abbrv] lucene-solr:jira/solr-9045: Revert "SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json"

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

This reverts commit 95d6fc2512d6525b2354165553f0d6cc4d0d6310.


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

Branch: refs/heads/jira/solr-9045
Commit: 30125f99daf38c4788a9763a89fddb3730c709af
Parents: 57a42e4
Author: Jan H�ydahl <ja...@apache.org>
Authored: Sat Feb 25 00:43:42 2017 +0100
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Sat Feb 25 00:43:42 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/30125f99/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 2c5f0db..0302615 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -134,8 +134,6 @@ New Features
   field must both be stored=false, indexed=false, docValues=true. (Ishan Chattopadhyaya, hossman, noble,
   shalin, yonik)
 
-* 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/30125f99/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 6115562..e3977d7 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -497,9 +497,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);
 
@@ -620,11 +618,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/30125f99/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 d185bc9..fdd4408 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;
@@ -195,14 +193,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());
@@ -219,41 +212,12 @@ 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;
     }
 
   }
 
-  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 SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
     HttpClientUtil.addRequestInterceptor(interceptor);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/30125f99/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 0dfb0ea..4f6bae0 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -560,7 +560,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);
@@ -1081,7 +1081,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/30125f99/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/30125f99/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/30125f99/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/30125f99/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 bbfc048..8c6eb60 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;
@@ -559,12 +558,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},
@@ -598,31 +591,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);
 
@@ -637,8 +617,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);
                 }
@@ -657,15 +636,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));
   }