You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2016/12/29 12:23:02 UTC

[1/6] lucene-solr:branch_6x: LUCENE-7607: FieldLeafComparator.setScorer() should throw IOException

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 529c60d91 -> e2aa2b638
  refs/heads/master fa959ad25 -> 3f24fd81c


LUCENE-7607: FieldLeafComparator.setScorer() should throw IOException


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

Branch: refs/heads/branch_6x
Commit: 16b1db4e99c018b8c9507cf0f62a79e56484d6c0
Parents: 183f998
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Dec 28 19:48:16 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 11:57:16 2016 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                                    |  3 +++
 .../org/apache/lucene/search/LeafFieldComparator.java |  2 +-
 .../apache/lucene/search/SimpleFieldComparator.java   |  2 +-
 .../apache/solr/search/CollapsingQParserPlugin.java   | 14 +++++++-------
 4 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/16b1db4e/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 5c4996f..5ab649e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -10,6 +10,9 @@ API Changes
 * LUCENE-7533: Classic query parser no longer allows autoGeneratePhraseQueries
   to be set to true when splitOnWhitespace is false (and vice-versa).
 
+* LUCENE-7607: LeafFieldComparator.setScorer and SimpleFieldComparator.setScorer
+  are declared as throwing IOException (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/16b1db4e/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java b/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
index 7f84953..b016c46 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
@@ -114,6 +114,6 @@ public interface LeafFieldComparator {
    * 
    * @param scorer Scorer instance that you should use to
    * obtain the current hit's score, if necessary. */
-  void setScorer(Scorer scorer);
+  void setScorer(Scorer scorer) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/16b1db4e/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java b/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
index 3f1e6c2..a258e24 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
@@ -38,5 +38,5 @@ public abstract class SimpleFieldComparator<T> extends FieldComparator<T> implem
   }
 
   @Override
-  public void setScorer(Scorer scorer) {}
+  public void setScorer(Scorer scorer) throws IOException {}
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/16b1db4e/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
index 7c88fad..b6bdc43 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -915,7 +915,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     @Override public boolean needsScores() { return needsScores || super.needsScores(); }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.collapseStrategy.setScorer(scorer);
     }
 
@@ -1077,7 +1077,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     @Override public boolean needsScores() { return needsScores || super.needsScores(); }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.collapseStrategy.setScorer(scorer);
     }
 
@@ -1421,7 +1421,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       return collapsedSet;
     }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.scorer = scorer;
     }
 
@@ -1818,7 +1818,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
 
     @Override
-    public void setScorer(Scorer s) {
+    public void setScorer(Scorer s) throws IOException {
       super.setScorer(s);
       this.compareState.setScorer(s);
     }
@@ -1966,7 +1966,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       return collapsedSet;
     }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.scorer = scorer;
     }
 
@@ -2367,7 +2367,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
 
     @Override
-    public void setScorer(Scorer s) {
+    public void setScorer(Scorer s) throws IOException {
       super.setScorer(s);
       this.compareState.setScorer(s);
     }
@@ -2518,7 +2518,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         leafFieldComparators[clause] = fieldComparators[clause].getLeafComparator(context);
       }
     }
