You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2016/04/25 22:20:42 UTC

lucene-solr:master: SOLR-9014: Deprecate and reduce usage of ClusterState methods which may make calls to ZK via the lazy collection reference

Repository: lucene-solr
Updated Branches:
  refs/heads/master 837264a42 -> 922265b47


SOLR-9014: Deprecate and reduce usage of ClusterState methods which may make calls to ZK via the lazy collection reference


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

Branch: refs/heads/master
Commit: 922265b478296992189434040517368cf93d1b09
Parents: 837264a
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Tue Apr 26 01:50:26 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Tue Apr 26 01:50:26 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 ++
 .../src/java/org/apache/solr/cloud/Assign.java  |  8 ++--
 .../OverseerAutoReplicaFailoverThread.java      |  1 +
 .../cloud/OverseerCollectionMessageHandler.java | 40 ++++++++--------
 .../cloud/OverseerConfigSetMessageHandler.java  |  1 +
 .../cloud/overseer/ClusterStateMutator.java     | 16 +++----
 .../solr/cloud/overseer/CollectionMutator.java  |  6 +--
 .../apache/solr/cloud/overseer/NodeMutator.java |  9 ++--
 .../solr/cloud/overseer/ReplicaMutator.java     | 36 ++++++++-------
 .../solr/cloud/overseer/SliceMutator.java       | 39 ++++++++--------
 .../solr/cloud/overseer/ZkStateWriter.java      |  3 +-
 .../src/java/org/apache/solr/core/SolrCore.java |  5 +-
 .../solr/handler/CdcrUpdateLogSynchronizer.java |  4 +-
 .../org/apache/solr/handler/SQLHandler.java     |  1 +
 .../solr/handler/admin/CollectionsHandler.java  | 27 ++++++-----
 .../solr/handler/admin/CoreAdminOperation.java  | 11 +++--
 .../org/apache/solr/servlet/HttpSolrCall.java   | 11 +++--
 .../processor/DistributedUpdateProcessor.java   |  9 ++--
 .../src/java/org/apache/solr/util/SolrCLI.java  |  1 +
 .../org/apache/solr/util/SolrLogLayout.java     | 10 ++--
 .../test/org/apache/solr/cloud/AssignTest.java  |  2 +-
 .../cloud/CollectionsAPIDistributedZkTest.java  |  8 ++--
 .../solr/cloud/CollectionsAPISolrJTest.java     |  2 +-
 .../org/apache/solr/cloud/OverseerTest.java     |  3 +-
 .../org/apache/solr/handler/TestSQLHandler.java |  1 +
 .../client/solrj/io/stream/TopicStream.java     |  1 +
 .../apache/solr/common/cloud/ClusterState.java  | 48 ++++++++++++++------
 .../solr/common/cloud/ClusterStateUtil.java     |  1 +
 .../apache/solr/common/cloud/DocCollection.java |  6 +++
 .../apache/solr/common/cloud/ZkStateReader.java |  3 +-
 .../solr/client/solrj/io/sql/JdbcTest.java      |  1 +
 31 files changed, 186 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 6034851..999bd73 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -169,6 +169,9 @@ Optimizations
 
 * SOLR-8973: Zookeeper frenzy when a core is first created. (Janmejay Singh, Scott Blum, shalin)
 
+* SOLR-9014: Deprecate and reduce usage of ClusterState methods which may make calls to ZK via
+  the lazy collection reference. (Scott Blum, shalin)
+
 Other Changes
 ----------------------
 * SOLR-7516: Improve javadocs for JavaBinCodec, ObjectResolver and enforce the single-usage policy.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/Assign.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Assign.java b/solr/core/src/java/org/apache/solr/cloud/Assign.java
index 6939270..92310ed 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Assign.java
@@ -46,8 +46,8 @@ import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 public class Assign {
   private static Pattern COUNT = Pattern.compile("core_node(\\d+)");
 
-  public static String assignNode(String collection, ClusterState state) {
-    Map<String, Slice> sliceMap = state.getSlicesMap(collection);
+  public static String assignNode(DocCollection collection) {
+    Map<String, Slice> sliceMap = collection != null ? collection.getSlicesMap() : null;
     if (sliceMap == null) {
       return "core_node1";
     }
@@ -70,12 +70,12 @@ public class Assign {
    *
    * @return the assigned shard id
    */
-  public static String assignShard(String collection, ClusterState state, Integer numShards) {
+  public static String assignShard(DocCollection collection, Integer numShards) {
     if (numShards == null) {
       numShards = 1;
     }
     String returnShardId = null;
-    Map<String, Slice> sliceMap = state.getActiveSlicesMap(collection);
+    Map<String, Slice> sliceMap = collection != null ? collection.getActiveSlicesMap() : null;
 
 
     // TODO: now that we create shards ahead of time, is this code needed?  Esp since hash ranges aren't assigned when creating via this method?

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
index f94ffcc..e8ac6c5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 6bff648..ce04841 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -945,16 +945,13 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
                 + router.getClass().getName());
       }
     } else {
-      parentSlice = clusterState.getSlice(collectionName, slice);
+      parentSlice = collection.getSlice(slice);
     }
     
     if (parentSlice == null) {
-      if (clusterState.hasCollection(collectionName)) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
-      } else {
-        throw new SolrException(ErrorCode.BAD_REQUEST,
-            "No collection with the specified name exists: " + collectionName);
-      }
+      // no chance of the collection being null because ClusterState#getCollection(String) would have thrown
+      // an exception already
+      throw new SolrException(ErrorCode.BAD_REQUEST, "No shard with the specified name exists: " + slice);
     }
     
     // find the leader for the shard
@@ -1039,7 +1036,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
         String subShardName = collectionName + "_" + subSlice + "_replica1";
         subShardNames.add(subShardName);
         
-        Slice oSlice = clusterState.getSlice(collectionName, subSlice);
+        Slice oSlice = collection.getSlice(subSlice);
         if (oSlice != null) {
           final Slice.State state = oSlice.getState();
           if (state == Slice.State.ACTIVE) {
@@ -1180,7 +1177,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       
       // TODO: Have replication factor decided in some other way instead of numShards for the parent
       
-      int repFactor = clusterState.getSlice(collectionName, slice).getReplicas().size();
+      int repFactor = parentSlice.getReplicas().size();
       
       // we need to look at every node and see how many cores it serves
       // add our new cores to existing nodes serving the least number of cores
@@ -1379,18 +1376,18 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
   }
 
   private void deleteShard(ClusterState clusterState, ZkNodeProps message, NamedList results) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     String sliceId = message.getStr(ZkStateReader.SHARD_ID_PROP);
     
     log.info("Delete shard invoked");
-    Slice slice = clusterState.getSlice(collection, sliceId);
+    Slice slice = clusterState.getSlice(collectionName, sliceId);
     
     if (slice == null) {
-      if (clusterState.hasCollection(collection)) {
+      if (clusterState.hasCollection(collectionName)) {
         throw new SolrException(ErrorCode.BAD_REQUEST,
-            "No shard with name " + sliceId + " exists for collection " + collection);
+            "No shard with name " + sliceId + " exists for collection " + collectionName);
       } else {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "No collection with the specified name exists: " + collection);
+        throw new SolrException(ErrorCode.BAD_REQUEST, "No collection with the specified name exists: " + collectionName);
       }
     }
     // For now, only allow for deletions of Inactive slices or custom hashes (range==null).
@@ -1421,7 +1418,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       processResponses(results, shardHandler, true, "Failed to delete shard", asyncId, requestMap, Collections.emptySet());
 
       ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETESHARD.toLower(), ZkStateReader.COLLECTION_PROP,
-          collection, ZkStateReader.SHARD_ID_PROP, sliceId);
+          collectionName, ZkStateReader.SHARD_ID_PROP, sliceId);
       Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer(Utils.toJSON(m));
       
       // wait for a while until we don't see the shard
