SOLR-13992: Refactor code to have collection, shard name in Replica,Slice (#1051)

* SOLR-13992: Refactor code to have collection name shard name in Replica,Slice
This commit is contained in:
Noble Paul 2019-12-03 13:10:19 +11:00 committed by GitHub
parent aebf7f7a46
commit d8f9f47ca0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 139 additions and 118 deletions

View File

@ -217,6 +217,8 @@ Other Changes
* SOLR-13885: Typos in the documentation. (KoenDG via Cassandra Targett)
* SOLR-13992: Code refactored to have collection name, slice name in Replica, Slice (noble)
================== 8.3.1 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -296,7 +296,7 @@ class ExclusiveSliceProperty {
if (newSlice != null) {
replica = newSlice.getReplica(replicaName);
} else {
newSlice = new Slice(origSlice.getName(), origSlice.getReplicasCopy(), origSlice.shallowCopy());
newSlice = new Slice(origSlice.getName(), origSlice.getReplicasCopy(), origSlice.shallowCopy(), origSlice.collection);
changedSlices.put(origSlice.getName(), newSlice);
replica = newSlice.getReplica(replicaName);
}

View File

@ -80,9 +80,9 @@ import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
import static org.apache.solr.common.params.CollectionAdminParams.ALIAS;
import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
import static org.apache.solr.common.params.CollectionAdminParams.COLOCATED_WITH;
import static org.apache.solr.common.params.CollectionAdminParams.ALIAS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MODIFYCOLLECTION;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;

View File

@ -18,20 +18,6 @@
package org.apache.solr.cloud.api.collections;
import static org.apache.solr.common.cloud.DocCollection.STATE_FORMAT;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
import java.lang.invoke.MethodHandles;
import java.net.URI;
import java.util.ArrayList;
@ -78,6 +64,20 @@ import org.apache.solr.handler.component.ShardHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.solr.common.cloud.DocCollection.STATE_FORMAT;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
import static org.apache.solr.common.params.CommonParams.NAME;
public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@ -199,7 +199,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
Map<String, Slice> newSlices = new LinkedHashMap<>(backupSlices.size());
for (Slice backupSlice : backupSlices) {
newSlices.put(backupSlice.getName(),
new Slice(backupSlice.getName(), Collections.emptyMap(), backupSlice.getProperties()));
new Slice(backupSlice.getName(), Collections.emptyMap(), backupSlice.getProperties(),restoreCollectionName));
}
propMap.put(OverseerCollectionMessageHandler.SHARDS_PROP, newSlices);
}

View File

@ -2386,7 +2386,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
props.put(ZkStateReader.CORE_NAME_PROP, ri.getCore());
props.put(ZkStateReader.REPLICA_TYPE, ri.getType().toString());
props.put(ZkStateReader.STATE_PROP, ri.getState().toString());
Replica r = new Replica(ri.getName(), props);
Replica r = new Replica(ri.getName(), props, ri.getCollection(), ri.getShard());
collMap.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
.computeIfAbsent(ri.getShard(), s -> new HashMap<>())
.put(ri.getName(), r);
@ -2410,7 +2410,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
Map<String, Slice> slices = new HashMap<>();
shards.forEach((s, replicas) -> {
Map<String, Object> sliceProps = sliceProperties.computeIfAbsent(coll, c -> new ConcurrentHashMap<>()).computeIfAbsent(s, sl -> new ConcurrentHashMap<>());
Slice slice = new Slice(s, replicas, sliceProps);
Slice slice = new Slice(s, replicas, sliceProps, coll);
slices.put(s, slice);
});
Map<String, Object> collProps = collProperties.computeIfAbsent(coll, c -> new ConcurrentHashMap<>());

View File