-    public void setScorer(Scorer s) {
+    public void setScorer(Scorer s) throws IOException {
       for (int clause = 0; clause < numClauses; clause++) {
         leafFieldComparators[clause].setScorer(s);
       }


[4/6] lucene-solr:master: SOLR-9132: Cut over some more tests

Posted by ro...@apache.org.
SOLR-9132: Cut over some more tests


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

Branch: refs/heads/master
Commit: 12aff1cfcc48d7c89008447d482bf610242e0431
Parents: fa959ad
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Oct 27 16:50:28 2016 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 12:22:40 2016 +0000

----------------------------------------------------------------------
 .../java/org/apache/solr/cloud/Overseer.java    |   2 +-
 .../solr/cloud/CollectionsAPISolrJTest.java     | 470 +++++++------------
 .../DeleteLastCustomShardedReplicaTest.java     | 104 +---
 .../org/apache/solr/cloud/DeleteShardTest.java  | 211 +++------
 .../cloud/OverseerModifyCollectionTest.java     |  92 ++--
 .../apache/solr/cloud/OverseerRolesTest.java    | 165 +++----
 .../apache/solr/cloud/OverseerStatusTest.java   |  55 +--
 .../apache/solr/cloud/RemoteQueryErrorTest.java |  53 +--
 .../solr/cloud/TestDownShardTolerantSearch.java |  40 +-
 .../TestExclusionRuleCollectionAccess.java      |  38 +-
 .../PKIAuthenticationIntegrationTest.java       |  40 +-
 .../solrj/request/CollectionAdminRequest.java   |   2 +
 12 files changed, 457 insertions(+), 815 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/java/org/apache/solr/cloud/Overseer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index d7285fa..a618874 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -371,7 +371,7 @@ public class Overseer implements Closeable {
             return Collections.singletonList(new SliceMutator(getZkStateReader()).updateShardState(clusterState, message));
           case QUIT:
             if (myId.equals(message.get("id"))) {
-              log.info("Quit command received {}", LeaderElector.getNodeName(myId));
+              log.info("Quit command received {} {}", message, LeaderElector.getNodeName(myId));
               overseerCollectionConfigSetProcessor.close();
               close();
             } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index b04bfbc..616b657 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -16,70 +16,47 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.codec.binary.StringUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.request.CoreStatus;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.CoreAdminResponse;
-import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.apache.solr.cloud.ReplicaPropertiesBase.verifyUniqueAcrossCollection;
-
 @LuceneTestCase.Slow
-public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
+public class CollectionsAPISolrJTest extends SolrCloudTestCase {
 
-  @Test
-  public void test() throws Exception {
-    testCreateAndDeleteCollection();
-    testCreateAndDeleteShard();
-    testReloadCollection();
-    testCreateAndDeleteAlias();
-    testSplitShard();
-    testCreateCollectionWithPropertyParam();
-    testAddAndDeleteReplica();
-    testClusterProp();
-    testAddAndRemoveRole();
-    testOverseerStatus();
-    testList();
-    testAddAndDeleteReplicaProp();
-    testBalanceShardUnique();
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(4)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
-  protected void testCreateAndDeleteCollection() throws Exception {
+  @Test
+  public void testCreateAndDeleteCollection() throws Exception {
     String collectionName = "solrj_test";
-    CollectionAdminRequest.Create createCollectionRequest = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setNumShards(2)
-            .setReplicationFactor(2)
-            .setConfigName("conf1")
-            .setRouterField("myOwnField")
-            .setStateFormat(1);
-
-    CollectionAdminResponse response = createCollectionRequest.process(cloudClient);
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
+        .setStateFormat(1)
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
@@ -91,57 +68,44 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
       assertTrue(status.get("QTime") > 0);
     }
 
-    cloudClient.setDefaultCollection(collectionName);
-    CollectionAdminRequest.Delete deleteCollectionRequest = new CollectionAdminRequest.Delete()
-            .setCollectionName(collectionName);
-    response = deleteCollectionRequest.process(cloudClient);
+    response = CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String,NamedList<Integer>> nodesStatus = response.getCollectionNodesStatus();
-    assertNull("Deleted collection " + collectionName + "still exists",
-        cloudClient.getZkStateReader().getClusterState().getCollectionOrNull(collectionName));
     assertEquals(4, nodesStatus.size());
-    
+
+    waitForState("Expected " + collectionName + " to disappear from cluster state", collectionName, (n, c) -> c == null);
+
     // Test Creating a collection with new stateformat.
     collectionName = "solrj_newstateformat";
-    createCollectionRequest = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setNumShards(2)
-            .setConfigName("conf1")
-            .setStateFormat(2);
 
-    response = createCollectionRequest.process(cloudClient);
+    response = CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
+        .setStateFormat(2)
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    waitForRecoveriesToFinish(collectionName, false);
-    assertTrue("Collection state does not exist",
-        cloudClient.getZkStateReader().getZkClient()
-            .exists(ZkStateReader.getCollectionPath(collectionName), true));
+    waitForState("Expected " + collectionName + " to appear in cluster state", collectionName, (n, c) -> c != null);
 
   }
-  
-  protected void testCreateAndDeleteShard() throws IOException, SolrServerException {
+
+  @Test
+  public void testCreateAndDeleteShard() throws IOException, SolrServerException {
+
     // Create an implicit collection
     String collectionName = "solrj_implicit";
-    CollectionAdminResponse response = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setShards("shardA,shardB")
-            .setConfigName("conf1")
-            .setRouterName("implicit").process(cloudClient);
+    CollectionAdminResponse response
+        = CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "shardA,shardB", 1)
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
     assertEquals(2, coresStatus.size());
 
-    cloudClient.setDefaultCollection(collectionName);
     // Add a shard to the implicit collection
-    response = new CollectionAdminRequest
-        .CreateShard()
-            .setCollectionName(collectionName)
-            .setShardName("shardC").process(cloudClient);
+    response = CollectionAdminRequest.createShard(collectionName, "shardC").process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
@@ -149,57 +113,38 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
     assertEquals(1, coresStatus.size());
     assertEquals(0, (int) coresStatus.get(collectionName + "_shardC_replica1").get("status"));
 
-    CollectionAdminRequest.DeleteShard deleteShardRequest = new CollectionAdminRequest
-        .DeleteShard()
-            .setCollectionName(collectionName)
-            .setShardName("shardC");
-    response = deleteShardRequest.process(cloudClient);
+    response = CollectionAdminRequest.deleteShard(collectionName, "shardC").process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> nodesStatus = response.getCollectionNodesStatus();
     assertEquals(1, nodesStatus.size());
   }
-  
-  protected void testReloadCollection() throws IOException, SolrServerException {
-    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
-    CollectionAdminRequest.Reload reloadCollectionRequest = new CollectionAdminRequest.Reload()
-            .setCollectionName("collection1");
-    CollectionAdminResponse response = reloadCollectionRequest.process(cloudClient);
 
-    assertEquals(0, response.getStatus());
-  }
-  
-  protected void testCreateAndDeleteAlias() throws IOException, SolrServerException {
-    CollectionAdminRequest.CreateAlias createAliasRequest = new CollectionAdminRequest
-        .CreateAlias()
-            .setAliasName("solrj_alias")
-            .setAliasedCollections(DEFAULT_COLLECTION);
-    CollectionAdminResponse response = createAliasRequest.process(cloudClient);
+  @Test
+  public void testCreateAndDeleteAlias() throws IOException, SolrServerException {
+
+    final String collection = "aliasedCollection";
+    CollectionAdminRequest.createCollection(collection, "conf", 1, 1).process(cluster.getSolrClient());
 
+    CollectionAdminResponse response
+        = CollectionAdminRequest.createAlias("solrj_alias", collection).process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    CollectionAdminRequest.DeleteAlias deleteAliasRequest = new CollectionAdminRequest.DeleteAlias()
-            .setAliasName("solrj_alias");
-    deleteAliasRequest.process(cloudClient);
-    
+    response = CollectionAdminRequest.deleteAlias("solrj_alias").process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
   }
-  
-  protected void testSplitShard() throws Exception {
-    String collectionName = "solrj_test_splitshard";
-    cloudClient.setDefaultCollection(collectionName);
-    
-    CollectionAdminRequest.Create createCollectionRequest = new CollectionAdminRequest.Create()
-            .setConfigName("conf1")
-            .setNumShards(2)
-            .setCollectionName(collectionName);
-    createCollectionRequest.process(cloudClient);
-    
-    CollectionAdminRequest.SplitShard splitShardRequest = new CollectionAdminRequest.SplitShard()
-            .setCollectionName(collectionName)
-            .setShardName("shard1");
-    CollectionAdminResponse response = splitShardRequest.process(cloudClient);
+
+  @Test
+  public void testSplitShard() throws Exception {
+
+    final String collectionName = "solrj_test_splitshard";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .process(cluster.getSolrClient());
+
+    CollectionAdminResponse response = CollectionAdminRequest.splitShard(collectionName)
+        .setShardName("shard1")
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
@@ -207,267 +152,204 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
     assertEquals(0, (int) coresStatus.get(collectionName + "_shard1_0_replica1").get("status"));
     assertEquals(0, (int) coresStatus.get(collectionName + "_shard1_1_replica1").get("status"));
 
-    waitForRecoveriesToFinish(collectionName, false);
-    waitForThingsToLevelOut(10);
+    waitForState("Expected all shards to be active and parent shard to be removed", collectionName, (n, c) -> {
+      if (c.getSlice("shard1").getState() == Slice.State.ACTIVE)
+        return false;
+      for (Replica r : c.getReplicas()) {
+        if (r.isActive(n) == false)
+          return false;
+      }
+      return true;
+    });
     
     // Test splitting using split.key
-    splitShardRequest = new CollectionAdminRequest.SplitShard()
-            .setCollectionName(collectionName)
-            .setSplitKey("b!");
-    response = splitShardRequest.process(cloudClient);
+    response = CollectionAdminRequest.splitShard(collectionName)
+        .setSplitKey("b!")
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    waitForRecoveriesToFinish(collectionName, false);
-    waitForThingsToLevelOut(10);
-    
-    ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-    Collection<Slice> slices = clusterState.getActiveSlices(collectionName);
-    assertEquals("ClusterState: "+ clusterState.getActiveSlices(collectionName), 5, slices.size());  
+    waitForState("Expected 5 slices to be active", collectionName, (n, c) -> c.getActiveSlices().size() == 5);
     
   }
 
-  private void testCreateCollectionWithPropertyParam() throws Exception {
+  @Test
+  public void testCreateCollectionWithPropertyParam() throws Exception {
+
     String collectionName = "solrj_test_core_props";
     
-    File tmpDir = createTempDir("testPropertyParamsForCreate").toFile();
-    File dataDir = new File(tmpDir, "dataDir-" + TestUtil.randomSimpleString(random(), 1, 5));
-    File ulogDir = new File(tmpDir, "ulogDir-" + TestUtil.randomSimpleString(random(), 1, 5));
+    Path tmpDir = createTempDir("testPropertyParamsForCreate");
+    Path dataDir = tmpDir.resolve("dataDir-" + TestUtil.randomSimpleString(random(), 1, 5));
+    Path ulogDir = tmpDir.resolve("ulogDir-" + TestUtil.randomSimpleString(random(), 1, 5));
 
-    Properties properties = new Properties();
-    properties.put(CoreAdminParams.DATA_DIR, dataDir.getAbsolutePath());
-    properties.put(CoreAdminParams.ULOG_DIR, ulogDir.getAbsolutePath());
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
+        .withProperty(CoreAdminParams.DATA_DIR, dataDir.toString())
+        .withProperty(CoreAdminParams.ULOG_DIR, ulogDir.toString())
+        .process(cluster.getSolrClient());
 
-    CollectionAdminRequest.Create createReq = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setNumShards(1)
-            .setConfigName("conf1")
-            .setProperties(properties);
-
-    CollectionAdminResponse response = createReq.process(cloudClient);
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
     assertEquals(1, coresStatus.size());
 
-    DocCollection testCollection = cloudClient.getZkStateReader()
-        .getClusterState().getCollection(collectionName);
+    DocCollection testCollection = getCollectionState(collectionName);
 
     Replica replica1 = testCollection.getReplica("core_node1");
+    CoreStatus coreStatus = getCoreStatus(replica1);
 
-    try (HttpSolrClient client = getHttpSolrClient(replica1.getStr("base_url"))) {
-      CoreAdminResponse status = CoreAdminRequest.getStatus(replica1.getStr("core"), client);
-      NamedList<Object> coreStatus = status.getCoreStatus(replica1.getStr("core"));
-      String dataDirStr = (String) coreStatus.get("dataDir");
-      assertEquals("Data dir does not match param given in property.dataDir syntax",
-          new File(dataDirStr).getAbsolutePath(), dataDir.getAbsolutePath());
-    }
+    assertEquals(Paths.get(coreStatus.getDataDirectory()).toString(), dataDir.toString());
 
-    CollectionAdminRequest.Delete deleteCollectionRequest = new CollectionAdminRequest.Delete();
-    deleteCollectionRequest.setCollectionName(collectionName);
-    deleteCollectionRequest.process(cloudClient);
   }
 
-  private void testAddAndDeleteReplica() throws Exception {
-    String collectionName = "solrj_replicatests";
-    createCollection(collectionName, cloudClient, 1, 2);
+  @Test
+  public void testAddAndDeleteReplica() throws Exception {
 
-    cloudClient.setDefaultCollection(collectionName);
+    final String collectionName = "solrj_replicatests";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2)
+        .process(cluster.getSolrClient());
 
-    String newReplicaName = Assign.assignNode(cloudClient.getZkStateReader().getClusterState().getCollection(collectionName));
-    ArrayList<String> nodeList = new ArrayList<>(cloudClient.getZkStateReader().getClusterState().getLiveNodes());
+    String newReplicaName = Assign.assignNode(getCollectionState(collectionName));
+    ArrayList<String> nodeList
+        = new ArrayList<>(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes());
     Collections.shuffle(nodeList, random());
-    CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()
-            .setCollectionName(collectionName)
-            .setShardName("shard1")
-            .setNode(nodeList.get(0));
-    CollectionAdminResponse response = addReplica.process(cloudClient);
+    final String node = nodeList.get(0);
+
+    CollectionAdminResponse response = CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
+        .setNode(node)
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
-    Replica newReplica = null;
-
-    while (! timeout.hasTimedOut() && newReplica == null) {
-      Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
-      newReplica = slice.getReplica(newReplicaName);
-    }
-
-    assertNotNull(newReplica);
-
-    assertEquals("Replica should be created on the right node",
-        cloudClient.getZkStateReader().getBaseUrlForNodeName(nodeList.get(0)),
-        newReplica.getStr(ZkStateReader.BASE_URL_PROP)
-    );
+    waitForState("Expected to see replica " + newReplicaName + " on node " + node, collectionName, (n, c) -> {
+      Replica r = c.getSlice("shard1").getReplica(newReplicaName);
+      return r != null && r.getNodeName().equals(node);
+    });
     
     // Test DELETEREPLICA
-    CollectionAdminRequest.DeleteReplica deleteReplicaRequest = new CollectionAdminRequest.DeleteReplica()
-            .setCollectionName(collectionName)
-            .setShardName("shard1")
-            .setReplica(newReplicaName);
-    response = deleteReplicaRequest.process(cloudClient);
-
+    response = CollectionAdminRequest.deleteReplica(collectionName, "shard1", newReplicaName)
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    timeout = new TimeOut(3, TimeUnit.SECONDS);
-
-    while (! timeout.hasTimedOut() && newReplica != null) {
-      Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
-      newReplica = slice.getReplica(newReplicaName);
-    }
+    waitForState("Expected replica " + newReplicaName + " to vanish from cluster state", collectionName,
+        (n, c) -> c.getSlice("shard1").getReplica(newReplicaName) == null);
 
-    assertNull(newReplica);
   }
 
-  private void testClusterProp() throws InterruptedException, IOException, SolrServerException {
-    CollectionAdminRequest.ClusterProp clusterPropRequest = new CollectionAdminRequest.ClusterProp()
-            .setPropertyName(ZkStateReader.LEGACY_CLOUD)
-            .setPropertyValue("false");
-    CollectionAdminResponse response = clusterPropRequest.process(cloudClient);
+  @Test
+  public void testClusterProp() throws InterruptedException, IOException, SolrServerException {
+
+    CollectionAdminResponse response = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, "false")
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
 
-    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
-    boolean changed = false;
-    
-    while(! timeout.hasTimedOut()){
-      Thread.sleep(10);
-      changed = Objects.equals("false",
-          cloudClient.getZkStateReader().getClusterProperty(ZkStateReader.LEGACY_CLOUD, "none"));
-      if(changed) break;
-    }
-    assertTrue("The Cluster property wasn't set", changed);
+    ClusterProperties props = new ClusterProperties(zkClient());
+    assertEquals("Cluster property was not set", props.getClusterProperty(ZkStateReader.LEGACY_CLOUD, "true"), "false");
     
     // Unset ClusterProp that we set.
-    clusterPropRequest = new CollectionAdminRequest.ClusterProp()
-            .setPropertyName(ZkStateReader.LEGACY_CLOUD)
-            .setPropertyValue(null);
-    clusterPropRequest.process(cloudClient);
-
-    timeout = new TimeOut(3, TimeUnit.SECONDS);
-    changed = false;
-    while(! timeout.hasTimedOut()) {
-      Thread.sleep(10);
-      changed = (cloudClient.getZkStateReader().getClusterProperty(ZkStateReader.LEGACY_CLOUD, (String) null) == null);
-      if(changed)  
-        break;
-    }
-    assertTrue("The Cluster property wasn't unset", changed);
+    CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, null).process(cluster.getSolrClient());
+    assertEquals("Cluster property was not unset", props.getClusterProperty(ZkStateReader.LEGACY_CLOUD, "true"), "true");
+
   }
 
-  private void testAddAndRemoveRole() throws InterruptedException, IOException, SolrServerException {
-    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
-    Replica replica = cloudClient.getZkStateReader().getLeaderRetry(DEFAULT_COLLECTION, SHARD1);
-    CollectionAdminRequest.AddRole addRoleRequest = new CollectionAdminRequest.AddRole()
-            .setNode(replica.getNodeName())
-            .setRole("overseer");
-    addRoleRequest.process(cloudClient);
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testAddAndRemoveRole() throws InterruptedException, IOException, SolrServerException {
+
+    String node = cluster.getRandomJetty(random()).getNodeName();
+
+    CollectionAdminRequest.addRole(node, "overseer").process(cluster.getSolrClient());
 
-    CollectionAdminRequest.ClusterStatus clusterStatusRequest = new CollectionAdminRequest.ClusterStatus()
-            .setCollectionName(DEFAULT_COLLECTION);
-    CollectionAdminResponse response = clusterStatusRequest.process(cloudClient);
+    CollectionAdminResponse response = CollectionAdminRequest.getClusterStatus().process(cluster.getSolrClient());
 
     NamedList<Object> rsp = response.getResponse();
-    NamedList<Object> cluster = (NamedList<Object>) rsp.get("cluster");
-    assertNotNull("Cluster state should not be null", cluster);
-    Map<String, Object> roles = (Map<String, Object>) cluster.get("roles");
+    NamedList<Object> cs = (NamedList<Object>) rsp.get("cluster");
+    assertNotNull("Cluster state should not be null", cs);
+    Map<String, Object> roles = (Map<String, Object>) cs.get("roles");
     assertNotNull("Role information should not be null", roles);
     List<String> overseer = (List<String>) roles.get("overseer");
     assertNotNull(overseer);
     assertEquals(1, overseer.size());
-    assertTrue(overseer.contains(replica.getNodeName()));
+    assertTrue(overseer.contains(node));
     
     // Remove role
-    new CollectionAdminRequest.RemoveRole()
-            .setNode(replica.getNodeName())
-            .setRole("overseer")
-            .process(cloudClient);
-
-    clusterStatusRequest = new CollectionAdminRequest.ClusterStatus();
-    clusterStatusRequest.setCollectionName(DEFAULT_COLLECTION);
-    response = clusterStatusRequest.process(cloudClient);
+    CollectionAdminRequest.removeRole(node, "overseer").process(cluster.getSolrClient());
 
+    response = CollectionAdminRequest.getClusterStatus().process(cluster.getSolrClient());
     rsp = response.getResponse();
-    cluster = (NamedList<Object>) rsp.get("cluster");
-    assertNotNull("Cluster state should not be null", cluster);
-    roles = (Map<String, Object>) cluster.get("roles");
+    cs = (NamedList<Object>) rsp.get("cluster");
+    assertNotNull("Cluster state should not be null", cs);
+    roles = (Map<String, Object>) cs.get("roles");
     assertNotNull("Role information should not be null", roles);
     overseer = (List<String>) roles.get("overseer");
-    assertFalse(overseer.contains(replica.getNodeName()));
+    assertFalse(overseer.contains(node));
   }
-  
-  private void testOverseerStatus() throws IOException, SolrServerException {
-    CollectionAdminResponse response = new CollectionAdminRequest.OverseerStatus().process(cloudClient);
+
+  @Test
+  public void testOverseerStatus() throws IOException, SolrServerException {
+    CollectionAdminResponse response = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
     assertNotNull("overseer_operations shouldn't be null", response.getResponse().get("overseer_operations"));
   }
-  
-  private void testList() throws IOException, SolrServerException {
-    CollectionAdminResponse response = new CollectionAdminRequest.List().process(cloudClient);
+
+  @Test
+  public void testList() throws IOException, SolrServerException {
+    CollectionAdminResponse response = new CollectionAdminRequest.List().process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
     assertNotNull("collection list should not be null", response.getResponse().get("collections"));
   }
-  
-  private void testAddAndDeleteReplicaProp() throws InterruptedException, IOException, SolrServerException {
-    Replica replica = cloudClient.getZkStateReader().getLeaderRetry(DEFAULT_COLLECTION, SHARD1);
-    CollectionAdminResponse response = new CollectionAdminRequest.AddReplicaProp()
-            .setCollectionName(DEFAULT_COLLECTION)
-            .setShardName(SHARD1)
-            .setReplica(replica.getName())
-            .setPropertyName("preferredleader")
-            .setPropertyValue("true").process(cloudClient);
-    assertEquals(0, response.getStatus());
 
-    TimeOut timeout = new TimeOut(20, TimeUnit.SECONDS);
-    String propertyValue = null;
-    
-    String replicaName = replica.getName();
-    while (! timeout.hasTimedOut()) {
-      ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-      replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
-      propertyValue = replica.getStr("property.preferredleader"); 
-      if(StringUtils.equals("true", propertyValue))
-        break;
-      Thread.sleep(50);
-    }
-    
-    assertEquals("Replica property was not updated, Latest value: " +
-        cloudClient.getZkStateReader().getClusterState().getReplica(DEFAULT_COLLECTION, replicaName),
-        "true",
-        propertyValue);
-
-    response = new CollectionAdminRequest.DeleteReplicaProp()
-            .setCollectionName(DEFAULT_COLLECTION)
-            .setShardName(SHARD1)
-            .setReplica(replicaName)
-            .setPropertyName("property.preferredleader").process(cloudClient);
+  @Test
+  public void testAddAndDeleteReplicaProp() throws InterruptedException, IOException, SolrServerException {
+
+    final String collection = "replicaProperties";
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    final Replica replica = getCollectionState(collection).getLeader("shard1");
+    CollectionAdminResponse response
+        = CollectionAdminRequest.addReplicaProperty(collection, "shard1", replica.getName(), "preferredleader", "true")
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    timeout = new TimeOut(20, TimeUnit.SECONDS);
-    boolean updated = false;
+    waitForState("Expecting property 'preferredleader' to appear on replica " + replica.getName(), collection,
+        (n, c) -> "true".equals(c.getReplica(replica.getName()).getStr("property.preferredleader")));
 
-    while (! timeout.hasTimedOut()) {
-      ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-      replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
-      updated = replica.getStr("property.preferredleader") == null;
-      if(updated)
-        break;
-      Thread.sleep(50);
-    }
+    response = CollectionAdminRequest.deleteReplicaProperty(collection, "shard1", replica.getName(), "property.preferredleader")
+        .process(cluster.getSolrClient());
+    assertEquals(0, response.getStatus());
 
-    assertTrue("Replica property was not removed", updated);
+    waitForState("Expecting property 'preferredleader' to be removed from replica " + replica.getName(), collection,
+        (n, c) -> c.getReplica(replica.getName()).getStr("property.preferredleader") == null);
     
   }
-  
-  private void testBalanceShardUnique() throws IOException,
+
+  @Test
+  public void testBalanceShardUnique() throws IOException,
       SolrServerException, KeeperException, InterruptedException {
-    CollectionAdminResponse response = new CollectionAdminRequest.BalanceShardUnique()
-            .setCollection(DEFAULT_COLLECTION)
-            .setPropertyName("preferredLeader").process(cloudClient);
+
+    final String collection = "balancedProperties";
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    CollectionAdminResponse response = CollectionAdminRequest.balanceReplicaProperty(collection, "preferredLeader")
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    verifyUniqueAcrossCollection(cloudClient, DEFAULT_COLLECTION, "property.preferredleader");    
+    waitForState("Expecting 'preferredleader' property to be balanced across all shards", collection, (n, c) -> {
+      for (Slice slice : c) {
+        int count = 0;
+        for (Replica replica : slice) {
+          if ("true".equals(replica.getStr("property.preferredleader")))
+            count += 1;
+        }
+        if (count != 1)
+          return false;
+      }
+      return true;
+    });
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
index dcc99a4..c46362e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
@@ -16,106 +16,40 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.Replica;
-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.SolrParams;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.util.TimeOut;
+import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
+public class DeleteLastCustomShardedReplicaTest extends SolrCloudTestCase {
 
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARDS_PROP;
-import static org.apache.solr.common.util.Utils.makeMap;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICA;
-
-public class DeleteLastCustomShardedReplicaTest extends AbstractFullDistribZkTestBase {
-
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  protected String getSolrXml() {
-    return "solr.xml";
-  }
-
-  public DeleteLastCustomShardedReplicaTest() {
-    sliceCount = 2;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
   @Test
-  @ShardsFixed(num = 2)
   public void test() throws Exception {
-    try (CloudSolrClient client = createCloudClient(null))  {
-      int replicationFactor = 1;
-      int maxShardsPerNode = 5;
-
-      Map<String, Object> props = Utils.makeMap(
-          "router.name", ImplicitDocRouter.NAME,
-          ZkStateReader.REPLICATION_FACTOR, replicationFactor,
-          ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode,
-          NUM_SLICES, 1,
-          SHARDS_PROP, "a,b");
 
-      Map<String,List<Integer>> collectionInfos = new HashMap<>();
+    final String collectionName = "customcollreplicadeletion";
 
-      String collectionName = "customcollreplicadeletion";
+    CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "a,b", 1)
+        .setMaxShardsPerNode(5)
+        .process(cluster.getSolrClient());
 
-      createCollection(collectionInfos, collectionName, props, client);
+    DocCollection collectionState = getCollectionState(collectionName);
+    Replica replica = getRandomReplica(collectionState.getSlice("a"));
 
-      waitForRecoveriesToFinish(collectionName, false);
+    CollectionAdminRequest.deleteReplica(collectionName, "a", replica.getName())
+        .process(cluster.getSolrClient());
 
-      DocCollection testcoll = getCommonCloudSolrClient().getZkStateReader()
-              .getClusterState().getCollection(collectionName);
-      Replica replica = testcoll.getSlice("a").getReplicas().iterator().next();
-
-      removeAndWaitForReplicaGone(client, collectionName, replica, "a", replicationFactor-1);
-    }
-  }
+    waitForState("Expected shard 'a' to have no replicas", collectionName, (n, c) -> {
+      return c.getSlice("a") == null || c.getSlice("a").getReplicas().size() == 0;
+    });
 
-  protected void removeAndWaitForReplicaGone(CloudSolrClient client, String COLL_NAME, Replica replica, String shard,
-      final int expectedNumReplicasRemaining)
-      throws SolrServerException, IOException, InterruptedException {
-    Map m = makeMap("collection", COLL_NAME, "action", DELETEREPLICA.toLower(), "shard",
-        shard, "replica", replica.getName());
-    SolrParams params = new MapSolrParams(m);
-    SolrRequest request = new QueryRequest(params);
-    request.setPath("/admin/collections");
-    client.request(request);
-    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
-    boolean success = false;
-    DocCollection testcoll = null;
-    while (! timeout.hasTimedOut()) {
-      testcoll = getCommonCloudSolrClient().getZkStateReader()
-          .getClusterState().getCollection(COLL_NAME);
-      // As of SOLR-5209 the last replica deletion no longer leads to
-      // the deletion of the slice.
-      final Slice slice = testcoll.getSlice(shard);
-      final int actualNumReplicasRemaining = (slice == null ? 0 : slice.getReplicas().size());
-      success = (actualNumReplicasRemaining == expectedNumReplicasRemaining);
-      if (success) {
-        log.info("replica cleaned up {}/{} core {}",
-            shard + "/" + replica.getName(), replica.getStr("core"));
-        log.info("current state {}", testcoll);
-        break;
-      }
-      Thread.sleep(100);
-    }
-    assertTrue("Replica not cleaned up", success);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
index 9097363..ed3d03b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
@@ -20,202 +20,123 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.CoreAdminRequest;
-import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.client.solrj.request.CoreStatus;
 import org.apache.solr.cloud.overseer.OverseerAction;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.Slice.State;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.FileUtils;
 import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class DeleteShardTest extends AbstractFullDistribZkTestBase {
-
-  public DeleteShardTest() {
-    super();
-    sliceCount = 2;
-  }
+public class DeleteShardTest extends SolrCloudTestCase {
 
   // TODO: Custom hash slice deletion test
 
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
   @Test
-  @ShardsFixed(num = 2)
   public void test() throws Exception {
-    ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
 
-    Slice slice1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
-    Slice slice2 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD2);
+    final String collection = "deleteShard";
 
-    assertNotNull("Shard1 not found", slice1);
-    assertNotNull("Shard2 not found", slice2);
-    assertSame("Shard1 is not active", Slice.State.ACTIVE, slice1.getState());
-    assertSame("Shard2 is not active", Slice.State.ACTIVE, slice2.getState());
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 1)
+        .process(cluster.getSolrClient());
 
-    try {
-      deleteShard(SHARD1);
-      fail("Deleting an active shard should not have succeeded");
-    } catch (HttpSolrClient.RemoteSolrException e) {
-      // expected
-    }
+    DocCollection state = getCollectionState(collection);
+    assertEquals(State.ACTIVE, state.getSlice("shard1").getState());
+    assertEquals(State.ACTIVE, state.getSlice("shard2").getState());
 
-    setSliceState(SHARD1, Slice.State.INACTIVE);
+    // Can't delete an ACTIVE shard
+    expectThrows(Exception.class, () -> {
+      CollectionAdminRequest.deleteShard(collection, "shard1").process(cluster.getSolrClient());
+    });
 
-    clusterState = cloudClient.getZkStateReader().getClusterState();
+    setSliceState(collection, "shard1", Slice.State.INACTIVE);
 
-    slice1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
+    // Can delete an INATIVE shard
+    CollectionAdminRequest.deleteShard(collection, "shard1").process(cluster.getSolrClient());
+    waitForState("Expected 'shard1' to be removed", collection, (n, c) -> {
+      return c.getSlice("shard1") == null;
+    });
 
-    assertSame("Shard1 is not inactive yet.", Slice.State.INACTIVE, slice1.getState());
-
-    deleteShard(SHARD1);
-
-    confirmShardDeletion(SHARD1);
-
-    setSliceState(SHARD2, Slice.State.CONSTRUCTION);
-    deleteShard(SHARD2);
-    confirmShardDeletion(SHARD2);
-  }
+    // Can delete a shard under construction
+    setSliceState(collection, "shard2", Slice.State.CONSTRUCTION);
+    CollectionAdminRequest.deleteShard(collection, "shard2").process(cluster.getSolrClient());
+    waitForState("Expected 'shard2' to be removed", collection, (n, c) -> {
+      return c.getSlice("shard2") == null;
+    });
 
-  protected void confirmShardDeletion(String shard) throws SolrServerException, KeeperException,
-      InterruptedException {
-    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
-    ClusterState clusterState = zkStateReader.getClusterState();
-    int counter = 10;
-    while (counter-- > 0) {
-      clusterState = zkStateReader.getClusterState();
-      if (clusterState.getSlice("collection1", shard) == null) {
-        break;
-      }
-      Thread.sleep(1000);
-    }
-
-    assertNull("Cluster still contains shard1 even after waiting for it to be deleted.",
-        clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1));
   }
 
-  protected void deleteShard(String shard) throws SolrServerException, IOException,
+  protected void setSliceState(String collection, String slice, State state) throws SolrServerException, IOException,
       KeeperException, InterruptedException {
 
-    ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set("action", CollectionParams.CollectionAction.DELETESHARD.toString());
-    params.set("collection", AbstractFullDistribZkTestBase.DEFAULT_COLLECTION);
-    params.set("shard", shard);
-    SolrRequest request = new QueryRequest(params);
-    request.setPath("/admin/collections");
-
-    String baseUrl = ((HttpSolrClient) shardToJetty.get(SHARD1).get(0).client.solrClient)
-        .getBaseURL();
-    baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
-
-    try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl)) {
-      baseServer.setConnectionTimeout(15000);
-      baseServer.setSoTimeout(60000);
-      baseServer.request(request);
-    }
-  }
+    CloudSolrClient client = cluster.getSolrClient();
 
-  protected void setSliceState(String slice, State state) throws SolrServerException, IOException,
-      KeeperException, InterruptedException {
-    DistributedQueue inQueue = Overseer.getStateUpdateQueue(cloudClient.getZkStateReader().getZkClient());
+    // TODO can this be encapsulated better somewhere?
+    DistributedQueue inQueue = Overseer.getStateUpdateQueue(client.getZkStateReader().getZkClient());
     Map<String, Object> propMap = new HashMap<>();
     propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
     propMap.put(slice, state.toString());
-    propMap.put(ZkStateReader.COLLECTION_PROP, "collection1");
+    propMap.put(ZkStateReader.COLLECTION_PROP, collection);
     ZkNodeProps m = new ZkNodeProps(propMap);
-    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     inQueue.offer(Utils.toJSON(m));
-    boolean transition = false;
-
-    for (int counter = 10; counter > 0; counter--) {
-      ClusterState clusterState = zkStateReader.getClusterState();
-      State sliceState = clusterState.getSlice("collection1", slice).getState();
-      if (sliceState == state) {
-        transition = true;
-        break;
-      }
-      Thread.sleep(1000);
-    }
-
-    if (!transition) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not set shard [" + slice + "] as " + state);
-    }
+
+    waitForState("Expected shard " + slice + " to be in state " + state.toString(), collection, (n, c) -> {
+      return c.getSlice(slice).getState() == state;
+    });
+
   }
 
   @Test
   public void testDirectoryCleanupAfterDeleteShard() throws InterruptedException, IOException, SolrServerException {
-    CollectionAdminResponse rsp = new CollectionAdminRequest.Create()
-        .setCollectionName("deleteshard_test")
-        .setRouterName("implicit")
-        .setShards("a,b,c")
-        .setReplicationFactor(1)
-        .setConfigName("conf1")
-        .process(cloudClient);
+
+    final String collection = "deleteshard_test";
+    CollectionAdminRequest.createCollectionWithImplicitRouter(collection, "conf", "a,b,c", 1)
+        .setMaxShardsPerNode(2)
+        .process(cluster.getSolrClient());
 
     // Get replica details
-    Replica leader = cloudClient.getZkStateReader().getLeaderRetry("deleteshard_test", "a");
-    String baseUrl = (String) leader.get("base_url");
-    String core = (String) leader.get("core");
+    Replica leader = getCollectionState(collection).getLeader("a");
 
-    String instanceDir;
-    String dataDir;
+    CoreStatus coreStatus = getCoreStatus(leader);
+    assertTrue("Instance directory doesn't exist", FileUtils.fileExists(coreStatus.getInstanceDirectory()));
+    assertTrue("Data directory doesn't exist", FileUtils.fileExists(coreStatus.getDataDirectory()));
 
-    try (HttpSolrClient client = getHttpSolrClient(baseUrl)) {
-      CoreAdminResponse statusResp = CoreAdminRequest.getStatus(core, client);
-      NamedList r = statusResp.getCoreStatus().get(core);
-      instanceDir = (String) r.findRecursive("instanceDir");
-      dataDir = (String) r.get("dataDir");
-    }
+    assertEquals(3, getCollectionState(collection).getActiveSlices().size());
 
-    assertTrue("Instance directory doesn't exist", FileUtils.fileExists(instanceDir));
-    assertTrue("Data directory doesn't exist", FileUtils.fileExists(dataDir));
+    // Delete shard 'a'
+    CollectionAdminRequest.deleteShard(collection, "a").process(cluster.getSolrClient());
 
-    assertEquals(3, cloudClient.getZkStateReader().getClusterState().getActiveSlices("deleteshard_test").size());
+    assertEquals(2, getCollectionState(collection).getActiveSlices().size());
+    assertFalse("Instance directory still exists", FileUtils.fileExists(coreStatus.getInstanceDirectory()));
+    assertFalse("Data directory still exists", FileUtils.fileExists(coreStatus.getDataDirectory()));
 
-    // Delete shard 'a'
-    new CollectionAdminRequest.DeleteShard()
-        .setCollectionName("deleteshard_test")
-        .setShardName("a")
-        .process(cloudClient);
-
-    assertEquals(2, cloudClient.getZkStateReader().getClusterState().getActiveSlices("deleteshard_test").size());
-    assertFalse("Instance directory still exists", FileUtils.fileExists(instanceDir));
-    assertFalse("Data directory still exists", FileUtils.fileExists(dataDir));
-
-    leader = cloudClient.getZkStateReader().getLeaderRetry("deleteshard_test", "b");
-    baseUrl = (String) leader.get("base_url");
-    core = (String) leader.get("core");
-
-    try (HttpSolrClient client = getHttpSolrClient(baseUrl)) {
-      CoreAdminResponse statusResp = CoreAdminRequest.getStatus(core, client);
-      NamedList r = statusResp.getCoreStatus().get(core);
-      instanceDir = (String) r.findRecursive("instanceDir");
-      dataDir = (String) r.get("dataDir");
-    }
+    leader = getCollectionState(collection).getLeader("b");
+    coreStatus = getCoreStatus(leader);
 
     // Delete shard 'b'
-    new CollectionAdminRequest.DeleteShard()
-        .setCollectionName("deleteshard_test")
-        .setShardName("b")
+    CollectionAdminRequest.deleteShard(collection, "b")
         .setDeleteDataDir(false)
         .setDeleteInstanceDir(false)
-        .process(cloudClient);
+        .process(cluster.getSolrClient());
 
-    assertEquals(1, cloudClient.getZkStateReader().getClusterState().getActiveSlices("deleteshard_test").size());
-    assertTrue("Instance directory still exists", FileUtils.fileExists(instanceDir));
-    assertTrue("Data directory still exists", FileUtils.fileExists(dataDir));
+    assertEquals(1, getCollectionState(collection).getActiveSlices().size());
+    assertTrue("Instance directory still exists", FileUtils.fileExists(coreStatus.getInstanceDirectory()));
+    assertTrue("Data directory still exists", FileUtils.fileExists(coreStatus.getDataDirectory()));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
index e902ab4..c9a90a5 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
@@ -17,78 +17,62 @@
 
 package org.apache.solr.cloud;
 
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
-
-import java.lang.invoke.MethodHandles;
 import java.util.Map;
 
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class OverseerModifyCollectionTest extends AbstractFullDistribZkTestBase {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
+
+public class OverseerModifyCollectionTest extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf1", configset("cloud-minimal"))
+        .addConfig("conf2", configset("cloud-minimal"))
+        .configure();
+  }
+
   @Test
   public void testModifyColl() throws Exception {
-    String collName = "modifyColl";
-    String newConfName = "conf" + random().nextInt();
-    String oldConfName = "conf1";
-    try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
-      CollectionAdminResponse rsp;
-      CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collName, oldConfName, 1, 2);
-      rsp = create.process(client);
-      assertEquals(0, rsp.getStatus());
-      assertTrue(rsp.isSuccess());
-      
-      ConfigSetAdminRequest.Create createConfig = new ConfigSetAdminRequest.Create()
-        .setBaseConfigSetName(oldConfName)
-        .setConfigSetName(newConfName);
-      
-      ConfigSetAdminResponse configRsp = createConfig.process(client);
-      
-      assertEquals(0, configRsp.getStatus());
-      
-      ModifiableSolrParams p = new ModifiableSolrParams();
-      p.add("collection", collName);
-      p.add("action", "MODIFYCOLLECTION");
-      p.add("collection.configName", newConfName);
-      client.request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p));
-    }
-    
-    assertEquals(newConfName, getConfigNameFromZk(collName));    
+
+    final String collName = "modifyColl";
+
+    CollectionAdminRequest.createCollection(collName, "conf1", 1, 2)
+        .process(cluster.getSolrClient());
+
+    // TODO create a modifyCollection() method on CollectionAdminRequest
+    ModifiableSolrParams p1 = new ModifiableSolrParams();
+    p1.add("collection", collName);
+    p1.add("action", "MODIFYCOLLECTION");
+    p1.add("collection.configName", "conf2");
+    cluster.getSolrClient().request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p1));
+
+    assertEquals("conf2", getConfigNameFromZk(collName));
     
     //Try an invalid config name
-    try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
-      ModifiableSolrParams p = new ModifiableSolrParams();
-      p.add("collection", collName);
-      p.add("action", "MODIFYCOLLECTION");
-      p.add("collection.configName", "notARealConfigName");
-      try{
-        client.request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p));
-        fail("Exception should be thrown");
-      } catch(RemoteSolrException e) {
-        assertTrue(e.getMessage(), e.getMessage().contains("Can not find the specified config set"));
-      }
-    }
+    ModifiableSolrParams p2 = new ModifiableSolrParams();
+    p2.add("collection", collName);
+    p2.add("action", "MODIFYCOLLECTION");
+    p2.add("collection.configName", "notARealConfigName");
+    Exception e = expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p2));
+    });
+
+    assertTrue(e.getMessage(), e.getMessage().contains("Can not find the specified config set"));
 
   }
   
   private String getConfigNameFromZk(String collName) throws KeeperException, InterruptedException {
-    byte[] b = cloudClient.getZkStateReader().getZkClient().getData(ZkStateReader.getCollectionPathRoot(collName), null, null, false);
+    byte[] b = zkClient().getData(ZkStateReader.getCollectionPathRoot(collName), null, null, false);
     Map confData = (Map) Utils.fromJSON(b);
     return (String) confData.get(ZkController.CONFIGNAME_PROP); 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
index dec54d9..762bbeb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
@@ -16,118 +16,95 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
-import org.apache.solr.client.solrj.SolrRequest;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.params.CollectionParams.CollectionAction;
-import org.apache.solr.common.params.MapSolrParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.data.Stat;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 import static org.apache.solr.cloud.OverseerCollectionConfigSetProcessor.getLeaderNode;
 import static org.apache.solr.cloud.OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames;
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
-import static org.apache.solr.common.util.Utils.makeMap;
-import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+import static org.hamcrest.CoreMatchers.not;
 
-@LuceneTestCase.Slow
-@SuppressSSL(bugUrl = "SOLR-5776")
-public class OverseerRolesTest  extends AbstractFullDistribZkTestBase{
+public class OverseerRolesTest extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  protected String getSolrXml() {
-    return "solr.xml";
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(4)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
-  public OverseerRolesTest() {
-    sliceCount = 2;
-    fixShardCount(TEST_NIGHTLY ? 6 : 2);
-  }
-
-  @Test
-  public void test() throws Exception {
-    try (CloudSolrClient client = createCloudClient(null))  {
-      testQuitCommand(client);
-      testOverseerRole(client);
+  @Before
+  public void clearAllOverseerRoles() throws Exception {
+    for (String node : OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zkClient())) {
+      CollectionAdminRequest.removeRole(node, "overseer").process(cluster.getSolrClient());
     }
   }
 
-  private void testQuitCommand(CloudSolrClient client) throws Exception{
-    String collectionName = "testOverseerQuit";
-
-    createCollection(collectionName, client);
-
-    waitForRecoveriesToFinish(collectionName, false);
+  @Test
+  public void testQuitCommand() throws Exception {
 
-    SolrZkClient zk = client.getZkStateReader().getZkClient();
-    byte[] data = new byte[0];
-    data = zk.getData("/overseer_elect/leader", null, new Stat(), true);
+    SolrZkClient zk = zkClient();
+    byte[] data = zk.getData("/overseer_elect/leader", null, new Stat(), true);
     Map m = (Map) Utils.fromJSON(data);
     String s = (String) m.get("id");
     String leader = LeaderElector.getNodeName(s);
-    Overseer.getStateUpdateQueue(zk).offer(Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower())));
+    log.info("Current overseer: {}", leader);
+    Overseer.getStateUpdateQueue(zk)
+        .offer(Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower(),
+                                            "id", s)));
     final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS);
-    String newLeader=null;
+    String newLeader = null;
     for(;! timeout.hasTimedOut();){
       newLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(zk);
-      if(newLeader!=null && !newLeader.equals(leader)) break;
+      if (newLeader != null && !newLeader.equals(leader))
+        break;
       Thread.sleep(100);
     }
-    assertNotSame( "Leader not changed yet",newLeader,leader);
-
+    assertThat("Leader not changed yet", newLeader, not(leader));
 
-
-    assertTrue("The old leader should have rejoined election ", OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zk).contains(leader));
+    assertTrue("The old leader should have rejoined election",
+        OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zk).contains(leader));
   }
 