@@ -1429,7 +1426,8 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       boolean removed = false;
       while (! timeout.hasTimedOut()) {
         Thread.sleep(100);
-        removed = zkStateReader.getClusterState().getSlice(collection, sliceId) == null;
+        DocCollection collection = zkStateReader.getClusterState().getCollection(collectionName);
+        removed = collection.getSlice(sliceId) == null;
         if (removed) {
           Thread.sleep(100); // just a bit of time so it's more likely other readers see on return
           break;
@@ -1437,16 +1435,16 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
       }
       if (!removed) {
         throw new SolrException(ErrorCode.SERVER_ERROR,
-            "Could not fully remove collection: " + collection + " shard: " + sliceId);
+            "Could not fully remove collection: " + collectionName + " shard: " + sliceId);
       }
       
-      log.info("Successfully deleted collection: " + collection + ", shard: " + sliceId);
+      log.info("Successfully deleted collection: " + collectionName + ", shard: " + sliceId);
       
     } catch (SolrException e) {
       throw e;
     } catch (Exception e) {
       throw new SolrException(ErrorCode.SERVER_ERROR,
-          "Error executing delete operation for collection: " + collection + " shard: " + sliceId, e);
+          "Error executing delete operation for collection: " + collectionName + " shard: " + sliceId, e);
     }
   }
 
