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 2017/05/27 02:04:44 UTC

lucene-solr:master: SOLR-10755: delete/refactor many solrj deprecations

Repository: lucene-solr
Updated Branches:
  refs/heads/master 1d2acdbea -> bc973ecdc


SOLR-10755: delete/refactor many solrj deprecations


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

Branch: refs/heads/master
Commit: bc973ecdcfacf39440da06b86139c77935e1e92e
Parents: 1d2acdb
Author: Chris Hostetter <ho...@apache.org>
Authored: Fri May 26 19:04:31 2017 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Fri May 26 19:04:31 2017 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../solr/highlight/DefaultSolrHighlighter.java  |   8 +-
 .../org/apache/solr/schema/IndexSchema.java     |   2 +-
 .../apache/solr/cloud/TestConfigSetsAPI.java    |   5 +-
 .../solr/cloud/TestMiniSolrCloudClusterSSL.java |   2 +-
 .../TestSolrCloudWithDelegationTokens.java      |   2 +-
 .../hadoop/TestDelegationWithHadoopAuth.java    |   2 +-
 .../solr/client/solrj/impl/CloudSolrClient.java | 271 +------------------
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |  51 +---
 .../impl/DelegationTokenHttpSolrClient.java     |  24 +-
 .../solr/client/solrj/impl/HttpSolrClient.java  |  43 +--
 .../client/solrj/impl/LBHttpSolrClient.java     |  26 +-
 .../impl/SolrHttpClientContextBuilder.java      |   8 -
 .../solrj/io/graph/ShortestPathStream.java      |  25 --
 .../client/solrj/io/stream/CloudSolrStream.java |  21 --
 .../client/solrj/io/stream/FacetStream.java     |  16 --
 .../client/solrj/io/stream/RandomStream.java    |   4 +-
 .../solr/client/solrj/io/stream/SolrStream.java |  14 -
 .../solr/client/solrj/io/stream/SqlStream.java  |   2 -
 .../client/solrj/io/stream/StatsStream.java     |  10 -
 .../client/solrj/io/stream/TopicStream.java     |  19 --
 .../solr/common/params/HighlightParams.java     |   2 -
 .../apache/solr/common/util/ExecutorUtil.java   |  41 ---
 .../org/apache/solr/common/util/NamedList.java  |  23 +-
 .../solr/common/util/SimpleOrderedMap.java      |   9 +-
 .../embedded/SolrExampleStreamingTest.java      |   2 +-
 .../solrj/impl/BasicHttpSolrClientTest.java     |   2 +-
 .../CloudSolrClientMultiConstructorTest.java    |   4 +-
 .../impl/ConcurrentUpdateSolrClientTest.java    |  23 +-
 .../solrj/io/graph/GraphExpressionTest.java     |   2 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    | 108 +++++---
 31 files changed, 147 insertions(+), 626 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7ad4122..d4e6eac 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -196,6 +196,8 @@ Other Changes
 
 * SOLR-10710: Fix LTR failing tests. (Diego Ceccarelli via Tomás Fernández Löbbe)
 
+* SOLR-10755: delete/refactor many solrj deprecations (hossman)
+
 ==================  6.7.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