+  @Test
+  public void testOverseerRole() throws Exception {
 
-
-
-  private void testOverseerRole(CloudSolrClient client) throws Exception {
-    String collectionName = "testOverseerCol";
-
-    createCollection(collectionName, client);
-
-    waitForRecoveriesToFinish(collectionName, false);
-    List<String> l = OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(client.getZkStateReader().getZkClient()) ;
+    List<String> l = OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zkClient()) ;
 
     log.info("All nodes {}", l);
-    String currentLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(client.getZkStateReader().getZkClient());
+    String currentLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());
     log.info("Current leader {} ", currentLeader);
     l.remove(currentLeader);
 
     Collections.shuffle(l, random());
     String overseerDesignate = l.get(0);
-    log.info("overseerDesignate {}",overseerDesignate);
-    setOverseerRole(client, CollectionAction.ADDROLE,overseerDesignate);
+    log.info("overseerDesignate {}", overseerDesignate);
+
+    CollectionAdminRequest.addRole(overseerDesignate, "overseer").process(cluster.getSolrClient());
 
     TimeOut timeout = new TimeOut(15, TimeUnit.SECONDS);
 
     boolean leaderchanged = false;
-    for(;!timeout.hasTimedOut();){
-      if(overseerDesignate.equals(OverseerCollectionConfigSetProcessor.getLeaderNode(client.getZkStateReader().getZkClient()))){
+    for (;!timeout.hasTimedOut();) {
+      if (overseerDesignate.equals(OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient()))) {
         log.info("overseer designate is the new overseer");
         leaderchanged =true;
         break;
@@ -136,36 +113,29 @@ public class OverseerRolesTest  extends AbstractFullDistribZkTestBase{
     }
     assertTrue("could not set the new overseer . expected "+
         overseerDesignate + " current order : " +
-        getSortedOverseerNodeNames(client.getZkStateReader().getZkClient()) +
-        " ldr :"+ OverseerCollectionConfigSetProcessor.getLeaderNode(client.getZkStateReader().getZkClient()) ,leaderchanged);
-
-
+        getSortedOverseerNodeNames(zkClient()) +
+        " ldr :"+ OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient()) ,leaderchanged);
 
     //add another node as overseer
-
-
     l.remove(overseerDesignate);
-
     Collections.shuffle(l, random());
 
     String anotherOverseer = l.get(0);
     log.info("Adding another overseer designate {}", anotherOverseer);
-    setOverseerRole(client, CollectionAction.ADDROLE, anotherOverseer);
+    CollectionAdminRequest.addRole(anotherOverseer, "overseer").process(cluster.getSolrClient());
 
-    String currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
+    String currentOverseer = getLeaderNode(zkClient());
 
     log.info("Current Overseer {}", currentOverseer);
 
-    String hostPort = currentOverseer.substring(0,currentOverseer.indexOf('_'));
+    String hostPort = currentOverseer.substring(0, currentOverseer.indexOf('_'));
 
     StringBuilder sb = new StringBuilder();
-//
-//
     log.info("hostPort : {}", hostPort);
 
     JettySolrRunner leaderJetty = null;
 
-    for (JettySolrRunner jetty : jettys) {
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       String s = jetty.getBaseUrl().toString();
       log.info("jetTy {}",s);
       sb.append(s).append(" , ");
@@ -178,49 +148,20 @@ public class OverseerRolesTest  extends AbstractFullDistribZkTestBase{
     assertNotNull("Could not find a jetty2 kill",  leaderJetty);
 
     log.info("leader node {}", leaderJetty.getBaseUrl());
-    log.info ("current election Queue",
-        OverseerCollectionConfigSetProcessor.getSortedElectionNodes(client.getZkStateReader().getZkClient(),
-            "/overseer_elect/election"));
+    log.info("current election Queue",
+        OverseerCollectionConfigSetProcessor.getSortedElectionNodes(zkClient(), "/overseer_elect/election"));
     ChaosMonkey.stop(leaderJetty);
     timeout = new TimeOut(10, TimeUnit.SECONDS);
     leaderchanged = false;
     for (; !timeout.hasTimedOut(); ) {
-      currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
+      currentOverseer = getLeaderNode(zkClient());
       if (anotherOverseer.equals(currentOverseer)) {
         leaderchanged = true;
         break;
       }
       Thread.sleep(100);
     }
-    assertTrue("New overseer designate has not become the overseer, expected : " + anotherOverseer + "actual : " + getLeaderNode(client.getZkStateReader().getZkClient()), leaderchanged);
+    assertTrue("New overseer designate has not become the overseer, expected : " + anotherOverseer + "actual : " + getLeaderNode(zkClient()), leaderchanged);
   }
 
-  private void setOverseerRole(CloudSolrClient client, CollectionAction action, String overseerDesignate) throws Exception, IOException {
-    log.info("Adding overseer designate {} ", overseerDesignate);
-    Map m = makeMap(
-        "action", action.toString().toLowerCase(Locale.ROOT),
-        "role", "overseer",
-        "node", overseerDesignate);
-    SolrParams params = new MapSolrParams(m);
-    SolrRequest request = new QueryRequest(params);
-    request.setPath("/admin/collections");
-    client.request(request);
-  }
-
-
-  protected void createCollection(String COLL_NAME, CloudSolrClient client) throws Exception {
-    int replicationFactor = 2;
-    int numShards = 4;
-    int maxShardsPerNode = ((((numShards+1) * replicationFactor) / getCommonCloudSolrClient()
-        .getZkStateReader().getClusterState().getLiveNodes().size())) + 1;
-
-    Map<String, Object> props = makeMap(
-        REPLICATION_FACTOR, replicationFactor,
-        MAX_SHARDS_PER_NODE, maxShardsPerNode,
-        NUM_SLICES, numShards);
-    Map<String,List<Integer>> collectionInfos = new HashMap<>();
-    createCollection(collectionInfos, COLL_NAME, props, client);
-  }
-
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
index b1899da..80fd38e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
@@ -17,74 +17,56 @@
 package org.apache.solr.cloud;
 
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class OverseerStatusTest extends BasicDistributedZkTest {
+public class OverseerStatusTest extends SolrCloudTestCase {
 
-  public OverseerStatusTest() {
-    schemaString = "schema15.xml";      // we need a string id
-    sliceCount = 1;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();;
   }
 
   @Test
-  @ShardsFixed(num = 1)
   public void test() throws Exception {
 
-    waitForThingsToLevelOut(15);
-
     // find existing command counts because collection may be created by base test class too
     int numCollectionCreates = 0, numOverseerCreates = 0;
-    NamedList<Object> resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
-    if (resp != null) {
-      NamedList<Object> collection_operations = (NamedList<Object>) resp.get("collection_operations");
-      if (collection_operations != null)  {
-        SimpleOrderedMap<Object> createcollection = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
-        if (createcollection != null && createcollection.get("requests") != null) {
-          numCollectionCreates = (Integer) createcollection.get("requests");
-        }
-        NamedList<Object> overseer_operations = (NamedList<Object>) resp.get("overseer_operations");
-        if (overseer_operations != null)  {
-          createcollection = (SimpleOrderedMap<Object>) overseer_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
-          if (createcollection != null && createcollection.get("requests") != null) {
-            numOverseerCreates = (Integer) createcollection.get("requests");
-          }
-        }
-      }
-    }
 
     String collectionName = "overseer_status_test";
-    CollectionAdminResponse response = createCollection(collectionName, 1, 1, 1);
-    resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1).process(cluster.getSolrClient());
+
+    NamedList<Object> resp = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient()).getResponse();
     NamedList<Object> collection_operations = (NamedList<Object>) resp.get("collection_operations");
     NamedList<Object> overseer_operations = (NamedList<Object>) resp.get("overseer_operations");
-    SimpleOrderedMap<Object> createcollection = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
+    SimpleOrderedMap<Object> createcollection
+        = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
     assertEquals("No stats for create in OverseerCollectionProcessor", numCollectionCreates + 1, createcollection.get("requests"));
     createcollection = (SimpleOrderedMap<Object>) overseer_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
     assertEquals("No stats for create in Overseer", numOverseerCreates + 1, createcollection.get("requests"));
 
     // Reload the collection
-    new CollectionAdminRequest.Reload().setCollectionName(collectionName).process(cloudClient);
-
+    CollectionAdminRequest.reloadCollection(collectionName).process(cluster.getSolrClient());
 
-    resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
+    resp = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient()).getResponse();
     collection_operations = (NamedList<Object>) resp.get("collection_operations");
     SimpleOrderedMap<Object> reload = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.RELOAD.toLower());
     assertEquals("No stats for reload in OverseerCollectionProcessor", 1, reload.get("requests"));
 
     try {
-      new CollectionAdminRequest.SplitShard()
-              .setCollectionName("non_existent_collection")
-              .setShardName("non_existent_shard")
-              .process(cloudClient);
+      CollectionAdminRequest.splitShard("non_existent_collection")
+          .setShardName("non_existent_shard")
+          .process(cluster.getSolrClient());
       fail("Split shard for non existent collection should have failed");
     } catch (Exception e) {
       // expected because we did not correctly specify required params for split
     }
-    resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
+    resp = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient()).getResponse();
     collection_operations = (NamedList<Object>) resp.get("collection_operations");
     SimpleOrderedMap<Object> split = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.SPLITSHARD.toLower());
     assertEquals("No stats for split in OverseerCollectionProcessor", 1, split.get("errors"));