@ -68,7 +68,7 @@ public class ClusterStateMutator {
Map<String, Slice> slices;
if (messageShardsObj instanceof Map) { // we are being explicitly told the slice data (e.g. coll restore)
slices = Slice.loadAllFromMap((Map<String, Object>)messageShardsObj);
slices = Slice.loadAllFromMap(cName, (Map<String, Object>)messageShardsObj);
} else {
List<String> shardNames = new ArrayList<>();
@ -89,7 +89,7 @@ public class ClusterStateMutator {
Map<String, Object> sliceProps = new LinkedHashMap<>(1);
sliceProps.put(Slice.RANGE, ranges == null ? null : ranges.get(i));
slices.put(sliceName, new Slice(sliceName, null, sliceProps));
slices.put(sliceName, new Slice(sliceName, null, sliceProps,cName));
}
}

View File

@ -77,7 +77,7 @@ public class CollectionMutator {
if (shardParentNode != null) {
sliceProps.put("shard_parent_node", shardParentNode);
}
collection = updateSlice(collectionName, collection, new Slice(shardId, replicas, sliceProps));
collection = updateSlice(collectionName, collection, new Slice(shardId, replicas, sliceProps, collectionName));
return new ZkWriteCommand(collectionName, collection);
} else {
log.error("Unable to create Shard: " + shardId + " because it already exists in collection: " + collectionName);

View File

@ -65,13 +65,13 @@ public class NodeMutator {
log.debug("Update replica state for " + replica + " to " + Replica.State.DOWN.toString());
Map<String, Object> props = replica.shallowCopy();
props.put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
Replica newReplica = new Replica(replica.getName(), props);
Replica newReplica = new Replica(replica.getName(), props, collection, slice.getName());
newReplicas.put(replica.getName(), newReplica);
needToUpdateCollection = true;
}
}
Slice newSlice = new Slice(slice.getName(), newReplicas, slice.shallowCopy());
Slice newSlice = new Slice(slice.getName(), newReplicas, slice.shallowCopy(),collection);
slicesCopy.put(slice.getName(), newSlice);
}

View File

