mirror of https://github.com/apache/lucene.git
SOLR-9014: Deprecate ClusterState.getCollections and introduce a new ClusterState.getCollectionsMap instead
(cherry picked from commit f5497a3
)
This commit is contained in:
parent
ee45e83439
commit
7bfaa51079
|
@ -233,8 +233,9 @@ public class Assign {
|
|||
}
|
||||
DocCollection coll = clusterState.getCollection(collectionName);
|
||||
Integer maxShardsPerNode = coll.getInt(MAX_SHARDS_PER_NODE, 1);
|
||||
for (String s : clusterState.getCollections()) {
|
||||
DocCollection c = clusterState.getCollection(s);
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection c = entry.getValue();
|
||||
//identify suitable nodes by checking the no:of cores in each of them
|
||||
for (Slice slice : c.getSlices()) {
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
|
@ -242,7 +243,7 @@ public class Assign {
|
|||
ReplicaCount count = nodeNameVsShardCount.get(replica.getNodeName());
|
||||
if (count != null) {
|
||||
count.totalNodes++; // Used ot "weigh" whether this node should be used later.
|
||||
if (s.equals(collectionName)) {
|
||||
if (entry.getKey().equals(collectionName)) {
|
||||
count.thisCollectionNodes++;
|
||||
if (count.thisCollectionNodes >= maxShardsPerNode) nodeNameVsShardCount.remove(replica.getNodeName());
|
||||
}
|
||||
|
|
|
@ -162,10 +162,10 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
|
|||
|
||||
liveNodes = clusterState.getLiveNodes();
|
||||
lastClusterStateVersion = clusterState.getZkClusterStateVersion();
|
||||
Set<String> collections = clusterState.getCollections();
|
||||
for (final String collection : collections) {
|
||||
log.debug("look at collection={}", collection);
|
||||
DocCollection docCollection = clusterState.getCollection(collection);
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
log.debug("look at collection={}", entry.getKey());
|
||||
DocCollection docCollection = entry.getValue();
|
||||
if (!docCollection.getAutoAddReplicas()) {
|
||||
log.debug("Collection {} is not setup to use autoAddReplicas, skipping..", docCollection.getName());
|
||||
continue;
|
||||
|
@ -174,7 +174,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
|
|||
log.debug("Skipping collection because it has no defined replicationFactor, name={}", docCollection.getName());
|
||||
continue;
|
||||
}
|
||||
log.debug("Found collection, name={} replicationFactor={}", collection, docCollection.getReplicationFactor());
|
||||
log.debug("Found collection, name={} replicationFactor={}", entry.getKey(), docCollection.getReplicationFactor());
|
||||
|
||||
Collection<Slice> slices = docCollection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
|
@ -188,7 +188,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
|
|||
|
||||
if (downReplicas.size() > 0 && goodReplicas < docCollection.getReplicationFactor()) {
|
||||
// badReplicaMap.put(collection, badReplicas);
|
||||
processBadReplicas(collection, downReplicas);
|
||||
processBadReplicas(entry.getKey(), downReplicas);
|
||||
} else if (goodReplicas > docCollection.getReplicationFactor()) {
|
||||
log.debug("There are too many replicas");
|
||||
}
|
||||
|
@ -313,10 +313,11 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
|
|||
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
if (clusterState != null) {
|
||||
Set<String> collections = clusterState.getCollections();
|
||||
for (String collection : collections) {
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
String collection = entry.getKey();
|
||||
log.debug("look at collection {} as possible create candidate", collection);
|
||||
DocCollection docCollection = clusterState.getCollection(collection);
|
||||
DocCollection docCollection = entry.getValue();
|
||||
// TODO - only operate on collections with sharedfs failover = true ??
|
||||
Collection<Slice> slices = docCollection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
|
|
|
@ -350,10 +350,10 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
|
|||
throw new SolrException(ErrorCode.BAD_REQUEST, "ConfigSet does not exist to delete: " + configSetName);
|
||||
}
|
||||
|
||||
for (String s : zkStateReader.getClusterState().getCollections()) {
|
||||
if (configSetName.equals(zkStateReader.readConfigName(s)))
|
||||
for (Map.Entry<String, DocCollection> entry : zkStateReader.getClusterState().getCollectionsMap().entrySet()) {
|
||||
if (configSetName.equals(zkStateReader.readConfigName(entry.getKey())))
|
||||
throw new SolrException(ErrorCode.BAD_REQUEST,
|
||||
"Can not delete ConfigSet as it is currently being used by collection [" + s + "]");
|
||||
"Can not delete ConfigSet as it is currently being used by collection [" + entry.getKey() + "]");
|
||||
}
|
||||
|
||||
String propertyPath = ConfigSetProperties.DEFAULT_FILENAME;
|
||||
|
|
|
@ -689,14 +689,12 @@ public final class ZkController {
|
|||
long now = System.nanoTime();
|
||||
long timeout = now + TimeUnit.NANOSECONDS.convert(WAIT_DOWN_STATES_TIMEOUT_SECONDS, TimeUnit.SECONDS);
|
||||
boolean foundStates = true;
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Set<String> collections = clusterState.getCollections();
|
||||
|
||||
while (System.nanoTime() < timeout) {
|
||||
clusterState = zkStateReader.getClusterState();
|
||||
collections = clusterState.getCollections();
|
||||
for (String collectionName : collections) {
|
||||
DocCollection collection = clusterState.getCollection(collectionName);
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection collection = entry.getValue();
|
||||
Collection<Slice> slices = collection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
|
|
|
@ -49,14 +49,13 @@ public class NodeMutator {
|
|||
|
||||
log.info("DownNode state invoked for node: " + nodeName);
|
||||
|
||||
Set<String> collections = clusterState.getCollections();
|
||||
for (String collection : collections) {
|
||||
DocCollection docCollection = clusterState.getCollection(collection);
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection docCollection = entry.getValue();
|
||||
Map<String,Slice> slicesCopy = new LinkedHashMap<>(docCollection.getSlicesMap());
|
||||
|
||||
Set<Entry<String,Slice>> entries = slicesCopy.entrySet();
|
||||
for (Entry<String,Slice> entry : entries) {
|
||||
Slice slice = docCollection.getSlice(entry.getKey());
|
||||
for (Entry<String,Slice> sliceEntry : slicesCopy.entrySet()) {
|
||||
Slice slice = docCollection.getSlice(sliceEntry.getKey());
|
||||
Map<String,Replica> newReplicas = new HashMap<String,Replica>();
|
||||
|
||||
Collection<Replica> replicas = slice.getReplicas();
|
||||
|
@ -77,7 +76,7 @@ public class NodeMutator {
|
|||
|
||||
}
|
||||
|
||||
zkWriteCommands.add(new ZkWriteCommand(collection, docCollection.copyWithSlices(slicesCopy)));
|
||||
zkWriteCommands.add(new ZkWriteCommand(entry.getKey(), docCollection.copyWithSlices(slicesCopy)));
|
||||
}
|
||||
|
||||
return zkWriteCommands;
|
||||
|
|
|
@ -250,13 +250,9 @@ public class ZkStateWriter {
|
|||
assert clusterState.getZkClusterStateVersion() >= 0;
|
||||
byte[] data = Utils.toJSON(clusterState);
|
||||
Stat stat = reader.getZkClient().setData(ZkStateReader.CLUSTER_STATE, data, clusterState.getZkClusterStateVersion(), true);
|
||||
Set<String> collectionNames = clusterState.getCollections();
|
||||
Map<String, DocCollection> collectionStates = new HashMap<>(collectionNames.size());
|
||||
for (String c : collectionNames) {
|
||||
collectionStates.put(c, clusterState.getCollection(c));
|
||||
}
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
// use the reader's live nodes because our cluster state's live nodes may be stale
|
||||
clusterState = new ClusterState(stat.getVersion(), reader.getClusterState().getLiveNodes(), collectionStates);
|
||||
clusterState = new ClusterState(stat.getVersion(), reader.getClusterState().getLiveNodes(), collections);
|
||||
isClusterStateModified = false;
|
||||
}
|
||||
lastUpdatedTime = System.nanoTime();
|
||||
|
|
|
@ -104,8 +104,9 @@ public class ReplicaAssigner {
|
|||
validateTags(nodeVsTags);
|
||||
|
||||
if (clusterState != null) {
|
||||
for (String s : clusterState.getCollections()) {
|
||||
DocCollection coll = clusterState.getCollection(s);
|
||||
Map<String, DocCollection> collections = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection coll = entry.getValue();
|
||||
for (Slice slice : coll.getSlices()) {
|
||||
for (Replica replica : slice.getReplicas()) {
|
||||
AtomicInteger count = nodeVsCores.get(replica.getNodeName());
|
||||
|
|
|
@ -1515,8 +1515,9 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware , Pe
|
|||
CloudSolrClient cloudSolrClient = this.context.getSolrClientCache().getCloudSolrClient(this.zkHost);
|
||||
cloudSolrClient.connect();
|
||||
ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
|
||||
if (zkStateReader.getClusterState().getCollections().size() != 0) {
|
||||
this.tables.addAll(zkStateReader.getClusterState().getCollections());
|
||||
Map<String, DocCollection> collections = zkStateReader.getClusterState().getCollectionsMap();
|
||||
if (collections.size() != 0) {
|
||||
this.tables.addAll(collections.keySet());
|
||||
}
|
||||
Collections.sort(this.tables);
|
||||
}
|
||||
|
|
|
@ -89,20 +89,22 @@ public class ClusterStatus {
|
|||
byte[] bytes = Utils.toJSON(clusterState);
|
||||
Map<String, Object> stateMap = (Map<String,Object>) Utils.fromJSON(bytes);
|
||||
|
||||
Set<String> collections;
|
||||
String routeKey = message.getStr(ShardParams._ROUTE_);
|
||||
String shard = message.getStr(ZkStateReader.SHARD_ID_PROP);
|
||||
|
||||
Map<String, DocCollection> collectionsMap = null;
|
||||
if (collection == null) {
|
||||
collections = new HashSet<>(clusterState.getCollections());
|
||||
collectionsMap = clusterState.getCollectionsMap();
|
||||
} else {
|
||||
collections = Collections.singleton(collection);
|
||||
collectionsMap = Collections.singletonMap(collection, clusterState.getCollectionOrNull(collection));
|
||||
}
|
||||
|
||||
NamedList<Object> collectionProps = new SimpleOrderedMap<>();
|
||||
|
||||
for (String name : collections) {
|
||||
for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
|
||||
Map<String, Object> collectionStatus;
|
||||
DocCollection clusterStateCollection = clusterState.getCollectionOrNull(name);
|
||||
String name = entry.getKey();
|
||||
DocCollection clusterStateCollection = entry.getValue();
|
||||
if (clusterStateCollection == null) {
|
||||
if (collection != null) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + name + " not found");
|
||||
|
|
|
@ -679,11 +679,8 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
|
|||
@Override
|
||||
Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, CollectionsHandler handler) throws Exception {
|
||||
NamedList<Object> results = new NamedList<>();
|
||||
Set<String> collections = handler.coreContainer.getZkController().getZkStateReader().getClusterState().getCollections();
|
||||
List<String> collectionList = new ArrayList<>();
|
||||
for (String collection : collections) {
|
||||
collectionList.add(collection);
|
||||
}
|
||||
Map<String, DocCollection> collections = handler.coreContainer.getZkController().getZkStateReader().getClusterState().getCollectionsMap();
|
||||
List<String> collectionList = new ArrayList<>(collections.keySet());
|
||||
results.add("collections", collectionList);
|
||||
SolrResponse response = new OverseerSolrResponse(results);
|
||||
rsp.getValues().addAll(response.getResponse());
|
||||
|
|
|
@ -814,15 +814,15 @@ public class HttpSolrCall {
|
|||
private void getSlicesForCollections(ClusterState clusterState,
|
||||
Collection<Slice> slices, boolean activeSlices) {
|
||||
if (activeSlices) {
|
||||
for (String collection : clusterState.getCollections()) {
|
||||
final Collection<Slice> activeCollectionSlices = clusterState.getActiveSlices(collection);
|
||||
for (Map.Entry<String, DocCollection> entry : clusterState.getCollectionsMap().entrySet()) {
|
||||
final Collection<Slice> activeCollectionSlices = entry.getValue().getActiveSlices();
|
||||
if (activeCollectionSlices != null) {
|
||||
slices.addAll(activeCollectionSlices);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (String collection : clusterState.getCollections()) {
|
||||
final Collection<Slice> collectionSlices = clusterState.getSlices(collection);
|
||||
for (Map.Entry<String, DocCollection> entry : clusterState.getCollectionsMap().entrySet()) {
|
||||
final Collection<Slice> collectionSlices = entry.getValue().getSlices();
|
||||
if (collectionSlices != null) {
|
||||
slices.addAll(collectionSlices);
|
||||
}
|
||||
|
|
|
@ -1884,7 +1884,7 @@ public class SolrCLI {
|
|||
log.warn("Skipping safety checks, configuration directory "+configName+" will be deleted with impunity.");
|
||||
} else {
|
||||
// need to scan all Collections to see if any are using the config
|
||||
Set<String> collections = zkStateReader.getClusterState().getCollections();
|
||||
Set<String> collections = zkStateReader.getClusterState().getCollectionsMap().keySet();
|
||||
|
||||
// give a little note to the user if there are many collections in case it takes a while
|
||||
if (collections.size() > 50)
|
||||
|
|
|
@ -66,21 +66,21 @@ public class ClusterStateTest extends SolrTestCaseJ4 {
|
|||
|
||||
assertEquals("Provided liveNodes not used properly", 2, loadedClusterState
|
||||
.getLiveNodes().size());
|
||||
assertEquals("No collections found", 2, loadedClusterState.getCollections().size());
|
||||
assertEquals("Poperties not copied properly", replica.getStr("prop1"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop1"));
|
||||
assertEquals("Poperties not copied properly", replica.getStr("prop2"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop2"));
|
||||
assertEquals("No collections found", 2, loadedClusterState.getCollectionsMap().size());
|
||||
assertEquals("Properties not copied properly", replica.getStr("prop1"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop1"));
|
||||
assertEquals("Properties not copied properly", replica.getStr("prop2"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop2"));
|
||||
|
||||
loadedClusterState = ClusterState.load(-1, new byte[0], liveNodes);
|
||||
|
||||
assertEquals("Provided liveNodes not used properly", 2, loadedClusterState
|
||||
.getLiveNodes().size());
|
||||
assertEquals("Should not have collections", 0, loadedClusterState.getCollections().size());
|
||||
assertEquals("Should not have collections", 0, loadedClusterState.getCollectionsMap().size());
|
||||
|
||||
loadedClusterState = ClusterState.load(-1, (byte[])null, liveNodes);
|
||||
|
||||
assertEquals("Provided liveNodes not used properly", 2, loadedClusterState
|
||||
.getLiveNodes().size());
|
||||
assertEquals("Should not have collections", 0, loadedClusterState.getCollections().size());
|
||||
assertEquals("Should not have collections", 0, loadedClusterState.getCollectionsMap().size());
|
||||
}
|
||||
|
||||
public static ZkStateReader getMockZkStateReader(final Set<String> collections) {
|
||||
|
|
|
@ -1087,7 +1087,7 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa
|
|||
}
|
||||
} else {
|
||||
throw new IllegalArgumentException("Could not find collection in :"
|
||||
+ clusterState.getCollections());
|
||||
+ clusterState.getCollectionsMap());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -525,7 +525,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
|||
int maxIterations = 100;
|
||||
while (0 < maxIterations--) {
|
||||
final ClusterState state = stateReader.getClusterState();
|
||||
Set<String> availableCollections = state.getCollections();
|
||||
Set<String> availableCollections = state.getCollectionsMap().keySet();
|
||||
int availableCount = 0;
|
||||
for(String requiredCollection: collections) {
|
||||
if(availableCollections.contains(requiredCollection)) {
|
||||
|
@ -911,8 +911,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
|||
ClusterState state = reader.getClusterState();
|
||||
|
||||
int numFound = 0;
|
||||
for (String c : state.getCollections()) {
|
||||
DocCollection collection = state.getCollection(c);
|
||||
Map<String, DocCollection> collectionsMap = state.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
|
||||
DocCollection collection = entry.getValue();
|
||||
for (Slice slice : collection.getSlices()) {
|
||||
if (slice.getReplicasMap().get("core_node1") != null) {
|
||||
numFound++;
|
||||
|
|
|
@ -2496,7 +2496,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase {
|
|||
assertEquals(2, tuples.size());
|
||||
|
||||
List<String> collections = new ArrayList<>();
|
||||
collections.addAll(cloudClient.getZkStateReader().getClusterState().getCollections());
|
||||
collections.addAll(cloudClient.getZkStateReader().getClusterState().getCollectionsMap().keySet());
|
||||
Collections.sort(collections);
|
||||
for (Tuple tuple : tuples) {
|
||||
assertEquals(zkServer.getZkAddress(), tuple.getString("TABLE_CAT"));
|
||||
|
@ -2510,7 +2510,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase {
|
|||
assertEquals(2, tuples.size());
|
||||
|
||||
collections = new ArrayList<>();
|
||||
collections.addAll(cloudClient.getZkStateReader().getClusterState().getCollections());
|
||||
collections.addAll(cloudClient.getZkStateReader().getClusterState().getCollectionsMap().keySet());
|
||||
Collections.sort(collections);
|
||||
for (Tuple tuple : tuples) {
|
||||
assertEquals(zkServer.getZkAddress(), tuple.getString("TABLE_CAT"));
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParamete
|
|||
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
|
||||
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
|
||||
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;
|
||||
|
@ -329,16 +330,17 @@ public class CloudSolrStream extends TupleStream implements Expressible {
|
|||
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
|
||||
|
||||
if(slices == null) {
|
||||
if (slices == null) {
|
||||
//Try case insensitive match
|
||||
for(String col : clusterState.getCollections()) {
|
||||
if(col.equalsIgnoreCase(collection)) {
|
||||
slices = clusterState.getActiveSlices(col);
|
||||
Map<String, DocCollection> collectionsMap = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
|
||||
if (entry.getKey().equalsIgnoreCase(collection)) {
|
||||
slices = entry.getValue().getActiveSlices();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if(slices == null) {
|
||||
if (slices == null) {
|
||||
throw new Exception("Collection not found:" + this.collection);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -463,16 +463,17 @@ public class TopicStream extends CloudSolrStream implements Expressible {
|
|||
|
||||
Collection<Slice> slices = clusterState.getActiveSlices(this.collection);
|
||||
|
||||
if(slices == null) {
|
||||
if (slices == null) {
|
||||
//Try case insensitive match
|
||||
for(String col : clusterState.getCollections()) {
|
||||
if(col.equalsIgnoreCase(collection)) {
|
||||
slices = clusterState.getActiveSlices(col);
|
||||
Map<String, DocCollection> collectionsMap = clusterState.getCollectionsMap();
|
||||
for (Map.Entry<String, DocCollection> entry : collectionsMap.entrySet()) {
|
||||
if (entry.getKey().equalsIgnoreCase(collection)) {
|
||||
slices = entry.getValue().getActiveSlices();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if(slices == null) {
|
||||
if (slices == null) {
|
||||
throw new Exception("Collection not found:" + this.collection);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -218,7 +218,10 @@ public class ClusterState implements JSONWriter.Writable {
|
|||
* {@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 see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
|
||||
*
|
||||
* @deprecated use {@link #getCollectionsMap()} to avoid a second lookup for lazy collections
|
||||
*/
|
||||
@Deprecated
|
||||
public Set<String> getCollections() {
|
||||
Set<String> result = new HashSet<>();
|
||||
for (Entry<String, CollectionRef> entry : collectionStates.entrySet()) {
|
||||
|
@ -229,6 +232,27 @@ public class ClusterState implements JSONWriter.Writable {
|
|||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a map of collection name vs DocCollection objects
|
||||
*
|
||||
* 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 see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Map<String, DocCollection> getCollectionsMap() {
|
||||
Map<String, DocCollection> result = new HashMap<>(collectionStates.size());
|
||||
for (Entry<String, CollectionRef> entry : collectionStates.entrySet()) {
|
||||
DocCollection collection = entry.getValue().get();
|
||||
if (collection != null) {
|
||||
result.put(entry.getKey(), collection);
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get names of the currently live nodes.
|
||||
*/
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.lang.invoke.MethodHandles;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -66,14 +67,14 @@ public class ClusterStateUtil {
|
|||
success = true;
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
if (clusterState != null) {
|
||||
Set<String> collections;
|
||||
Map<String, DocCollection> collections = null;
|
||||
if (collection != null) {
|
||||
collections = Collections.singleton(collection);
|
||||
collections = Collections.singletonMap(collection, clusterState.getCollection(collection));
|
||||
} else {
|
||||
collections = clusterState.getCollections();
|
||||
collections = clusterState.getCollectionsMap();
|
||||
}
|
||||
for (String coll : collections) {
|
||||
DocCollection docCollection = clusterState.getCollection(coll);
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection docCollection = entry.getValue();
|
||||
Collection<Slice> slices = docCollection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
// only look at active shards
|
||||
|
@ -178,14 +179,14 @@ public class ClusterStateUtil {
|
|||
success = true;
|
||||
ClusterState clusterState = zkStateReader.getClusterState();
|
||||
if (clusterState != null) {
|
||||
Set<String> collections;
|
||||
if (collection == null) {
|
||||
collections = clusterState.getCollections();
|
||||
Map<String, DocCollection> collections = null;
|
||||
if (collection != null) {
|
||||
collections = Collections.singletonMap(collection, clusterState.getCollection(collection));
|
||||
} else {
|
||||
collections = Collections.singleton(collection);
|
||||
collections = clusterState.getCollectionsMap();
|
||||
}
|
||||
for (String coll : collections) {
|
||||
DocCollection docCollection = clusterState.getCollection(coll);
|
||||
for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
|
||||
DocCollection docCollection = entry.getValue();
|
||||
Collection<Slice> slices = docCollection.getSlices();
|
||||
for (Slice slice : slices) {
|
||||
// only look at active shards
|
||||
|
|
|
@ -520,7 +520,8 @@ public class ZkStateReader implements Closeable {
|
|||
* In fact this is a clever way to avoid doing a ZK exists check on
|
||||
* the /collections/collection_name/state.json znode
|
||||
* Such an exists check is done in {@link ClusterState#hasCollection(String)} and
|
||||
* {@link ClusterState#getCollections()} method as a safeguard against exposing wrong collection names to the users
|
||||
* {@link ClusterState#getCollections()} and {@link ClusterState#getCollectionsMap()} methods
|
||||
* have a safeguard against exposing wrong collection names to the users
|
||||
*/
|
||||
private void refreshCollectionList(Watcher watcher) throws KeeperException, InterruptedException {
|
||||
synchronized (refreshCollectionListLock) {
|
||||
|
@ -715,8 +716,7 @@ public class ZkStateReader implements Closeable {
|
|||
Map<String,Slice> slices = clusterState.getSlicesMap(collection);
|
||||
if (slices == null) {
|
||||
throw new ZooKeeperException(ErrorCode.BAD_REQUEST,
|
||||
"Could not find collection in zk: " + collection + " "
|
||||
+ clusterState.getCollections());
|
||||
"Could not find collection in zk: " + collection);
|
||||
}
|
||||
|
||||
Slice replicas = slices.get(shardId);
|
||||
|
|
|
@ -478,7 +478,7 @@ public class JdbcTest extends SolrCloudTestCase {
|
|||
}
|
||||
|
||||
List<String> collections = new ArrayList<>();
|
||||
collections.addAll(cluster.getSolrClient().getZkStateReader().getClusterState().getCollections());
|
||||
collections.addAll(cluster.getSolrClient().getZkStateReader().getClusterState().getCollectionsMap().keySet());
|
||||
Collections.sort(collections);
|
||||
|
||||
try(ResultSet rs = databaseMetaData.getSchemas()) {
|
||||
|
|
|
@ -1353,7 +1353,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
|||
|
||||
if (slices == null) {
|
||||
throw new RuntimeException("Could not find collection "
|
||||
+ DEFAULT_COLLECTION + " in " + clusterState.getCollections());
|
||||
+ DEFAULT_COLLECTION + " in " + clusterState.getCollectionsMap().keySet());
|
||||
}
|
||||
|
||||
for (CloudJettyRunner cjetty : cloudJettys) {
|
||||
|
@ -1920,9 +1920,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
|||
if (collection != null) {
|
||||
cs = clusterState.getCollection(collection).toString();
|
||||
} else {
|
||||
Map<String,DocCollection> map = new HashMap<>();
|
||||
for (String coll : clusterState.getCollections())
|
||||
map.put(coll, clusterState.getCollection(coll));
|
||||
Map<String,DocCollection> map = clusterState.getCollectionsMap();
|
||||
CharArr out = new CharArr();
|
||||
new JSONWriter(out, 2).write(map);
|
||||
cs = out.toString();
|
||||
|
|
Loading…
Reference in New Issue