@@ -111,6 +93,5 @@ public class OverseerStatusTest extends BasicDistributedZkTest {
     assertNotNull(updateState.get("errors"));
     assertNotNull(updateState.get("avgTimePerRequest"));
 
-    waitForThingsToLevelOut(15);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java b/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
index 24f9696..54503bf 100644
--- a/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
@@ -16,58 +16,43 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import static org.junit.internal.matchers.StringContains.containsString;
 
 /**
  * Verify that remote (proxied) queries return proper error messages
  */
-@Slow
-public class RemoteQueryErrorTest extends AbstractFullDistribZkTestBase {
+public class RemoteQueryErrorTest extends SolrCloudTestCase {
 
-  public RemoteQueryErrorTest() {
-    super();
-    sliceCount = 1;
-    fixShardCount(random().nextBoolean() ? 3 : 4);
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(3)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
+  // TODO add test for CloudSolrClient as well
+
   @Test
   public void test() throws Exception {
-    handle.clear();
-    handle.put("timestamp", SKIPVAL);
-    
-    waitForThingsToLevelOut(15);
 
-    del("*:*");
-    
-    createCollection("collection2", 2, 1, 10);
-    
-    List<Integer> numShardsNumReplicaList = new ArrayList<>(2);
-    numShardsNumReplicaList.add(2);
-    numShardsNumReplicaList.add(1);
-    checkForCollection("collection2", numShardsNumReplicaList, null);
-    waitForRecoveriesToFinish("collection2", true);
+    CollectionAdminRequest.createCollection("collection", "conf", 2, 1).process(cluster.getSolrClient());
 
-    for (SolrClient solrClient : clients) {
-      try {
-        SolrInputDocument emptyDoc = new SolrInputDocument();
-        solrClient.add(emptyDoc);
-        fail("Expected unique key exception");
-      } catch (SolrException ex) {
-        assertThat(ex.getMessage(), containsString("Document is missing mandatory uniqueKey field: id"));
-      } catch(Exception ex) {
-        fail("Expected a SolrException to occur, instead received: " + ex.getClass());
-      } finally {
-        solrClient.close();
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      try (SolrClient client = jetty.newClient()) {
+        SolrException e = expectThrows(SolrException.class, () -> {
+          client.add("collection", new SolrInputDocument());
+        });
+        assertThat(e.getMessage(), containsString("Document is missing mandatory uniqueKey field: id"));
       }
     }
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java b/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
index 01c4440..415d4e4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
@@ -16,13 +16,15 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.params.ShardParams;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,41 +36,47 @@ import static org.hamcrest.CoreMatchers.is;
  * and also asserts that a meaningful exception is thrown when shards.tolerant=false
  * See SOLR-7566
  */
-public class TestDownShardTolerantSearch extends AbstractFullDistribZkTestBase {
+public class TestDownShardTolerantSearch extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public TestDownShardTolerantSearch() {
-    sliceCount = 2;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
   @Test
-  @ShardsFixed(num = 2)
   public void searchingShouldFailWithoutTolerantSearchSetToTrue() throws Exception {
-    waitForRecoveriesToFinish(true);
 
-    indexAbunchOfDocs();
-    commit();
-    QueryResponse response = cloudClient.query(new SolrQuery("*:*").setRows(1));
+    CollectionAdminRequest.createCollection("tolerant", "conf", 2, 1)
+        .process(cluster.getSolrClient());
+
+    UpdateRequest update = new UpdateRequest();
+    for (int i = 0; i < 100; i++) {
+      update.add("id", Integer.toString(i));
+    }
+    update.commit(cluster.getSolrClient(), "tolerant");
+
+    QueryResponse response = cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1));
     assertThat(response.getStatus(), is(0));
-    assertThat(response.getResults().getNumFound(), is(66L));
+    assertThat(response.getResults().getNumFound(), is(100L));
 
-    ChaosMonkey.kill(shardToJetty.get(SHARD1).get(0));
+    cluster.stopJettySolrRunner(0);
 
-    response = cloudClient.query(new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, true));
+    response = cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, true));
     assertThat(response.getStatus(), is(0));
     assertTrue(response.getResults().getNumFound() > 0);
 
     try {
-      cloudClient.query(new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, false));
+      cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, false));
       fail("Request should have failed because we killed shard1 jetty");
     } catch (SolrServerException e) {
       log.info("error from server", e);
       assertNotNull(e.getCause());
       assertTrue("Error message from server should have the name of the down shard",
-          e.getCause().getMessage().contains(SHARD1));
-    } catch (IOException e) {
-      e.printStackTrace();
+          e.getCause().getMessage().contains("shard"));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java b/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
index 9ef2dcd..5bf77c1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
@@ -16,34 +16,32 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-@LuceneTestCase.Slow
-public class TestExclusionRuleCollectionAccess extends AbstractFullDistribZkTestBase {
+public class TestExclusionRuleCollectionAccess extends SolrCloudTestCase {
 
-  public TestExclusionRuleCollectionAccess() {
-    schemaString = "schema15.xml";      // we need a string id
-    sliceCount = 1;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
   @Test
   public void doTest() throws Exception {
-    CollectionAdminRequest.Create req = new CollectionAdminRequest.Create();
-    req.setCollectionName("css33");
-    req.setNumShards(1);
-    req.process(cloudClient);
-    
-    waitForRecoveriesToFinish("css33", false);
-    
-    try (SolrClient c = createCloudClient("css33")) {
-      c.add(getDoc("id", "1"));
-      c.commit();
-
-      assertEquals("Should have returned 1 result", 1, c.query(params("q", "*:*", "collection", "css33")).getResults().getNumFound());
-    }
+
+    CollectionAdminRequest.createCollection("css33", "conf", 1, 1).process(cluster.getSolrClient());
+
+    new UpdateRequest()
+        .add("id", "1")
+        .commit(cluster.getSolrClient(), "css33");
+
+    assertEquals("Should have returned 1 result", 1,
+        cluster.getSolrClient().query("css33", params("q", "*:*", "collection", "css33")).getResults().getNumFound());
+
   }
   
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
index 30fe933..bc4f4e5 100644
--- a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
@@ -18,19 +18,21 @@ package org.apache.solr.security;
 
 import javax.servlet.ServletRequest;
 import javax.servlet.http.HttpServletRequest;
-
 import java.lang.invoke.MethodHandles;
 import java.security.Principal;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Predicate;
 
-import org.apache.solr.SolrTestCaseJ4;
+import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.Utils;
+import org.junit.After;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,27 +41,32 @@ import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.util.Utils.makeMap;
 import static org.apache.solr.security.TestAuthorizationFramework.verifySecurityStatus;
 
-@SolrTestCaseJ4.SuppressSSL
-public class PKIAuthenticationIntegrationTest extends AbstractFullDistribZkTestBase {
+public class PKIAuthenticationIntegrationTest extends SolrCloudTestCase {
+
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  static final int TIMEOUT = 10000;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
 
   @Test
   public void testPkiAuth() throws Exception {
-    waitForThingsToLevelOut(10);
 
+    CollectionAdminRequest.createCollection("collection", "conf", 2, 1).process(cluster.getSolrClient());
+
+    // TODO make a SolrJ helper class for this
     byte[] bytes = Utils.toJSON(makeMap("authorization", singletonMap("class", MockAuthorizationPlugin.class.getName()),
         "authentication", singletonMap("class", MockAuthenticationPlugin.class.getName())));
+    zkClient().setData(ZkStateReader.SOLR_SECURITY_CONF_PATH, bytes, true);
 
-    try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
-        TIMEOUT, TIMEOUT)) {
-      zkStateReader.getZkClient().setData(ZkStateReader.SOLR_SECURITY_CONF_PATH, bytes, true);
-    }
-    for (JettySolrRunner jetty : jettys) {
+    HttpClient httpClient = cluster.getSolrClient().getHttpClient();
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       String baseUrl = jetty.getBaseUrl().toString();
-      verifySecurityStatus(cloudClient.getLbClient().getHttpClient(), baseUrl + "/admin/authorization", "authorization/class", MockAuthorizationPlugin.class.getName(), 20);
-      verifySecurityStatus(cloudClient.getLbClient().getHttpClient(), baseUrl + "/admin/authentication", "authentication.enabled", "true", 20);
+      verifySecurityStatus(httpClient, baseUrl + "/admin/authorization", "authorization/class", MockAuthorizationPlugin.class.getName(), 20);
+      verifySecurityStatus(httpClient, baseUrl + "/admin/authentication", "authentication.enabled", "true", 20);
     }
     log.info("Starting test");
     ModifiableSolrParams params = new ModifiableSolrParams();
@@ -95,13 +102,12 @@ public class PKIAuthenticationIntegrationTest extends AbstractFullDistribZkTestB
       }
     };
     QueryRequest query = new QueryRequest(params);
-    query.process(cloudClient);
+    query.process(cluster.getSolrClient(), "collection");
     assertTrue("all nodes must get the user solr , no:of nodes got solr : " + count.get(),count.get() > 2);
   }
 
-  @Override
+  @After
   public void distribTearDown() throws Exception {
-    super.distribTearDown();
     MockAuthenticationPlugin.predicate = null;
     MockAuthorizationPlugin.predicate = null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12aff1cf/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index 0beaa55..94750c0a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -280,6 +280,8 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
 
     public CollectionAdminRoleRequest(CollectionAction action, String node, String role) {
       super(action);
+      this.node = node;
+      this.role = role;
     }
 
     @Override


[3/6] lucene-solr:branch_6x: LUCENE-5325: Add LongValuesSource and DoubleValuesSource in core

Posted by ro...@apache.org.
LUCENE-5325: Add LongValuesSource and DoubleValuesSource in core


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

Branch: refs/heads/branch_6x
Commit: e2aa2b638468f8ed7555533be25f4dc86780b840
Parents: 16b1db4
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Dec 28 20:10:47 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 12:05:32 2016 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../org/apache/lucene/search/DoubleValues.java  |  38 +++
 .../lucene/search/DoubleValuesSource.java       | 301 +++++++++++++++++++
 .../org/apache/lucene/search/LongValues.java    |  38 +++
 .../apache/lucene/search/LongValuesSource.java  | 204 +++++++++++++
 .../lucene/search/TestDoubleValuesSource.java   | 158 ++++++++++
 .../lucene/search/TestLongValuesSource.java     | 140 +++++++++
 7 files changed, 883 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2aa2b63/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 5ab649e..7a566ba 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -34,6 +34,10 @@ New features
   http://blog.mikemccandless.com/2012/04/lucenes-tokenstreams-are-actually.html
   for details. (Mike McCandless)
 
+* LUCENE-5325: Added LongValuesSource and DoubleValuesSource, intended as
+  type-safe replacements for ValueSource in the queries module.  These
+  expose per-segment LongValues or DoubleValues iterators. (Alan Woodward, Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2aa2b63/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
new file mode 100644
index 0000000..4f12390
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
@@ -0,0 +1,38 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+
+/**
+ * Per-segment, per-document double values, which can be calculated at search-time
+ */
+public abstract class DoubleValues {
+
+  /**
+   * Get the double value for the current document
+   */
+  public abstract double doubleValue() throws IOException;
+
+  /**
+   * Advance this instance to the given document id
+   * @return true if there is a value for this document
+   */
+  public abstract boolean advanceExact(int doc) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2aa2b63/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
new file mode 100644
index 0000000..4361243
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
@@ -0,0 +1,301 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.function.DoubleToLongFunction;
+import java.util.function.LongToDoubleFunction;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/**
+ * Base class for producing {@link DoubleValues}
+ *
+ * To obtain a {@link DoubleValues} object for a leaf reader, clients should
+ * call {@link #getValues(LeafReaderContext, DoubleValues)}.
+ *
+ * DoubleValuesSource objects for NumericDocValues fields can be obtained by calling
+ * {@link #fromDoubleField(String)}, {@link #fromFloatField(String)}, {@link #fromIntField(String)}
+ * or {@link #fromLongField(String)}, or from {@link #fromField(String, LongToDoubleFunction)} if
+ * special long-to-double encoding is required.
+ *
+ * Scores may be used as a source for value calculations by wrapping a {@link Scorer} using
+ * {@link #fromScorer(Scorer)} and passing the resulting DoubleValues to {@link #getValues(LeafReaderContext, DoubleValues)}.
+ * The scores can then be accessed using the {@link #SCORES} DoubleValuesSource.
+ */
+public abstract class DoubleValuesSource {
+
+  /**
+   * Returns a {@link DoubleValues} instance for the passed-in LeafReaderContext and scores
+   *
+   * If scores are not needed to calculate the values (ie {@link #needsScores() returns false}, callers
+   * may safely pass {@code null} for the {@code scores} parameter.
+   */
+  public abstract DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException;
+
+  /**
+   * Return true if document scores are needed to calculate values
+   */
+  public abstract boolean needsScores();
+
+  /**
+   * Create a sort field based on the value of this producer
+   * @param reverse true if the sort should be decreasing
+   */
+  public SortField getSortField(boolean reverse) {
+    return new DoubleValuesSortField(this, reverse);
+  }
+
+  /**
+   * Convert to a LongValuesSource by casting the double values to longs
+   */
+  public final LongValuesSource toLongValuesSource() {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        DoubleValues in = DoubleValuesSource.this.getValues(ctx, scores);
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            return (long) in.doubleValue();
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return in.advanceExact(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return DoubleValuesSource.this.needsScores();
+      }
+    };
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a generic NumericDocValues field
+   *
+   * @param field the field to wrap, must have NumericDocValues
+   * @param decoder a function to convert the long-valued doc values to doubles
+   */
+  public static DoubleValuesSource fromField(String field, LongToDoubleFunction decoder) {
+    return new FieldValuesSource(field, decoder);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a double-valued field
+   */
+  public static DoubleValuesSource fromDoubleField(String field) {
+    return fromField(field, Double::longBitsToDouble);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a float-valued field
+   */
+  public static DoubleValuesSource fromFloatField(String field) {
+    return fromField(field, (v) -> (double)Float.intBitsToFloat((int)v));
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a long-valued field
+   */
+  public static DoubleValuesSource fromLongField(String field) {
+    return fromField(field, (v) -> (double) v);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps an int-valued field
+   */
+  public static DoubleValuesSource fromIntField(String field) {
+    return fromLongField(field);
+  }
+
+  /**
+   * A DoubleValuesSource that exposes a document's score
+   *
+   * If this source is used as part of a values calculation, then callers must not
+   * pass {@code null} as the {@link DoubleValues} parameter on {@link #getValues(LeafReaderContext, DoubleValues)}
+   */
+  public static final DoubleValuesSource SCORES = new DoubleValuesSource() {
+    @Override
+    public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      assert scores != null;
+      return scores;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
+  };
+
+  /**
+   * Returns a DoubleValues instance that wraps scores returned by a Scorer
+   */
+  public static DoubleValues fromScorer(Scorer scorer) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return scorer.score();
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        assert scorer.docID() == doc;
+        return true;
+      }
+    };
+  }
+
+  private static class FieldValuesSource extends DoubleValuesSource {
+
+    final String field;
+    final LongToDoubleFunction decoder;
+
+    private FieldValuesSource(String field, LongToDoubleFunction decoder) {
+      this.field = field;
+      this.decoder = decoder;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      FieldValuesSource that = (FieldValuesSource) o;
+      return Objects.equals(field, that.field) &&
+          Objects.equals(decoder, that.decoder);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(field, decoder);
+    }
+
+    @Override
+    public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      final NumericDocValues values = DocValues.getNumeric(ctx.reader(), field);
+      return toDoubleValues(values, decoder::applyAsDouble);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+  }
+
+  private static class DoubleValuesSortField extends SortField {
+
+    final DoubleValuesSource producer;
+
+    public DoubleValuesSortField(DoubleValuesSource producer, boolean reverse) {
+      super(producer.toString(), new DoubleValuesComparatorSource(producer), reverse);
+      this.producer = producer;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return producer.needsScores();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder("<");
+      buffer.append(getField()).append(">");
+      if (reverse)
+        buffer.append("!");
+      return buffer.toString();
+    }
+
+  }
+
+  private static class DoubleValuesHolder {
+    DoubleValues values;
+  }
+
+  private static class DoubleValuesComparatorSource extends FieldComparatorSource {
+    private final DoubleValuesSource producer;
+
+    public DoubleValuesComparatorSource(DoubleValuesSource producer) {
+      this.producer = producer;
+    }
+
+    @Override
+    public FieldComparator<Double> newComparator(String fieldname, int numHits,
+                                               int sortPos, boolean reversed) throws IOException {
+      return new FieldComparator.DoubleComparator(numHits, fieldname, 0.0){
+
+        LeafReaderContext ctx;
+        DoubleValuesHolder holder = new DoubleValuesHolder();
+
+        @Override
+        protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+          ctx = context;
+          return asNumericDocValues(holder, missingValue, Double::doubleToLongBits);
+        }
+
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          holder.values = producer.getValues(ctx, fromScorer(scorer));
+        }
+      };
+    }
+  }
+
+  private static DoubleValues toDoubleValues(NumericDocValues in, LongToDoubleFunction map) {
+    return new DoubleValues() {
+
+      int current = -1;
+
+      @Override
+      public double doubleValue() throws IOException {
+        return map.applyAsDouble(in.get(current));
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        current = target;
+        return true;
+      }
+
+    };
+  }
+
+  private static NumericDocValues asNumericDocValues(DoubleValuesHolder in, Double missingValue, DoubleToLongFunction converter) {
+    long missing = converter.applyAsLong(missingValue);
+    return new NumericDocValues() {
+      @Override
+      public long get(int docID) {
+        try {
+          if (in.values.advanceExact(docID))
+            return converter.applyAsLong(in.values.doubleValue());
+        }
+        catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        return missing;
+      }
+
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2aa2b63/lucene/core/src/java/org/apache/lucene/search/LongValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValues.java b/lucene/core/src/java/org/apache/lucene/search/LongValues.java
new file mode 100644
index 0000000..54d3189
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValues.java
@@ -0,0 +1,38 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+
+/**
+ * Per-segment, per-document long values, which can be calculated at search-time
+ */
+public abstract class LongValues {
+
+  /**
+   * Get the long value for the current document
+   */
+  public abstract long longValue() throws IOException;
+
+  /**
+   * Advance this instance to the given document id
+   * @return true if there is a value for this document
+   */
+  public abstract boolean advanceExact(int doc) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2aa2b63/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
new file mode 100644
index 0000000..4165e90
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
@@ -0,0 +1,204 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/**
+ * Base class for producing {@link LongValues}
+ *
+ * To obtain a {@link LongValues} object for a leaf reader, clients should
+ * call {@link #getValues(LeafReaderContext, DoubleValues)}.
+ *
+ * LongValuesSource objects for long and int-valued NumericDocValues fields can
+ * be obtained by calling {@link #fromLongField(String)} and {@link #fromIntField(String)}.
+ *
+ * To obtain a LongValuesSource from a float or double-valued NumericDocValues field,
+ * use {@link DoubleValuesSource#fromFloatField(String)} or {@link DoubleValuesSource#fromDoubleField(String)}
+ * and then call {@link DoubleValuesSource#toLongValuesSource()}.
+ */
+public abstract class LongValuesSource {
+
+  /**
+   * Returns a {@link LongValues} instance for the passed-in LeafReaderContext and scores
+   *
+   * If scores are not needed to calculate the values (ie {@link #needsScores() returns false}, callers
+   * may safely pass {@code null} for the {@code scores} parameter.
+   */
+  public abstract LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException;
+
+  /**
+   * Return true if document scores are needed to calculate values
+   */
+  public abstract boolean needsScores();
+
+  /**
+   * Create a sort field based on the value of this producer
+   * @param reverse true if the sort should be decreasing
+   */
+  public SortField getSortField(boolean reverse) {
+    return new LongValuesSortField(this, reverse);
+  }
+
+  /**
+   * Creates a LongValuesSource that wraps a long-valued field
+   */
+  public static LongValuesSource fromLongField(String field) {
+    return new FieldValuesSource(field);
+  }
+
+  /**
+   * Creates a LongValuesSource that wraps an int-valued field
+   */
+  public static LongValuesSource fromIntField(String field) {
+    return fromLongField(field);
+  }
+
+  private static class FieldValuesSource extends LongValuesSource {
+
+    final String field;
+
+    private FieldValuesSource(String field) {
+      this.field = field;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      FieldValuesSource that = (FieldValuesSource) o;
+      return Objects.equals(field, that.field);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(field);
+    }
+
+    @Override
+    public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      final NumericDocValues values = DocValues.getNumeric(ctx.reader(), field);
+      return toLongValues(values);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+  }
+
+  private static class LongValuesSortField extends SortField {
+
+    final LongValuesSource producer;
+
+    public LongValuesSortField(LongValuesSource producer, boolean reverse) {
+      super(producer.toString(), new LongValuesComparatorSource(producer), reverse);
+      this.producer = producer;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return producer.needsScores();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder("<");
+      buffer.append(getField()).append(">");
+      if (reverse)
+        buffer.append("!");
+      return buffer.toString();
+    }
+
+  }
+
+  private static class LongValuesHolder {
+    LongValues values;
+  }
+
+  private static class LongValuesComparatorSource extends FieldComparatorSource {
+    private final LongValuesSource producer;
+
+    public LongValuesComparatorSource(LongValuesSource producer) {
+      this.producer = producer;
+    }
+
+    @Override
+    public FieldComparator<Long> newComparator(String fieldname, int numHits,
+                                                 int sortPos, boolean reversed) throws IOException {
+      return new FieldComparator.LongComparator(numHits, fieldname, 0L){
+
+        LeafReaderContext ctx;
+        LongValuesHolder holder = new LongValuesHolder();
+
+        @Override
+        protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+          ctx = context;
+          return asNumericDocValues(holder, missingValue);
+        }
+
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          holder.values = producer.getValues(ctx, DoubleValuesSource.fromScorer(scorer));
+        }
+      };
+    }
+  }
+
+  private static LongValues toLongValues(NumericDocValues in) {
+    return new LongValues() {
+
+      int current = -1;
+
+      @Override
+      public long longValue() throws IOException {
+        return in.get(current);
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        current = target;
+        return true;
+      }
+
+    };
+  }
+
+  private static NumericDocValues asNumericDocValues(LongValuesHolder in, Long missingValue) {
+    return new NumericDocValues() {
+      @Override
+      public long get(int docID) {
+        try {
+          if (in.values.advanceExact(docID))
+            return in.values.longValue();
+        }
+        catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        return missingValue;
+      }
+
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2aa2b63/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
new file mode 100644
index 0000000..3860963
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
@@ -0,0 +1,158 @@
+/*
+ * 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.lucene.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestDoubleValuesSource extends LuceneTestCase {
+
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    int numDocs = TestUtil.nextInt(random(), 2049, 4000);
+    for (int i = 0; i < numDocs; i++) {
+      Document document = new Document();
+      document.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
+      document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
+      document.add(new NumericDocValuesField("int", random().nextInt()));
+      document.add(new NumericDocValuesField("long", random().nextLong()));
+      document.add(new FloatDocValuesField("float", random().nextFloat()));
+      document.add(new DoubleDocValuesField("double", random().nextDouble()));
+      iw.addDocument(document);
+    }
+    reader = iw.getReader();
+    iw.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+
+  public void testSimpleFieldEquivalences() throws Exception {
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("float", SortField.Type.FLOAT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("double", SortField.Type.DOUBLE, random().nextBoolean())));
+  }
+
+  public void testHashCodeAndEquals() {
+    DoubleValuesSource vs1 = DoubleValuesSource.fromDoubleField("double");
+    DoubleValuesSource vs2 = DoubleValuesSource.fromDoubleField("double");
+    assertEquals(vs1, vs2);
+    assertEquals(vs1.hashCode(), vs2.hashCode());
+    DoubleValuesSource v3 = DoubleValuesSource.fromLongField("long");
+    assertFalse(vs1.equals(v3));
+  }
+
+  public void testSimpleFieldSortables() throws Exception {
+    int n = atLeast(4);
+    for (int i = 0; i < n; i++) {
+      Sort sort = randomSort();
+      checkSorts(new MatchAllDocsQuery(), sort);
+      checkSorts(new TermQuery(new Term("english", "one")), sort);
+    }
+  }
+
+  Sort randomSort() throws Exception {
+    boolean reversed = random().nextBoolean();
+    SortField fields[] = new SortField[] {
+        new SortField("int", SortField.Type.INT, reversed),
+        new SortField("long", SortField.Type.LONG, reversed),
+        new SortField("float", SortField.Type.FLOAT, reversed),
+        new SortField("double", SortField.Type.DOUBLE, reversed),
+        new SortField("score", SortField.Type.SCORE)
+    };
+    Collections.shuffle(Arrays.asList(fields), random());
+    int numSorts = TestUtil.nextInt(random(), 1, fields.length);
+    return new Sort(Arrays.copyOfRange(fields, 0, numSorts));
+  }
+
+  // Take a Sort, and replace any field sorts with Sortables
+  Sort convertSortToSortable(Sort sort) {
+    SortField original[] = sort.getSort();
+    SortField mutated[] = new SortField[original.length];
+    for (int i = 0; i < mutated.length; i++) {
+      if (random().nextInt(3) > 0) {
+        SortField s = original[i];
+        boolean reverse = s.getType() == SortField.Type.SCORE || s.getReverse();
+        switch (s.getType()) {
+          case INT:
+            mutated[i] = DoubleValuesSource.fromIntField(s.getField()).getSortField(reverse);
+            break;
+          case LONG:
+            mutated[i] = DoubleValuesSource.fromLongField(s.getField()).getSortField(reverse);
+            break;
+          case FLOAT:
+            mutated[i] = DoubleValuesSource.fromFloatField(s.getField()).getSortField(reverse);
+            break;
+          case DOUBLE:
+            mutated[i] = DoubleValuesSource.fromDoubleField(s.getField()).getSortField(reverse);
+            break;
+          case SCORE:
+            mutated[i] = DoubleValuesSource.SCORES.getSortField(reverse);
+            break;
+          default:
+            mutated[i] = original[i];
+        }
+      } else {
+        mutated[i] = original[i];
+      }
+    }
+
+    return new Sort(mutated);
+  }
+
+  void checkSorts(Query query, Sort sort) throws Exception {
+    int size = TestUtil.nextInt(random(), 1, searcher.getIndexReader().maxDoc() / 5);
+    TopDocs expected = searcher.search(query, size, sort, random().nextBoolean(), random().nextBoolean());
+    Sort mutatedSort = convertSortToSortable(sort);
+    TopDocs actual = searcher.search(query, size, mutatedSort, random().nextBoolean(), random().nextBoolean());
+
+    CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+
+    if (size < actual.totalHits) {
+      expected = searcher.searchAfter(expected.scoreDocs[size-1], query, size, sort);
+      actual = searcher.searchAfter(actual.scoreDocs[size-1], query, size, mutatedSort);
+      CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e2aa2b63/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
new file mode 100644
index 0000000..9148ad5
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
@@ -0,0 +1,140 @@
+/*
+ * 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.lucene.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestLongValuesSource extends LuceneTestCase {
+
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    int numDocs = TestUtil.nextInt(random(), 2049, 4000);
+    for (int i = 0; i < numDocs; i++) {
+      Document document = new Document();
+      document.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
+      document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
+      document.add(new NumericDocValuesField("int", random().nextInt()));
+      document.add(new NumericDocValuesField("long", random().nextLong()));
+      iw.addDocument(document);
+    }
+    reader = iw.getReader();
+    iw.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+
+  public void testSimpleFieldEquivalences() throws Exception {
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));
+  }
+
+  public void testHashCodeAndEquals() {
+    LongValuesSource vs1 = LongValuesSource.fromLongField("long");
+    LongValuesSource vs2 = LongValuesSource.fromLongField("long");
+    assertEquals(vs1, vs2);
+    assertEquals(vs1.hashCode(), vs2.hashCode());
+    LongValuesSource v3 = LongValuesSource.fromLongField("int");
+    assertFalse(vs1.equals(v3));
+  }
+
+  public void testSimpleFieldSortables() throws Exception {
+    int n = atLeast(4);
+    for (int i = 0; i < n; i++) {
+      Sort sort = randomSort();
+      checkSorts(new MatchAllDocsQuery(), sort);
+      checkSorts(new TermQuery(new Term("english", "one")), sort);
+    }
+  }
+
+  Sort randomSort() throws Exception {
+    boolean reversed = random().nextBoolean();
+    SortField fields[] = new SortField[] {
+        new SortField("int", SortField.Type.INT, reversed),
+        new SortField("long", SortField.Type.LONG, reversed)
+    };
+    Collections.shuffle(Arrays.asList(fields), random());
+    int numSorts = TestUtil.nextInt(random(), 1, fields.length);
+    return new Sort(Arrays.copyOfRange(fields, 0, numSorts));
+  }
+
+  // Take a Sort, and replace any field sorts with Sortables
+  Sort convertSortToSortable(Sort sort) {
+    SortField original[] = sort.getSort();
+    SortField mutated[] = new SortField[original.length];
+    for (int i = 0; i < mutated.length; i++) {
+      if (random().nextInt(3) > 0) {
+        SortField s = original[i];
+        boolean reverse = s.getType() == SortField.Type.SCORE || s.getReverse();
+        switch (s.getType()) {
+          case INT:
+            mutated[i] = LongValuesSource.fromIntField(s.getField()).getSortField(reverse);
+            break;
+          case LONG:
+            mutated[i] = LongValuesSource.fromLongField(s.getField()).getSortField(reverse);
+            break;
+          default:
+            mutated[i] = original[i];
+        }
+      } else {
+        mutated[i] = original[i];
+      }
+    }
+
+    return new Sort(mutated);
+  }
+
+  void checkSorts(Query query, Sort sort) throws Exception {
+    int size = TestUtil.nextInt(random(), 1, searcher.getIndexReader().maxDoc() / 5);
+    Sort mutatedSort = convertSortToSortable(sort);
+    TopDocs actual = searcher.search(query, size, mutatedSort, random().nextBoolean(), random().nextBoolean());
+    TopDocs expected = searcher.search(query, size, sort, random().nextBoolean(), random().nextBoolean());
+
+    CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+
+    if (size < actual.totalHits) {
+      expected = searcher.searchAfter(expected.scoreDocs[size-1], query, size, sort);
+      actual = searcher.searchAfter(actual.scoreDocs[size-1], query, size, mutatedSort);
+      CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+    }
+  }
+}


[6/6] lucene-solr:master: LUCENE-7607: FieldLeafComparator.setScorer() should throw IOException

Posted by ro...@apache.org.
LUCENE-7607: FieldLeafComparator.setScorer() should throw IOException


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

Branch: refs/heads/master
Commit: 87b6c2c8fcdc3a5f4adc3516f249af89b479d77a
Parents: 12aff1c
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Dec 28 19:48:16 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 12:22:40 2016 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                                    |  3 +++
 .../org/apache/lucene/search/LeafFieldComparator.java |  2 +-
 .../apache/lucene/search/SimpleFieldComparator.java   |  2 +-
 .../apache/solr/search/CollapsingQParserPlugin.java   | 14 +++++++-------
 4 files changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/87b6c2c8/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7a118f1..35314db 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -65,6 +65,9 @@ API Changes
 * LUCENE-7533: Classic query parser no longer allows autoGeneratePhraseQueries
   to be set to true when splitOnWhitespace is false (and vice-versa).
 
+* LUCENE-7607: LeafFieldComparator.setScorer and SimpleFieldComparator.setScorer
+  are declared as throwing IOException (Alan Woodward)
+
 New features
 
 * LUCENE-5867: Added BooleanSimilarity. (Robert Muir, Adrien Grand)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/87b6c2c8/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java b/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
index 6019376..677639c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LeafFieldComparator.java
@@ -114,6 +114,6 @@ public interface LeafFieldComparator {
    * 
    * @param scorer Scorer instance that you should use to
    * obtain the current hit's score, if necessary. */
-  void setScorer(Scorer scorer);
+  void setScorer(Scorer scorer) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/87b6c2c8/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java b/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
index 3f1e6c2..a258e24 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SimpleFieldComparator.java
@@ -38,5 +38,5 @@ public abstract class SimpleFieldComparator<T> extends FieldComparator<T> implem
   }
 
   @Override
-  public void setScorer(Scorer scorer) {}
+  public void setScorer(Scorer scorer) throws IOException {}
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/87b6c2c8/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
index f6bc18e..44aade5 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -965,7 +965,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     @Override public boolean needsScores() { return needsScores || super.needsScores(); }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.collapseStrategy.setScorer(scorer);
     }
 
@@ -1147,7 +1147,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
 
     @Override public boolean needsScores() { return needsScores || super.needsScores(); }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.collapseStrategy.setScorer(scorer);
     }
 
@@ -1523,7 +1523,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       return collapsedSet;
     }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.scorer = scorer;
     }
 
@@ -1952,7 +1952,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
 
     @Override
-    public void setScorer(Scorer s) {
+    public void setScorer(Scorer s) throws IOException {
       super.setScorer(s);
       this.compareState.setScorer(s);
     }
@@ -2100,7 +2100,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
       return collapsedSet;
     }
 
-    public void setScorer(Scorer scorer) {
+    public void setScorer(Scorer scorer) throws IOException {
       this.scorer = scorer;
     }
 
@@ -2522,7 +2522,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
 
     @Override
-    public void setScorer(Scorer s) {
+    public void setScorer(Scorer s) throws IOException {
       super.setScorer(s);
       this.compareState.setScorer(s);
     }
@@ -2673,7 +2673,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
         leafFieldComparators[clause] = fieldComparators[clause].getLeafComparator(context);
       }
     }
-    public void setScorer(Scorer s) {
+    public void setScorer(Scorer s) throws IOException {
       for (int clause = 0; clause < numClauses; clause++) {
         leafFieldComparators[clause].setScorer(s);
       }


[5/6] lucene-solr:master: LUCENE-5325: Add LongValuesSource and DoubleValuesSource in core

Posted by ro...@apache.org.
LUCENE-5325: Add LongValuesSource and DoubleValuesSource in core


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

Branch: refs/heads/master
Commit: 3f24fd81c836982be96b9b60082b53177fffe504
Parents: 87b6c2c
Author: Alan Woodward <ro...@apache.org>
Authored: Wed Dec 28 20:10:47 2016 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 12:22:40 2016 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../org/apache/lucene/search/DoubleValues.java  |  38 +++
 .../lucene/search/DoubleValuesSource.java       | 313 +++++++++++++++++++
 .../org/apache/lucene/search/LongValues.java    |  38 +++
 .../apache/lucene/search/LongValuesSource.java  | 217 +++++++++++++
 .../lucene/search/TestDoubleValuesSource.java   | 158 ++++++++++
 .../lucene/search/TestLongValuesSource.java     | 140 +++++++++
 7 files changed, 909 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 35314db..12b615d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -89,6 +89,11 @@ New features
   http://blog.mikemccandless.com/2012/04/lucenes-tokenstreams-are-actually.html
   for details. (Mike McCandless)
 
+* LUCENE-5325: Added LongValuesSource and DoubleValuesSource, intended as
+  type-safe replacements for ValueSource in the queries module.  These
+  expose per-segment LongValues or DoubleValues iterators, similar to the
+  existing DocValues iterator API. (Alan Woodward, Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
new file mode 100644
index 0000000..4f12390
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValues.java
@@ -0,0 +1,38 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+
+/**
+ * Per-segment, per-document double values, which can be calculated at search-time
+ */
+public abstract class DoubleValues {
+
+  /**
+   * Get the double value for the current document
+   */
+  public abstract double doubleValue() throws IOException;
+
+  /**
+   * Advance this instance to the given document id
+   * @return true if there is a value for this document
+   */
+  public abstract boolean advanceExact(int doc) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
new file mode 100644
index 0000000..4ac8fc1
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
@@ -0,0 +1,313 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.function.DoubleToLongFunction;
+import java.util.function.LongToDoubleFunction;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/**
+ * Base class for producing {@link DoubleValues}
+ *
+ * To obtain a {@link DoubleValues} object for a leaf reader, clients should
+ * call {@link #getValues(LeafReaderContext, DoubleValues)}.
+ *
+ * DoubleValuesSource objects for NumericDocValues fields can be obtained by calling
+ * {@link #fromDoubleField(String)}, {@link #fromFloatField(String)}, {@link #fromIntField(String)}
+ * or {@link #fromLongField(String)}, or from {@link #fromField(String, LongToDoubleFunction)} if
+ * special long-to-double encoding is required.
+ *
+ * Scores may be used as a source for value calculations by wrapping a {@link Scorer} using
+ * {@link #fromScorer(Scorer)} and passing the resulting DoubleValues to {@link #getValues(LeafReaderContext, DoubleValues)}.
+ * The scores can then be accessed using the {@link #SCORES} DoubleValuesSource.
+ */
+public abstract class DoubleValuesSource {
+
+  /**
+   * Returns a {@link DoubleValues} instance for the passed-in LeafReaderContext and scores
+   *
+   * If scores are not needed to calculate the values (ie {@link #needsScores() returns false}, callers
+   * may safely pass {@code null} for the {@code scores} parameter.
+   */
+  public abstract DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException;
+
+  /**
+   * Return true if document scores are needed to calculate values
+   */
+  public abstract boolean needsScores();
+
+  /**
+   * Create a sort field based on the value of this producer
+   * @param reverse true if the sort should be decreasing
+   */
+  public SortField getSortField(boolean reverse) {
+    return new DoubleValuesSortField(this, reverse);
+  }
+
+  /**
+   * Convert to a LongValuesSource by casting the double values to longs
+   */
+  public final LongValuesSource toLongValuesSource() {
+    return new LongValuesSource() {
+      @Override
+      public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+        DoubleValues in = DoubleValuesSource.this.getValues(ctx, scores);
+        return new LongValues() {
+          @Override
+          public long longValue() throws IOException {
+            return (long) in.doubleValue();
+          }
+
+          @Override
+          public boolean advanceExact(int doc) throws IOException {
+            return in.advanceExact(doc);
+          }
+        };
+      }
+
+      @Override
+      public boolean needsScores() {
+        return DoubleValuesSource.this.needsScores();
+      }
+    };
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a generic NumericDocValues field
+   *
+   * @param field the field to wrap, must have NumericDocValues
+   * @param decoder a function to convert the long-valued doc values to doubles
+   */
+  public static DoubleValuesSource fromField(String field, LongToDoubleFunction decoder) {
+    return new FieldValuesSource(field, decoder);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a double-valued field
+   */
+  public static DoubleValuesSource fromDoubleField(String field) {
+    return fromField(field, Double::longBitsToDouble);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a float-valued field
+   */
+  public static DoubleValuesSource fromFloatField(String field) {
+    return fromField(field, (v) -> (double)Float.intBitsToFloat((int)v));
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps a long-valued field
+   */
+  public static DoubleValuesSource fromLongField(String field) {
+    return fromField(field, (v) -> (double) v);
+  }
+
+  /**
+   * Creates a DoubleValuesSource that wraps an int-valued field
+   */
+  public static DoubleValuesSource fromIntField(String field) {
+    return fromLongField(field);
+  }
+
+  /**
+   * A DoubleValuesSource that exposes a document's score
+   *
+   * If this source is used as part of a values calculation, then callers must not
+   * pass {@code null} as the {@link DoubleValues} parameter on {@link #getValues(LeafReaderContext, DoubleValues)}
+   */
+  public static final DoubleValuesSource SCORES = new DoubleValuesSource() {
+    @Override
+    public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      assert scores != null;
+      return scores;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
+  };
+
+  /**
+   * Returns a DoubleValues instance that wraps scores returned by a Scorer
+   */
+  public static DoubleValues fromScorer(Scorer scorer) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return scorer.score();
+      }
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        assert scorer.docID() == doc;
+        return true;
+      }
+    };
+  }
+
+  private static class FieldValuesSource extends DoubleValuesSource {
+
+    final String field;
+    final LongToDoubleFunction decoder;
+
+    private FieldValuesSource(String field, LongToDoubleFunction decoder) {
+      this.field = field;
+      this.decoder = decoder;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      FieldValuesSource that = (FieldValuesSource) o;
+      return Objects.equals(field, that.field) &&
+          Objects.equals(decoder, that.decoder);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(field, decoder);
+    }
+
+    @Override
+    public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      final NumericDocValues values = DocValues.getNumeric(ctx.reader(), field);
+      return toDoubleValues(values, decoder::applyAsDouble);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+  }
+
+  private static class DoubleValuesSortField extends SortField {
+
+    final DoubleValuesSource producer;
+
+    public DoubleValuesSortField(DoubleValuesSource producer, boolean reverse) {
+      super(producer.toString(), new DoubleValuesComparatorSource(producer), reverse);
+      this.producer = producer;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return producer.needsScores();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder("<");
+      buffer.append(getField()).append(">");
+      if (reverse)
+        buffer.append("!");
+      return buffer.toString();
+    }
+
+  }
+
+  private static class DoubleValuesHolder {
+    DoubleValues values;
+  }
+
+  private static class DoubleValuesComparatorSource extends FieldComparatorSource {
+    private final DoubleValuesSource producer;
+
+    public DoubleValuesComparatorSource(DoubleValuesSource producer) {
+      this.producer = producer;
+    }
+
+    @Override
+    public FieldComparator<Double> newComparator(String fieldname, int numHits,
+                                               int sortPos, boolean reversed) throws IOException {
+      return new FieldComparator.DoubleComparator(numHits, fieldname, 0.0){
+
+        LeafReaderContext ctx;
+        DoubleValuesHolder holder = new DoubleValuesHolder();
+
+        @Override
+        protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+          ctx = context;
+          return asNumericDocValues(holder, Double::doubleToLongBits);
+        }
+
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          holder.values = producer.getValues(ctx, fromScorer(scorer));
+        }
+      };
+    }
+  }
+
+  private static DoubleValues toDoubleValues(NumericDocValues in, LongToDoubleFunction map) {
+    return new DoubleValues() {
+      @Override
+      public double doubleValue() throws IOException {
+        return map.applyAsDouble(in.longValue());
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.advanceExact(target);
+      }
+
+    };
+  }
+
+  private static NumericDocValues asNumericDocValues(DoubleValuesHolder in, DoubleToLongFunction converter) {
+    return new NumericDocValues() {
+      @Override
+      public long longValue() throws IOException {
+        return converter.applyAsLong(in.values.doubleValue());
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.values.advanceExact(target);
+      }
+
+      @Override
+      public int docID() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long cost() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/LongValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValues.java b/lucene/core/src/java/org/apache/lucene/search/LongValues.java
new file mode 100644
index 0000000..54d3189
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValues.java
@@ -0,0 +1,38 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+
+/**
+ * Per-segment, per-document long values, which can be calculated at search-time
+ */
+public abstract class LongValues {
+
+  /**
+   * Get the long value for the current document
+   */
+  public abstract long longValue() throws IOException;
+
+  /**
+   * Advance this instance to the given document id
+   * @return true if there is a value for this document
+   */
+  public abstract boolean advanceExact(int doc) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
new file mode 100644
index 0000000..9d00355
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/LongValuesSource.java
@@ -0,0 +1,217 @@
+/*
+ * 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.lucene.search;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/**
+ * Base class for producing {@link LongValues}
+ *
+ * To obtain a {@link LongValues} object for a leaf reader, clients should
+ * call {@link #getValues(LeafReaderContext, DoubleValues)}.
+ *
+ * LongValuesSource objects for long and int-valued NumericDocValues fields can
+ * be obtained by calling {@link #fromLongField(String)} and {@link #fromIntField(String)}.
+ *
+ * To obtain a LongValuesSource from a float or double-valued NumericDocValues field,
+ * use {@link DoubleValuesSource#fromFloatField(String)} or {@link DoubleValuesSource#fromDoubleField(String)}
+ * and then call {@link DoubleValuesSource#toLongValuesSource()}.
+ */
+public abstract class LongValuesSource {
+
+  /**
+   * Returns a {@link LongValues} instance for the passed-in LeafReaderContext and scores
+   *
+   * If scores are not needed to calculate the values (ie {@link #needsScores() returns false}, callers
+   * may safely pass {@code null} for the {@code scores} parameter.
+   */
+  public abstract LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException;
+
+  /**
+   * Return true if document scores are needed to calculate values
+   */
+  public abstract boolean needsScores();
+
+  /**
+   * Create a sort field based on the value of this producer
+   * @param reverse true if the sort should be decreasing
+   */
+  public SortField getSortField(boolean reverse) {
+    return new LongValuesSortField(this, reverse);
+  }
+
+  /**
+   * Creates a LongValuesSource that wraps a long-valued field
+   */
+  public static LongValuesSource fromLongField(String field) {
+    return new FieldValuesSource(field);
+  }
+
+  /**
+   * Creates a LongValuesSource that wraps an int-valued field
+   */
+  public static LongValuesSource fromIntField(String field) {
+    return fromLongField(field);
+  }
+
+  private static class FieldValuesSource extends LongValuesSource {
+
+    final String field;
+
+    private FieldValuesSource(String field) {
+      this.field = field;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      FieldValuesSource that = (FieldValuesSource) o;
+      return Objects.equals(field, that.field);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(field);
+    }
+
+    @Override
+    public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
+      final NumericDocValues values = DocValues.getNumeric(ctx.reader(), field);
+      return toLongValues(values);
+    }
+
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
+  }
+
+  private static class LongValuesSortField extends SortField {
+
+    final LongValuesSource producer;
+
+    public LongValuesSortField(LongValuesSource producer, boolean reverse) {
+      super(producer.toString(), new LongValuesComparatorSource(producer), reverse);
+      this.producer = producer;
+    }
+
+    @Override
+    public boolean needsScores() {
+      return producer.needsScores();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder("<");
+      buffer.append(getField()).append(">");
+      if (reverse)
+        buffer.append("!");
+      return buffer.toString();
+    }
+
+  }
+
+  private static class LongValuesHolder {
+    LongValues values;
+  }
+
+  private static class LongValuesComparatorSource extends FieldComparatorSource {
+    private final LongValuesSource producer;
+
+    public LongValuesComparatorSource(LongValuesSource producer) {
+      this.producer = producer;
+    }
+
+    @Override
+    public FieldComparator<Long> newComparator(String fieldname, int numHits,
+                                                 int sortPos, boolean reversed) throws IOException {
+      return new FieldComparator.LongComparator(numHits, fieldname, 0L){
+
+        LeafReaderContext ctx;
+        LongValuesHolder holder = new LongValuesHolder();
+
+        @Override
+        protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
+          ctx = context;
+          return asNumericDocValues(holder);
+        }
+
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          holder.values = producer.getValues(ctx, DoubleValuesSource.fromScorer(scorer));
+        }
+      };
+    }
+  }
+
+  private static LongValues toLongValues(NumericDocValues in) {
+    return new LongValues() {
+      @Override
+      public long longValue() throws IOException {
+        return in.longValue();
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.advanceExact(target);
+      }
+
+    };
+  }
+
+  private static NumericDocValues asNumericDocValues(LongValuesHolder in) {
+    return new NumericDocValues() {
+      @Override
+      public long longValue() throws IOException {
+        return in.values.longValue();
+      }
+
+      @Override
+      public boolean advanceExact(int target) throws IOException {
+        return in.values.advanceExact(target);
+      }
+
+      @Override
+      public int docID() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public long cost() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
new file mode 100644
index 0000000..3860963
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDoubleValuesSource.java
@@ -0,0 +1,158 @@
+/*
+ * 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.lucene.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestDoubleValuesSource extends LuceneTestCase {
+
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    int numDocs = TestUtil.nextInt(random(), 2049, 4000);
+    for (int i = 0; i < numDocs; i++) {
+      Document document = new Document();
+      document.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
+      document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
+      document.add(new NumericDocValuesField("int", random().nextInt()));
+      document.add(new NumericDocValuesField("long", random().nextLong()));
+      document.add(new FloatDocValuesField("float", random().nextFloat()));
+      document.add(new DoubleDocValuesField("double", random().nextDouble()));
+      iw.addDocument(document);
+    }
+    reader = iw.getReader();
+    iw.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+
+  public void testSimpleFieldEquivalences() throws Exception {
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("float", SortField.Type.FLOAT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("double", SortField.Type.DOUBLE, random().nextBoolean())));
+  }
+
+  public void testHashCodeAndEquals() {
+    DoubleValuesSource vs1 = DoubleValuesSource.fromDoubleField("double");
+    DoubleValuesSource vs2 = DoubleValuesSource.fromDoubleField("double");
+    assertEquals(vs1, vs2);
+    assertEquals(vs1.hashCode(), vs2.hashCode());
+    DoubleValuesSource v3 = DoubleValuesSource.fromLongField("long");
+    assertFalse(vs1.equals(v3));
+  }
+
+  public void testSimpleFieldSortables() throws Exception {
+    int n = atLeast(4);
+    for (int i = 0; i < n; i++) {
+      Sort sort = randomSort();
+      checkSorts(new MatchAllDocsQuery(), sort);
+      checkSorts(new TermQuery(new Term("english", "one")), sort);
+    }
+  }
+
+  Sort randomSort() throws Exception {
+    boolean reversed = random().nextBoolean();
+    SortField fields[] = new SortField[] {
+        new SortField("int", SortField.Type.INT, reversed),
+        new SortField("long", SortField.Type.LONG, reversed),
+        new SortField("float", SortField.Type.FLOAT, reversed),
+        new SortField("double", SortField.Type.DOUBLE, reversed),
+        new SortField("score", SortField.Type.SCORE)
+    };
+    Collections.shuffle(Arrays.asList(fields), random());
+    int numSorts = TestUtil.nextInt(random(), 1, fields.length);
+    return new Sort(Arrays.copyOfRange(fields, 0, numSorts));
+  }
+
+  // Take a Sort, and replace any field sorts with Sortables
+  Sort convertSortToSortable(Sort sort) {
+    SortField original[] = sort.getSort();
+    SortField mutated[] = new SortField[original.length];
+    for (int i = 0; i < mutated.length; i++) {
+      if (random().nextInt(3) > 0) {
+        SortField s = original[i];
+        boolean reverse = s.getType() == SortField.Type.SCORE || s.getReverse();
+        switch (s.getType()) {
+          case INT:
+            mutated[i] = DoubleValuesSource.fromIntField(s.getField()).getSortField(reverse);
+            break;
+          case LONG:
+            mutated[i] = DoubleValuesSource.fromLongField(s.getField()).getSortField(reverse);
+            break;
+          case FLOAT:
+            mutated[i] = DoubleValuesSource.fromFloatField(s.getField()).getSortField(reverse);
+            break;
+          case DOUBLE:
+            mutated[i] = DoubleValuesSource.fromDoubleField(s.getField()).getSortField(reverse);
+            break;
+          case SCORE:
+            mutated[i] = DoubleValuesSource.SCORES.getSortField(reverse);
+            break;
+          default:
+            mutated[i] = original[i];
+        }
+      } else {
+        mutated[i] = original[i];
+      }
+    }
+
+    return new Sort(mutated);
+  }
+
+  void checkSorts(Query query, Sort sort) throws Exception {
+    int size = TestUtil.nextInt(random(), 1, searcher.getIndexReader().maxDoc() / 5);
+    TopDocs expected = searcher.search(query, size, sort, random().nextBoolean(), random().nextBoolean());
+    Sort mutatedSort = convertSortToSortable(sort);
+    TopDocs actual = searcher.search(query, size, mutatedSort, random().nextBoolean(), random().nextBoolean());
+
+    CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+
+    if (size < actual.totalHits) {
+      expected = searcher.searchAfter(expected.scoreDocs[size-1], query, size, sort);
+      actual = searcher.searchAfter(actual.scoreDocs[size-1], query, size, mutatedSort);
+      CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3f24fd81/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
new file mode 100644
index 0000000..9148ad5
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLongValuesSource.java
@@ -0,0 +1,140 @@
+/*
+ * 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.lucene.search;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestLongValuesSource extends LuceneTestCase {
+
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    int numDocs = TestUtil.nextInt(random(), 2049, 4000);
+    for (int i = 0; i < numDocs; i++) {
+      Document document = new Document();
+      document.add(newTextField("english", English.intToEnglish(i), Field.Store.NO));
+      document.add(newTextField("oddeven", (i % 2 == 0) ? "even" : "odd", Field.Store.NO));
+      document.add(new NumericDocValuesField("int", random().nextInt()));
+      document.add(new NumericDocValuesField("long", random().nextLong()));
+      iw.addDocument(document);
+    }
+    reader = iw.getReader();
+    iw.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+
+  public void testSimpleFieldEquivalences() throws Exception {
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("int", SortField.Type.INT, random().nextBoolean())));
+    checkSorts(new MatchAllDocsQuery(), new Sort(new SortField("long", SortField.Type.LONG, random().nextBoolean())));
+  }
+
+  public void testHashCodeAndEquals() {
+    LongValuesSource vs1 = LongValuesSource.fromLongField("long");
+    LongValuesSource vs2 = LongValuesSource.fromLongField("long");
+    assertEquals(vs1, vs2);
+    assertEquals(vs1.hashCode(), vs2.hashCode());
+    LongValuesSource v3 = LongValuesSource.fromLongField("int");
+    assertFalse(vs1.equals(v3));
+  }
+
+  public void testSimpleFieldSortables() throws Exception {
+    int n = atLeast(4);
+    for (int i = 0; i < n; i++) {
+      Sort sort = randomSort();
+      checkSorts(new MatchAllDocsQuery(), sort);
+      checkSorts(new TermQuery(new Term("english", "one")), sort);
+    }
+  }
+
+  Sort randomSort() throws Exception {
+    boolean reversed = random().nextBoolean();
+    SortField fields[] = new SortField[] {
+        new SortField("int", SortField.Type.INT, reversed),
+        new SortField("long", SortField.Type.LONG, reversed)
+    };
+    Collections.shuffle(Arrays.asList(fields), random());
+    int numSorts = TestUtil.nextInt(random(), 1, fields.length);
+    return new Sort(Arrays.copyOfRange(fields, 0, numSorts));
+  }
+
+  // Take a Sort, and replace any field sorts with Sortables
+  Sort convertSortToSortable(Sort sort) {
+    SortField original[] = sort.getSort();
+    SortField mutated[] = new SortField[original.length];
+    for (int i = 0; i < mutated.length; i++) {
+      if (random().nextInt(3) > 0) {
+        SortField s = original[i];
+        boolean reverse = s.getType() == SortField.Type.SCORE || s.getReverse();
+        switch (s.getType()) {
+          case INT:
+            mutated[i] = LongValuesSource.fromIntField(s.getField()).getSortField(reverse);
+            break;
+          case LONG:
+            mutated[i] = LongValuesSource.fromLongField(s.getField()).getSortField(reverse);
+            break;
+          default:
+            mutated[i] = original[i];
+        }
+      } else {
+        mutated[i] = original[i];
+      }
+    }
+
+    return new Sort(mutated);
+  }
+
+  void checkSorts(Query query, Sort sort) throws Exception {
+    int size = TestUtil.nextInt(random(), 1, searcher.getIndexReader().maxDoc() / 5);
+    Sort mutatedSort = convertSortToSortable(sort);
+    TopDocs actual = searcher.search(query, size, mutatedSort, random().nextBoolean(), random().nextBoolean());
+    TopDocs expected = searcher.search(query, size, sort, random().nextBoolean(), random().nextBoolean());
+
+    CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+
+    if (size < actual.totalHits) {
+      expected = searcher.searchAfter(expected.scoreDocs[size-1], query, size, sort);
+      actual = searcher.searchAfter(actual.scoreDocs[size-1], query, size, mutatedSort);
+      CheckHits.checkEqual(query, expected.scoreDocs, actual.scoreDocs);
+    }
+  }
+}


[2/6] lucene-solr:branch_6x: SOLR-9132: Cut over some more tests

Posted by ro...@apache.org.
SOLR-9132: Cut over some more tests


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

Branch: refs/heads/branch_6x
Commit: 183f998098b0764117d2f858df9909a4ee139cc0
Parents: 529c60d
Author: Alan Woodward <ro...@apache.org>
Authored: Thu Oct 27 16:50:28 2016 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Thu Dec 29 11:57:16 2016 +0000

----------------------------------------------------------------------
 .../java/org/apache/solr/cloud/Overseer.java    |   2 +-
 .../solr/cloud/CollectionsAPISolrJTest.java     | 470 +++++++------------
 .../DeleteLastCustomShardedReplicaTest.java     | 104 +---
 .../org/apache/solr/cloud/DeleteShardTest.java  | 211 +++------
 .../cloud/OverseerModifyCollectionTest.java     |  92 ++--
 .../apache/solr/cloud/OverseerRolesTest.java    | 165 +++----
 .../apache/solr/cloud/OverseerStatusTest.java   |  55 +--
 .../apache/solr/cloud/RemoteQueryErrorTest.java |  53 +--
 .../solr/cloud/TestDownShardTolerantSearch.java |  40 +-
 .../TestExclusionRuleCollectionAccess.java      |  38 +-
 .../PKIAuthenticationIntegrationTest.java       |  40 +-
 .../solrj/request/CollectionAdminRequest.java   |   2 +
 12 files changed, 457 insertions(+), 815 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/java/org/apache/solr/cloud/Overseer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index d7285fa..a618874 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -371,7 +371,7 @@ public class Overseer implements Closeable {
             return Collections.singletonList(new SliceMutator(getZkStateReader()).updateShardState(clusterState, message));
           case QUIT:
             if (myId.equals(message.get("id"))) {
-              log.info("Quit command received {}", LeaderElector.getNodeName(myId));
+              log.info("Quit command received {} {}", message, LeaderElector.getNodeName(myId));
               overseerCollectionConfigSetProcessor.close();
               close();
             } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index b04bfbc..616b657 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -16,70 +16,47 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.codec.binary.StringUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.request.CoreStatus;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.CoreAdminResponse;
-import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.apache.solr.cloud.ReplicaPropertiesBase.verifyUniqueAcrossCollection;
-
 @LuceneTestCase.Slow
-public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
+public class CollectionsAPISolrJTest extends SolrCloudTestCase {
 
-  @Test
-  public void test() throws Exception {
-    testCreateAndDeleteCollection();
-    testCreateAndDeleteShard();
-    testReloadCollection();
-    testCreateAndDeleteAlias();
-    testSplitShard();
-    testCreateCollectionWithPropertyParam();
-    testAddAndDeleteReplica();
-    testClusterProp();
-    testAddAndRemoveRole();
-    testOverseerStatus();
-    testList();
-    testAddAndDeleteReplicaProp();
-    testBalanceShardUnique();
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(4)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
-  protected void testCreateAndDeleteCollection() throws Exception {
+  @Test
+  public void testCreateAndDeleteCollection() throws Exception {
     String collectionName = "solrj_test";
-    CollectionAdminRequest.Create createCollectionRequest = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setNumShards(2)
-            .setReplicationFactor(2)
-            .setConfigName("conf1")
-            .setRouterField("myOwnField")
-            .setStateFormat(1);
-
-    CollectionAdminResponse response = createCollectionRequest.process(cloudClient);
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
+        .setStateFormat(1)
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
@@ -91,57 +68,44 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
       assertTrue(status.get("QTime") > 0);
     }
 
-    cloudClient.setDefaultCollection(collectionName);
-    CollectionAdminRequest.Delete deleteCollectionRequest = new CollectionAdminRequest.Delete()
-            .setCollectionName(collectionName);
-    response = deleteCollectionRequest.process(cloudClient);
+    response = CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String,NamedList<Integer>> nodesStatus = response.getCollectionNodesStatus();
-    assertNull("Deleted collection " + collectionName + "still exists",
-        cloudClient.getZkStateReader().getClusterState().getCollectionOrNull(collectionName));
     assertEquals(4, nodesStatus.size());
-    
+
+    waitForState("Expected " + collectionName + " to disappear from cluster state", collectionName, (n, c) -> c == null);
+
     // Test Creating a collection with new stateformat.
     collectionName = "solrj_newstateformat";
-    createCollectionRequest = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setNumShards(2)
-            .setConfigName("conf1")
-            .setStateFormat(2);
 
-    response = createCollectionRequest.process(cloudClient);
+    response = CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
+        .setStateFormat(2)
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    waitForRecoveriesToFinish(collectionName, false);
-    assertTrue("Collection state does not exist",
-        cloudClient.getZkStateReader().getZkClient()
-            .exists(ZkStateReader.getCollectionPath(collectionName), true));
+    waitForState("Expected " + collectionName + " to appear in cluster state", collectionName, (n, c) -> c != null);
 
   }
-  
-  protected void testCreateAndDeleteShard() throws IOException, SolrServerException {
+
+  @Test
+  public void testCreateAndDeleteShard() throws IOException, SolrServerException {
+
     // Create an implicit collection
     String collectionName = "solrj_implicit";
-    CollectionAdminResponse response = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setShards("shardA,shardB")
-            .setConfigName("conf1")
-            .setRouterName("implicit").process(cloudClient);
+    CollectionAdminResponse response
+        = CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "shardA,shardB", 1)
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
     assertEquals(2, coresStatus.size());
 
-    cloudClient.setDefaultCollection(collectionName);
     // Add a shard to the implicit collection
-    response = new CollectionAdminRequest
-        .CreateShard()
-            .setCollectionName(collectionName)
-            .setShardName("shardC").process(cloudClient);
+    response = CollectionAdminRequest.createShard(collectionName, "shardC").process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
@@ -149,57 +113,38 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
     assertEquals(1, coresStatus.size());
     assertEquals(0, (int) coresStatus.get(collectionName + "_shardC_replica1").get("status"));
 
-    CollectionAdminRequest.DeleteShard deleteShardRequest = new CollectionAdminRequest
-        .DeleteShard()
-            .setCollectionName(collectionName)
-            .setShardName("shardC");
-    response = deleteShardRequest.process(cloudClient);
+    response = CollectionAdminRequest.deleteShard(collectionName, "shardC").process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> nodesStatus = response.getCollectionNodesStatus();
     assertEquals(1, nodesStatus.size());
   }
-  
-  protected void testReloadCollection() throws IOException, SolrServerException {
-    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
-    CollectionAdminRequest.Reload reloadCollectionRequest = new CollectionAdminRequest.Reload()
-            .setCollectionName("collection1");
-    CollectionAdminResponse response = reloadCollectionRequest.process(cloudClient);
 
-    assertEquals(0, response.getStatus());
-  }
-  
-  protected void testCreateAndDeleteAlias() throws IOException, SolrServerException {
-    CollectionAdminRequest.CreateAlias createAliasRequest = new CollectionAdminRequest
-        .CreateAlias()
-            .setAliasName("solrj_alias")
-            .setAliasedCollections(DEFAULT_COLLECTION);
-    CollectionAdminResponse response = createAliasRequest.process(cloudClient);
+  @Test
+  public void testCreateAndDeleteAlias() throws IOException, SolrServerException {
+
+    final String collection = "aliasedCollection";
+    CollectionAdminRequest.createCollection(collection, "conf", 1, 1).process(cluster.getSolrClient());
 
+    CollectionAdminResponse response
+        = CollectionAdminRequest.createAlias("solrj_alias", collection).process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    CollectionAdminRequest.DeleteAlias deleteAliasRequest = new CollectionAdminRequest.DeleteAlias()
-            .setAliasName("solrj_alias");
-    deleteAliasRequest.process(cloudClient);
-    
+    response = CollectionAdminRequest.deleteAlias("solrj_alias").process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
   }
-  
-  protected void testSplitShard() throws Exception {
-    String collectionName = "solrj_test_splitshard";
-    cloudClient.setDefaultCollection(collectionName);
-    
-    CollectionAdminRequest.Create createCollectionRequest = new CollectionAdminRequest.Create()
-            .setConfigName("conf1")
-            .setNumShards(2)
-            .setCollectionName(collectionName);
-    createCollectionRequest.process(cloudClient);
-    
-    CollectionAdminRequest.SplitShard splitShardRequest = new CollectionAdminRequest.SplitShard()
-            .setCollectionName(collectionName)
-            .setShardName("shard1");
-    CollectionAdminResponse response = splitShardRequest.process(cloudClient);
+
+  @Test
+  public void testSplitShard() throws Exception {
+
+    final String collectionName = "solrj_test_splitshard";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .process(cluster.getSolrClient());
+
+    CollectionAdminResponse response = CollectionAdminRequest.splitShard(collectionName)
+        .setShardName("shard1")
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
@@ -207,267 +152,204 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
     assertEquals(0, (int) coresStatus.get(collectionName + "_shard1_0_replica1").get("status"));
     assertEquals(0, (int) coresStatus.get(collectionName + "_shard1_1_replica1").get("status"));
 
-    waitForRecoveriesToFinish(collectionName, false);
-    waitForThingsToLevelOut(10);
+    waitForState("Expected all shards to be active and parent shard to be removed", collectionName, (n, c) -> {
+      if (c.getSlice("shard1").getState() == Slice.State.ACTIVE)
+        return false;
+      for (Replica r : c.getReplicas()) {
+        if (r.isActive(n) == false)
+          return false;
+      }
+      return true;
+    });
     
     // Test splitting using split.key
-    splitShardRequest = new CollectionAdminRequest.SplitShard()
-            .setCollectionName(collectionName)
-            .setSplitKey("b!");
-    response = splitShardRequest.process(cloudClient);
+    response = CollectionAdminRequest.splitShard(collectionName)
+        .setSplitKey("b!")
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    waitForRecoveriesToFinish(collectionName, false);
-    waitForThingsToLevelOut(10);
-    
-    ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-    Collection<Slice> slices = clusterState.getActiveSlices(collectionName);
-    assertEquals("ClusterState: "+ clusterState.getActiveSlices(collectionName), 5, slices.size());  
+    waitForState("Expected 5 slices to be active", collectionName, (n, c) -> c.getActiveSlices().size() == 5);
     
   }
 
-  private void testCreateCollectionWithPropertyParam() throws Exception {
+  @Test
+  public void testCreateCollectionWithPropertyParam() throws Exception {
+
     String collectionName = "solrj_test_core_props";
     
-    File tmpDir = createTempDir("testPropertyParamsForCreate").toFile();
-    File dataDir = new File(tmpDir, "dataDir-" + TestUtil.randomSimpleString(random(), 1, 5));
-    File ulogDir = new File(tmpDir, "ulogDir-" + TestUtil.randomSimpleString(random(), 1, 5));
+    Path tmpDir = createTempDir("testPropertyParamsForCreate");
+    Path dataDir = tmpDir.resolve("dataDir-" + TestUtil.randomSimpleString(random(), 1, 5));
+    Path ulogDir = tmpDir.resolve("ulogDir-" + TestUtil.randomSimpleString(random(), 1, 5));
 
-    Properties properties = new Properties();
-    properties.put(CoreAdminParams.DATA_DIR, dataDir.getAbsolutePath());
-    properties.put(CoreAdminParams.ULOG_DIR, ulogDir.getAbsolutePath());
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1)
+        .withProperty(CoreAdminParams.DATA_DIR, dataDir.toString())
+        .withProperty(CoreAdminParams.ULOG_DIR, ulogDir.toString())
+        .process(cluster.getSolrClient());
 
-    CollectionAdminRequest.Create createReq = new CollectionAdminRequest.Create()
-            .setCollectionName(collectionName)
-            .setNumShards(1)
-            .setConfigName("conf1")
-            .setProperties(properties);
-
-    CollectionAdminResponse response = createReq.process(cloudClient);
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
     Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
     assertEquals(1, coresStatus.size());
 
-    DocCollection testCollection = cloudClient.getZkStateReader()
-        .getClusterState().getCollection(collectionName);
+    DocCollection testCollection = getCollectionState(collectionName);
 
     Replica replica1 = testCollection.getReplica("core_node1");
+    CoreStatus coreStatus = getCoreStatus(replica1);
 
-    try (HttpSolrClient client = getHttpSolrClient(replica1.getStr("base_url"))) {
-      CoreAdminResponse status = CoreAdminRequest.getStatus(replica1.getStr("core"), client);
-      NamedList<Object> coreStatus = status.getCoreStatus(replica1.getStr("core"));
-      String dataDirStr = (String) coreStatus.get("dataDir");
-      assertEquals("Data dir does not match param given in property.dataDir syntax",
-          new File(dataDirStr).getAbsolutePath(), dataDir.getAbsolutePath());
-    }
+    assertEquals(Paths.get(coreStatus.getDataDirectory()).toString(), dataDir.toString());
 
-    CollectionAdminRequest.Delete deleteCollectionRequest = new CollectionAdminRequest.Delete();
-    deleteCollectionRequest.setCollectionName(collectionName);
-    deleteCollectionRequest.process(cloudClient);
   }
 
-  private void testAddAndDeleteReplica() throws Exception {
-    String collectionName = "solrj_replicatests";
-    createCollection(collectionName, cloudClient, 1, 2);
+  @Test
+  public void testAddAndDeleteReplica() throws Exception {
 
-    cloudClient.setDefaultCollection(collectionName);
+    final String collectionName = "solrj_replicatests";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 2)
+        .process(cluster.getSolrClient());
 
-    String newReplicaName = Assign.assignNode(cloudClient.getZkStateReader().getClusterState().getCollection(collectionName));
-    ArrayList<String> nodeList = new ArrayList<>(cloudClient.getZkStateReader().getClusterState().getLiveNodes());
+    String newReplicaName = Assign.assignNode(getCollectionState(collectionName));
+    ArrayList<String> nodeList
+        = new ArrayList<>(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes());
     Collections.shuffle(nodeList, random());
-    CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()
-            .setCollectionName(collectionName)
-            .setShardName("shard1")
-            .setNode(nodeList.get(0));
-    CollectionAdminResponse response = addReplica.process(cloudClient);
+    final String node = nodeList.get(0);
+
+    CollectionAdminResponse response = CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
+        .setNode(node)
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
 
-    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
-    Replica newReplica = null;
-
-    while (! timeout.hasTimedOut() && newReplica == null) {
-      Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
-      newReplica = slice.getReplica(newReplicaName);
-    }
-
-    assertNotNull(newReplica);
-
-    assertEquals("Replica should be created on the right node",
-        cloudClient.getZkStateReader().getBaseUrlForNodeName(nodeList.get(0)),
-        newReplica.getStr(ZkStateReader.BASE_URL_PROP)
-    );
+    waitForState("Expected to see replica " + newReplicaName + " on node " + node, collectionName, (n, c) -> {
+      Replica r = c.getSlice("shard1").getReplica(newReplicaName);
+      return r != null && r.getNodeName().equals(node);
+    });
     
     // Test DELETEREPLICA
-    CollectionAdminRequest.DeleteReplica deleteReplicaRequest = new CollectionAdminRequest.DeleteReplica()
-            .setCollectionName(collectionName)
-            .setShardName("shard1")
-            .setReplica(newReplicaName);
-    response = deleteReplicaRequest.process(cloudClient);
-
+    response = CollectionAdminRequest.deleteReplica(collectionName, "shard1", newReplicaName)
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    timeout = new TimeOut(3, TimeUnit.SECONDS);
-
-    while (! timeout.hasTimedOut() && newReplica != null) {
-      Slice slice = cloudClient.getZkStateReader().getClusterState().getSlice(collectionName, "shard1");
-      newReplica = slice.getReplica(newReplicaName);
-    }
+    waitForState("Expected replica " + newReplicaName + " to vanish from cluster state", collectionName,
+        (n, c) -> c.getSlice("shard1").getReplica(newReplicaName) == null);
 
-    assertNull(newReplica);
   }
 
-  private void testClusterProp() throws InterruptedException, IOException, SolrServerException {
-    CollectionAdminRequest.ClusterProp clusterPropRequest = new CollectionAdminRequest.ClusterProp()
-            .setPropertyName(ZkStateReader.LEGACY_CLOUD)
-            .setPropertyValue("false");
-    CollectionAdminResponse response = clusterPropRequest.process(cloudClient);
+  @Test
+  public void testClusterProp() throws InterruptedException, IOException, SolrServerException {
+
+    CollectionAdminResponse response = CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, "false")
+        .process(cluster.getSolrClient());
 
     assertEquals(0, response.getStatus());
 
-    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
-    boolean changed = false;
-    
-    while(! timeout.hasTimedOut()){
-      Thread.sleep(10);
-      changed = Objects.equals("false",
-          cloudClient.getZkStateReader().getClusterProperty(ZkStateReader.LEGACY_CLOUD, "none"));
-      if(changed) break;
-    }
-    assertTrue("The Cluster property wasn't set", changed);
+    ClusterProperties props = new ClusterProperties(zkClient());
+    assertEquals("Cluster property was not set", props.getClusterProperty(ZkStateReader.LEGACY_CLOUD, "true"), "false");
     
     // Unset ClusterProp that we set.
-    clusterPropRequest = new CollectionAdminRequest.ClusterProp()
-            .setPropertyName(ZkStateReader.LEGACY_CLOUD)
-            .setPropertyValue(null);
-    clusterPropRequest.process(cloudClient);
-
-    timeout = new TimeOut(3, TimeUnit.SECONDS);
-    changed = false;
-    while(! timeout.hasTimedOut()) {
-      Thread.sleep(10);
-      changed = (cloudClient.getZkStateReader().getClusterProperty(ZkStateReader.LEGACY_CLOUD, (String) null) == null);
-      if(changed)  
-        break;
-    }
-    assertTrue("The Cluster property wasn't unset", changed);
+    CollectionAdminRequest.setClusterProperty(ZkStateReader.LEGACY_CLOUD, null).process(cluster.getSolrClient());
+    assertEquals("Cluster property was not unset", props.getClusterProperty(ZkStateReader.LEGACY_CLOUD, "true"), "true");
+
   }
 
-  private void testAddAndRemoveRole() throws InterruptedException, IOException, SolrServerException {
-    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
-    Replica replica = cloudClient.getZkStateReader().getLeaderRetry(DEFAULT_COLLECTION, SHARD1);
-    CollectionAdminRequest.AddRole addRoleRequest = new CollectionAdminRequest.AddRole()
-            .setNode(replica.getNodeName())
-            .setRole("overseer");
-    addRoleRequest.process(cloudClient);
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testAddAndRemoveRole() throws InterruptedException, IOException, SolrServerException {
+
+    String node = cluster.getRandomJetty(random()).getNodeName();
+
+    CollectionAdminRequest.addRole(node, "overseer").process(cluster.getSolrClient());
 
-    CollectionAdminRequest.ClusterStatus clusterStatusRequest = new CollectionAdminRequest.ClusterStatus()
-            .setCollectionName(DEFAULT_COLLECTION);
-    CollectionAdminResponse response = clusterStatusRequest.process(cloudClient);
+    CollectionAdminResponse response = CollectionAdminRequest.getClusterStatus().process(cluster.getSolrClient());
 
     NamedList<Object> rsp = response.getResponse();
-    NamedList<Object> cluster = (NamedList<Object>) rsp.get("cluster");
-    assertNotNull("Cluster state should not be null", cluster);
-    Map<String, Object> roles = (Map<String, Object>) cluster.get("roles");
+    NamedList<Object> cs = (NamedList<Object>) rsp.get("cluster");
+    assertNotNull("Cluster state should not be null", cs);
+    Map<String, Object> roles = (Map<String, Object>) cs.get("roles");
     assertNotNull("Role information should not be null", roles);
     List<String> overseer = (List<String>) roles.get("overseer");
     assertNotNull(overseer);
     assertEquals(1, overseer.size());
-    assertTrue(overseer.contains(replica.getNodeName()));
+    assertTrue(overseer.contains(node));
     
     // Remove role
-    new CollectionAdminRequest.RemoveRole()
-            .setNode(replica.getNodeName())
-            .setRole("overseer")
-            .process(cloudClient);
-
-    clusterStatusRequest = new CollectionAdminRequest.ClusterStatus();
-    clusterStatusRequest.setCollectionName(DEFAULT_COLLECTION);
-    response = clusterStatusRequest.process(cloudClient);
+    CollectionAdminRequest.removeRole(node, "overseer").process(cluster.getSolrClient());
 
+    response = CollectionAdminRequest.getClusterStatus().process(cluster.getSolrClient());
     rsp = response.getResponse();
-    cluster = (NamedList<Object>) rsp.get("cluster");
-    assertNotNull("Cluster state should not be null", cluster);
-    roles = (Map<String, Object>) cluster.get("roles");
+    cs = (NamedList<Object>) rsp.get("cluster");
+    assertNotNull("Cluster state should not be null", cs);
+    roles = (Map<String, Object>) cs.get("roles");
     assertNotNull("Role information should not be null", roles);
     overseer = (List<String>) roles.get("overseer");
-    assertFalse(overseer.contains(replica.getNodeName()));
+    assertFalse(overseer.contains(node));
   }
-  
-  private void testOverseerStatus() throws IOException, SolrServerException {
-    CollectionAdminResponse response = new CollectionAdminRequest.OverseerStatus().process(cloudClient);
+
+  @Test
+  public void testOverseerStatus() throws IOException, SolrServerException {
+    CollectionAdminResponse response = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
     assertNotNull("overseer_operations shouldn't be null", response.getResponse().get("overseer_operations"));
   }
-  
-  private void testList() throws IOException, SolrServerException {
-    CollectionAdminResponse response = new CollectionAdminRequest.List().process(cloudClient);
+
+  @Test
+  public void testList() throws IOException, SolrServerException {
+    CollectionAdminResponse response = new CollectionAdminRequest.List().process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
     assertNotNull("collection list should not be null", response.getResponse().get("collections"));
   }
-  
-  private void testAddAndDeleteReplicaProp() throws InterruptedException, IOException, SolrServerException {
-    Replica replica = cloudClient.getZkStateReader().getLeaderRetry(DEFAULT_COLLECTION, SHARD1);
-    CollectionAdminResponse response = new CollectionAdminRequest.AddReplicaProp()
-            .setCollectionName(DEFAULT_COLLECTION)
-            .setShardName(SHARD1)
-            .setReplica(replica.getName())
-            .setPropertyName("preferredleader")
-            .setPropertyValue("true").process(cloudClient);
-    assertEquals(0, response.getStatus());
 
-    TimeOut timeout = new TimeOut(20, TimeUnit.SECONDS);
-    String propertyValue = null;
-    
-    String replicaName = replica.getName();
-    while (! timeout.hasTimedOut()) {
-      ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-      replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
-      propertyValue = replica.getStr("property.preferredleader"); 
-      if(StringUtils.equals("true", propertyValue))
-        break;
-      Thread.sleep(50);
-    }
-    
-    assertEquals("Replica property was not updated, Latest value: " +
-        cloudClient.getZkStateReader().getClusterState().getReplica(DEFAULT_COLLECTION, replicaName),
-        "true",
-        propertyValue);
-
-    response = new CollectionAdminRequest.DeleteReplicaProp()
-            .setCollectionName(DEFAULT_COLLECTION)
-            .setShardName(SHARD1)
-            .setReplica(replicaName)
-            .setPropertyName("property.preferredleader").process(cloudClient);
+  @Test
+  public void testAddAndDeleteReplicaProp() throws InterruptedException, IOException, SolrServerException {
+
+    final String collection = "replicaProperties";
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    final Replica replica = getCollectionState(collection).getLeader("shard1");
+    CollectionAdminResponse response
+        = CollectionAdminRequest.addReplicaProperty(collection, "shard1", replica.getName(), "preferredleader", "true")
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    timeout = new TimeOut(20, TimeUnit.SECONDS);
-    boolean updated = false;
+    waitForState("Expecting property 'preferredleader' to appear on replica " + replica.getName(), collection,
+        (n, c) -> "true".equals(c.getReplica(replica.getName()).getStr("property.preferredleader")));
 
-    while (! timeout.hasTimedOut()) {
-      ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
-      replica = clusterState.getReplica(DEFAULT_COLLECTION, replicaName);
-      updated = replica.getStr("property.preferredleader") == null;
-      if(updated)
-        break;
-      Thread.sleep(50);
-    }
+    response = CollectionAdminRequest.deleteReplicaProperty(collection, "shard1", replica.getName(), "property.preferredleader")
+        .process(cluster.getSolrClient());
+    assertEquals(0, response.getStatus());
 
-    assertTrue("Replica property was not removed", updated);
+    waitForState("Expecting property 'preferredleader' to be removed from replica " + replica.getName(), collection,
+        (n, c) -> c.getReplica(replica.getName()).getStr("property.preferredleader") == null);
     
   }
-  
-  private void testBalanceShardUnique() throws IOException,
+
+  @Test
+  public void testBalanceShardUnique() throws IOException,
       SolrServerException, KeeperException, InterruptedException {
-    CollectionAdminResponse response = new CollectionAdminRequest.BalanceShardUnique()
-            .setCollection(DEFAULT_COLLECTION)
-            .setPropertyName("preferredLeader").process(cloudClient);
+
+    final String collection = "balancedProperties";
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    CollectionAdminResponse response = CollectionAdminRequest.balanceReplicaProperty(collection, "preferredLeader")
+        .process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());
 
-    verifyUniqueAcrossCollection(cloudClient, DEFAULT_COLLECTION, "property.preferredleader");    
+    waitForState("Expecting 'preferredleader' property to be balanced across all shards", collection, (n, c) -> {
+      for (Slice slice : c) {
+        int count = 0;
+        for (Replica replica : slice) {
+          if ("true".equals(replica.getStr("property.preferredleader")))
+            count += 1;
+        }
+        if (count != 1)
+          return false;
+      }
+      return true;
+    });
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
index dcc99a4..c46362e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteLastCustomShardedReplicaTest.java
@@ -16,106 +16,40 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.Replica;
-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.SolrParams;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.util.TimeOut;
+import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
+public class DeleteLastCustomShardedReplicaTest extends SolrCloudTestCase {
 
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.SHARDS_PROP;
-import static org.apache.solr.common.util.Utils.makeMap;
-import static org.apache.solr.common.params.CollectionParams.CollectionAction.DELETEREPLICA;
-
-public class DeleteLastCustomShardedReplicaTest extends AbstractFullDistribZkTestBase {
-
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  protected String getSolrXml() {
-    return "solr.xml";
-  }
-
-  public DeleteLastCustomShardedReplicaTest() {
-    sliceCount = 2;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
   @Test
-  @ShardsFixed(num = 2)
   public void test() throws Exception {
-    try (CloudSolrClient client = createCloudClient(null))  {
-      int replicationFactor = 1;
-      int maxShardsPerNode = 5;
-
-      Map<String, Object> props = Utils.makeMap(
-          "router.name", ImplicitDocRouter.NAME,
-          ZkStateReader.REPLICATION_FACTOR, replicationFactor,
-          ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode,
-          NUM_SLICES, 1,
-          SHARDS_PROP, "a,b");
 
-      Map<String,List<Integer>> collectionInfos = new HashMap<>();
+    final String collectionName = "customcollreplicadeletion";
 
-      String collectionName = "customcollreplicadeletion";
+    CollectionAdminRequest.createCollectionWithImplicitRouter(collectionName, "conf", "a,b", 1)
+        .setMaxShardsPerNode(5)
+        .process(cluster.getSolrClient());
 
-      createCollection(collectionInfos, collectionName, props, client);
+    DocCollection collectionState = getCollectionState(collectionName);
+    Replica replica = getRandomReplica(collectionState.getSlice("a"));
 
-      waitForRecoveriesToFinish(collectionName, false);
+    CollectionAdminRequest.deleteReplica(collectionName, "a", replica.getName())
+        .process(cluster.getSolrClient());
 
-      DocCollection testcoll = getCommonCloudSolrClient().getZkStateReader()
-              .getClusterState().getCollection(collectionName);
-      Replica replica = testcoll.getSlice("a").getReplicas().iterator().next();
-
-      removeAndWaitForReplicaGone(client, collectionName, replica, "a", replicationFactor-1);
-    }
-  }
+    waitForState("Expected shard 'a' to have no replicas", collectionName, (n, c) -> {
+      return c.getSlice("a") == null || c.getSlice("a").getReplicas().size() == 0;
+    });
 
-  protected void removeAndWaitForReplicaGone(CloudSolrClient client, String COLL_NAME, Replica replica, String shard,
-      final int expectedNumReplicasRemaining)
-      throws SolrServerException, IOException, InterruptedException {
-    Map m = makeMap("collection", COLL_NAME, "action", DELETEREPLICA.toLower(), "shard",
-        shard, "replica", replica.getName());
-    SolrParams params = new MapSolrParams(m);
-    SolrRequest request = new QueryRequest(params);
-    request.setPath("/admin/collections");
-    client.request(request);
-    TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS);
-    boolean success = false;
-    DocCollection testcoll = null;
-    while (! timeout.hasTimedOut()) {
-      testcoll = getCommonCloudSolrClient().getZkStateReader()
-          .getClusterState().getCollection(COLL_NAME);
-      // As of SOLR-5209 the last replica deletion no longer leads to
-      // the deletion of the slice.
-      final Slice slice = testcoll.getSlice(shard);
-      final int actualNumReplicasRemaining = (slice == null ? 0 : slice.getReplicas().size());
-      success = (actualNumReplicasRemaining == expectedNumReplicasRemaining);
-      if (success) {
-        log.info("replica cleaned up {}/{} core {}",
-            shard + "/" + replica.getName(), replica.getStr("core"));
-        log.info("current state {}", testcoll);
-        break;
-      }
-      Thread.sleep(100);
-    }
-    assertTrue("Replica not cleaned up", success);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
index 9097363..ed3d03b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteShardTest.java
@@ -20,202 +20,123 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.CoreAdminRequest;
-import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.client.solrj.request.CoreStatus;
 import org.apache.solr.cloud.overseer.OverseerAction;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.Slice.State;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.FileUtils;
 import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class DeleteShardTest extends AbstractFullDistribZkTestBase {
-
-  public DeleteShardTest() {
-    super();
-    sliceCount = 2;
-  }
+public class DeleteShardTest extends SolrCloudTestCase {
 
   // TODO: Custom hash slice deletion test
 
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
   @Test
-  @ShardsFixed(num = 2)
   public void test() throws Exception {
-    ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
 
-    Slice slice1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
-    Slice slice2 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD2);
+    final String collection = "deleteShard";
 
-    assertNotNull("Shard1 not found", slice1);
-    assertNotNull("Shard2 not found", slice2);
-    assertSame("Shard1 is not active", Slice.State.ACTIVE, slice1.getState());
-    assertSame("Shard2 is not active", Slice.State.ACTIVE, slice2.getState());
+    CollectionAdminRequest.createCollection(collection, "conf", 2, 1)
+        .process(cluster.getSolrClient());
 
-    try {
-      deleteShard(SHARD1);
-      fail("Deleting an active shard should not have succeeded");
-    } catch (HttpSolrClient.RemoteSolrException e) {
-      // expected
-    }
+    DocCollection state = getCollectionState(collection);
+    assertEquals(State.ACTIVE, state.getSlice("shard1").getState());
+    assertEquals(State.ACTIVE, state.getSlice("shard2").getState());
 
-    setSliceState(SHARD1, Slice.State.INACTIVE);
+    // Can't delete an ACTIVE shard
+    expectThrows(Exception.class, () -> {
+      CollectionAdminRequest.deleteShard(collection, "shard1").process(cluster.getSolrClient());
+    });
 
-    clusterState = cloudClient.getZkStateReader().getClusterState();
+    setSliceState(collection, "shard1", Slice.State.INACTIVE);
 
-    slice1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
+    // Can delete an INATIVE shard
+    CollectionAdminRequest.deleteShard(collection, "shard1").process(cluster.getSolrClient());
+    waitForState("Expected 'shard1' to be removed", collection, (n, c) -> {
+      return c.getSlice("shard1") == null;
+    });
 
-    assertSame("Shard1 is not inactive yet.", Slice.State.INACTIVE, slice1.getState());
-
-    deleteShard(SHARD1);
-
-    confirmShardDeletion(SHARD1);
-
-    setSliceState(SHARD2, Slice.State.CONSTRUCTION);
-    deleteShard(SHARD2);
-    confirmShardDeletion(SHARD2);
-  }
+    // Can delete a shard under construction
+    setSliceState(collection, "shard2", Slice.State.CONSTRUCTION);
+    CollectionAdminRequest.deleteShard(collection, "shard2").process(cluster.getSolrClient());
+    waitForState("Expected 'shard2' to be removed", collection, (n, c) -> {
+      return c.getSlice("shard2") == null;
+    });
 
-  protected void confirmShardDeletion(String shard) throws SolrServerException, KeeperException,
-      InterruptedException {
-    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
-    ClusterState clusterState = zkStateReader.getClusterState();
-    int counter = 10;
-    while (counter-- > 0) {
-      clusterState = zkStateReader.getClusterState();
-      if (clusterState.getSlice("collection1", shard) == null) {
-        break;
-      }
-      Thread.sleep(1000);
-    }
-
-    assertNull("Cluster still contains shard1 even after waiting for it to be deleted.",
-        clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1));
   }
 
-  protected void deleteShard(String shard) throws SolrServerException, IOException,
+  protected void setSliceState(String collection, String slice, State state) throws SolrServerException, IOException,
       KeeperException, InterruptedException {
 
-    ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set("action", CollectionParams.CollectionAction.DELETESHARD.toString());
-    params.set("collection", AbstractFullDistribZkTestBase.DEFAULT_COLLECTION);
-    params.set("shard", shard);
-    SolrRequest request = new QueryRequest(params);
-    request.setPath("/admin/collections");
-
-    String baseUrl = ((HttpSolrClient) shardToJetty.get(SHARD1).get(0).client.solrClient)
-        .getBaseURL();
-    baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
-
-    try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl)) {
-      baseServer.setConnectionTimeout(15000);
-      baseServer.setSoTimeout(60000);
-      baseServer.request(request);
-    }
-  }
+    CloudSolrClient client = cluster.getSolrClient();
 
-  protected void setSliceState(String slice, State state) throws SolrServerException, IOException,
-      KeeperException, InterruptedException {
-    DistributedQueue inQueue = Overseer.getStateUpdateQueue(cloudClient.getZkStateReader().getZkClient());
+    // TODO can this be encapsulated better somewhere?
+    DistributedQueue inQueue = Overseer.getStateUpdateQueue(client.getZkStateReader().getZkClient());
     Map<String, Object> propMap = new HashMap<>();
     propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
     propMap.put(slice, state.toString());
-    propMap.put(ZkStateReader.COLLECTION_PROP, "collection1");
+    propMap.put(ZkStateReader.COLLECTION_PROP, collection);
     ZkNodeProps m = new ZkNodeProps(propMap);
-    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
     inQueue.offer(Utils.toJSON(m));
-    boolean transition = false;
-
-    for (int counter = 10; counter > 0; counter--) {
-      ClusterState clusterState = zkStateReader.getClusterState();
-      State sliceState = clusterState.getSlice("collection1", slice).getState();
-      if (sliceState == state) {
-        transition = true;
-        break;
-      }
-      Thread.sleep(1000);
-    }
-
-    if (!transition) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not set shard [" + slice + "] as " + state);
-    }
+
+    waitForState("Expected shard " + slice + " to be in state " + state.toString(), collection, (n, c) -> {
+      return c.getSlice(slice).getState() == state;
+    });
+
   }
 
   @Test
   public void testDirectoryCleanupAfterDeleteShard() throws InterruptedException, IOException, SolrServerException {
-    CollectionAdminResponse rsp = new CollectionAdminRequest.Create()
-        .setCollectionName("deleteshard_test")
-        .setRouterName("implicit")
-        .setShards("a,b,c")
-        .setReplicationFactor(1)
-        .setConfigName("conf1")
-        .process(cloudClient);
+
+    final String collection = "deleteshard_test";
+    CollectionAdminRequest.createCollectionWithImplicitRouter(collection, "conf", "a,b,c", 1)
+        .setMaxShardsPerNode(2)
+        .process(cluster.getSolrClient());
 
     // Get replica details
-    Replica leader = cloudClient.getZkStateReader().getLeaderRetry("deleteshard_test", "a");
-    String baseUrl = (String) leader.get("base_url");
-    String core = (String) leader.get("core");
+    Replica leader = getCollectionState(collection).getLeader("a");
 
-    String instanceDir;
-    String dataDir;
+    CoreStatus coreStatus = getCoreStatus(leader);
+    assertTrue("Instance directory doesn't exist", FileUtils.fileExists(coreStatus.getInstanceDirectory()));
+    assertTrue("Data directory doesn't exist", FileUtils.fileExists(coreStatus.getDataDirectory()));
 
-    try (HttpSolrClient client = getHttpSolrClient(baseUrl)) {
-      CoreAdminResponse statusResp = CoreAdminRequest.getStatus(core, client);
-      NamedList r = statusResp.getCoreStatus().get(core);
-      instanceDir = (String) r.findRecursive("instanceDir");
-      dataDir = (String) r.get("dataDir");
-    }
+    assertEquals(3, getCollectionState(collection).getActiveSlices().size());
 
-    assertTrue("Instance directory doesn't exist", FileUtils.fileExists(instanceDir));
-    assertTrue("Data directory doesn't exist", FileUtils.fileExists(dataDir));
+    // Delete shard 'a'
+    CollectionAdminRequest.deleteShard(collection, "a").process(cluster.getSolrClient());
 
-    assertEquals(3, cloudClient.getZkStateReader().getClusterState().getActiveSlices("deleteshard_test").size());
+    assertEquals(2, getCollectionState(collection).getActiveSlices().size());
+    assertFalse("Instance directory still exists", FileUtils.fileExists(coreStatus.getInstanceDirectory()));
+    assertFalse("Data directory still exists", FileUtils.fileExists(coreStatus.getDataDirectory()));
 
-    // Delete shard 'a'
-    new CollectionAdminRequest.DeleteShard()
-        .setCollectionName("deleteshard_test")
-        .setShardName("a")
-        .process(cloudClient);
-
-    assertEquals(2, cloudClient.getZkStateReader().getClusterState().getActiveSlices("deleteshard_test").size());
-    assertFalse("Instance directory still exists", FileUtils.fileExists(instanceDir));
-    assertFalse("Data directory still exists", FileUtils.fileExists(dataDir));
-
-    leader = cloudClient.getZkStateReader().getLeaderRetry("deleteshard_test", "b");
-    baseUrl = (String) leader.get("base_url");
-    core = (String) leader.get("core");
-
-    try (HttpSolrClient client = getHttpSolrClient(baseUrl)) {
-      CoreAdminResponse statusResp = CoreAdminRequest.getStatus(core, client);
-      NamedList r = statusResp.getCoreStatus().get(core);
-      instanceDir = (String) r.findRecursive("instanceDir");
-      dataDir = (String) r.get("dataDir");
-    }
+    leader = getCollectionState(collection).getLeader("b");
+    coreStatus = getCoreStatus(leader);
 
     // Delete shard 'b'
-    new CollectionAdminRequest.DeleteShard()
-        .setCollectionName("deleteshard_test")
-        .setShardName("b")
+    CollectionAdminRequest.deleteShard(collection, "b")
         .setDeleteDataDir(false)
         .setDeleteInstanceDir(false)
-        .process(cloudClient);
+        .process(cluster.getSolrClient());
 
-    assertEquals(1, cloudClient.getZkStateReader().getClusterState().getActiveSlices("deleteshard_test").size());
-    assertTrue("Instance directory still exists", FileUtils.fileExists(instanceDir));
-    assertTrue("Data directory still exists", FileUtils.fileExists(dataDir));
+    assertEquals(1, getCollectionState(collection).getActiveSlices().size());
+    assertTrue("Instance directory still exists", FileUtils.fileExists(coreStatus.getInstanceDirectory()));
+    assertTrue("Data directory still exists", FileUtils.fileExists(coreStatus.getDataDirectory()));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
index e902ab4..c9a90a5 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerModifyCollectionTest.java
@@ -17,78 +17,62 @@
 
 package org.apache.solr.cloud;
 
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
-
-import java.lang.invoke.MethodHandles;
 import java.util.Map;
 
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
-import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class OverseerModifyCollectionTest extends AbstractFullDistribZkTestBase {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
+
+public class OverseerModifyCollectionTest extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf1", configset("cloud-minimal"))
+        .addConfig("conf2", configset("cloud-minimal"))
+        .configure();
+  }
+
   @Test
   public void testModifyColl() throws Exception {
-    String collName = "modifyColl";
-    String newConfName = "conf" + random().nextInt();
-    String oldConfName = "conf1";
-    try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
-      CollectionAdminResponse rsp;
-      CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collName, oldConfName, 1, 2);
-      rsp = create.process(client);
-      assertEquals(0, rsp.getStatus());
-      assertTrue(rsp.isSuccess());
-      
-      ConfigSetAdminRequest.Create createConfig = new ConfigSetAdminRequest.Create()
-        .setBaseConfigSetName(oldConfName)
-        .setConfigSetName(newConfName);
-      
-      ConfigSetAdminResponse configRsp = createConfig.process(client);
-      
-      assertEquals(0, configRsp.getStatus());
-      
-      ModifiableSolrParams p = new ModifiableSolrParams();
-      p.add("collection", collName);
-      p.add("action", "MODIFYCOLLECTION");
-      p.add("collection.configName", newConfName);
-      client.request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p));
-    }
-    
-    assertEquals(newConfName, getConfigNameFromZk(collName));    
+
+    final String collName = "modifyColl";
+
+    CollectionAdminRequest.createCollection(collName, "conf1", 1, 2)
+        .process(cluster.getSolrClient());
+
+    // TODO create a modifyCollection() method on CollectionAdminRequest
+    ModifiableSolrParams p1 = new ModifiableSolrParams();
+    p1.add("collection", collName);
+    p1.add("action", "MODIFYCOLLECTION");
+    p1.add("collection.configName", "conf2");
+    cluster.getSolrClient().request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p1));
+
+    assertEquals("conf2", getConfigNameFromZk(collName));
     
     //Try an invalid config name
-    try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
-      ModifiableSolrParams p = new ModifiableSolrParams();
-      p.add("collection", collName);
-      p.add("action", "MODIFYCOLLECTION");
-      p.add("collection.configName", "notARealConfigName");
-      try{
-        client.request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p));
-        fail("Exception should be thrown");
-      } catch(RemoteSolrException e) {
-        assertTrue(e.getMessage(), e.getMessage().contains("Can not find the specified config set"));
-      }
-    }
+    ModifiableSolrParams p2 = new ModifiableSolrParams();
+    p2.add("collection", collName);
+    p2.add("action", "MODIFYCOLLECTION");
+    p2.add("collection.configName", "notARealConfigName");
+    Exception e = expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p2));
+    });
+
+    assertTrue(e.getMessage(), e.getMessage().contains("Can not find the specified config set"));
 
   }
   
   private String getConfigNameFromZk(String collName) throws KeeperException, InterruptedException {
-    byte[] b = cloudClient.getZkStateReader().getZkClient().getData(ZkStateReader.getCollectionPathRoot(collName), null, null, false);
+    byte[] b = zkClient().getData(ZkStateReader.getCollectionPathRoot(collName), null, null, false);
     Map confData = (Map) Utils.fromJSON(b);
     return (String) confData.get(ZkController.CONFIGNAME_PROP); 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
index dec54d9..762bbeb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerRolesTest.java
@@ -16,118 +16,95 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
-import org.apache.solr.client.solrj.SolrRequest;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
-import org.apache.solr.common.params.CollectionParams.CollectionAction;
-import org.apache.solr.common.params.MapSolrParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.data.Stat;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 import static org.apache.solr.cloud.OverseerCollectionConfigSetProcessor.getLeaderNode;
 import static org.apache.solr.cloud.OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames;
-import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
-import static org.apache.solr.common.util.Utils.makeMap;
-import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
-import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+import static org.hamcrest.CoreMatchers.not;
 
-@LuceneTestCase.Slow
-@SuppressSSL(bugUrl = "SOLR-5776")
-public class OverseerRolesTest  extends AbstractFullDistribZkTestBase{
+public class OverseerRolesTest extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  protected String getSolrXml() {
-    return "solr.xml";
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(4)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
-  public OverseerRolesTest() {
-    sliceCount = 2;
-    fixShardCount(TEST_NIGHTLY ? 6 : 2);
-  }
-
-  @Test
-  public void test() throws Exception {
-    try (CloudSolrClient client = createCloudClient(null))  {
-      testQuitCommand(client);
-      testOverseerRole(client);
+  @Before
+  public void clearAllOverseerRoles() throws Exception {
+    for (String node : OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zkClient())) {
+      CollectionAdminRequest.removeRole(node, "overseer").process(cluster.getSolrClient());
     }
   }
 
-  private void testQuitCommand(CloudSolrClient client) throws Exception{
-    String collectionName = "testOverseerQuit";
-
-    createCollection(collectionName, client);
-
-    waitForRecoveriesToFinish(collectionName, false);
+  @Test
+  public void testQuitCommand() throws Exception {
 
-    SolrZkClient zk = client.getZkStateReader().getZkClient();
-    byte[] data = new byte[0];
-    data = zk.getData("/overseer_elect/leader", null, new Stat(), true);
+    SolrZkClient zk = zkClient();
+    byte[] data = zk.getData("/overseer_elect/leader", null, new Stat(), true);
     Map m = (Map) Utils.fromJSON(data);
     String s = (String) m.get("id");
     String leader = LeaderElector.getNodeName(s);
-    Overseer.getStateUpdateQueue(zk).offer(Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower())));
+    log.info("Current overseer: {}", leader);
+    Overseer.getStateUpdateQueue(zk)
+        .offer(Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower(),
+                                            "id", s)));
     final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS);
-    String newLeader=null;
+    String newLeader = null;
     for(;! timeout.hasTimedOut();){
       newLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(zk);
-      if(newLeader!=null && !newLeader.equals(leader)) break;
+      if (newLeader != null && !newLeader.equals(leader))
+        break;
       Thread.sleep(100);
     }
-    assertNotSame( "Leader not changed yet",newLeader,leader);
-
+    assertThat("Leader not changed yet", newLeader, not(leader));
 
-
-    assertTrue("The old leader should have rejoined election ", OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zk).contains(leader));
+    assertTrue("The old leader should have rejoined election",
+        OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zk).contains(leader));
   }
 
+  @Test
+  public void testOverseerRole() throws Exception {
 
-
-
-  private void testOverseerRole(CloudSolrClient client) throws Exception {
-    String collectionName = "testOverseerCol";
-
-    createCollection(collectionName, client);
-
-    waitForRecoveriesToFinish(collectionName, false);
-    List<String> l = OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(client.getZkStateReader().getZkClient()) ;
+    List<String> l = OverseerCollectionConfigSetProcessor.getSortedOverseerNodeNames(zkClient()) ;
 
     log.info("All nodes {}", l);
-    String currentLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(client.getZkStateReader().getZkClient());
+    String currentLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient());
     log.info("Current leader {} ", currentLeader);
     l.remove(currentLeader);
 
     Collections.shuffle(l, random());
     String overseerDesignate = l.get(0);
-    log.info("overseerDesignate {}",overseerDesignate);
-    setOverseerRole(client, CollectionAction.ADDROLE,overseerDesignate);
+    log.info("overseerDesignate {}", overseerDesignate);
+
+    CollectionAdminRequest.addRole(overseerDesignate, "overseer").process(cluster.getSolrClient());
 
     TimeOut timeout = new TimeOut(15, TimeUnit.SECONDS);
 
     boolean leaderchanged = false;
-    for(;!timeout.hasTimedOut();){
-      if(overseerDesignate.equals(OverseerCollectionConfigSetProcessor.getLeaderNode(client.getZkStateReader().getZkClient()))){
+    for (;!timeout.hasTimedOut();) {
+      if (overseerDesignate.equals(OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient()))) {
         log.info("overseer designate is the new overseer");
         leaderchanged =true;
         break;
@@ -136,36 +113,29 @@ public class OverseerRolesTest  extends AbstractFullDistribZkTestBase{
     }
     assertTrue("could not set the new overseer . expected "+
         overseerDesignate + " current order : " +
-        getSortedOverseerNodeNames(client.getZkStateReader().getZkClient()) +
-        " ldr :"+ OverseerCollectionConfigSetProcessor.getLeaderNode(client.getZkStateReader().getZkClient()) ,leaderchanged);
-
-
+        getSortedOverseerNodeNames(zkClient()) +
+        " ldr :"+ OverseerCollectionConfigSetProcessor.getLeaderNode(zkClient()) ,leaderchanged);
 
     //add another node as overseer
-
-
     l.remove(overseerDesignate);
-
     Collections.shuffle(l, random());
 
     String anotherOverseer = l.get(0);
     log.info("Adding another overseer designate {}", anotherOverseer);
-    setOverseerRole(client, CollectionAction.ADDROLE, anotherOverseer);
+    CollectionAdminRequest.addRole(anotherOverseer, "overseer").process(cluster.getSolrClient());
 
-    String currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
+    String currentOverseer = getLeaderNode(zkClient());
 
     log.info("Current Overseer {}", currentOverseer);
 
-    String hostPort = currentOverseer.substring(0,currentOverseer.indexOf('_'));
+    String hostPort = currentOverseer.substring(0, currentOverseer.indexOf('_'));
 
     StringBuilder sb = new StringBuilder();
-//
-//
     log.info("hostPort : {}", hostPort);
 
     JettySolrRunner leaderJetty = null;
 
-    for (JettySolrRunner jetty : jettys) {
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       String s = jetty.getBaseUrl().toString();
       log.info("jetTy {}",s);
       sb.append(s).append(" , ");
@@ -178,49 +148,20 @@ public class OverseerRolesTest  extends AbstractFullDistribZkTestBase{
     assertNotNull("Could not find a jetty2 kill",  leaderJetty);
 
     log.info("leader node {}", leaderJetty.getBaseUrl());
-    log.info ("current election Queue",
-        OverseerCollectionConfigSetProcessor.getSortedElectionNodes(client.getZkStateReader().getZkClient(),
-            "/overseer_elect/election"));
+    log.info("current election Queue",
+        OverseerCollectionConfigSetProcessor.getSortedElectionNodes(zkClient(), "/overseer_elect/election"));
     ChaosMonkey.stop(leaderJetty);
     timeout = new TimeOut(10, TimeUnit.SECONDS);
     leaderchanged = false;
     for (; !timeout.hasTimedOut(); ) {
-      currentOverseer = getLeaderNode(client.getZkStateReader().getZkClient());
+      currentOverseer = getLeaderNode(zkClient());
       if (anotherOverseer.equals(currentOverseer)) {
         leaderchanged = true;
         break;
       }
       Thread.sleep(100);
     }
-    assertTrue("New overseer designate has not become the overseer, expected : " + anotherOverseer + "actual : " + getLeaderNode(client.getZkStateReader().getZkClient()), leaderchanged);
+    assertTrue("New overseer designate has not become the overseer, expected : " + anotherOverseer + "actual : " + getLeaderNode(zkClient()), leaderchanged);
   }
 
-  private void setOverseerRole(CloudSolrClient client, CollectionAction action, String overseerDesignate) throws Exception, IOException {
-    log.info("Adding overseer designate {} ", overseerDesignate);
-    Map m = makeMap(
-        "action", action.toString().toLowerCase(Locale.ROOT),
-        "role", "overseer",
-        "node", overseerDesignate);
-    SolrParams params = new MapSolrParams(m);
-    SolrRequest request = new QueryRequest(params);
-    request.setPath("/admin/collections");
-    client.request(request);
-  }
-
-
-  protected void createCollection(String COLL_NAME, CloudSolrClient client) throws Exception {
-    int replicationFactor = 2;
-    int numShards = 4;
-    int maxShardsPerNode = ((((numShards+1) * replicationFactor) / getCommonCloudSolrClient()
-        .getZkStateReader().getClusterState().getLiveNodes().size())) + 1;
-
-    Map<String, Object> props = makeMap(
-        REPLICATION_FACTOR, replicationFactor,
-        MAX_SHARDS_PER_NODE, maxShardsPerNode,
-        NUM_SLICES, numShards);
-    Map<String,List<Integer>> collectionInfos = new HashMap<>();
-    createCollection(collectionInfos, COLL_NAME, props, client);
-  }
-
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
index b1899da..80fd38e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerStatusTest.java
@@ -17,74 +17,56 @@
 package org.apache.solr.cloud;
 
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class OverseerStatusTest extends BasicDistributedZkTest {
+public class OverseerStatusTest extends SolrCloudTestCase {
 
-  public OverseerStatusTest() {
-    schemaString = "schema15.xml";      // we need a string id
-    sliceCount = 1;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();;
   }
 
   @Test
-  @ShardsFixed(num = 1)
   public void test() throws Exception {
 
-    waitForThingsToLevelOut(15);
-
     // find existing command counts because collection may be created by base test class too
     int numCollectionCreates = 0, numOverseerCreates = 0;
-    NamedList<Object> resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
-    if (resp != null) {
-      NamedList<Object> collection_operations = (NamedList<Object>) resp.get("collection_operations");
-      if (collection_operations != null)  {
-        SimpleOrderedMap<Object> createcollection = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
-        if (createcollection != null && createcollection.get("requests") != null) {
-          numCollectionCreates = (Integer) createcollection.get("requests");
-        }
-        NamedList<Object> overseer_operations = (NamedList<Object>) resp.get("overseer_operations");
-        if (overseer_operations != null)  {
-          createcollection = (SimpleOrderedMap<Object>) overseer_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
-          if (createcollection != null && createcollection.get("requests") != null) {
-            numOverseerCreates = (Integer) createcollection.get("requests");
-          }
-        }
-      }
-    }
 
     String collectionName = "overseer_status_test";
-    CollectionAdminResponse response = createCollection(collectionName, 1, 1, 1);
-    resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1).process(cluster.getSolrClient());
+
+    NamedList<Object> resp = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient()).getResponse();
     NamedList<Object> collection_operations = (NamedList<Object>) resp.get("collection_operations");
     NamedList<Object> overseer_operations = (NamedList<Object>) resp.get("overseer_operations");
-    SimpleOrderedMap<Object> createcollection = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
+    SimpleOrderedMap<Object> createcollection
+        = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
     assertEquals("No stats for create in OverseerCollectionProcessor", numCollectionCreates + 1, createcollection.get("requests"));
     createcollection = (SimpleOrderedMap<Object>) overseer_operations.get(CollectionParams.CollectionAction.CREATE.toLower());
     assertEquals("No stats for create in Overseer", numOverseerCreates + 1, createcollection.get("requests"));
 
     // Reload the collection
-    new CollectionAdminRequest.Reload().setCollectionName(collectionName).process(cloudClient);
-
+    CollectionAdminRequest.reloadCollection(collectionName).process(cluster.getSolrClient());
 
-    resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
+    resp = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient()).getResponse();
     collection_operations = (NamedList<Object>) resp.get("collection_operations");
     SimpleOrderedMap<Object> reload = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.RELOAD.toLower());
     assertEquals("No stats for reload in OverseerCollectionProcessor", 1, reload.get("requests"));
 
     try {
-      new CollectionAdminRequest.SplitShard()
-              .setCollectionName("non_existent_collection")
-              .setShardName("non_existent_shard")
-              .process(cloudClient);
+      CollectionAdminRequest.splitShard("non_existent_collection")
+          .setShardName("non_existent_shard")
+          .process(cluster.getSolrClient());
       fail("Split shard for non existent collection should have failed");
     } catch (Exception e) {
       // expected because we did not correctly specify required params for split
     }
-    resp = new CollectionAdminRequest.OverseerStatus().process(cloudClient).getResponse();
+    resp = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient()).getResponse();
     collection_operations = (NamedList<Object>) resp.get("collection_operations");
     SimpleOrderedMap<Object> split = (SimpleOrderedMap<Object>) collection_operations.get(CollectionParams.CollectionAction.SPLITSHARD.toLower());
     assertEquals("No stats for split in OverseerCollectionProcessor", 1, split.get("errors"));
@@ -111,6 +93,5 @@ public class OverseerStatusTest extends BasicDistributedZkTest {
     assertNotNull(updateState.get("errors"));
     assertNotNull(updateState.get("avgTimePerRequest"));
 
-    waitForThingsToLevelOut(15);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java b/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
index 24f9696..54503bf 100644
--- a/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/RemoteQueryErrorTest.java
@@ -16,58 +16,43 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import static org.junit.internal.matchers.StringContains.containsString;
 
 /**
  * Verify that remote (proxied) queries return proper error messages
  */
-@Slow
-public class RemoteQueryErrorTest extends AbstractFullDistribZkTestBase {
+public class RemoteQueryErrorTest extends SolrCloudTestCase {
 
-  public RemoteQueryErrorTest() {
-    super();
-    sliceCount = 1;
-    fixShardCount(random().nextBoolean() ? 3 : 4);
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(3)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
+  // TODO add test for CloudSolrClient as well
+
   @Test
   public void test() throws Exception {
-    handle.clear();
-    handle.put("timestamp", SKIPVAL);
-    
-    waitForThingsToLevelOut(15);
 
-    del("*:*");
-    
-    createCollection("collection2", 2, 1, 10);
-    
-    List<Integer> numShardsNumReplicaList = new ArrayList<>(2);
-    numShardsNumReplicaList.add(2);
-    numShardsNumReplicaList.add(1);
-    checkForCollection("collection2", numShardsNumReplicaList, null);
-    waitForRecoveriesToFinish("collection2", true);
+    CollectionAdminRequest.createCollection("collection", "conf", 2, 1).process(cluster.getSolrClient());
 
-    for (SolrClient solrClient : clients) {
-      try {
-        SolrInputDocument emptyDoc = new SolrInputDocument();
-        solrClient.add(emptyDoc);
-        fail("Expected unique key exception");
-      } catch (SolrException ex) {
-        assertThat(ex.getMessage(), containsString("Document is missing mandatory uniqueKey field: id"));
-      } catch(Exception ex) {
-        fail("Expected a SolrException to occur, instead received: " + ex.getClass());
-      } finally {
-        solrClient.close();
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      try (SolrClient client = jetty.newClient()) {
+        SolrException e = expectThrows(SolrException.class, () -> {
+          client.add("collection", new SolrInputDocument());
+        });
+        assertThat(e.getMessage(), containsString("Document is missing mandatory uniqueKey field: id"));
       }
     }
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java b/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
index 01c4440..415d4e4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestDownShardTolerantSearch.java
@@ -16,13 +16,15 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.params.ShardParams;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,41 +36,47 @@ import static org.hamcrest.CoreMatchers.is;
  * and also asserts that a meaningful exception is thrown when shards.tolerant=false
  * See SOLR-7566
  */
-public class TestDownShardTolerantSearch extends AbstractFullDistribZkTestBase {
+public class TestDownShardTolerantSearch extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public TestDownShardTolerantSearch() {
-    sliceCount = 2;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
   @Test
-  @ShardsFixed(num = 2)
   public void searchingShouldFailWithoutTolerantSearchSetToTrue() throws Exception {
-    waitForRecoveriesToFinish(true);
 
-    indexAbunchOfDocs();
-    commit();
-    QueryResponse response = cloudClient.query(new SolrQuery("*:*").setRows(1));
+    CollectionAdminRequest.createCollection("tolerant", "conf", 2, 1)
+        .process(cluster.getSolrClient());
+
+    UpdateRequest update = new UpdateRequest();
+    for (int i = 0; i < 100; i++) {
+      update.add("id", Integer.toString(i));
+    }
+    update.commit(cluster.getSolrClient(), "tolerant");
+
+    QueryResponse response = cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1));
     assertThat(response.getStatus(), is(0));
-    assertThat(response.getResults().getNumFound(), is(66L));
+    assertThat(response.getResults().getNumFound(), is(100L));
 
-    ChaosMonkey.kill(shardToJetty.get(SHARD1).get(0));
+    cluster.stopJettySolrRunner(0);
 
-    response = cloudClient.query(new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, true));
+    response = cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, true));
     assertThat(response.getStatus(), is(0));
     assertTrue(response.getResults().getNumFound() > 0);
 
     try {
-      cloudClient.query(new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, false));
+      cluster.getSolrClient().query("tolerant", new SolrQuery("*:*").setRows(1).setParam(ShardParams.SHARDS_TOLERANT, false));
       fail("Request should have failed because we killed shard1 jetty");
     } catch (SolrServerException e) {
       log.info("error from server", e);
       assertNotNull(e.getCause());
       assertTrue("Error message from server should have the name of the down shard",
-          e.getCause().getMessage().contains(SHARD1));
-    } catch (IOException e) {
-      e.printStackTrace();
+          e.getCause().getMessage().contains("shard"));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java b/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
index 9ef2dcd..5bf77c1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestExclusionRuleCollectionAccess.java
@@ -16,34 +16,32 @@
  */
 package org.apache.solr.cloud;
 
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-@LuceneTestCase.Slow
-public class TestExclusionRuleCollectionAccess extends AbstractFullDistribZkTestBase {
+public class TestExclusionRuleCollectionAccess extends SolrCloudTestCase {
 
-  public TestExclusionRuleCollectionAccess() {
-    schemaString = "schema15.xml";      // we need a string id
-    sliceCount = 1;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
   }
 
   @Test
   public void doTest() throws Exception {
-    CollectionAdminRequest.Create req = new CollectionAdminRequest.Create();
-    req.setCollectionName("css33");
-    req.setNumShards(1);
-    req.process(cloudClient);
-    
-    waitForRecoveriesToFinish("css33", false);
-    
-    try (SolrClient c = createCloudClient("css33")) {
-      c.add(getDoc("id", "1"));
-      c.commit();
-
-      assertEquals("Should have returned 1 result", 1, c.query(params("q", "*:*", "collection", "css33")).getResults().getNumFound());
-    }
+
+    CollectionAdminRequest.createCollection("css33", "conf", 1, 1).process(cluster.getSolrClient());
+
+    new UpdateRequest()
+        .add("id", "1")
+        .commit(cluster.getSolrClient(), "css33");
+
+    assertEquals("Should have returned 1 result", 1,
+        cluster.getSolrClient().query("css33", params("q", "*:*", "collection", "css33")).getResults().getNumFound());
+
   }
   
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
index 30fe933..bc4f4e5 100644
--- a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
@@ -18,19 +18,21 @@ package org.apache.solr.security;
 
 import javax.servlet.ServletRequest;
 import javax.servlet.http.HttpServletRequest;
-
 import java.lang.invoke.MethodHandles;
 import java.security.Principal;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Predicate;
 
-import org.apache.solr.SolrTestCaseJ4;
+import org.apache.http.client.HttpClient;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.Utils;
+import org.junit.After;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,27 +41,32 @@ import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.util.Utils.makeMap;
 import static org.apache.solr.security.TestAuthorizationFramework.verifySecurityStatus;
 
-@SolrTestCaseJ4.SuppressSSL
-public class PKIAuthenticationIntegrationTest extends AbstractFullDistribZkTestBase {
+public class PKIAuthenticationIntegrationTest extends SolrCloudTestCase {
+
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  static final int TIMEOUT = 10000;
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
 
   @Test
   public void testPkiAuth() throws Exception {
-    waitForThingsToLevelOut(10);
 
+    CollectionAdminRequest.createCollection("collection", "conf", 2, 1).process(cluster.getSolrClient());
+
+    // TODO make a SolrJ helper class for this
     byte[] bytes = Utils.toJSON(makeMap("authorization", singletonMap("class", MockAuthorizationPlugin.class.getName()),
         "authentication", singletonMap("class", MockAuthenticationPlugin.class.getName())));
+    zkClient().setData(ZkStateReader.SOLR_SECURITY_CONF_PATH, bytes, true);
 
-    try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
-        TIMEOUT, TIMEOUT)) {
-      zkStateReader.getZkClient().setData(ZkStateReader.SOLR_SECURITY_CONF_PATH, bytes, true);
-    }
-    for (JettySolrRunner jetty : jettys) {
+    HttpClient httpClient = cluster.getSolrClient().getHttpClient();
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       String baseUrl = jetty.getBaseUrl().toString();
-      verifySecurityStatus(cloudClient.getLbClient().getHttpClient(), baseUrl + "/admin/authorization", "authorization/class", MockAuthorizationPlugin.class.getName(), 20);
-      verifySecurityStatus(cloudClient.getLbClient().getHttpClient(), baseUrl + "/admin/authentication", "authentication.enabled", "true", 20);
+      verifySecurityStatus(httpClient, baseUrl + "/admin/authorization", "authorization/class", MockAuthorizationPlugin.class.getName(), 20);
+      verifySecurityStatus(httpClient, baseUrl + "/admin/authentication", "authentication.enabled", "true", 20);
     }
     log.info("Starting test");
     ModifiableSolrParams params = new ModifiableSolrParams();
@@ -95,13 +102,12 @@ public class PKIAuthenticationIntegrationTest extends AbstractFullDistribZkTestB
       }
     };
     QueryRequest query = new QueryRequest(params);
-    query.process(cloudClient);
+    query.process(cluster.getSolrClient(), "collection");
     assertTrue("all nodes must get the user solr , no:of nodes got solr : " + count.get(),count.get() > 2);
   }
 
-  @Override
+  @After
   public void distribTearDown() throws Exception {
-    super.distribTearDown();
     MockAuthenticationPlugin.predicate = null;
     MockAuthorizationPlugin.predicate = null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/183f9980/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index 0beaa55..94750c0a 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -280,6 +280,8 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
 
     public CollectionAdminRoleRequest(CollectionAction action, String node, String role) {
       super(action);
+      this.node = node;
+      this.role = role;
     }
 
     @Override