@@ -1561,7 +1559,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     boolean added = false;
     while (! waitUntil.hasTimedOut()) {
       Thread.sleep(100);
-      Map<String, RoutingRule> rules = zkStateReader.getClusterState().getSlice(sourceCollection.getName(), sourceSlice.getName()).getRoutingRules();
+      sourceCollection = zkStateReader.getClusterState().getCollection(sourceCollection.getName());
+      sourceSlice = sourceCollection.getSlice(sourceSlice.getName());
+      Map<String, RoutingRule> rules = sourceSlice.getRoutingRules();
       if (rules != null) {
         RoutingRule rule = rules.get(SolrIndexSplitter.getRouteKey(splitKey) + "!");
         if (rule != null && rule.getRouteRanges().contains(splitRange)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
index 1f972ff..15fed42 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
@@ -30,6 +30,7 @@ import java.util.Set;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkConfigManager;
 import org.apache.solr.common.cloud.ZkNodeProps;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
index 7ffa8c1..0a76d91 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
@@ -122,7 +122,7 @@ public class ClusterStateMutator {
   public static ClusterState newState(ClusterState state, String name, DocCollection collection) {
     ClusterState newClusterState = null;
     if (collection == null) {
-      newClusterState = state.copyWith(name, (DocCollection) null);
+      newClusterState = state.copyWith(name, null);
     } else {
       newClusterState = state.copyWith(name, collection);
     }
@@ -153,9 +153,8 @@ public class ClusterStateMutator {
   /*
        * Return an already assigned id or null if not assigned
        */
-  public static String getAssignedId(final ClusterState state, final String nodeName,
-                              final ZkNodeProps coreState) {
-    Collection<Slice> slices = state.getSlices(coreState.getStr(ZkStateReader.COLLECTION_PROP));
+  public static String getAssignedId(final DocCollection collection, final String nodeName) {
+    Collection<Slice> slices = collection != null ? collection.getSlices() : null;
     if (slices != null) {
       for (Slice slice : slices) {
         if (slice.getReplicasMap().get(nodeName) != null) {
@@ -166,18 +165,15 @@ public class ClusterStateMutator {
     return null;
   }
 
-  public static String getAssignedCoreNodeName(ClusterState state, ZkNodeProps message) {
-    Collection<Slice> slices = state.getSlices(message.getStr(ZkStateReader.COLLECTION_PROP));
+  public static String getAssignedCoreNodeName(DocCollection collection, String forNodeName, String forCoreName) {
+    Collection<Slice> slices = collection != null ? collection.getSlices() : null;
     if (slices != null) {
       for (Slice slice : slices) {
         for (Replica replica : slice.getReplicas()) {
           String nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
           String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
 
-          String msgNodeName = message.getStr(ZkStateReader.NODE_NAME_PROP);
-          String msgCore = message.getStr(ZkStateReader.CORE_NAME_PROP);
-
-          if (nodeName.equals(msgNodeName) && core.equals(msgCore)) {
+          if (nodeName.equals(forNodeName) && core.equals(forCoreName)) {
             return replica.getName();
           }
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
index 4f7cb52..3d950fe 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
@@ -51,7 +51,8 @@ public class CollectionMutator {
     String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
     String shardId = message.getStr(ZkStateReader.SHARD_ID_PROP);
-    Slice slice = clusterState.getSlice(collectionName, shardId);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice slice = collection.getSlice(shardId);
     if (slice == null) {
       Map<String, Replica> replicas = Collections.EMPTY_MAP;
       Map<String, Object> sliceProps = new HashMap<>();
@@ -63,8 +64,7 @@ public class CollectionMutator {
       if (shardParent != null) {
         sliceProps.put(Slice.PARENT, shardParent);
       }
-      DocCollection collection = updateSlice(collectionName,
-          clusterState.getCollection(collectionName), new Slice(shardId, replicas, sliceProps));
+      collection = updateSlice(collectionName, collection, new Slice(shardId, replicas, sliceProps));
       return new ZkWriteCommand(collectionName, collection);
     } else {
       log.error("Unable to create Shard: " + shardId + " because it already exists in collection: " + collectionName);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
index 0784cd4..5dd27c1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
@@ -27,6 +27,7 @@ import java.util.Set;
 import java.util.Map.Entry;
 
 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.ZkNodeProps;
@@ -50,12 +51,12 @@ public class NodeMutator {
 
     Set<String> collections = clusterState.getCollections();
     for (String collection : collections) {
-
-      Map<String,Slice> slicesCopy = new LinkedHashMap<>(clusterState.getSlicesMap(collection));
+      DocCollection docCollection = clusterState.getCollection(collection);
+      Map<String,Slice> slicesCopy = new LinkedHashMap<>(docCollection.getSlicesMap());
 
       Set<Entry<String,Slice>> entries = slicesCopy.entrySet();
       for (Entry<String,Slice> entry : entries) {
-        Slice slice = clusterState.getSlice(collection, entry.getKey());
+        Slice slice = docCollection.getSlice(entry.getKey());
         Map<String,Replica> newReplicas = new HashMap<String,Replica>();
 
         Collection<Replica> replicas = slice.getReplicas();
@@ -76,7 +77,7 @@ public class NodeMutator {
 
       }
 
-      zkWriteCommands.add(new ZkWriteCommand(collection, clusterState.getCollection(collection).copyWithSlices(slicesCopy)));
+      zkWriteCommands.add(new ZkWriteCommand(collection, docCollection.copyWithSlices(slicesCopy)));
     }
 
     return zkWriteCommands;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
index a3efc80..5147f43 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
@@ -127,7 +127,8 @@ public class ReplicaMutator {
       isUnique = Boolean.parseBoolean(shardUnique);
     }
 
-    Replica replica = clusterState.getReplica(collectionName, replicaName);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Replica replica = collection.getReplica(replicaName);
 
     if (replica == null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not find collection/slice/replica " +
@@ -138,7 +139,7 @@ public class ReplicaMutator {
     if (StringUtils.equalsIgnoreCase(replica.getStr(property), propVal)) return ZkStateWriter.NO_OP; // already the value we're going to set
 
     // OK, there's no way we won't change the cluster state now
-    Map<String,Replica> replicas = clusterState.getSlice(collectionName, sliceName).getReplicasCopy();
+    Map<String,Replica> replicas = collection.getSlice(sliceName).getReplicasCopy();
     if (isUnique == false) {
       replicas.get(replicaName).getProperties().put(property, propVal);
     } else { // Set prop for this replica, but remove it for all others.
@@ -150,8 +151,8 @@ public class ReplicaMutator {
         }
       }
     }
-    Slice newSlice = new Slice(sliceName, replicas, clusterState.getSlice(collectionName, sliceName).shallowCopy());
-    DocCollection newCollection = CollectionMutator.updateSlice(collectionName, clusterState.getCollection(collectionName),
+    Slice newSlice = new Slice(sliceName, replicas, collection.getSlice(sliceName).shallowCopy());
+    DocCollection newCollection = CollectionMutator.updateSlice(collectionName, collection,
         newSlice);
     return new ZkWriteCommand(collectionName, newCollection);
   }
@@ -174,7 +175,8 @@ public class ReplicaMutator {
       property = OverseerCollectionMessageHandler.COLL_PROP_PREFIX + property;
     }
 
-    Replica replica = clusterState.getReplica(collectionName, replicaName);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Replica replica = collection.getReplica(replicaName);
 
     if (replica == null) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not find collection/slice/replica " +
@@ -188,7 +190,6 @@ public class ReplicaMutator {
 
     log.info("Deleting property " + property + " for collection: " + collectionName +
         " slice " + sliceName + " replica " + replicaName + ". Full message: " + message);
-    DocCollection collection = clusterState.getCollection(collectionName);
     Slice slice = collection.getSlice(sliceName);
     DocCollection newCollection = SliceMutator.updateReplica(collection,
         slice, replicaName, unsetProperty(replica, property));
@@ -232,13 +233,15 @@ public class ReplicaMutator {
     String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP);
 
     String coreNodeName = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
+    DocCollection collection = prevState.getCollectionOrNull(collectionName);
     if (coreNodeName == null) {
-      coreNodeName = ClusterStateMutator.getAssignedCoreNodeName(prevState, message);
+      coreNodeName = ClusterStateMutator.getAssignedCoreNodeName(collection,
+          message.getStr(ZkStateReader.NODE_NAME_PROP), message.getStr(ZkStateReader.CORE_NAME_PROP));
       if (coreNodeName != null) {
         log.info("node=" + coreNodeName + " is already registered");
       } else {
         // if coreNodeName is null, auto assign one
-        coreNodeName = Assign.assignNode(collectionName, prevState);
+        coreNodeName = Assign.assignNode(collection);
       }
       message.getProperties().put(ZkStateReader.CORE_NODE_NAME_PROP,
           coreNodeName);
@@ -247,7 +250,7 @@ public class ReplicaMutator {
     // use the provided non null shardId
     if (sliceName == null) {
       //get shardId from ClusterState
-      sliceName = ClusterStateMutator.getAssignedId(prevState, coreNodeName, message);
+      sliceName = ClusterStateMutator.getAssignedId(collection, coreNodeName);
       if (sliceName != null) {
         log.info("shard=" + sliceName + " is already registered");
       }
@@ -256,14 +259,14 @@ public class ReplicaMutator {
       //request new shardId
       if (collectionExists) {
         // use existing numShards
-        numShards = prevState.getCollection(collectionName).getSlices().size();
+        numShards = collection.getSlices().size();
         log.info("Collection already exists with " + ZkStateReader.NUM_SHARDS_PROP + "=" + numShards);
       }
-      sliceName = Assign.assignShard(collectionName, prevState, numShards);
+      sliceName = Assign.assignShard(collection, numShards);
       log.info("Assigning new node to shard shard=" + sliceName);
     }
 
-    Slice slice = prevState.getSlice(collectionName, sliceName);
+    Slice slice = collection != null ?  collection.getSlice(sliceName) : null;
 
     Map<String, Object> replicaProps = new LinkedHashMap<>();
 
@@ -313,9 +316,7 @@ public class ReplicaMutator {
     Map<String, Object> sliceProps = null;
     Map<String, Replica> replicas;
 
-    DocCollection collection = prevState.getCollectionOrNull(collectionName);
     if (slice != null) {
-      collection = prevState.getCollection(collectionName);
       collection = checkAndCompleteShardSplit(prevState, collection, coreNodeName, sliceName, replica);
       // get the current slice again because it may have been updated due to checkAndCompleteShardSplit method
       slice = collection.getSlice(sliceName);
@@ -340,15 +341,16 @@ public class ReplicaMutator {
    * Handles non-legacy state updates
    */
   protected ZkWriteCommand updateStateNew(ClusterState clusterState, final ZkNodeProps message) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
     String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP);
 
-    if (collection == null || sliceName == null) {
+    if (collectionName == null || sliceName == null) {
       log.error("Invalid collection and slice {}", message);
       return ZkStateWriter.NO_OP;
     }
-    Slice slice = clusterState.getSlice(collection, sliceName);
+    DocCollection collection = clusterState.getCollectionOrNull(collectionName);
+    Slice slice = collection != null ? collection.getSlice(sliceName) : null;
     if (slice == null) {
       log.error("No such slice exists {}", message);
       return ZkStateWriter.NO_OP;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
index 63c37ef..836d014 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/SliceMutator.java
@@ -69,7 +69,7 @@ public class SliceMutator {
       return ZkStateWriter.NO_OP;
     }
 
-    String coreNodeName = Assign.assignNode(coll, clusterState);
+    String coreNodeName = Assign.assignNode(collection);
     Replica replica = new Replica(coreNodeName,
         makeMap(
             ZkStateReader.CORE_NAME_PROP, message.getStr(ZkStateReader.CORE_NAME_PROP),
@@ -149,18 +149,19 @@ public class SliceMutator {
   }
 
   public ZkWriteCommand updateShardState(ClusterState clusterState, ZkNodeProps message) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
-    log.info("Update shard state invoked for collection: " + collection + " with message: " + message);
+    log.info("Update shard state invoked for collection: " + collectionName + " with message: " + message);
 
-    Map<String, Slice> slicesCopy = new LinkedHashMap<>(clusterState.getSlicesMap(collection));
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Map<String, Slice> slicesCopy = new LinkedHashMap<>(collection.getSlicesMap());
     for (String key : message.keySet()) {
       if (ZkStateReader.COLLECTION_PROP.equals(key)) continue;
       if (Overseer.QUEUE_OPERATION.equals(key)) continue;
 
-      Slice slice = clusterState.getSlice(collection, key);
+      Slice slice = collection.getSlice(key);
       if (slice == null) {
-        throw new RuntimeException("Overseer.updateShardState unknown collection: " + collection + " slice: " + key);
+        throw new RuntimeException("Overseer.updateShardState unknown collection: " + collectionName + " slice: " + key);
       }
       log.info("Update shard state " + key + " to " + message.getStr(key));
       Map<String, Object> props = slice.shallowCopy();
@@ -174,11 +175,11 @@ public class SliceMutator {
       slicesCopy.put(slice.getName(), newSlice);
     }
 
-    return new ZkWriteCommand(collection, clusterState.getCollection(collection).copyWithSlices(slicesCopy));
+    return new ZkWriteCommand(collectionName, collection.copyWithSlices(slicesCopy));
   }
 
   public ZkWriteCommand addRoutingRule(final ClusterState clusterState, ZkNodeProps message) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
     String shard = message.getStr(ZkStateReader.SHARD_ID_PROP);
     String routeKey = message.getStr("routeKey");
@@ -187,9 +188,10 @@ public class SliceMutator {
     String targetShard = message.getStr("targetShard");
     String expireAt = message.getStr("expireAt");
 
-    Slice slice = clusterState.getSlice(collection, shard);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice slice = collection.getSlice(shard);
     if (slice == null) {
-      throw new RuntimeException("Overseer.addRoutingRule unknown collection: " + collection + " slice:" + shard);
+      throw new RuntimeException("Overseer.addRoutingRule unknown collection: " + collectionName + " slice:" + shard);
     }
 
     Map<String, RoutingRule> routingRules = slice.getRoutingRules();
@@ -215,22 +217,23 @@ public class SliceMutator {
     props.put("routingRules", routingRules);
 
     Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props);
-    return new ZkWriteCommand(collection,
-        CollectionMutator.updateSlice(collection, clusterState.getCollection(collection), newSlice));
+    return new ZkWriteCommand(collectionName,
+        CollectionMutator.updateSlice(collectionName, collection, newSlice));
   }
 
   public ZkWriteCommand removeRoutingRule(final ClusterState clusterState, ZkNodeProps message) {
-    String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
+    String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     if (!checkCollectionKeyExistence(message)) return ZkStateWriter.NO_OP;
     String shard = message.getStr(ZkStateReader.SHARD_ID_PROP);
     String routeKeyStr = message.getStr("routeKey");
 
-    log.info("Overseer.removeRoutingRule invoked for collection: " + collection
+    log.info("Overseer.removeRoutingRule invoked for collection: " + collectionName
         + " shard: " + shard + " routeKey: " + routeKeyStr);
 
-    Slice slice = clusterState.getSlice(collection, shard);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice slice = collection.getSlice(shard);
     if (slice == null) {
-      log.warn("Unknown collection: " + collection + " shard: " + shard);
+      log.warn("Unknown collection: " + collectionName + " shard: " + shard);
       return ZkStateWriter.NO_OP;
     }
     Map<String, RoutingRule> routingRules = slice.getRoutingRules();
@@ -239,8 +242,8 @@ public class SliceMutator {
       Map<String, Object> props = slice.shallowCopy();
       props.put("routingRules", routingRules);
       Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props);
-      return new ZkWriteCommand(collection,
-          CollectionMutator.updateSlice(collection, clusterState.getCollection(collection), newSlice));
+      return new ZkWriteCommand(collectionName,
+          CollectionMutator.updateSlice(collectionName, collection, newSlice));
     }
 
     return ZkStateWriter.NO_OP;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
index ec67ed7..e9edef1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkStateWriter.java
@@ -18,6 +18,7 @@ package org.apache.solr.cloud.overseer;
 
 import java.lang.invoke.MethodHandles;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -227,8 +228,8 @@ public class ZkStateWriter {
           } else if (c.getStateFormat() > 1) {
             byte[] data = Utils.toJSON(singletonMap(c.getName(), c));
             if (reader.getZkClient().exists(path, true)) {
-              assert c.getZNodeVersion() >= 0;
               log.info("going to update_collection {} version: {}", path, c.getZNodeVersion());
+              assert c.getZNodeVersion() >= 0;
               Stat stat = reader.getZkClient().setData(path, data, c.getZNodeVersion(), true);
               DocCollection newCollection = new DocCollection(name, c.getSlicesMap(), c.getProperties(), c.getRouter(), stat.getVersion(), path);
               clusterState = clusterState.copyWith(name, newCollection);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index bb0cd05..b94b3d8 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -55,6 +55,7 @@ import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 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.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.params.CommonParams;
@@ -834,8 +835,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
 
       // ZK pre-register would have already happened so we read slice properties now
       final ClusterState clusterState = cc.getZkController().getClusterState();
-      final Slice slice = clusterState.getSlice(coreDescriptor.getCloudDescriptor().getCollectionName(), 
-          coreDescriptor.getCloudDescriptor().getShardId());
+      final DocCollection collection = clusterState.getCollection(coreDescriptor.getCloudDescriptor().getCollectionName());
+      final Slice slice = collection.getSlice(coreDescriptor.getCloudDescriptor().getShardId());
       if (slice.getState() == Slice.State.CONSTRUCTION) {
         // set update log to buffer before publishing the core
         getUpdateHandler().getUpdateLog().bufferUpdates();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java b/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
index 7071908..48bfec0 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrUpdateLogSynchronizer.java
@@ -28,6 +28,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CommonParams;
@@ -113,7 +114,8 @@ class CdcrUpdateLogSynchronizer implements CdcrStateManager.CdcrStateObserver {
     private String getLeaderUrl() {
       ZkController zkController = core.getCoreDescriptor().getCoreContainer().getZkController();
       ClusterState cstate = zkController.getClusterState();
-      ZkNodeProps leaderProps = cstate.getLeader(collection, shardId);
+      DocCollection docCollection = cstate.getCollection(collection);
+      ZkNodeProps leaderProps = docCollection.getLeader(shardId);
       if (leaderProps == null) { // we might not have a leader yet, returns null
         return null;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
index aa40046..c26c5a8 100644
--- a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java
@@ -55,6 +55,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
 import org.apache.solr.client.solrj.io.stream.metrics.*;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 64b10ab..edafc54 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -425,7 +425,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
 
         ClusterState clusterState = h.coreContainer.getZkController().getClusterState();
 
-        ZkNodeProps leaderProps = clusterState.getLeader(collection, shard);
+        DocCollection docCollection = clusterState.getCollection(collection);
+        ZkNodeProps leaderProps = docCollection.getLeader(shard);
         ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
 
         try (HttpSolrClient client = new Builder(nodeProps.getBaseUrl()).build()) {
@@ -828,18 +829,15 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
 
   private static void forceLeaderElection(SolrQueryRequest req, CollectionsHandler handler) {
     ClusterState clusterState = handler.coreContainer.getZkController().getClusterState();
-    String collection = req.getParams().required().get(COLLECTION_PROP);
+    String collectionName = req.getParams().required().get(COLLECTION_PROP);
     String sliceId = req.getParams().required().get(SHARD_ID_PROP);
 
     log.info("Force leader invoked, state: {}", clusterState);
-    Slice slice = clusterState.getSlice(collection, sliceId);
+    DocCollection collection = clusterState.getCollection(collectionName);
+    Slice slice = collection.getSlice(sliceId);
     if (slice == null) {
-      if (clusterState.hasCollection(collection)) {
-        throw new SolrException(ErrorCode.BAD_REQUEST,
-            "No shard with name " + sliceId + " exists for collection " + collection);
-      } else {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "No collection with the specified name exists: " + collection);
-      }
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "No shard with name " + sliceId + " exists for collection " + collectionName);
     }
 
     try {
@@ -851,7 +849,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       }
 
       // Clear out any LIR state
-      String lirPath = handler.coreContainer.getZkController().getLeaderInitiatedRecoveryZnodePath(collection, sliceId);
+      String lirPath = handler.coreContainer.getZkController().getLeaderInitiatedRecoveryZnodePath(collectionName, sliceId);
       if (handler.coreContainer.getZkController().getZkClient().exists(lirPath, true)) {
         StringBuilder sb = new StringBuilder();
         handler.coreContainer.getZkController().getZkClient().printLayout(lirPath, 4, sb);
@@ -880,7 +878,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       for (int i = 0; i < 9; i++) {
         Thread.sleep(5000);
         clusterState = handler.coreContainer.getZkController().getClusterState();
-        slice = clusterState.getSlice(collection, sliceId);
+        collection = clusterState.getCollection(collectionName);
+        slice = collection.getSlice(sliceId);
         if (slice.getLeader() != null && slice.getLeader().getState() == State.ACTIVE) {
           success = true;
           break;
@@ -889,15 +888,15 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       }
 
       if (success) {
-        log.info("Successfully issued FORCELEADER command for collection: {}, shard: {}", collection, sliceId);
+        log.info("Successfully issued FORCELEADER command for collection: {}, shard: {}", collectionName, sliceId);
       } else {
-        log.info("Couldn't successfully force leader, collection: {}, shard: {}. Cluster state: {}", collection, sliceId, clusterState);
+        log.info("Couldn't successfully force leader, collection: {}, shard: {}. Cluster state: {}", collectionName, sliceId, clusterState);
       }
     } catch (SolrException e) {
       throw e;
     } catch (Exception e) {
       throw new SolrException(ErrorCode.SERVER_ERROR,
-          "Error executing FORCELEADER operation for collection: " + collection + " shard: " + sliceId, e);
+          "Error executing FORCELEADER operation for collection: " + collectionName + " shard: " + sliceId, e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
index e755b82..bdc9168 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
@@ -369,7 +369,7 @@ enum CoreAdminOperation {
           String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
           DocCollection collection = clusterState.getCollection(collectionName);
           String sliceName = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
-          Slice slice = clusterState.getSlice(collectionName, sliceName);
+          Slice slice = collection.getSlice(sliceName);
           router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
           if (ranges == null) {
             DocRouter.Range currentRange = slice.getRange();
@@ -461,7 +461,7 @@ enum CoreAdminOperation {
             // to accept updates
             CloudDescriptor cloudDescriptor = core.getCoreDescriptor()
                 .getCloudDescriptor();
-            String collection = cloudDescriptor.getCollectionName();
+            String collectionName = cloudDescriptor.getCollectionName();
 
             if (retry % 15 == 0) {
               if (retry > 0 && log.isInfoEnabled())
@@ -471,7 +471,7 @@ enum CoreAdminOperation {
                     waitForState + "; forcing ClusterState update from ZooKeeper");
 
               // force a cluster state update
-              coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collection);
+              coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName);
             }
 
             if (maxTries == 0) {
@@ -484,7 +484,8 @@ enum CoreAdminOperation {
             }
 
             ClusterState clusterState = coreContainer.getZkController().getClusterState();
-            Slice slice = clusterState.getSlice(collection, cloudDescriptor.getShardId());
+            DocCollection collection = clusterState.getCollection(collectionName);
+            Slice slice = collection.getSlice(cloudDescriptor.getShardId());
             if (slice != null) {
               final Replica replica = slice.getReplicasMap().get(coreNodeName);
               if (replica != null) {
@@ -508,7 +509,7 @@ enum CoreAdminOperation {
                 }
 
                 boolean onlyIfActiveCheckResult = onlyIfLeaderActive != null && onlyIfLeaderActive && localState != Replica.State.ACTIVE;
-                log.info("In WaitForState(" + waitForState + "): collection=" + collection + ", shard=" + slice.getName() +
+                log.info("In WaitForState(" + waitForState + "): collection=" + collectionName + ", shard=" + slice.getName() +
                     ", thisCore=" + core.getName() + ", leaderDoesNotNeedRecovery=" + leaderDoesNotNeedRecovery +
                     ", isLeader? " + core.getCoreDescriptor().getCloudDescriptor().isLeader() +
                     ", live=" + live + ", checkLive=" + checkLive + ", currentState=" + state.toString() + ", localState=" + localState + ", nodeName=" + nodeName +

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index eb90762..6ba571a 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -66,6 +66,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Aliases;
 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.ZkNodeProps;
@@ -756,11 +757,15 @@ public class HttpSolrCall {
     return result;
   }
 
-  private SolrCore getCoreByCollection(String collection) {
+  private SolrCore getCoreByCollection(String collectionName) {
     ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
 
     ClusterState clusterState = zkStateReader.getClusterState();
-    Map<String, Slice> slices = clusterState.getActiveSlicesMap(collection);
+    DocCollection collection = clusterState.getCollectionOrNull(collectionName);
+    if (collection == null) {
+      return null;
+    }
+    Map<String, Slice> slices = collection.getActiveSlicesMap();
     if (slices == null) {
       return null;
     }
@@ -773,7 +778,7 @@ public class HttpSolrCall {
     //For queries it doesn't matter and hence we don't distinguish here.
     for (Map.Entry<String, Slice> entry : entries) {
       // first see if we have the leader
-      Replica leaderProps = clusterState.getLeader(collection, entry.getKey());
+      Replica leaderProps = collection.getLeader(entry.getKey());
       if (leaderProps != null && liveNodes.contains(leaderProps.getNodeName()) && leaderProps.getState() == Replica.State.ACTIVE) {
         core = checkProps(leaderProps);
         if (core != null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 5f4e4f1..67c88dd 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -547,8 +547,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
                       throw new SolrException(ErrorCode.SERVER_ERROR,
                           "No active slices serving " + id + " found for target collection: " + rule.getTargetCollectionName());
                     }
-                    Replica targetLeader = cstate.getLeader(rule.getTargetCollectionName(), activeSlices.iterator().next().getName());
-                    if (nodes == null) nodes = new ArrayList<>(1);
+                    Replica targetLeader = targetColl.getLeader(activeSlices.iterator().next().getName());
+                    nodes = new ArrayList<>(1);
                     nodes.add(new StdNode(new ZkCoreNodeProps(targetLeader)));
                     break;
                   }
@@ -596,7 +596,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     ClusterState clusterState = zkController.getClusterState();
         
     CloudDescriptor cloudDescriptor = req.getCore().getCoreDescriptor().getCloudDescriptor();
-    Slice mySlice = clusterState.getSlice(collection, cloudDescriptor.getShardId());
+    DocCollection docCollection = clusterState.getCollection(collection);
+    Slice mySlice = docCollection.getSlice(cloudDescriptor.getShardId());
     boolean localIsLeader = cloudDescriptor.isLeader();
     if (DistribPhase.FROMLEADER == phase && localIsLeader && from != null) { // from will be null on log replay
       String fromShard = req.getParams().get(DISTRIB_FROM_PARENT);
@@ -606,7 +607,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
               "Request says it is coming from parent shard leader but we are in active state");
         }
         // shard splitting case -- check ranges to see if we are a sub-shard
-        Slice fromSlice = zkController.getClusterState().getCollection(collection).getSlice(fromShard);
+        Slice fromSlice = docCollection.getSlice(fromShard);
         DocRouter.Range parentRange = fromSlice.getRange();
         if (parentRange == null) parentRange = new DocRouter.Range(Integer.MIN_VALUE, Integer.MAX_VALUE);
         if (mySlice.getRange() != null && !mySlice.getRange().isSubsetOf(parentRange)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/util/SolrCLI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
index 9627671..d21400d 100644
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -88,6 +88,7 @@ import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 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.ZkCoreNodeProps;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java b/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java
index 8b1ca1f..b79ec0c 100644
--- a/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrLogLayout.java
@@ -28,6 +28,7 @@ import org.apache.log4j.spi.ThrowableInformation;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.StringUtils;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.SolrCore;
@@ -236,10 +237,11 @@ public class SolrLogLayout extends Layout {
     return sb.toString();
   }
 
-  private Map<String,Object> getReplicaProps(ZkController zkController, SolrCore core) {
-    final String collection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
-    Replica replica = zkController.getClusterState().getReplica(collection, zkController.getCoreNodeName(core.getCoreDescriptor()));
-    if(replica!=null) {
+  private Map<String, Object> getReplicaProps(ZkController zkController, SolrCore core) {
+    final String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
+    DocCollection collection = zkController.getClusterState().getCollectionOrNull(collectionName);
+    Replica replica = collection.getReplica(zkController.getCoreNodeName(core.getCoreDescriptor()));
+    if (replica != null) {
       return replica.getProperties();
     }
     return Collections.EMPTY_MAP;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
index 6069650..7593f3b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AssignTest.java
@@ -82,7 +82,7 @@ public class AssignTest extends SolrTestCaseJ4 {
     
     Set<String> liveNodes = new HashSet<>();
     ClusterState state = new ClusterState(-1,liveNodes, collectionStates);
-    String nodeName = Assign.assignNode("collection1", state);
+    String nodeName = Assign.assignNode(state.getCollection("collection1"));
     
     assertEquals("core_node2", nodeName);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
index accc36a..7dd77d9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java
@@ -1193,12 +1193,12 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
     String collectionName = "addReplicaColl";
     try (CloudSolrClient client = createCloudClient(null)) {
       createCollection(collectionName, client, 2, 2);
-      String newReplicaName = Assign.assignNode(collectionName, client.getZkStateReader().getClusterState());
+      String newReplicaName = Assign.assignNode(client.getZkStateReader().getClusterState().getCollection(collectionName));
       ArrayList<String> nodeList = new ArrayList<>(client.getZkStateReader().getClusterState().getLiveNodes());
       Collections.shuffle(nodeList, random());
 
       Replica newReplica = doAddReplica(collectionName, "shard1",
-          Assign.assignNode(collectionName, client.getZkStateReader().getClusterState()),
+          Assign.assignNode(client.getZkStateReader().getClusterState().getCollection(collectionName)),
           nodeList.get(0), client, null);
 
       log.info("newReplica {},\n{} ", newReplica, client.getZkStateReader().getBaseUrlForNodeName(nodeList.get(0)));
@@ -1210,7 +1210,7 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
       String instancePathStr = createTempDir().toString();
       props.put(CoreAdminParams.INSTANCE_DIR, instancePathStr); //Use name via the property.instanceDir method
       newReplica = doAddReplica(collectionName, "shard2",
-          Assign.assignNode(collectionName, client.getZkStateReader().getClusterState()),
+          Assign.assignNode(client.getZkStateReader().getClusterState().getCollection(collectionName)),
           null, client, props);
       assertNotNull(newReplica);
 
@@ -1244,7 +1244,7 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
       props.put(CoreAdminParams.NAME, "propertyDotName");
 
       newReplica = doAddReplica(collectionName, "shard1",
-          Assign.assignNode(collectionName, client.getZkStateReader().getClusterState()),
+          Assign.assignNode(client.getZkStateReader().getClusterState().getCollection(collectionName)),
           nodeList.get(0), client, props);
       assertEquals("'core' should be 'propertyDotName' ", "propertyDotName", newReplica.getStr("core"));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/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 52ae96f..0975b9a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -276,7 +276,7 @@ public class CollectionsAPISolrJTest extends AbstractFullDistribZkTestBase {
 
     cloudClient.setDefaultCollection(collectionName);
 
-    String newReplicaName = Assign.assignNode(collectionName, cloudClient.getZkStateReader().getClusterState());
+    String newReplicaName = Assign.assignNode(cloudClient.getZkStateReader().getClusterState().getCollection(collectionName));
     ArrayList<String> nodeList = new ArrayList<>(cloudClient.getZkStateReader().getClusterState().getLiveNodes());
     Collections.shuffle(nodeList, random());
     CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
index 85a88ec..7863899 100644
--- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
@@ -25,6 +25,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -687,7 +688,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
       while (version == getClusterStateVersion(zkClient));
       Thread.sleep(500);
       assertTrue(collection+" should remain after removal of the last core", // as of SOLR-5209 core removal does not cascade to remove the slice and collection
-          reader.getClusterState().getCollections().contains(collection));
+          reader.getClusterState().hasCollection(collection));
       assertTrue(core_node+" should be gone after publishing the null state",
           null == reader.getClusterState().getCollection(collection).getReplica(core_node));
     } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
index 9eca6db..893b6fe 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java
@@ -30,6 +30,7 @@ import org.apache.solr.client.solrj.io.stream.ExceptionStream;
 import org.apache.solr.client.solrj.io.stream.SolrStream;
 import org.apache.solr.client.solrj.io.stream.TupleStream;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.params.CommonParams;
 import org.junit.After;
 import org.junit.AfterClass;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
index 1a6139e..97a804d 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
@@ -51,6 +51,7 @@ import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrInputDocument;
 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.ZkCoreNodeProps;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
index 2495c41..d60cccb 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
@@ -90,7 +90,9 @@ public class ClusterState implements JSONWriter.Writable {
 
   /**
    * Get the lead replica for specific collection, or null if one currently doesn't exist.
+   * @deprecated Use {@link DocCollection#getLeader(String)} instead
    */
+  @Deprecated
   public Replica getLeader(String collection, String sliceName) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
@@ -98,14 +100,6 @@ public class ClusterState implements JSONWriter.Writable {
     if (slice == null) return null;
     return slice.getLeader();
   }
-  private Replica getReplica(DocCollection coll, String replicaName) {
-    if (coll == null) return null;
-    for (Slice slice : coll.getSlices()) {
-      Replica replica = slice.getReplica(replicaName);
-      if (replica != null) return replica;
-    }
-    return null;
-  }
 
   /**
    * Returns true if the specified collection name exists, false otherwise.
@@ -113,48 +107,76 @@ public class ClusterState implements JSONWriter.Writable {
    * Implementation note: This method resolves the collection reference by calling
    * {@link CollectionRef#get()} which can make a call to ZooKeeper. This is necessary
    * because the semantics of how collection list is loaded have changed in SOLR-6629.
-   * Please javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
+   * Please see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
    */
   public boolean hasCollection(String collectionName) {
     return getCollectionOrNull(collectionName) != null;
   }
 
   /**
-   * Gets the replica by the core name (assuming the slice is unknown) or null if replica is not found.
+   * Gets the replica by the core node name (assuming the slice is unknown) or null if replica is not found.
    * If the slice is known, do not use this method.
    * coreNodeName is the same as replicaName
+   *
+   * @deprecated use {@link DocCollection#getReplica(String)} instead
    */
+  @Deprecated
   public Replica getReplica(final String collection, final String coreNodeName) {
-    return getReplica(getCollectionOrNull(collection), coreNodeName);
+    DocCollection coll = getCollectionOrNull(collection);
+    if (coll == null) return null;
+    for (Slice slice : coll.getSlices()) {
+      Replica replica = slice.getReplica(coreNodeName);
+      if (replica != null) return replica;
+    }
+    return null;
   }
 
   /**
    * Get the named Slice for collection, or null if not found.
+   *
+   * @deprecated use {@link DocCollection#getSlice(String)} instead
    */
+  @Deprecated
   public Slice getSlice(String collection, String sliceName) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
     return coll.getSlice(sliceName);
   }
 
+  /**
+   * @deprecated use {@link DocCollection#getSlicesMap()} instead
+   */
+  @Deprecated
   public Map<String, Slice> getSlicesMap(String collection) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
     return coll.getSlicesMap();
   }
-  
+
+  /**
+   * @deprecated use {@link DocCollection#getActiveSlicesMap()} instead
+   */
+  @Deprecated
   public Map<String, Slice> getActiveSlicesMap(String collection) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
     return coll.getActiveSlicesMap();
   }
 
+  /**
+   * @deprecated use {@link DocCollection#getSlices()} instead
+   */
+  @Deprecated
   public Collection<Slice> getSlices(String collection) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
     return coll.getSlices();
   }
 
+  /**
+   * @deprecated use {@link DocCollection#getActiveSlices()} instead
+   */
+  @Deprecated
   public Collection<Slice> getActiveSlices(String collection) {
     DocCollection coll = getCollectionOrNull(collection);
     if (coll == null) return null;
@@ -195,7 +217,7 @@ public class ClusterState implements JSONWriter.Writable {
    * Implementation note: This method resolves the collection reference by calling
    * {@link CollectionRef#get()} which can make a call to ZooKeeper. This is necessary
    * because the semantics of how collection list is loaded have changed in SOLR-6629.
-   * Please javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
+   * Please see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
    */
   public Set<String> getCollections() {
     Set<String> result = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
index 1628756..a298cb3 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
@@ -20,6 +20,7 @@ import java.lang.invoke.MethodHandles;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
index e8f26e1..9e4418c 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
@@ -209,4 +209,10 @@ public class DocCollection extends ZkNodeProps {
     }
     return null;
   }
+
+  public Replica getLeader(String sliceName) {
+    Slice slice = getSlice(sliceName);
+    if (slice == null) return null;
+    return slice.getLeader();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index c6f88c0..568c791 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -646,7 +646,8 @@ public class ZkStateReader implements Closeable {
 
   public Replica getLeader(String collection, String shard) {
     if (clusterState != null) {
-      Replica replica = clusterState.getLeader(collection, shard);
+      DocCollection docCollection = clusterState.getCollectionOrNull(collection);
+      Replica replica = docCollection != null ? docCollection.getLeader(shard) : null;
       if (replica != null && getClusterState().liveNodesContain(replica.getNodeName())) {
         return replica;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/922265b4/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
index 9050092..d38661e 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/sql/JdbcTest.java
@@ -34,6 +34,7 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
 import org.apache.solr.cloud.AbstractZkTestCase;
+import org.apache.solr.common.cloud.DocCollection;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;