index 7e56ee4..18d9b44 100644
--- a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
+++ b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
@@ -84,6 +84,12 @@ import org.slf4j.LoggerFactory;
 public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInfoInitialized
 {
 
+  /** 
+   * This constant was formerly part of HighlightParams.  After deprecation it was removed so clients 
+   * would no longer use it, but we still support it server side.
+   */
+  private static final String USE_FVH = HighlightParams.HIGHLIGHT + ".useFastVectorHighlighter";
+  
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   protected final SolrCore solrCore;
@@ -492,7 +498,7 @@ public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInf
     boolean methodFvh =
         HighlightComponent.HighlightMethod.FAST_VECTOR.getMethodName().equals(
             params.getFieldParam(schemaField.getName(), HighlightParams.METHOD))
-        || params.getFieldBool(schemaField.getName(), HighlightParams.USE_FVH, false);
+        || params.getFieldBool(schemaField.getName(), USE_FVH, false);
     if (!methodFvh) return false;
     boolean termPosOff = schemaField.storeTermPositions() && schemaField.storeTermOffsets();
     if (!termPosOff) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
index 13385b7..04f64d5 100644
--- a/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
@@ -373,7 +373,7 @@ public class IndexSchema {
   void persist(Writer writer) throws IOException {
     final SolrQueryResponse response = new SolrQueryResponse();
     response.add(IndexSchema.SCHEMA, getNamedPropertyValues());
-    final NamedList args = new NamedList(Arrays.<Object>asList("indent", "on"));
+    final SolrParams args = (new ModifiableSolrParams()).set("indent", "on");
     final LocalSolrQueryRequest req = new LocalSolrQueryRequest(null, args);
     final SchemaXmlWriter schemaXmlWriter = new SchemaXmlWriter(writer, req, response);
     schemaXmlWriter.setEmitManagedSchemaDoNotEditWarning(true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
index 6c20ccc..875c0ef 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
@@ -279,8 +279,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
 
   @Test
   public void testUploadErrors() throws Exception {
-    final SolrClient solrClient = new HttpSolrClient(
-        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+    final SolrClient solrClient = getHttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
 
     ByteBuffer emptyData = ByteBuffer.allocate(0);
 
@@ -504,7 +503,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
 
   private void xsltRequest(String collection) throws SolrServerException, IOException {
     String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
-    try (HttpSolrClient client = new HttpSolrClient(baseUrl + "/" + collection)) {
+    try (HttpSolrClient client = getHttpSolrClient(baseUrl + "/" + collection)) {
       String xml = 
           "<random>" +
               " <document>" +

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
index a53b39f..98f952a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudClusterSSL.java
@@ -332,7 +332,7 @@ public class TestMiniSolrCloudClusterSSL extends SolrTestCaseJ4 {
     // that "optimize" the test client construction in a way that would prevent us from finding bugs with
     // regular HttpSolrClient instantiation.
     if (random().nextBoolean()) {
-      return new HttpSolrClient(url);
+      return (new HttpSolrClient.Builder(url)).build();
     } // else...
     return getHttpSolrClient(url);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithDelegationTokens.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithDelegationTokens.java b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithDelegationTokens.java
index f8f3f7e..ddbbb64 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithDelegationTokens.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestSolrCloudWithDelegationTokens.java
@@ -394,7 +394,7 @@ public class TestSolrCloudWithDelegationTokens extends SolrTestCaseJ4 {
     }
 
     ss = new HttpSolrClient.Builder(solrClientPrimary.getBaseURL().toString())
-        .withDelegationToken(token)
+        .withKerberosDelegationToken(token)
         .withResponseParser(solrClientPrimary.getParser())
         .build();
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java b/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
index c799296..37d9cdf 100644
--- a/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
+++ b/solr/core/src/test/org/apache/solr/security/hadoop/TestDelegationWithHadoopAuth.java
@@ -380,7 +380,7 @@ public class TestDelegationWithHadoopAuth extends SolrCloudTestCase {
     }
 
     ss = new HttpSolrClient.Builder(primarySolrClient.getBaseURL().toString())
-        .withDelegationToken(token)
+        .withKerberosDelegationToken(token)
         .withResponseParser(primarySolrClient.getParser())
         .build();
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index ff7b06a..1271655 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.net.ConnectException;
 import java.net.SocketException;
-import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -239,168 +238,6 @@ public class CloudSolrClient extends SolrClient {
    * of the SolrCloud state. If there is a fully redundant Zookeeper quorum and
    * SolrCloud has enough replicas for every shard in a collection, there is no
    * single point of failure. Updates will be sent to shard leaders by default.
-   * 
-   * @param zkHost
-   *          The client endpoint of the zookeeper quorum containing the cloud
-   *          state. The full specification for this string is one or more comma
-   *          separated HOST:PORT values, followed by an optional chroot value
-   *          that starts with a forward slash. Using a chroot allows multiple
-   *          applications to coexist in one ensemble. For full details, see the
-   *          Zookeeper documentation. Some examples:
-   *          <p>
-   *          "host1:2181"
-   *          <p>
-   *          "host1:2181,host2:2181,host3:2181/mysolrchroot"
-   *          <p>
-   *          "zoo1.example.com:2181,zoo2.example.com:2181,zoo3.example.com:2181"
-   *          
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public CloudSolrClient(String zkHost) {
-    this.stateProvider = new ZkClientClusterStateProvider(zkHost);
-      this.clientIsInternal = true;
-      this.myClient = HttpClientUtil.createClient(null);
-      this.lbClient = new LBHttpSolrClient.Builder()
-          .withHttpClient(myClient)
-          .build();
-      this.lbClient.setRequestWriter(new BinaryRequestWriter());
-      this.lbClient.setParser(new BinaryResponseParser());
-      this.updatesToLeaders = true;
-      this.directUpdatesToLeadersOnly = false;
-      shutdownLBHttpSolrServer = true;
-      lbClient.addQueryParams(STATE_VERSION);
-  }
-
-  /**
-   * Create a new client object that connects to Zookeeper and is always aware
-   * of the SolrCloud state. If there is a fully redundant Zookeeper quorum and
-   * SolrCloud has enough replicas for every shard in a collection, there is no
-   * single point of failure. Updates will be sent to shard leaders by default.
-   *
-   * @param zkHost
-   *          The client endpoint of the zookeeper quorum containing the cloud
-   *          state. The full specification for this string is one or more comma
-   *          separated HOST:PORT values, followed by an optional chroot value
-   *          that starts with a forward slash. Using a chroot allows multiple
-   *          applications to coexist in one ensemble. For full details, see the
-   *          Zookeeper documentation. Some examples:
-   *          <p>
-   *          "host1:2181"
-   *          <p>
-   *          "host1:2181,host2:2181,host3:2181/mysolrchroot"
-   *          <p>
-   *          "zoo1.example.com:2181,zoo2.example.com:2181,zoo3.example.com:2181"
-   * @param httpClient
-   *          the {@link HttpClient} instance to be used for all requests. The
-   *          provided httpClient should use a multi-threaded connection manager.
-   *          
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public CloudSolrClient(String zkHost, HttpClient httpClient) {
-    this.stateProvider = new ZkClientClusterStateProvider(zkHost);
-    this.clientIsInternal = httpClient == null;
-    this.myClient = httpClient == null ? HttpClientUtil.createClient(null) : httpClient;
-    this.lbClient = createLBHttpSolrClient(myClient);
-    this.updatesToLeaders = true;
-    this.directUpdatesToLeadersOnly = false;
-    shutdownLBHttpSolrServer = true;
-    lbClient.addQueryParams(STATE_VERSION);
-  }
-  
-  /**
-   * Create a new client object using multiple string values in a Collection
-   * instead of a standard zkHost connection string. Note that this method will
-   * not be used if there is only one String argument - that will use
-   * {@link #CloudSolrClient(String)} instead.
-   * 
-   * @param zkHosts
-   *          A Java Collection (List, Set, etc) of HOST:PORT strings, one for
-   *          each host in the zookeeper ensemble. Note that with certain
-   *          Collection types like HashSet, the order of hosts in the final
-   *          connect string may not be in the same order you added them.
-   * @param chroot
-   *          A chroot value for zookeeper, starting with a forward slash. If no
-   *          chroot is required, use null.
-   * @throws IllegalArgumentException
-   *           if the chroot value does not start with a forward slash.
-   * @see #CloudSolrClient(String)
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public CloudSolrClient(Collection<String> zkHosts, String chroot) {
-    this(zkHosts, chroot, null);
-  }
-
-  /**
-   * Create a new client object using multiple string values in a Collection
-   * instead of a standard zkHost connection string. Note that this method will
-   * not be used if there is only one String argument - that will use
-   * {@link #CloudSolrClient(String)} instead.
-   *
-   * @param zkHosts
-   *          A Java Collection (List, Set, etc) of HOST:PORT strings, one for
-   *          each host in the zookeeper ensemble. Note that with certain
-   *          Collection types like HashSet, the order of hosts in the final
-   *          connect string may not be in the same order you added them.
-   * @param chroot
-   *          A chroot value for zookeeper, starting with a forward slash. If no
-   *          chroot is required, use null.
-   * @param httpClient
-   *          the {@link HttpClient} instance to be used for all requests. The provided httpClient should use a
-   *          multi-threaded connection manager.
-   * @throws IllegalArgumentException
-   *           if the chroot value does not start with a forward slash.
-   * @see #CloudSolrClient(String)
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public CloudSolrClient(Collection<String> zkHosts, String chroot, HttpClient httpClient) {
-    this.stateProvider = new ZkClientClusterStateProvider(zkHosts, chroot);
-    this.clientIsInternal = httpClient == null;
-    this.myClient = httpClient == null ? HttpClientUtil.createClient(null) : httpClient;
-    this.lbClient = createLBHttpSolrClient(myClient);
-    this.updatesToLeaders = true;
-    this.directUpdatesToLeadersOnly = false;
-    shutdownLBHttpSolrServer = true;
-  }
-  
-  /**
-   * Create a new client object that connects to Zookeeper and is always aware
-   * of the SolrCloud state. If there is a fully redundant Zookeeper quorum and
-   * SolrCloud has enough replicas for every shard in a collection, there is no
-   * single point of failure. Updates will be sent to shard leaders by default.
-   * 
-   * @param zkHosts
-   *          A Java Collection (List, Set, etc) of HOST:PORT strings, one for
-   *          each host in the zookeeper ensemble. Note that with certain
-   *          Collection types like HashSet, the order of hosts in the final
-   *          connect string may not be in the same order you added them.
-   * @param chroot
-   *          A chroot value for zookeeper, starting with a forward slash. If no
-   *          chroot is required, use null.
-   * @param httpClient
-   *          the {@link HttpClient} instance to be used for all requests. The provided httpClient should use a
-   *          multi-threaded connection manager.  If null, a default HttpClient will be used.
-   * @param lbSolrClient
-   *          LBHttpSolrClient instance for requests.  If null, a default LBHttpSolrClient will be used.
-   * @param updatesToLeaders
-   *          If true, sends updates to shard leaders.
-   *
-   * @deprecated use {@link Builder} instead.  This will soon be a protected method, and will only
-   * be available for use in implementing subclasses.
-   */
-  @Deprecated
-  public CloudSolrClient(Collection<String> zkHosts, String chroot, HttpClient httpClient, LBHttpSolrClient lbSolrClient, boolean updatesToLeaders) {
-    this(zkHosts, chroot, null, httpClient, lbSolrClient, null, updatesToLeaders, false, null);
-  }
-
-  /**
-   * Create a new client object that connects to Zookeeper and is always aware
-   * of the SolrCloud state. If there is a fully redundant Zookeeper quorum and
-   * SolrCloud has enough replicas for every shard in a collection, there is no
-   * single point of failure. Updates will be sent to shard leaders by default.
    *
    * @param zkHosts
    *          A Java Collection (List, Set, etc) of HOST:PORT strings, one for
@@ -467,46 +304,6 @@ public class CloudSolrClient extends SolrClient {
     this.updatesToLeaders = updatesToLeaders;
     this.directUpdatesToLeadersOnly = directUpdatesToLeadersOnly;
   }
-  
-  /**
-   * @param zkHost
-   *          A zookeeper client endpoint.
-   * @param updatesToLeaders
-   *          If true, sends updates only to shard leaders.
-   * @see #CloudSolrClient(String) for full description and details on zkHost
-   * @deprecated use {@link CloudSolrClient.Builder} instead.
-   */
-  @Deprecated
-  public CloudSolrClient(String zkHost, boolean updatesToLeaders) {
-    this(zkHost, updatesToLeaders, null);
-  }
-
-  /**
-   * @param zkHost
-   *          A zookeeper client endpoint.
-   * @param updatesToLeaders
-   *          If true, sends updates only to shard leaders.
-   * @param httpClient
-   *          the {@link HttpClient} instance to be used for all requests. The provided httpClient should use a
-   *          multi-threaded connection manager.
-   * @see #CloudSolrClient(String) for full description and details on zkHost
-   * @deprecated use {@link CloudSolrClient.Builder} instead.
-   */
-  @Deprecated
-  public CloudSolrClient(String zkHost, boolean updatesToLeaders, HttpClient httpClient) {
-    this.stateProvider = new ZkClientClusterStateProvider(zkHost);
-    this.clientIsInternal = httpClient == null;
-    this.myClient = httpClient == null ? HttpClientUtil.createClient(null) : httpClient;
-    this.lbClient = new LBHttpSolrClient.Builder()
-        .withHttpClient(myClient)
-        .build();
-    this.lbClient.setRequestWriter(new BinaryRequestWriter());
-    this.lbClient.setParser(new BinaryResponseParser());
-    this.updatesToLeaders = updatesToLeaders;
-    this.directUpdatesToLeadersOnly = false;
-    shutdownLBHttpSolrServer = true;
-    lbClient.addQueryParams(STATE_VERSION);
-  }
 
   /**Sets the cache ttl for DocCollection Objects cached  . This is only applicable for collections which are persisted outside of clusterstate.json
    * @param seconds ttl value in seconds
@@ -515,40 +312,6 @@ public class CloudSolrClient extends SolrClient {
     assert seconds > 0;
     this.collectionStateCache.timeToLive = seconds * 1000L;
   }
-
-  /**
-   * @param zkHost
-   *          A zookeeper client endpoint.
-   * @param lbClient
-   *          LBHttpSolrServer instance for requests.
-   * @see #CloudSolrClient(String) for full description and details on zkHost
-   * @deprecated use {@link CloudSolrClient.Builder} instead.
-   */
-  @Deprecated
-  public CloudSolrClient(String zkHost, LBHttpSolrClient lbClient) {
-    this(zkHost, lbClient, true);
-  }
-  
-  /**
-   * @param zkHost
-   *          A zookeeper client endpoint.
-   * @param lbClient
-   *          LBHttpSolrServer instance for requests.
-   * @param updatesToLeaders
-   *          If true, sends updates only to shard leaders.
-   * @see #CloudSolrClient(String) for full description and details on zkHost
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public CloudSolrClient(String zkHost, LBHttpSolrClient lbClient, boolean updatesToLeaders) {
-    this.lbClient = lbClient;
-    this.stateProvider = new ZkClientClusterStateProvider(zkHost);
-    this.updatesToLeaders = updatesToLeaders;
-    this.directUpdatesToLeadersOnly = false;
-    shutdownLBHttpSolrServer = false;
-    this.clientIsInternal = false;
-    lbClient.addQueryParams(STATE_VERSION);
-  }
   
   public ResponseParser getParser() {
     return lbClient.getParser();
@@ -660,25 +423,6 @@ public class CloudSolrClient extends SolrClient {
     this.parallelUpdates = parallelUpdates;
   }
 
-  /**
-   * Upload a set of config files to Zookeeper and give it a name
-   *
-   * NOTE: You should only allow trusted users to upload configs.  If you
-   * are allowing client access to zookeeper, you should protect the
-   * /configs node against unauthorised write access.
-   *
-   *  @deprecated Please use {@link ZkClientClusterStateProvider#uploadConfig(Path, String)} instead
-   *
-   * @param configPath {@link java.nio.file.Path} to the config files
-   * @param configName the name of the config
-   * @throws IOException if an IO error occurs
-   */
-  @Deprecated
-  public void uploadConfig(Path configPath, String configName) throws IOException {
-    stateProvider.connect();
-    assertZKStateProvider().uploadConfig(configPath, configName);
-  }
-
   private ZkClientClusterStateProvider assertZKStateProvider() {
     if (stateProvider instanceof ZkClientClusterStateProvider) {
       return (ZkClientClusterStateProvider) stateProvider;
@@ -686,20 +430,7 @@ public class CloudSolrClient extends SolrClient {
     throw new IllegalArgumentException("This client does not use ZK");
 
   }
-
-  /**
-   * Download a named config from Zookeeper to a location on the filesystem
-   *
-   * @deprecated Please use {@link ZkClientClusterStateProvider#downloadConfig(String, Path)} instead
-   * @param configName    the name of the config
-   * @param downloadPath  the path to write config files to
-   * @throws IOException  if an I/O exception occurs
-   */
-  @Deprecated
-  public void downloadConfig(String configName, Path downloadPath) throws IOException {
-    assertZKStateProvider().downloadConfig(configName, downloadPath);
-  }
-
+  
   /**
    * Block until a collection state matches a predicate, or a timeout
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
index bc37c13..d6675f2 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
@@ -96,56 +96,11 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
   AtomicInteger emptyQueueLoops;
   
   /**
-   * Uses an internally managed HttpClient instance.
-   * 
-   * @param solrServerUrl
-   *          The Solr server URL
-   * @param queueSize
-   *          The buffer size before the documents are sent to the server
-   * @param threadCount
-   *          The number of background threads used to empty the queue
-   *          
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public ConcurrentUpdateSolrClient(String solrServerUrl, int queueSize,
-                                    int threadCount) {
-    this(solrServerUrl, null, queueSize, threadCount);
-    shutdownExecutor = true;
-    internalHttpClient = true;
-  }
-  
-  /**
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public ConcurrentUpdateSolrClient(String solrServerUrl,
-                                    HttpClient client, int queueSize, int threadCount) {
-    this(solrServerUrl, client, queueSize, threadCount, ExecutorUtil.newMDCAwareCachedThreadPool(
-        new SolrjNamedThreadFactory("concurrentUpdateScheduler")));
-    shutdownExecutor = true;
-  }
-
-  /**
-   * Uses the supplied HttpClient to send documents to the Solr server.
-   * 
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public ConcurrentUpdateSolrClient(String solrServerUrl,
-                                    HttpClient client, int queueSize, int threadCount, ExecutorService es) {
-    this(solrServerUrl, client, queueSize, threadCount, es, false);
-  }
-  
-  /**
    * Uses the supplied HttpClient to send documents to the Solr server.
-   * 
-   * @deprecated use {@link Builder} instead.  This will soon be a
-   * protected method, and will only be available for use in implementing subclasses.
    */
-  @Deprecated
-  public ConcurrentUpdateSolrClient(String solrServerUrl,
-                                    HttpClient client, int queueSize, int threadCount, ExecutorService es, boolean streamDeletes) {
+  protected ConcurrentUpdateSolrClient(String solrServerUrl,
+                                       HttpClient client, int queueSize, int threadCount,
+                                       ExecutorService es, boolean streamDeletes) {
     this.internalHttpClient = (client == null);
     this.client = new HttpSolrClient.Builder(solrServerUrl)
         .withHttpClient(client)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java
index ab8175d..fc83391 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/DelegationTokenHttpSolrClient.java
@@ -35,21 +35,15 @@ public class DelegationTokenHttpSolrClient extends HttpSolrClient {
   public final static String DELEGATION_TOKEN_PARAM = "delegation";
 
   /**
-   * This constructor is deprecated in favor of passing delegation token via
-   * {@linkplain org.apache.solr.client.solrj.impl.HttpSolrClient.Builder#withInvariantParams(ModifiableSolrParams)}.
-   *
-   * @param baseURL The base url to communicate with the Solr server
-   * @param client Http client instance to use for communication
-   * @param parser Response parser instance to use to decode response from Solr server
-   * @param allowCompression Should compression be allowed ?
-   * @param delegationToken The delegation token string.
+   * Package protected constructor for use by 
+   * {@linkplain org.apache.solr.client.solrj.impl.HttpSolrClient.Builder}.
+   * @lucene.internal
    */
-  @Deprecated
-  public DelegationTokenHttpSolrClient(String baseURL,
-                                       HttpClient client,
-                                       ResponseParser parser,
-                                       boolean allowCompression,
-                                       String delegationToken) {
+  DelegationTokenHttpSolrClient(String baseURL,
+                                HttpClient client,
+                                ResponseParser parser,
+                                boolean allowCompression,
+                                String delegationToken) {
     super(baseURL, client, parser, allowCompression);
     if (delegationToken == null) {
       throw new IllegalArgumentException("Delegation token cannot be null");
@@ -100,4 +94,4 @@ public class DelegationTokenHttpSolrClient extends HttpSolrClient {
     }
     super.setQueryParams(queryParams);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
----------------------------------------------------------------------
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 dea1711..c1e9576 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
@@ -158,40 +158,7 @@ public class HttpSolrClient extends SolrClient {
   private volatile Integer connectionTimeout;
   private volatile Integer soTimeout;
   
-  /**
-   * @param baseURL
-   *          The URL of the Solr server. For example, "
-   *          <code>http://localhost:8983/solr/</code>" if you are using the
-   *          standard distribution Solr webapp on your local machine.
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public HttpSolrClient(String baseURL) {
-    this(baseURL, null, new BinaryResponseParser());
-  }
-  
-  /**
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public HttpSolrClient(String baseURL, HttpClient client) {
-    this(baseURL, client, new BinaryResponseParser());
-  }
-  
-  /**
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public HttpSolrClient(String baseURL, HttpClient client, ResponseParser parser) {
-    this(baseURL, client, parser, false);
-  }
-  
-  /**
-   * @deprecated use {@link Builder} instead.  This will soon be a 'protected'
-   * method, and will only be available for use in implementing subclasses.
-   */
-  @Deprecated
-  public HttpSolrClient(String baseURL, HttpClient client, ResponseParser parser, boolean allowCompression) {
+  protected HttpSolrClient(String baseURL, HttpClient client, ResponseParser parser, boolean allowCompression) {
     this.baseUrl = baseURL;
     if (baseUrl.endsWith("/")) {
       baseUrl = baseUrl.substring(0, baseUrl.length() - 1);
@@ -840,14 +807,6 @@ public class HttpSolrClient extends SolrClient {
      * Use a delegation token for authenticating via the KerberosPlugin
      */
     public Builder withKerberosDelegationToken(String delegationToken) {
-      return withDelegationToken(delegationToken);
-    }
-
-    @Deprecated
-    /**
-     * @deprecated use {@link withKerberosDelegationToken(String)} instead
-     */
-    public Builder withDelegationToken(String delegationToken) {
       if (this.invariantParams.get(DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM) != null) {
         throw new IllegalStateException(DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM + " is already defined!");
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
index 8dc2fd9..7706bf6 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
@@ -237,28 +237,9 @@ public class LBHttpSolrClient extends SolrClient {
   }
 
   /**
-   * @deprecated use {@link Builder} instead.
-   */
-  @Deprecated
-  public LBHttpSolrClient(String... solrServerUrls) throws MalformedURLException {
-    this(null, solrServerUrls);
-  }
-  
-  /**
-   * The provided httpClient should use a multi-threaded connection manager
-   * @deprecated use {@link Builder} instead.
-   */ 
-  @Deprecated
-  public LBHttpSolrClient(HttpClient httpClient, String... solrServerUrl) {
-    this(httpClient, new BinaryResponseParser(), solrServerUrl);
-  }
-
-  /**
    * The provided httpClient should use a multi-threaded connection manager
-   * @deprecated use {@link Builder} instead.  This will soon be a protected
-   * method and will only be available for use in implementing subclasses.
    */
-  public LBHttpSolrClient(HttpSolrClient.Builder httpSolrClientBuilder,
+  protected LBHttpSolrClient(HttpSolrClient.Builder httpSolrClientBuilder,
                           HttpClient httpClient, String... solrServerUrl) {
     clientIsInternal = httpClient == null;
     this.httpSolrClientBuilder = httpSolrClientBuilder;
@@ -275,11 +256,8 @@ public class LBHttpSolrClient extends SolrClient {
 
   /**
    * The provided httpClient should use a multi-threaded connection manager
-   * @deprecated use {@link Builder} instead.  This will soon be a protected
-   * method and will only be available for use in implementing subclasses.
    */
-  @Deprecated
-  public LBHttpSolrClient(HttpClient httpClient, ResponseParser parser, String... solrServerUrl) {
+  protected LBHttpSolrClient(HttpClient httpClient, ResponseParser parser, String... solrServerUrl) {
     clientIsInternal = (httpClient == null);
     this.httpClient = httpClient == null ? constructClient(solrServerUrl) : httpClient;
     this.parser = parser;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientContextBuilder.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientContextBuilder.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientContextBuilder.java
index f57848d..7ae98e5 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientContextBuilder.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrHttpClientContextBuilder.java
@@ -77,14 +77,6 @@ public class SolrHttpClientContextBuilder {
     return credentialsProviderProvider;
   }
 
-  /**
-   * @deprecated use {@link #createContext(Object)}
-   */
-  @Deprecated
-  public HttpClientContext createContext() {
-    return createContext(null);
-  }
-  
   public HttpClientContext createContext(Object userToken) {
     HttpClientContext context = new HttpClientContext();
     if (getCredentialsProviderProvider() != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java
index 5075330..03595c2 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java
@@ -46,7 +46,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
-import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
@@ -72,30 +71,6 @@ public class ShortestPathStream extends TupleStream implements Expressible {
   private int threads;
   private SolrParams queryParams;
 
-  @Deprecated
-  public ShortestPathStream(String zkHost,
-                            String collection,
-                            String fromNode,
-                            String toNode,
-                            String fromField,
-                            String toField,
-                            Map queryParams,
-                            int joinBatchSize,
-                            int threads,
-                            int maxDepth) {
-
-    init(zkHost,
-        collection,
-        fromNode,
-        toNode,
-        fromField,
-        toField,
-        new MapSolrParams(queryParams),
-        joinBatchSize,
-        threads,
-        maxDepth);
-  }
-
   public ShortestPathStream(String zkHost,
                             String collection,
                             String fromNode,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
index 6d1764a..62e4079 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java
@@ -51,7 +51,6 @@ import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
@@ -89,32 +88,12 @@ public class CloudSolrStream extends TupleStream implements Expressible {
     
   }
 
-
-  /**
-   * @param zkHost         Zookeeper ensemble connection string
-   * @param collectionName Name of the collection to operate on
-   * @param params         Map&lt;String, String&gt; of parameter/value pairs
-   * @throws IOException Something went wrong
-   *                     <p>
-   *                     This form does not allow specifying multiple clauses, say "fq" clauses, use the form that
-   *                     takes a SolrParams. Transition code can call the preferred method that takes SolrParams
-   *                     by calling CloudSolrStream(zkHost, collectionName,
-   *                     new ModifiableSolrParams(SolrParams.toMultiMap(new NamedList(Map&lt;String, String&gt;)));
-   * @deprecated         Use the constructor that has a SolrParams obj rather than a Map
-   */
-
-  @Deprecated
-  public CloudSolrStream(String zkHost, String collectionName, Map params) throws IOException {
-    init(collectionName, zkHost, new MapSolrParams(params));
-  }
-
   /**
    * @param zkHost         Zookeeper ensemble connection string
    * @param collectionName Name of the collection to operate on
    * @param params         Map&lt;String, String[]&gt; of parameter/value pairs
    * @throws IOException Something went wrong
    */
-
   public CloudSolrStream(String zkHost, String collectionName, SolrParams params) throws IOException {
     init(collectionName, zkHost, params);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
index 0180764..c5bd56b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java
@@ -46,7 +46,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.client.solrj.io.stream.metrics.Bucket;
 import org.apache.solr.client.solrj.io.stream.metrics.Metric;
 import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
@@ -72,21 +71,6 @@ public class FacetStream extends TupleStream implements Expressible  {
   protected transient SolrClientCache cache;
   protected transient CloudSolrClient cloudSolrClient;
 
-  /*
-   *
-   * @deprecated. Use the form that takes a SolrParams rather than Map&ltString, String&gt;
-   */
-  @Deprecated
-  public FacetStream(String zkHost,
-                     String collection,
-                     Map<String, String> props,
-                     Bucket[] buckets,
-                     Metric[] metrics,
-                     FieldComparator[] bucketSorts,
-                     int bucketSizeLimit) throws IOException {
-    init(collection, new MapSolrParams(props), buckets, bucketSorts, metrics, bucketSizeLimit, zkHost);
-  }
-
   public FacetStream(String zkHost,
                      String collection,
                      SolrParams params,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RandomStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RandomStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RandomStream.java
index 5ba485d..269a800 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RandomStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RandomStream.java
@@ -174,7 +174,7 @@ public class RandomStream extends TupleStream implements Expressible  {
     if(cache != null) {
       cloudSolrClient = cache.getCloudSolrClient(zkHost);
     } else {
-      cloudSolrClient = new CloudSolrClient(zkHost);
+      cloudSolrClient = (new CloudSolrClient.Builder()).withZkHost(zkHost).build();
     }
 
     ModifiableSolrParams params = getParams(this.props);
@@ -236,4 +236,4 @@ public class RandomStream extends TupleStream implements Expressible  {
   public StreamComparator getStreamSort() {
     return null;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java
index 31d1913..ab029af 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SolrStream.java
@@ -41,7 +41,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
@@ -74,19 +73,6 @@ public class SolrStream extends TupleStream {
   /**
    * @param baseUrl Base URL of the stream.
    * @param params  Map&lt;String, String&gt; of parameters
-   * @deprecated, use the form that thakes SolrParams. Existing code can use
-   * new ModifiableSolrParams(SolrParams.toMultiMap(new NamedList(params)))
-   * for existing calls that use Map&lt;String, String&gt;
-   */
-  @Deprecated
-  public SolrStream(String baseUrl, Map params) {
-    this.baseUrl = baseUrl;
-    this.params = new ModifiableSolrParams(new MapSolrParams(params));
-  }
-
-  /**
-   * @param baseUrl Base URL of the stream.
-   * @param params  Map&lt;String, String&gt; of parameters
    */
 
   public SolrStream(String baseUrl, SolrParams params) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SqlStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SqlStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SqlStream.java
index d7c10e4..d2296b7 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SqlStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SqlStream.java
@@ -60,9 +60,7 @@ public class SqlStream extends TupleStream implements Expressible {
    *                     takes a SolrParams. Transition code can call the preferred method that takes SolrParams
    *                     by calling CloudSolrStream(zkHost, collectionName,
    *                     new ModifiableSolrParams(SolrParams.toMultiMap(new NamedList(Map&lt;String, String&gt;)));
-   * @deprecated         Use the constructor that has a SolrParams obj rather than a Map
    */
-  
   public SqlStream(String zkHost, String collectionName, SolrParams params) throws IOException {
     init(collectionName, zkHost, params);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StatsStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StatsStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StatsStream.java
index cb46db4..c20429d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StatsStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StatsStream.java
@@ -42,7 +42,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.client.solrj.io.stream.metrics.Metric;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
@@ -63,15 +62,6 @@ public class StatsStream extends TupleStream implements Expressible  {
   protected transient CloudSolrClient cloudSolrClient;
   protected StreamContext streamContext;
 
-  // Use StatsStream(String, String, SolrParams, Metric[]
-  @Deprecated
-  public StatsStream(String zkHost,
-                     String collection,
-                     Map<String, String> props,
-                     Metric[] metrics) {
-    init(zkHost, collection, new MapSolrParams(props), metrics);
-  }
-
   public StatsStream(String zkHost,
                      String collection,
                      SolrParams params,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
index 58063d0..7fff901 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
@@ -55,7 +55,6 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
@@ -83,24 +82,6 @@ public class TopicStream extends CloudSolrStream implements Expressible  {
   private String checkpointCollection;
   private long initialCheckpoint = -1;
 
-  // Use TopicStream that takes a SolrParams
-  @Deprecated
-  public TopicStream(String zkHost,
-                     String checkpointCollection,
-                     String collection,
-                     String id,
-                     long initialCheckpoint,
-                     long checkpointEvery,
-                     Map<String, String> params) {
-    init(zkHost,
-         checkpointCollection,
-         collection,
-         id,
-         initialCheckpoint,
-         checkpointEvery,
-         new MapSolrParams(params));
-  }
-
   public TopicStream(String zkHost,
                      String checkpointCollection,
                      String collection,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java b/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
index ef254cc..e09a2dc 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/HighlightParams.java
@@ -24,8 +24,6 @@ public interface HighlightParams {
   // primary
   public static final String HIGHLIGHT   = "hl";
   public static final String METHOD      = HIGHLIGHT+".method"; // original|fastVector|postings|unified
-  @Deprecated // see hl.method
-  public static final String USE_FVH     = HIGHLIGHT + ".useFastVectorHighlighter";
   public static final String FIELDS      = HIGHLIGHT+".fl";
   public static final String SNIPPETS    = HIGHLIGHT+".snippets";
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java b/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
index 5f307a8..a045726 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
@@ -71,47 +71,6 @@ public class ExecutorUtil {
     public void clean(AtomicReference<?> ctx);
   }
 
-  // ** This will interrupt the threads! ** Lucene and Solr do not like this because it can close channels, so only use
-  // this if you know what you are doing - you probably want shutdownAndAwaitTermination.
-  // Marked as Deprecated to discourage use.
-  @Deprecated
-  public static void shutdownWithInterruptAndAwaitTermination(ExecutorService pool) {
-    pool.shutdownNow(); // Cancel currently executing tasks - NOTE: this interrupts!
-    boolean shutdown = false;
-    while (!shutdown) {
-      try {
-        // Wait a while for existing tasks to terminate
-        shutdown = pool.awaitTermination(60, TimeUnit.SECONDS);
-      } catch (InterruptedException ie) {
-        // Preserve interrupt status
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
-  
-  // ** This will interrupt the threads! ** Lucene and Solr do not like this because it can close channels, so only use
-  // this if you know what you are doing - you probably want shutdownAndAwaitTermination.
-  // Marked as Deprecated to discourage use.
-  @Deprecated
-  public static void shutdownAndAwaitTerminationWithInterrupt(ExecutorService pool) {
-    pool.shutdown(); // Disable new tasks from being submitted
-    boolean shutdown = false;
-    boolean interrupted = false;
-    while (!shutdown) {
-      try {
-        // Wait a while for existing tasks to terminate
-        shutdown = pool.awaitTermination(60, TimeUnit.SECONDS);
-      } catch (InterruptedException ie) {
-        // Preserve interrupt status
-        Thread.currentThread().interrupt();
-      }
-      if (!shutdown && !interrupted) {
-        pool.shutdownNow(); // Cancel currently executing tasks - NOTE: this interrupts!
-        interrupted = true;
-      }
-    }
-  }
-
   public static void shutdownAndAwaitTermination(ExecutorService pool) {
     pool.shutdown(); // Disable new tasks from being submitted
     boolean shutdown = false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java b/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
index 0313d3a..d34d8e7 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
@@ -122,12 +122,17 @@ public class NamedList<T> implements Cloneable, Serializable, Iterable<Map.Entry
    * When using this constructor, runtime type safety is only guaranteed if
    * all even numbered elements of the input list are of type "T".
    * </p>
-   *
-   * @param nameValuePairs underlying List which should be used to implement a NamedList
-   * @deprecated Use {@link #NamedList(java.util.Map.Entry[])} for the NamedList instantiation
+   * <p>
+   * This method is package protected and exists solely so SimpleOrderedMap and clone() can utilize it
+   * </p>
+   * <p>
+   * TODO: this method was formerly public, now that it's not we can change the impl details of 
+   * this class to be based on a Map.Entry[] 
+   * </p>
+   * @lucene.internal
+   * @see #nameValueMapToList
    */
-  @Deprecated
-  public NamedList(List<Object> nameValuePairs) {
+  NamedList(List<Object> nameValuePairs) {
     nvPairs=nameValuePairs;
   }
 
@@ -136,12 +141,14 @@ public class NamedList<T> implements Cloneable, Serializable, Iterable<Map.Entry
    * indexed elements (0,2,4. ..etc) are Strings and odd elements (1,3,5,) are of
    * the type "T".
    *
-   * @return Modified List as per the above description
-   * @deprecated This a temporary placeholder method until the guts of the class
+   * <p>
+   * NOTE: This a temporary placeholder method until the guts of the class
    * are actually replaced by List&lt;String, ?&gt;.
+   * </p>
+   *
+   * @return Modified List as per the above description
    * @see <a href="https://issues.apache.org/jira/browse/SOLR-912">SOLR-912</a>
    */
-  @Deprecated
   private List<Object> nameValueMapToList(Map.Entry<String, ? extends T>[] nameValuePairs) {
     List<Object> result = new ArrayList<>();
     for (Map.Entry<String, ?> ent : nameValuePairs) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/java/org/apache/solr/common/util/SimpleOrderedMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/SimpleOrderedMap.java b/solr/solrj/src/java/org/apache/solr/common/util/SimpleOrderedMap.java
index 3fee6da..701cdc4 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/SimpleOrderedMap.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/SimpleOrderedMap.java
@@ -50,10 +50,15 @@ public class SimpleOrderedMap<T> extends NamedList<T> {
    * Creates an instance backed by an explicitly specified list of
    * pairwise names/values.
    *
+   * <p>
+   * TODO: this method was formerly public, now that it's not we can change the impl details of 
+   * this class to be based on a Map.Entry[] 
+   * </p>
+   *
    * @param nameValuePairs underlying List which should be used to implement a SimpleOrderedMap; modifying this List will affect the SimpleOrderedMap.
+   * @lucene.internal
    */
-  @Deprecated
-  public SimpleOrderedMap(List<Object> nameValuePairs) {
+  private SimpleOrderedMap(List<Object> nameValuePairs) {
     super(nameValuePairs);
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java
index 02ed7be..c2314f8 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java
@@ -50,7 +50,7 @@ public class SolrExampleStreamingTest extends SolrExampleTests {
     public Throwable lastError = null;
 
     public ErrorTrackingConcurrentUpdateSolrClient(String solrServerUrl, int queueSize, int threadCount) {
-      super(solrServerUrl, queueSize, threadCount);
+      super(solrServerUrl, null, queueSize, threadCount, null, false);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrClientTest.java
----------------------------------------------------------------------
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 06ae8b8..4addce3 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
@@ -839,7 +839,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
 
     try(HttpSolrClient createdClient = new HttpSolrClient.Builder()
         .withBaseSolrUrl(jetty.getBaseUrl().toString())
-        .withDelegationToken("mydt")
+        .withKerberosDelegationToken("mydt")
         .withInvariantParams(SolrTestCaseJ4.params(DelegationTokenHttpSolrClient.DELEGATION_TOKEN_PARAM, "mydt"))
         .build()) {
       fail();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientMultiConstructorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientMultiConstructorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientMultiConstructorTest.java
index 3a132d7..e1831f9 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientMultiConstructorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientMultiConstructorTest.java
@@ -67,7 +67,7 @@ public class CloudSolrClientMultiConstructorTest extends LuceneTestCase {
       clientChroot = "/mychroot";
     }
 
-    try (CloudSolrClient client = new CloudSolrClient(hosts, clientChroot)) {
+    try (CloudSolrClient client = (new CloudSolrClient.Builder()).withZkHost(hosts).withZkChroot(clientChroot).build()) {
       assertEquals(sb.toString(), client.getZkHost());
     }
 
@@ -77,6 +77,6 @@ public class CloudSolrClientMultiConstructorTest extends LuceneTestCase {
   public void testBadChroot() {
     hosts = new ArrayList<>();
     hosts.add("host1:2181");
-    new CloudSolrClient(hosts, "foo");
+    (new CloudSolrClient.Builder()).withZkHost(hosts).withZkChroot("foo").build();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java
index 24f08d2..4b061d5 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java
@@ -194,7 +194,11 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
     int cussThreadCount = 2;
     int cussQueueSize = 10;
 
-    try (ConcurrentUpdateSolrClient concurrentClient = new ConcurrentUpdateSolrClient(jetty.getBaseUrl().toString(), cussQueueSize, cussThreadCount)) {
+    try (ConcurrentUpdateSolrClient concurrentClient
+         = (new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString()))
+         .withQueueSize(cussQueueSize)
+         .withThreadCount(cussThreadCount).build()) {
+      
       SolrInputDocument doc = new SolrInputDocument();
       doc.addField("id", "collection");
       concurrentClient.add("collection1", doc);
@@ -203,7 +207,11 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
       assertEquals(1, concurrentClient.query("collection1", new SolrQuery("id:collection")).getResults().getNumFound());
     }
 
-    try (ConcurrentUpdateSolrClient concurrentClient = new ConcurrentUpdateSolrClient(jetty.getBaseUrl().toString() + "/collection1", cussQueueSize, cussThreadCount)) {
+    try (ConcurrentUpdateSolrClient concurrentClient
+         = (new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString() + "/collection1"))
+         .withQueueSize(cussQueueSize)
+         .withThreadCount(cussThreadCount).build()) {
+         
       assertEquals(1, concurrentClient.query(new SolrQuery("id:collection")).getResults().getNumFound());
     }
 
@@ -218,7 +226,10 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
     int numRunnables = 5;
     int expected = numDocs * numRunnables;
 
-    try (ConcurrentUpdateSolrClient concurrentClient = new ConcurrentUpdateSolrClient(jetty.getBaseUrl().toString(), cussQueueSize, cussThreadCount)) {
+    try (ConcurrentUpdateSolrClient concurrentClient
+         = (new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString()))
+         .withQueueSize(cussQueueSize)
+         .withThreadCount(cussThreadCount).build()) {
       concurrentClient.setPollQueueTime(0);
 
       // ensure it doesn't block where there's nothing to do yet
@@ -246,7 +257,11 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase {
       concurrentClient.shutdownNow();
     }
 
-    try (ConcurrentUpdateSolrClient concurrentClient = new ConcurrentUpdateSolrClient(jetty.getBaseUrl().toString() + "/collection1", cussQueueSize, cussThreadCount)) {
+    try (ConcurrentUpdateSolrClient concurrentClient
+         = (new ConcurrentUpdateSolrClient.Builder(jetty.getBaseUrl().toString() + "/collection1"))
+         .withQueueSize(cussQueueSize)
+         .withThreadCount(cussThreadCount).build()) {
+
       assertEquals(expected, concurrentClient.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
index 33781ef..9b11783 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/graph/GraphExpressionTest.java
@@ -862,7 +862,7 @@ public class GraphExpressionTest extends SolrCloudTestCase {
     JettySolrRunner runner = runners.get(0);
     String url = runner.getBaseUrl().toString();
 
-    HttpSolrClient client = new HttpSolrClient(url);
+    HttpSolrClient client = getHttpSolrClient(url);
     ModifiableSolrParams params = new ModifiableSolrParams();
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bc973ecd/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index a55e2ea..419f94f 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -2222,7 +2222,14 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
   public static Object skewed(Object likely, Object unlikely) {
     return (0 == TestUtil.nextInt(random(), 0, 9)) ? unlikely : likely;
   }
-  
+
+  /**
+   * A variant of {@link  org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} that will randomize which nodes recieve updates 
+   * unless otherwise specified by the caller.
+   *
+   * @see #sendDirectUpdatesToAnyShardReplica
+   * @see #sendDirectUpdatesToShardLeadersOnly
+   */
   public static class CloudSolrClientBuilder extends CloudSolrClient.Builder {
 
     private boolean configuredDUTflag = false;
@@ -2267,30 +2274,35 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     }
   }
 
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
+   */ 
   public static CloudSolrClient getCloudSolrClient(String zkHost) {
-    if (random().nextBoolean()) {
-      return new CloudSolrClient(zkHost);
-    }
     return new CloudSolrClientBuilder()
         .withZkHost(zkHost)
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
+   */ 
   public static CloudSolrClient getCloudSolrClient(String zkHost, HttpClient httpClient) {
-    if (random().nextBoolean()) {
-      return new CloudSolrClient(zkHost, httpClient);
-    }
     return new CloudSolrClientBuilder()
         .withZkHost(zkHost)
         .withHttpClient(httpClient)
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
+   */ 
   public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly) {
-    if (random().nextBoolean()) {
-      return new CloudSolrClient(zkHost, shardLeadersOnly);
-    }
-    
     if (shardLeadersOnly) {
       return new CloudSolrClientBuilder()
           .withZkHost(zkHost)
@@ -2303,11 +2315,12 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
+   */ 
   public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly, HttpClient httpClient) {
-    if (random().nextBoolean()) {
-      return new CloudSolrClient(zkHost, shardLeadersOnly, httpClient);
-    }
-    
     if (shardLeadersOnly) {
       return new CloudSolrClientBuilder()
           .withZkHost(zkHost)
@@ -2322,20 +2335,24 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient.Builder} class directly
+   */ 
   public static ConcurrentUpdateSolrClient getConcurrentUpdateSolrClient(String baseSolrUrl, int queueSize, int threadCount) {
-    if (random().nextBoolean()) {
-      return new ConcurrentUpdateSolrClient(baseSolrUrl, queueSize, threadCount);
-    }
     return new ConcurrentUpdateSolrClient.Builder(baseSolrUrl)
         .withQueueSize(queueSize)
         .withThreadCount(threadCount)
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient.Builder} class directly
+   */ 
   public static ConcurrentUpdateSolrClient getConcurrentUpdateSolrClient(String baseSolrUrl, HttpClient httpClient, int queueSize, int threadCount) {
-    if (random().nextBoolean()) {
-      return new ConcurrentUpdateSolrClient(baseSolrUrl, httpClient, queueSize, threadCount);
-    }
     return new ConcurrentUpdateSolrClient.Builder(baseSolrUrl)
         .withHttpClient(httpClient)
         .withQueueSize(queueSize)
@@ -2343,30 +2360,35 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.LBHttpSolrClient.Builder} class directly
+   */ 
   public static LBHttpSolrClient getLBHttpSolrClient(HttpClient client, String... solrUrls) {
-    if (random().nextBoolean()) {
-      return new LBHttpSolrClient(client, solrUrls);
-    }
-    
     return new LBHttpSolrClient.Builder()
         .withHttpClient(client)
         .withBaseSolrUrls(solrUrls)
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.LBHttpSolrClient.Builder} class directly
+   */ 
   public static LBHttpSolrClient getLBHttpSolrClient(String... solrUrls) throws MalformedURLException {
-    if (random().nextBoolean()) {
-      return new LBHttpSolrClient(solrUrls);
-    }
     return new LBHttpSolrClient.Builder()
         .withBaseSolrUrls(solrUrls)
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
+   */ 
   public static HttpSolrClient getHttpSolrClient(String url, HttpClient httpClient, ResponseParser responseParser, boolean compression) {
-    if(random().nextBoolean()) {
-      return new HttpSolrClient(url, httpClient, responseParser, compression);
-    }
     return new Builder(url)
         .withHttpClient(httpClient)
         .withResponseParser(responseParser)
@@ -2374,29 +2396,35 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
+   */ 
   public static HttpSolrClient getHttpSolrClient(String url, HttpClient httpClient, ResponseParser responseParser) {
-    if(random().nextBoolean()) {
-      return new HttpSolrClient(url, httpClient, responseParser);
-    }
     return new Builder(url)
         .withHttpClient(httpClient)
         .withResponseParser(responseParser)
         .build();
   }
   
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
+   */ 
   public static HttpSolrClient getHttpSolrClient(String url, HttpClient httpClient) {
-    if(random().nextBoolean()) {
-      return new HttpSolrClient(url, httpClient);
-    }
     return new Builder(url)
         .withHttpClient(httpClient)
         .build();
   }
 
+  /**
+   * This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
+   * Tests that do not wish to have any randomized behavior should use the 
+   * {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
+   */ 
   public static HttpSolrClient getHttpSolrClient(String url) {
-    if(random().nextBoolean()) {
-      return new HttpSolrClient(url);
-    }
     return new Builder(url)
         .build();
   }