@ -72,7 +72,7 @@ public class ReplicaMutator {
Map<String, Object> replicaProps = new LinkedHashMap<>(replica.getProperties());
replicaProps.put(key, value);
return new Replica(replica.getName(), replicaProps);
return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getSlice());
}
protected Replica unsetProperty(Replica replica, String key) {
@ -81,7 +81,7 @@ public class ReplicaMutator {
if (!replica.containsKey(key)) return replica;
Map<String, Object> replicaProps = new LinkedHashMap<>(replica.getProperties());
replicaProps.remove(key);
return new Replica(replica.getName(), replicaProps);
return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getSlice());
}
protected Replica setLeader(Replica replica) {
@ -144,10 +144,11 @@ public class ReplicaMutator {
}
log.info("Setting property {} with value {} for collection {}", property, propVal, collectionName);
log.debug("Full message: {}", message);
if (StringUtils.equalsIgnoreCase(replica.getStr(property), propVal)) return ZkStateWriter.NO_OP; // already the value we're going to set
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 = collection.getSlice(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.
@ -159,7 +160,7 @@ public class ReplicaMutator {
}
}
}
Slice newSlice = new Slice(sliceName, replicas, collection.getSlice(sliceName).shallowCopy());
Slice newSlice = new Slice(sliceName, replicas, collection.getSlice(sliceName).shallowCopy(),collectionName);
DocCollection newCollection = CollectionMutator.updateSlice(collectionName, collection,
newSlice);
return new ZkWriteCommand(collectionName, newCollection);
@ -326,8 +327,8 @@ public class ReplicaMutator {
String shardParent = (String) replicaProps.remove(ZkStateReader.SHARD_PARENT_PROP);
Replica replica = new Replica(coreNodeName, replicaProps);
Replica replica = new Replica(coreNodeName, replicaProps, collectionName, sliceName);
log.debug("Will update state for replica: {}", replica);
Map<String, Object> sliceProps = null;
@ -347,7 +348,7 @@ public class ReplicaMutator {
sliceProps.put(Slice.PARENT, shardParent);
}
replicas.put(replica.getName(), replica);
slice = new Slice(sliceName, replicas, sliceProps);
slice = new Slice(sliceName, replicas, sliceProps, collectionName);
DocCollection newCollection = CollectionMutator.updateSlice(collectionName, collection, slice);
log.debug("Collection is now: {}", newCollection);
@ -424,10 +425,10 @@ public class ReplicaMutator {
String parentSliceName = (String) sliceProps.remove(Slice.PARENT);
// now lets see if the parent leader is still the same or else there's a chance of data loss
// see SOLR-9438 for details
String shardParentZkSession = (String) sliceProps.remove("shard_parent_zk_session");
String shardParentZkSession = (String) sliceProps.remove("shard_parent_zk_session");
String shardParentNode = (String) sliceProps.remove("shard_parent_node");
boolean isLeaderSame = true;
if (shardParentNode != null && shardParentZkSession != null) {
if (shardParentNode != null && shardParentZkSession != null) {
log.info("Checking whether sub-shard leader node is still the same one at {} with ZK session id {}", shardParentNode, shardParentZkSession);
try {
VersionedData leaderZnode = null;
@ -437,7 +438,7 @@ public class ReplicaMutator {
} catch (NoSuchElementException e) {
// ignore
}
if (leaderZnode == null) {
if (leaderZnode == null) {
log.error("The shard leader node: {} is not live anymore!", shardParentNode);
isLeaderSame = false;
} else if (!shardParentZkSession.equals(leaderZnode.getOwner())) {
@ -471,7 +472,7 @@ public class ReplicaMutator {
log.info("TIMINGS Sub-shard " + subShardSlice.getName() + " not available: " + subShardSlice);
}
}
} else {
} else {
// we must mark the shard split as failed by switching sub-shards to recovery_failed state
propMap.put(sliceName, Slice.State.RECOVERY_FAILED.toString());
for (Slice subShardSlice : subShardSlices) {

View File

@ -80,7 +80,7 @@ public class SliceMutator {
ZkStateReader.BASE_URL_PROP, message.getStr(ZkStateReader.BASE_URL_PROP),
ZkStateReader.STATE_PROP, message.getStr(ZkStateReader.STATE_PROP),
ZkStateReader.NODE_NAME_PROP, message.getStr(ZkStateReader.NODE_NAME_PROP),
ZkStateReader.REPLICA_TYPE, message.get(ZkStateReader.REPLICA_TYPE)));
ZkStateReader.REPLICA_TYPE, message.get(ZkStateReader.REPLICA_TYPE)), coll, slice);
return new ZkWriteCommand(coll, updateReplica(collection, sl, replica.getName(), replica));
}
@ -103,7 +103,7 @@ public class SliceMutator {
if (replica != null && (baseUrl == null || baseUrl.equals(replica.getBaseUrl()))) {
Map<String, Replica> newReplicas = slice.getReplicasCopy();
newReplicas.remove(cnn);
slice = new Slice(slice.getName(), newReplicas, slice.getProperties());
slice = new Slice(slice.getName(), newReplicas, slice.getProperties(),collection);
}
newSlices.put(slice.getName(), slice);
}
@ -150,7 +150,7 @@ public class SliceMutator {
Map<String, Object> newSliceProps = slice.shallowCopy();
newSliceProps.put(Slice.REPLICAS, newReplicas);
slice = new Slice(slice.getName(), newReplicas, slice.getProperties());
slice = new Slice(slice.getName(), newReplicas, slice.getProperties(), collectionName);
return new ZkWriteCommand(collectionName, CollectionMutator.updateSlice(collectionName, coll, slice));
}
@ -180,7 +180,7 @@ public class SliceMutator {
props.put(ZkStateReader.STATE_PROP, message.getStr(key));
// we need to use epoch time so that it's comparable across Overseer restarts
props.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(cloudManager.getTimeSource().getEpochTimeNs()));
Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props);
Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props,collectionName);
slicesCopy.put(slice.getName(), newSlice);
}
@ -224,7 +224,7 @@ public class SliceMutator {
Map<String, Object> props = slice.shallowCopy();
props.put("routingRules", routingRules);
Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props);
Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props,collectionName);
return new ZkWriteCommand(collectionName,
CollectionMutator.updateSlice(collectionName, collection, newSlice));
}
@ -249,7 +249,7 @@ public class SliceMutator {
routingRules.remove(routeKeyStr); // no rules left
Map<String, Object> props = slice.shallowCopy();
props.put("routingRules", routingRules);
Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props);
Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props,collectionName);
return new ZkWriteCommand(collectionName,
CollectionMutator.updateSlice(collectionName, collection, newSlice));
}
@ -264,7 +264,7 @@ public class SliceMutator {
} else {
replicasCopy.put(replica.getName(), replica);
}
Slice newSlice = new Slice(slice.getName(), replicasCopy, slice.getProperties());
Slice newSlice = new Slice(slice.getName(), replicasCopy, slice.getProperties(), collection.getName());
log.debug("Old Slice: {}", slice);
log.debug("New Slice: {}", newSlice);
return CollectionMutator.updateSlice(collection.getName(), collection, newSlice);

View File

@ -112,6 +112,8 @@ public class ClusterStateMockUtil {
collectionProps.put(ZkStateReader.REPLICATION_FACTOR, Integer.toString(replicationFactor));
Map<String,DocCollection> collectionStates = new HashMap<>();
DocCollection docCollection = null;
String collName = null;
String sliceName = null;
Slice slice = null;
int replicaCount = 1;
@ -121,12 +123,13 @@ public class ClusterStateMockUtil {
switch (m.group(1)) {
case "c":
slices = new HashMap<>();
docCollection = new DocCollection("collection" + (collectionStates.size() + 1), slices, collectionProps, null);
docCollection = new DocCollection(collName = "collection" + (collectionStates.size() + 1), slices, collectionProps, null);
collectionStates.put(docCollection.getName(), docCollection);
break;
case "s":
replicas = new HashMap<>();
slice = new Slice("slice" + (slices.size() + 1), replicas, null);
if(collName == null) collName = "collection" + (collectionStates.size() + 1);
slice = new Slice(sliceName = "slice" + (slices.size() + 1), replicas, null, collName);
slices.put(slice.getName(), slice);
break;
case "r":
@ -168,8 +171,9 @@ public class ClusterStateMockUtil {
replicaPropMap.put(ZkStateReader.NODE_NAME_PROP, nodeName);
replicaPropMap.put(ZkStateReader.BASE_URL_PROP, "http://baseUrl" + node);
replicaPropMap.put(ZkStateReader.STATE_PROP, state.toString());
replica = new Replica(replicaName, replicaPropMap);
if(collName == null) collName = "collection" + (collectionStates.size() + 1);
if(sliceName == null) collName = "slice" + (slices.size() + 1);
replica = new Replica(replicaName, replicaPropMap, collName, sliceName);
replicas.put(replica.getName(), replica);
break;

View File

@ -25,8 +25,8 @@ import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.util.Utils;
import org.junit.Test;
@ -45,11 +45,11 @@ public class ClusterStateTest extends SolrTestCaseJ4 {
props.put("prop1", "value");
props.put("prop2", "value2");
Replica replica = new Replica("node1", props);
Replica replica = new Replica("node1", props, "collection1", "shard1");
sliceToProps.put("node1", replica);
Slice slice = new Slice("shard1", sliceToProps, null);
Slice slice = new Slice("shard1", sliceToProps, null, "collection1");
slices.put("shard1", slice);
Slice slice2 = new Slice("shard2", sliceToProps, null);
Slice slice2 = new Slice("shard2", sliceToProps, null, "collection1");
slices.put("shard2", slice2);
collectionStates.put("collection1", new DocCollection("collection1", slices, null, DocRouter.DEFAULT));
collectionStates.put("collection2", new DocCollection("collection2", slices, null, DocRouter.DEFAULT));

View File

@ -309,12 +309,12 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
String slice = replica.getStr(ZkStateReader.SHARD_ID_PROP);
if (!slices.containsKey(slice)) slices.put(slice, new HashMap<>());
String replicaName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
slices.get(slice).put(replicaName, new Replica(replicaName, replica.getProperties()));
slices.get(slice).put(replicaName, new Replica(replicaName, replica.getProperties(), docCollection.getName(), slice));
}
Map<String, Slice> slicesMap = new HashMap<>();
for (Map.Entry<String, Map<String, Replica>> entry : slices.entrySet()) {
slicesMap.put(entry.getKey(), new Slice(entry.getKey(), entry.getValue(), null));
slicesMap.put(entry.getKey(), new Slice(entry.getKey(), entry.getValue(), null,docCollection.getName()));
}
return docCollection.copyWithSlices(slicesMap);

View File

@ -16,6 +16,11 @@
*/
package org.apache.solr.cloud;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
@ -25,11 +30,6 @@ import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.util.Utils;
import org.junit.Test;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
public class SliceStateTest extends SolrTestCaseJ4 {
@Test
@ -42,9 +42,9 @@ public class SliceStateTest extends SolrTestCaseJ4 {
Map<String, Replica> sliceToProps = new HashMap<>();
Map<String, Object> props = new HashMap<>();
Replica replica = new Replica("node1", props);
Replica replica = new Replica("node1", props, "collection1", "shard1");
sliceToProps.put("node1", replica);
Slice slice = new Slice("shard1", sliceToProps, null);
Slice slice = new Slice("shard1", sliceToProps, null, "collection1");
assertSame("Default state not set to active", Slice.State.ACTIVE, slice.getState());
slices.put("shard1", slice);
collectionStates.put("collection1", new DocCollection("collection1", slices, null, DocRouter.DEFAULT));

View File

@ -273,7 +273,7 @@ public class TestHashPartitioner extends SolrTestCaseJ4 {
Map<String,Slice> slices = new HashMap<>();
for (int i=0; i<ranges.size(); i++) {
Range range = ranges.get(i);
Slice slice = new Slice("shard"+(i+1), null, map("range",range));
Slice slice = new Slice("shard"+(i+1), null, map("range",range), "collections1");
slices.put(slice.getName(), slice);
}

View File

@ -16,13 +16,6 @@
*/
package org.apache.solr.cloud.api.collections;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collections;
@ -53,6 +46,13 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class AssignTest extends SolrTestCaseJ4 {
@Override
@ -147,8 +147,8 @@ public class AssignTest extends SolrTestCaseJ4 {
// TODO: fix this to be independent of ZK
ZkDistribStateManager stateManager = new ZkDistribStateManager(zkClient);
Map<String, Slice> slices = new HashMap<>();
slices.put("shard1", new Slice("shard1", new HashMap<>(), null));
slices.put("shard2", new Slice("shard2", new HashMap<>(), null));
slices.put("shard1", new Slice("shard1", new HashMap<>(), null,"collection1"));
slices.put("shard2", new Slice("shard2", new HashMap<>(), null,"collection1"));
DocCollection docCollection = new DocCollection("collection1", slices, null, DocRouter.DEFAULT);
assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildSolrCoreName(stateManager, docCollection, "shard1", Replica.Type.NRT));

View File

@ -151,7 +151,7 @@ public class CoreSorterTest extends SolrTestCaseJ4 {
}
public Replica getReplica(String node) {
return new Replica(replicaName, Utils.makeMap("core", replicaName, "node_name", node));
return new Replica(replicaName, Utils.makeMap("core", replicaName, "node_name", node), cd.getCollectionName(), cd.getShardId());
}
public boolean equals(String coll, String slice) {

View File

@ -31,7 +31,6 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
import org.noggit.JSONWriter;
/**
@ -104,7 +103,6 @@ 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 see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
*/
public boolean hasCollection(String collectionName) {
return getCollectionOrNull(collectionName) != null;
@ -141,7 +139,6 @@ public class ClusterState implements JSONWriter.Writable {
* Implementation note: This method resolves the collection reference by calling
* {@link CollectionRef#get()} which may 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)}
*/
public DocCollection getCollectionOrNull(String collectionName, boolean allowCached) {
CollectionRef ref = collectionStates.get(collectionName);
@ -154,7 +151,6 @@ 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 see javadocs in {@link ZkStateReader#refreshCollectionList(Watcher)}
*
* @return a map of collection name vs DocCollection object
*/
@ -258,13 +254,13 @@ public class ClusterState implements JSONWriter.Writable {
Map<String,Object> props;
Map<String,Slice> slices;
Map<String,Object> sliceObjs = (Map<String,Object>)objs.get(DocCollection.SHARDS);
Map<String, Object> sliceObjs = (Map<String, Object>) objs.get(DocCollection.SHARDS);
if (sliceObjs == null) {
// legacy format from 4.0... there was no separate "shards" level to contain the collection shards.
slices = Slice.loadAllFromMap(objs);
slices = Slice.loadAllFromMap(name, objs);
props = Collections.emptyMap();
} else {
slices = Slice.loadAllFromMap(sliceObjs);
slices = Slice.loadAllFromMap(name, sliceObjs);
props = new HashMap<>(objs);
objs.remove(DocCollection.SHARDS);
}

View File

@ -110,9 +110,12 @@ public class Replica extends ZkNodeProps {
private final String nodeName;
private final State state;
private final Type type;
public final String slice, collection;
public Replica(String name, Map<String,Object> propMap) {
public Replica(String name, Map<String,Object> propMap, String collection, String slice) {
super(propMap);
this.collection = collection;
this.slice = slice;
this.name = name;
this.nodeName = (String) propMap.get(ZkStateReader.NODE_NAME_PROP);
if (propMap.get(ZkStateReader.STATE_PROP) != null) {
@ -124,6 +127,13 @@ public class Replica extends ZkNodeProps {
type = Type.get((String) propMap.get(ZkStateReader.REPLICA_TYPE));
}
public String getCollection(){
return collection;
}
public String getSlice(){
return slice;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;

View File

@ -37,9 +37,10 @@ import static org.apache.solr.common.util.Utils.toJSONString;
* A Slice contains immutable information about a logical shard (all replicas that share the same shard id).
*/
public class Slice extends ZkNodeProps implements Iterable<Replica> {
public final String collection;
/** Loads multiple slices into a Map from a generic Map that probably came from deserialized JSON. */
public static Map<String,Slice> loadAllFromMap(Map<String, Object> genericSlices) {
public static Map<String,Slice> loadAllFromMap(String collection, Map<String, Object> genericSlices) {
if (genericSlices == null) return Collections.emptyMap();
Map<String, Slice> result = new LinkedHashMap<>(genericSlices.size());
for (Map.Entry<String, Object> entry : genericSlices.entrySet()) {
@ -48,7 +49,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
if (val instanceof Slice) {
result.put(name, (Slice) val);
} else if (val instanceof Map) {
result.put(name, new Slice(name, null, (Map<String, Object>) val));
result.put(name, new Slice(name, null, (Map<String, Object>) val, collection));
}
}
return result;
@ -128,9 +129,10 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
* @param replicas The replicas of the slice. This is used directly and a copy is not made. If null, replicas will be constructed from props.
* @param props The properties of the slice - a shallow copy will always be made.
*/
public Slice(String name, Map<String,Replica> replicas, Map<String,Object> props) {
public Slice(String name, Map<String,Replica> replicas, Map<String,Object> props, String collection) {
super( props==null ? new LinkedHashMap<String,Object>(2) : new LinkedHashMap<>(props));
this.name = name;
this.collection = collection;
Object rangeObj = propMap.get(RANGE);
if (propMap.get(ZkStateReader.STATE_PROP) != null) {
@ -162,7 +164,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
replicationFactor = null; // future
// add the replicas *after* the other properties (for aesthetics, so it's easy to find slice properties in the JSON output)
this.replicas = replicas != null ? replicas : makeReplicas((Map<String,Object>)propMap.get(REPLICAS));
this.replicas = replicas != null ? replicas : makeReplicas(collection,name, (Map<String,Object>)propMap.get(REPLICAS));
propMap.put(REPLICAS, this.replicas);
Map<String, Object> rules = (Map<String, Object>) propMap.get("routingRules");
@ -186,7 +188,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
}
private Map<String,Replica> makeReplicas(Map<String,Object> genericReplicas) {
private Map<String,Replica> makeReplicas(String collection, String slice,Map<String,Object> genericReplicas) {
if (genericReplicas == null) return new HashMap<>(1);
Map<String,Replica> result = new LinkedHashMap<>(genericReplicas.size());
for (Map.Entry<String,Object> entry : genericReplicas.entrySet()) {
@ -196,7 +198,7 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
if (val instanceof Replica) {
r = (Replica)val;
} else {
r = new Replica(name, (Map<String,Object>)val);
r = new Replica(name, (Map<String,Object>)val, collection, slice);
}
result.put(name, r);
}
@ -213,6 +215,9 @@ public class Slice extends ZkNodeProps implements Iterable<Replica> {
return null;
}
public String getCollection() {
return collection;
}
/**
* Return slice name (shard id).
*/

View File

@ -944,6 +944,11 @@ public class ZkStateReader implements SolrCloseable {
return null;
}
public boolean isNodeLive(String node) {
return liveNodes.contains(node);
}
/**
* Get shard leader properties, with retry if none exist.
*/

View File

@ -18,16 +18,6 @@
package org.apache.solr.client.solrj.cloud.autoscaling;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.CLUSTER_PREFERENCES;
import static org.apache.solr.client.solrj.cloud.autoscaling.TestPolicy2.loadFromResource;
import static org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type.CORES;
import static org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type.FREEDISK;
import static org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type.REPLICA;
import static org.apache.solr.common.cloud.ZkStateReader.CLUSTER_STATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
import java.io.IOException;
import java.io.StringWriter;
import java.lang.invoke.MethodHandles;
@ -45,6 +35,8 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrResponse;
@ -83,8 +75,15 @@ import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.solr.client.solrj.cloud.autoscaling.Policy.CLUSTER_PREFERENCES;
import static org.apache.solr.client.solrj.cloud.autoscaling.TestPolicy2.loadFromResource;
import static org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type.CORES;
import static org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type.FREEDISK;
import static org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type.REPLICA;
import static org.apache.solr.common.cloud.ZkStateReader.CLUSTER_STATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
public class TestPolicy extends SolrTestCaseJ4 {
boolean useNodeset ;
@ -2595,13 +2594,13 @@ public class TestPolicy extends SolrTestCaseJ4 {
if (node.equals("node1")) {
Map m = Utils.makeMap("newColl",
Utils.makeMap("shard1", Collections.singletonList(new ReplicaInfo("r1", "shard1",
new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node1")),
new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node1"), "newColl", "shard1"),
Utils.makeMap(FREEDISK.perReplicaValue, 200)))));
return m;
} else if (node.equals("node2")) {
Map m = Utils.makeMap("newColl",
Utils.makeMap("shard2", Collections.singletonList(new ReplicaInfo("r1", "shard2",
new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node2")),
new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node2"),"newColl", "shard2"),
Utils.makeMap(FREEDISK.perReplicaValue, 200)))));
return m;
}
@ -2624,9 +2623,9 @@ public class TestPolicy extends SolrTestCaseJ4 {
@Override
public Replica getLeader(String sliceName) {
if (sliceName.equals("shard1"))
return new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node1"));
return new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node1"), name, "shard1");
if (sliceName.equals("shard2"))
return new Replica("r2", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node2"));
return new Replica("r2", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node2"),name, "shard2");
return null;
}
};
@ -2643,20 +2642,20 @@ public class TestPolicy extends SolrTestCaseJ4 {
public void testMoveReplicaLeaderlast() {
List<Pair<ReplicaInfo, Row>> validReplicas = new ArrayList<>();
Replica replica = new Replica("r1", Utils.makeMap("leader", "true"));
ReplicaInfo replicaInfo = new ReplicaInfo("c1", "s1", replica, new HashMap<>());
Replica replica = new Replica("r1", Utils.makeMap("leader", "true"), "c1", "s1");
ReplicaInfo replicaInfo = new ReplicaInfo(replica.collection, replica.slice ,replica, new HashMap<>());
validReplicas.add(new Pair<>(replicaInfo, null));
replicaInfo = new ReplicaInfo("r4", "c1_s2_r1", "c1", "s2", Replica.Type.NRT, "n1", Collections.singletonMap("leader", "true"));
validReplicas.add(new Pair<>(replicaInfo, null));
replica = new Replica("r2", Utils.makeMap("leader", false));
replicaInfo = new ReplicaInfo("c1", "s1", replica, new HashMap<>());
replica = new Replica("r2", Utils.makeMap("leader", false),"c1","s1");
replicaInfo = new ReplicaInfo(replica.collection, replica.slice, replica, new HashMap<>());
validReplicas.add(new Pair<>(replicaInfo, null));
replica = new Replica("r3", Utils.makeMap("leader", false));
replicaInfo = new ReplicaInfo("c1", "s1", replica, new HashMap<>());
replica = new Replica("r3", Utils.makeMap("leader", false),"c1","s1");
replicaInfo = new ReplicaInfo(replica.collection,replica.slice, replica, new HashMap<>());
validReplicas.add(new Pair<>(replicaInfo, null));

View File

@ -75,7 +75,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
ZkStateReader.NODE_NAME_PROP, "node4",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "TLOG"
)
),"collection1","shard1"
)
);
@ -125,7 +125,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
ZkStateReader.NODE_NAME_PROP, "node1",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "NRT"
)
),"collection1","shard1"
)
);
replicas.add(
@ -136,7 +136,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
ZkStateReader.NODE_NAME_PROP, "node2",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "TLOG"
)
),"collection1","shard1"
)
);
replicas.add(
@ -147,7 +147,7 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
ZkStateReader.NODE_NAME_PROP, "node3",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "PULL"
)
),"collection1","shard1"
)
);
return replicas;

View File

@ -23,7 +23,6 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.solr.SolrTestCase;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.params.ModifiableSolrParams;
@ -128,7 +127,7 @@ public class ReplicaListTransformerTest extends SolrTestCase {
final Map<String,Object> propMap = new HashMap<String,Object>();
propMap.put("base_url", url);
// a skeleton replica, good enough for this test's purposes
final Replica replica = new Replica(name, propMap);
final Replica replica = new Replica(name, propMap,"c1","s1");
inputs.add(replica);
if (url.matches(regex)) {

View File

@ -84,7 +84,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
ZkStateReader.NODE_NAME_PROP, "node4",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "TLOG"
)
), "c1","s1"
)
);
@ -122,7 +122,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
ZkStateReader.NODE_NAME_PROP, "node1",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "NRT"
)
),"c1","s1"
)
);
replicas.add(
@ -133,7 +133,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
ZkStateReader.NODE_NAME_PROP, "node2",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "TLOG"
)
),"c1","s1"
)
);
replicas.add(
@ -144,7 +144,7 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
ZkStateReader.NODE_NAME_PROP, "node3",
ZkStateReader.CORE_NAME_PROP, "collection1",
ZkStateReader.REPLICA_TYPE, "PULL"
)
),"c1","s1"
)
);
return replicas;

View File

@ -35,7 +35,7 @@ public class ShufflingReplicaListTransformerTest extends SolrTestCase {
public void testTransformReplicas() throws Exception {
final List<Replica> replicas = new ArrayList<>();
for (final String url : createRandomUrls()) {
replicas.add(new Replica(url, new HashMap<String,Object>()));
replicas.add(new Replica(url, new HashMap<String,Object>(),"c1","s1"));
}
implTestTransform(replicas);
}