SOLR-9321: Remove deprecated methods of ClusterState

This commit is contained in:
Cao Manh Dat 2017-08-01 14:49:57 +07:00
parent 3a405971b9
commit 93ed4770ac
48 changed files with 1296 additions and 243 deletions

View File

@ -253,6 +253,9 @@ Upgrading from Solr 6.x
replaced with 'sourceNode' and 'targetNode' instead. The old names will continue to work for back-compatibility
but they will be removed in 8.0. See SOLR-11068 for more details.
* All deperated methods of ClusterState (except getZkClusterStateVersion())
have been removed. Use DocCollection methods instead.
New Features
----------------------
* SOLR-9857, SOLR-9858: Collect aggregated metrics from nodes and shard leaders in overseer. (ab)
@ -596,6 +599,8 @@ Other Changes
* SOLR-10033: Provide a clear exception when attempting to facet with facet.mincount=0 over points fields.
(Steve Rowe)
* SOLR-9321: Remove deprecated methods of ClusterState. (Jason Gerlowski, ishan, Cao Manh Dat)
================== 6.7.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -27,6 +27,7 @@ import java.util.Map;
import org.apache.commons.io.FileUtils;
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.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
@ -57,9 +58,9 @@ public class CloudUtil {
log.debug("checkSharedFSFailoverReplaced running for coreNodeName={} baseUrl={}", thisCnn, thisBaseUrl);
// if we see our core node name on a different base url, unload
Map<String,Slice> slicesMap = zkController.getClusterState().getSlicesMap(desc.getCloudDescriptor().getCollectionName());
if (slicesMap != null) {
final DocCollection docCollection = zkController.getClusterState().getCollectionOrNull(desc.getCloudDescriptor().getCollectionName());
if (docCollection != null && docCollection.getSlicesMap() != null) {
Map<String,Slice> slicesMap = docCollection.getSlicesMap();
for (Slice slice : slicesMap.values()) {
for (Replica replica : slice.getReplicas()) {

View File

@ -65,16 +65,10 @@ public class DeleteShardCmd implements Cmd {
String sliceId = message.getStr(ZkStateReader.SHARD_ID_PROP);
log.info("Delete shard invoked");
Slice slice = clusterState.getSlice(collectionName, sliceId);
Slice slice = clusterState.getCollection(collectionName).getSlice(sliceId);
if (slice == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"No shard with name " + sliceId + " exists for collection " + collectionName);
if (slice == null) {
if (clusterState.hasCollection(collectionName)) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"No shard with name " + sliceId + " exists for collection " + collectionName);
} else {
throw new SolrException(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).
// TODO: Add check for range gaps on Slice deletion
final Slice.State state = slice.getState();

View File

@ -31,6 +31,7 @@ import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
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.SolrZkClient;
@ -498,8 +499,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
ZkStateReader zkStateReader = zkController.getZkStateReader();
zkStateReader.forceUpdateCollection(collection);
ClusterState clusterState = zkStateReader.getClusterState();
Replica rep = (clusterState == null) ? null
: clusterState.getReplica(collection, leaderProps.getStr(ZkStateReader.CORE_NODE_NAME_PROP));
Replica rep = getReplica(clusterState, collection, leaderProps.getStr(ZkStateReader.CORE_NODE_NAME_PROP));
if (rep != null && rep.getState() != Replica.State.ACTIVE
&& rep.getState() != Replica.State.RECOVERING) {
log.debug("We have become the leader after core registration but are not in an ACTIVE state - publishing ACTIVE");
@ -507,6 +507,13 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
}
}
}
private Replica getReplica(ClusterState clusterState, String collectionName, String replicaName) {
if (clusterState == null) return null;
final DocCollection docCollection = clusterState.getCollectionOrNull(collectionName);
if (docCollection == null) return null;
return docCollection.getReplica(replicaName);
}
public void checkLIR(String coreName, boolean allReplicasInLine)
throws InterruptedException, KeeperException, IOException {
@ -604,7 +611,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
long timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutms, TimeUnit.MILLISECONDS);
final String shardsElectZkPath = electionPath + LeaderElector.ELECTION_NODE;
Slice slices = zkController.getClusterState().getSlice(collection, shardId);
DocCollection docCollection = zkController.getClusterState().getCollectionOrNull(collection);
Slice slices = (docCollection == null) ? null : docCollection.getSlice(shardId);
int cnt = 0;
while (!isClosed && !cc.isShutDown()) {
// wait for everyone to be up
@ -649,7 +657,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
}
Thread.sleep(500);
slices = zkController.getClusterState().getSlice(collection, shardId);
docCollection = zkController.getClusterState().getCollectionOrNull(collection);
slices = (docCollection == null) ? null : docCollection.getSlice(shardId);
cnt++;
}
return false;
@ -658,9 +667,10 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
// returns true if all replicas are found to be up, false if not
private boolean areAllReplicasParticipating() throws InterruptedException {
final String shardsElectZkPath = electionPath + LeaderElector.ELECTION_NODE;
Slice slices = zkController.getClusterState().getSlice(collection, shardId);
final DocCollection docCollection = zkController.getClusterState().getCollectionOrNull(collection);
if (slices != null) {
if (docCollection != null && docCollection.getSlice(shardId) != null) {
final Slice slices = docCollection.getSlice(shardId);
int found = 0;
try {
found = zkClient.getChildren(shardsElectZkPath, null, true).size();

View File

@ -522,10 +522,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
String waitForCoreNodeName(String collectionName, String msgNodeName, String msgCore) {
int retryCount = 320;
while (retryCount-- > 0) {
Map<String,Slice> slicesMap = zkStateReader.getClusterState()
.getSlicesMap(collectionName);
if (slicesMap != null) {
final DocCollection docCollection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
if (docCollection != null && docCollection.getSlicesMap() != null) {
Map<String,Slice> slicesMap = docCollection.getSlicesMap();
for (Slice slice : slicesMap.values()) {
for (Replica replica : slice.getReplicas()) {
// TODO: for really large clusters, we could 'index' on this

View File

@ -545,8 +545,8 @@ public class RecoveryStrategy implements Runnable, Closeable {
zkController.publish(core.getCoreDescriptor(), Replica.State.RECOVERING);
final Slice slice = zkStateReader.getClusterState().getSlice(cloudDesc.getCollectionName(),
cloudDesc.getShardId());
final Slice slice = zkStateReader.getClusterState().getCollection(cloudDesc.getCollectionName())
.getSlice(cloudDesc.getShardId());
try {
prevSendPreRecoveryHttpUriRequest.abort();

View File

@ -942,7 +942,8 @@ public class ZkController {
try {
// If we're a preferred leader, insert ourselves at the head of the queue
boolean joinAtHead = false;
Replica replica = zkStateReader.getClusterState().getReplica(collection, coreZkNodeName);
final DocCollection docCollection = zkStateReader.getClusterState().getCollectionOrNull(collection);
Replica replica = (docCollection == null) ? null : docCollection.getReplica(coreZkNodeName);
if (replica != null) {
joinAtHead = replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false);
}
@ -994,7 +995,7 @@ public class ZkController {
// we will call register again after zk expiration and on reload
if (!afterExpiration && !core.isReloaded() && ulog != null && !isTlogReplicaAndNotLeader) {
// disable recovery in case shard is in construction state (for shard splits)
Slice slice = getClusterState().getSlice(collection, shardId);
Slice slice = getClusterState().getCollection(collection).getSlice(shardId);
if (slice.getState() != Slice.State.CONSTRUCTION || !isLeader) {
Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler().getUpdateLog().recoverFromLog();
if (recoveryFuture != null) {
@ -1354,7 +1355,8 @@ public class ZkController {
assert false : "No collection was specified [" + collection + "]";
return;
}
Replica replica = zkStateReader.getClusterState().getReplica(collection, coreNodeName);
final DocCollection docCollection = zkStateReader.getClusterState().getCollectionOrNull(collection);
Replica replica = (docCollection == null) ? null : docCollection.getReplica(coreNodeName);
if (replica == null || replica.getType() != Type.PULL) {
ElectionContext context = electionContexts.remove(new ContextKey(collection, coreNodeName));
@ -1408,10 +1410,10 @@ public class ZkController {
int retryCount = 320;
log.debug("look for our core node name");
while (retryCount-- > 0) {
Map<String, Slice> slicesMap = zkStateReader.getClusterState()
.getSlicesMap(descriptor.getCloudDescriptor().getCollectionName());
if (slicesMap != null) {
final DocCollection docCollection = zkStateReader.getClusterState()
.getCollectionOrNull(descriptor.getCloudDescriptor().getCollectionName());
if (docCollection != null && docCollection.getSlicesMap() != null) {
final Map<String, Slice> slicesMap = docCollection.getSlicesMap();
for (Slice slice : slicesMap.values()) {
for (Replica replica : slice.getReplicas()) {
// TODO: for really large clusters, we could 'index' on this

View File

@ -42,6 +42,7 @@ import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.cloud.ZkController;
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.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
@ -397,7 +398,8 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
log.warn("Error when updating cluster state", e);
}
ClusterState cstate = zkController.getClusterState();
Collection<Slice> shards = cstate.getActiveSlices(collection);
DocCollection docCollection = cstate.getCollectionOrNull(collection);
Collection<Slice> shards = docCollection == null? null : docCollection.getActiveSlices();
ExecutorService parallelExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(new DefaultSolrThreadFactory("parallelCdcrExecutor"));

View File

@ -47,6 +47,7 @@ import org.apache.solr.cloud.ZkController;
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.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.params.CommonParams;
@ -789,8 +790,9 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
List<String> activeReplicaCoreUrls = new ArrayList<>();
ClusterState clusterState = zkController.getZkStateReader().getClusterState();
Set<String> liveNodes = clusterState.getLiveNodes();
Collection<Slice> activeSlices = clusterState.getActiveSlices(collection);
if (activeSlices != null && activeSlices.size() > 0) {
final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
if (docCollection != null && docCollection.getActiveSlices() != null && docCollection.getActiveSlices().size() > 0) {
final Collection<Slice> activeSlices = docCollection.getActiveSlices();
for (Slice next : activeSlices) {
Map<String, Replica> replicasMap = next.getReplicasMap();
if (replicasMap != null) {

View File

@ -1036,8 +1036,10 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
for (int i = 0; i < numRetries; i++) {
ClusterState clusterState = zkStateReader.getClusterState();
Collection<Slice> shards = clusterState.getSlices(collectionName);
if (shards != null) {
final DocCollection docCollection = clusterState.getCollectionOrNull(collectionName);
if (docCollection != null && docCollection.getSlices() != null) {
Collection<Slice> shards = docCollection.getSlices();
replicaNotAlive = null;
for (Slice shard : shards) {
Collection<Replica> replicas;

View File

@ -387,7 +387,7 @@ public class HttpShardHandler extends ShardHandler {
} else {
if (clusterState == null) {
clusterState = zkController.getClusterState();
slices = clusterState.getSlicesMap(cloudDescriptor.getCollectionName());
slices = clusterState.getCollection(cloudDescriptor.getCollectionName()).getSlicesMap();
}
String sliceName = rb.slices[i];

View File

@ -53,6 +53,7 @@ import org.apache.solr.cloud.ZkSolrResourceLoader;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
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.SolrZkClient;
@ -287,8 +288,9 @@ public final class ManagedIndexSchema extends IndexSchema {
ZkStateReader zkStateReader = zkController.getZkStateReader();
ClusterState clusterState = zkStateReader.getClusterState();
Set<String> liveNodes = clusterState.getLiveNodes();
Collection<Slice> activeSlices = clusterState.getActiveSlices(collection);
if (activeSlices != null && activeSlices.size() > 0) {
final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
if (docCollection != null && docCollection.getActiveSlices() != null && docCollection.getActiveSlices().size() > 0) {
final Collection<Slice> activeSlices = docCollection.getActiveSlices();
for (Slice next : activeSlices) {
Map<String, Replica> replicasMap = next.getReplicasMap();
if (replicasMap != null) {

View File

@ -310,7 +310,7 @@ public class ScoreJoinQParserPlugin extends QParserPlugin {
String fromReplica = null;
String nodeName = zkController.getNodeName();
for (Slice slice : zkController.getClusterState().getActiveSlices(fromIndex)) {
for (Slice slice : zkController.getClusterState().getCollection(fromIndex).getActiveSlices()) {
if (fromReplica != null)
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"SolrCloud join: multiple shards not yet supported " + fromIndex);

View File

@ -898,7 +898,8 @@ public class HttpSolrCall {
private String getRemotCoreUrl(String collectionName, String origCorename) {
ClusterState clusterState = cores.getZkController().getClusterState();
Collection<Slice> slices = clusterState.getActiveSlices(collectionName);
final DocCollection docCollection = clusterState.getCollectionOrNull(collectionName);
Collection<Slice> slices = (docCollection != null) ? docCollection.getActiveSlices() : null;
boolean byCoreName = false;
if (slices == null) {

View File

@ -551,8 +551,9 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
if (id == null) {
for (Entry<String, RoutingRule> entry : routingRules.entrySet()) {
String targetCollectionName = entry.getValue().getTargetCollectionName();
Collection<Slice> activeSlices = cstate.getActiveSlices(targetCollectionName);
if (activeSlices != null && !activeSlices.isEmpty()) {
final DocCollection docCollection = cstate.getCollectionOrNull(targetCollectionName);
if (docCollection != null && docCollection.getActiveSlices() != null && !docCollection.getActiveSlices().isEmpty()) {
final Collection<Slice> activeSlices = docCollection.getActiveSlices();
Slice any = activeSlices.iterator().next();
if (nodes == null) nodes = new ArrayList<>();
nodes.add(new StdNode(new ZkCoreNodeProps(any.getLeader())));
@ -1973,11 +1974,12 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
private List<Node> getCollectionUrls(SolrQueryRequest req, String collection, EnumSet<Replica.Type> types) {
ClusterState clusterState = req.getCore()
.getCoreContainer().getZkController().getClusterState();
Map<String,Slice> slices = clusterState.getSlicesMap(collection);
if (slices == null) {
final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
if (collection == null || docCollection.getSlicesMap() == null) {
throw new ZooKeeperException(ErrorCode.BAD_REQUEST,
"Could not find collection in zk: " + clusterState);
}
Map<String,Slice> slices = docCollection.getSlicesMap();
final List<Node> urls = new ArrayList<>(slices.size());
for (Map.Entry<String,Slice> sliceEntry : slices.entrySet()) {
Slice replicas = slices.get(sliceEntry.getKey());

View File

@ -469,7 +469,7 @@ public final class DocExpirationUpdateProcessorFactory
CloudDescriptor desc = core.getCoreDescriptor().getCloudDescriptor();
String col = desc.getCollectionName();
List<Slice> slices = new ArrayList<Slice>(zk.getClusterState().getActiveSlices(col));
List<Slice> slices = new ArrayList<Slice>(zk.getClusterState().getCollection(col).getActiveSlices());
Collections.sort(slices, COMPARE_SLICES_BY_NAME);
if (slices.isEmpty()) {
log.error("Collection {} has no active Slices?", col);

View File

@ -106,6 +106,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.SolrZkClient;
@ -1170,10 +1171,11 @@ public class SolrCLI {
ClusterState clusterState = zkStateReader.getClusterState();
Set<String> liveNodes = clusterState.getLiveNodes();
Collection<Slice> slices = clusterState.getSlices(collection);
if (slices == null)
final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
if (docCollection == null || docCollection.getSlices() == null)
throw new IllegalArgumentException("Collection "+collection+" not found!");
Collection<Slice> slices = docCollection.getSlices();
// Test http code using a HEAD request first, fail fast if authentication failure
String urlForColl = zkStateReader.getLeaderUrl(collection, slices.stream().findFirst().get().getName(), 1000);
attemptHttpHead(urlForColl, cloudSolrClient.getHttpClient());

View File

@ -649,7 +649,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
ZkNodeProps leader = clusterState.getLeader(collection, slice);
ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
if (leader == null) {
throw new RuntimeException("Could not find leader:" + collection + " " + slice);
}
@ -850,10 +850,11 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
// we added a role of none on these creates - check for it
ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
zkStateReader.forceUpdateCollection(oneInstanceCollection2);
Map<String,Slice> slices = zkStateReader.getClusterState().getSlicesMap(oneInstanceCollection2);
Map<String,Slice> slices = zkStateReader.getClusterState().getCollection(oneInstanceCollection2).getSlicesMap();
assertNotNull(slices);
ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState().getLeader(oneInstanceCollection2, "shard1"));
ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState()
.getCollection(oneInstanceCollection2).getLeader("shard1"));
// now test that unloading a core gets us a new leader
try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {

View File

@ -60,7 +60,7 @@ public class ChaosMonkeyShardSplitTest extends ShardSplitTest {
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
final DocRouter router = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getRouter();
Slice shard1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
Slice shard1 = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getSlice(SHARD1);
DocRouter.Range shard1Range = shard1.getRange() != null ? shard1.getRange() : router.fullRange();
final List<DocRouter.Range> ranges = router.partitionRange(2, shard1Range);
final int[] docCounts = new int[ranges.size()];

View File

@ -62,8 +62,8 @@ public class ClusterStateTest extends SolrTestCaseJ4 {
assertEquals("Provided liveNodes not used properly", 2, loadedClusterState
.getLiveNodes().size());
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"));
assertEquals("Properties not copied properly", replica.getStr("prop1"), loadedClusterState.getCollection("collection1").getSlice("shard1").getReplicasMap().get("node1").getStr("prop1"));
assertEquals("Properties not copied properly", replica.getStr("prop2"), loadedClusterState.getCollection("collection1").getSlice("shard1").getReplicasMap().get("node1").getStr("prop2"));
loadedClusterState = ClusterState.load(-1, new byte[0], liveNodes);

View File

@ -24,6 +24,7 @@ import java.util.Set;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
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.ZkStateReader;
@ -75,7 +76,8 @@ public class ClusterStateUpdateTest extends SolrCloudTestCase {
Map<String,Slice> slices = null;
for (int i = 75; i > 0; i--) {
clusterState2 = zkController2.getClusterState();
slices = clusterState2.getSlicesMap("testcore");
DocCollection docCollection = clusterState2.getCollectionOrNull("testcore");
slices = docCollection == null ? null : docCollection.getSlicesMap();
if (slices != null && slices.containsKey("shard1")
&& slices.get("shard1").getReplicasMap().size() > 0) {

View File

@ -125,7 +125,7 @@ public class CollectionsAPIAsyncDistributedZkTest extends SolrCloudTestCase {
assertSame("AddReplica did not complete", RequestStatusState.COMPLETED, state);
//cloudClient watch might take a couple of seconds to reflect it
Slice shard1 = client.getZkStateReader().getClusterState().getSlice(collection, "shard1");
Slice shard1 = client.getZkStateReader().getClusterState().getCollection(collection).getSlice("shard1");
int count = 0;
while (shard1.getReplicas().size() != 2) {
if (count++ > 1000) {
@ -163,7 +163,7 @@ public class CollectionsAPIAsyncDistributedZkTest extends SolrCloudTestCase {
}
}
shard1 = client.getZkStateReader().getClusterState().getSlice(collection, "shard1");
shard1 = client.getZkStateReader().getClusterState().getCollection(collection).getSlice("shard1");
String replicaName = shard1.getReplicas().iterator().next().getName();
state = CollectionAdminRequest.deleteReplica(collection, "shard1", replicaName)
.processAndWait(client, MAX_TIMEOUT_SECONDS);

View File

@ -102,7 +102,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
"; clusterState: " + printClusterStateInfo(), 0, numActiveReplicas);
int numReplicasOnLiveNodes = 0;
for (Replica rep : clusterState.getSlice(testCollectionName, SHARD1).getReplicas()) {
for (Replica rep : clusterState.getCollection(testCollectionName).getSlice(SHARD1).getReplicas()) {
if (clusterState.getLiveNodes().contains(rep.getNodeName())) {
numReplicasOnLiveNodes++;
}
@ -110,8 +110,8 @@ public class ForceLeaderTest extends HttpPartitionTest {
assertEquals(2, numReplicasOnLiveNodes);
log.info("Before forcing leader: " + printClusterStateInfo());
// Assert there is no leader yet
assertNull("Expected no leader right now. State: " + clusterState.getSlice(testCollectionName, SHARD1),
clusterState.getSlice(testCollectionName, SHARD1).getLeader());
assertNull("Expected no leader right now. State: " + clusterState.getCollection(testCollectionName).getSlice(SHARD1),
clusterState.getCollection(testCollectionName).getSlice(SHARD1).getLeader());
assertSendDocFails(3);
@ -122,9 +122,9 @@ public class ForceLeaderTest extends HttpPartitionTest {
cloudClient.getZkStateReader().forceUpdateCollection(testCollectionName);
clusterState = cloudClient.getZkStateReader().getClusterState();
log.info("After forcing leader: " + clusterState.getSlice(testCollectionName, SHARD1));
log.info("After forcing leader: " + clusterState.getCollection(testCollectionName).getSlice(SHARD1));
// we have a leader
Replica newLeader = clusterState.getSlice(testCollectionName, SHARD1).getLeader();
Replica newLeader = clusterState.getCollectionOrNull(testCollectionName).getSlice(SHARD1).getLeader();
assertNotNull(newLeader);
// leader is active
assertEquals(State.ACTIVE, newLeader.getState());
@ -216,7 +216,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
boolean transition = false;
for (int counter = 10; counter > 0; counter--) {
clusterState = zkStateReader.getClusterState();
Replica newLeader = clusterState.getSlice(collection, slice).getLeader();
Replica newLeader = clusterState.getCollection(collection).getSlice(slice).getLeader();
if (newLeader == null) {
transition = true;
break;
@ -250,7 +250,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
Replica.State replicaState = null;
for (int counter = 10; counter > 0; counter--) {
ClusterState clusterState = zkStateReader.getClusterState();
replicaState = clusterState.getSlice(collection, slice).getReplica(replica.getName()).getState();
replicaState = clusterState.getCollection(collection).getSlice(slice).getReplica(replica.getName()).getState();
if (replicaState == state) {
transition = true;
break;
@ -349,7 +349,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
for (State lirState : lirStates)
if (Replica.State.DOWN.equals(lirState) == false)
allDown = false;
if (allDown && clusterState.getSlice(collectionName, shard).getLeader() == null) {
if (allDown && clusterState.getCollection(collectionName).getSlice(shard).getLeader() == null) {
break;
}
log.warn("Attempt " + i + ", waiting on for 1 sec to settle down in the steady state. State: " +
@ -381,7 +381,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
waitForRecoveriesToFinish(collection, cloudClient.getZkStateReader(), true);
cloudClient.getZkStateReader().forceUpdateCollection(collection);
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
log.info("After bringing back leader: " + clusterState.getSlice(collection, SHARD1));
log.info("After bringing back leader: " + clusterState.getCollection(collection).getSlice(SHARD1));
int numActiveReplicas = getNumberOfActiveReplicas(clusterState, collection, SHARD1);
assertEquals(1+notLeaders.size(), numActiveReplicas);
log.info("Sending doc "+docid+"...");
@ -423,7 +423,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
protected int getNumberOfActiveReplicas(ClusterState clusterState, String collection, String sliceId) {
int numActiveReplicas = 0;
// Assert all replicas are active
for (Replica rep : clusterState.getSlice(collection, sliceId).getReplicas()) {
for (Replica rep : clusterState.getCollection(collection).getSlice(sliceId).getReplicas()) {
if (rep.getState().equals(State.ACTIVE)) {
numActiveReplicas++;
}

View File

@ -42,6 +42,7 @@ import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.common.SolrException;
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.SolrZkClient;
@ -233,7 +234,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
ZkStateReader zkr = cloudClient.getZkStateReader();
zkr.forceUpdateCollection(testCollectionName);; // force the state to be fresh
ClusterState cs = zkr.getClusterState();
Collection<Slice> slices = cs.getActiveSlices(testCollectionName);
Collection<Slice> slices = cs.getCollection(testCollectionName).getActiveSlices();
Slice slice = slices.iterator().next();
Replica partitionedReplica = slice.getReplica(notLeaders.get(0).getName());
assertEquals("The partitioned replica did not get marked down",
@ -522,7 +523,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
ZkStateReader zkr = cloudClient.getZkStateReader();
ClusterState cs = zkr.getClusterState();
assertNotNull(cs);
for (Slice shard : cs.getActiveSlices(testCollectionName)) {
for (Slice shard : cs.getCollection(testCollectionName).getActiveSlices()) {
if (shard.getName().equals(shardId)) {
for (Replica replica : shard.getReplicas()) {
final Replica.State state = replica.getState();
@ -629,14 +630,15 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
ZkStateReader zkr = cloudClient.getZkStateReader();
zkr.forceUpdateCollection(testCollectionName);
ClusterState cs = zkr.getClusterState();
Collection<Slice> slices = cs.getActiveSlices(testCollectionName);
boolean allReplicasUp = false;
long waitMs = 0L;
long maxWaitMs = maxWaitSecs * 1000L;
while (waitMs < maxWaitMs && !allReplicasUp) {
cs = cloudClient.getZkStateReader().getClusterState();
assertNotNull(cs);
Slice shard = cs.getSlice(testCollectionName, shardId);
final DocCollection docCollection = cs.getCollectionOrNull(testCollectionName);
assertNotNull(docCollection);
Slice shard = docCollection.getSlice(shardId);
assertNotNull("No Slice for "+shardId, shard);
allReplicasUp = true; // assume true

View File

@ -205,7 +205,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
}
private String getShardId(String collection, String coreNodeName) {
Map<String,Slice> slices = zkStateReader.getClusterState().getSlicesMap(collection);
DocCollection dc = zkStateReader.getClusterState().getCollectionOrNull(collection);
if (dc == null) return null;
Map<String,Slice> slices = dc.getSlicesMap();
if (slices != null) {
for (Slice slice : slices.values()) {
for (Replica replica : slice.getReplicas()) {
@ -291,10 +293,10 @@ public class OverseerTest extends SolrTestCaseJ4 {
for (int i = 0; i < numShards; i++) {
assertNotNull("shard got no id?", zkController.publishState(COLLECTION, "core" + (i+1), "node" + (i+1), "shard"+((i%3)+1), Replica.State.ACTIVE, 3));
}
final Map<String,Replica> rmap = reader.getClusterState().getSlice(COLLECTION, "shard1").getReplicasMap();
final Map<String,Replica> rmap = reader.getClusterState().getCollection(COLLECTION).getSlice("shard1").getReplicasMap();
assertEquals(rmap.toString(), 2, rmap.size());
assertEquals(rmap.toString(), 2, reader.getClusterState().getSlice(COLLECTION, "shard2").getReplicasMap().size());
assertEquals(rmap.toString(), 2, reader.getClusterState().getSlice(COLLECTION, "shard3").getReplicasMap().size());
assertEquals(rmap.toString(), 2, reader.getClusterState().getCollection(COLLECTION).getSlice("shard2").getReplicasMap().size());
assertEquals(rmap.toString(), 2, reader.getClusterState().getCollection(COLLECTION).getSlice("shard3").getReplicasMap().size());
//make sure leaders are in cloud state
assertNotNull(reader.getLeaderUrl(COLLECTION, "shard1", 15000));
@ -343,9 +345,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
"node" + (i+1), "shard"+((i%3)+1) , Replica.State.ACTIVE, 3));
}
assertEquals(1, reader.getClusterState().getSlice(COLLECTION, "shard1").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getSlice(COLLECTION, "shard2").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getSlice(COLLECTION, "shard3").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getCollection(COLLECTION).getSlice("shard1").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getCollection(COLLECTION).getSlice("shard2").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getCollection(COLLECTION).getSlice("shard3").getReplicasMap().size());
//make sure leaders are in cloud state
assertNotNull(reader.getLeaderUrl(COLLECTION, "shard1", 15000));
@ -364,9 +366,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
"core" + (i + 1), "node" + (i + 1),"shard"+((i%3)+1), Replica.State.ACTIVE, 3));
}
assertEquals(1, reader.getClusterState().getSlice("collection2", "shard1").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getSlice("collection2", "shard2").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getSlice("collection2", "shard3").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getCollection("collection2").getSlice("shard1").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getCollection("collection2").getSlice("shard2").getReplicasMap().size());
assertEquals(1, reader.getClusterState().getCollection("collection2").getSlice("shard3").getReplicasMap().size());
//make sure leaders are in cloud state
assertNotNull(reader.getLeaderUrl("collection2", "shard1", 15000));
@ -474,7 +476,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
private void verifyShardLeader(ZkStateReader reader, String collection, String shard, String expectedCore) throws InterruptedException, KeeperException {
int maxIterations = 200;
while(maxIterations-->0) {
ZkNodeProps props = reader.getClusterState().getLeader(collection, shard);
ZkNodeProps props = reader.getClusterState().getCollection(collection).getLeader(shard);
if(props!=null) {
if(expectedCore.equals(props.getStr(ZkStateReader.CORE_NAME_PROP))) {
return;
@ -482,8 +484,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
}
Thread.sleep(200);
}
assertEquals("Unexpected shard leader coll:" + collection + " shard:" + shard, expectedCore, (reader.getClusterState().getLeader(collection, shard)!=null)?reader.getClusterState().getLeader(collection, shard).getStr(ZkStateReader.CORE_NAME_PROP):null);
DocCollection docCollection = reader.getClusterState().getCollection(collection);
assertEquals("Unexpected shard leader coll:" + collection + " shard:" + shard, expectedCore,
(docCollection.getLeader(shard)!=null)?docCollection.getLeader(shard).getStr(ZkStateReader.CORE_NAME_PROP):null);
}
@Test
@ -553,7 +556,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
assertEquals("Live nodes count does not match", 1, reader
.getClusterState().getLiveNodes().size());
assertEquals(shard+" replica count does not match", 1, reader.getClusterState()
.getSlice(COLLECTION, shard).getReplicasMap().size());
.getCollection(COLLECTION).getSlice(shard).getReplicasMap().size());
version = getClusterStateVersion(zkClient);
mockController.publishState(COLLECTION, core, core_node, "shard1", null, numShards);
while (version == getClusterStateVersion(zkClient));
@ -1004,12 +1007,13 @@ public class OverseerTest extends SolrTestCaseJ4 {
queue.offer(Utils.toJSON(m));
for(int i=0;i<100;i++) {
Slice s = reader.getClusterState().getSlice(COLLECTION, "shard1");
DocCollection dc = reader.getClusterState().getCollectionOrNull(COLLECTION);
Slice s = dc == null? null : dc.getSlice("shard1");
if(s!=null && s.getReplicasMap().size()==3) break;
Thread.sleep(100);
}
assertNotNull(reader.getClusterState().getSlice(COLLECTION, "shard1"));
assertEquals(3, reader.getClusterState().getSlice(COLLECTION, "shard1").getReplicasMap().size());
assertNotNull(reader.getClusterState().getCollection(COLLECTION).getSlice("shard1"));
assertEquals(3, reader.getClusterState().getCollection(COLLECTION).getSlice("shard1").getReplicasMap().size());
} finally {
close(overseerClient);
close(zkClient);
@ -1278,7 +1282,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
{
int iterationsLeft = 100;
while (iterationsLeft-- > 0) {
final Slice slice = zkStateReader.getClusterState().getSlice(COLLECTION, "shard"+ss);
final Slice slice = zkStateReader.getClusterState().getCollection(COLLECTION).getSlice("shard"+ss);
if (null == slice || null == slice.getReplicasMap().get("core_node"+N)) {
break;
}

View File

@ -57,7 +57,8 @@ public abstract class ReplicaPropertiesBase extends AbstractFullDistribZkTestBas
Replica replica = null;
for (int idx = 0; idx < 300; ++idx) {
clusterState = client.getZkStateReader().getClusterState();
replica = clusterState.getReplica(collectionName, replicaName);
final DocCollection docCollection = clusterState.getCollectionOrNull(collectionName);
replica = (docCollection == null) ? null : docCollection.getReplica(replicaName);
if (replica == null) {
fail("Could not find collection/replica pair! " + collectionName + "/" + replicaName);
}
@ -82,7 +83,8 @@ public abstract class ReplicaPropertiesBase extends AbstractFullDistribZkTestBas
for (int idx = 0; idx < 300; ++idx) { // Keep trying while Overseer writes the ZK state for up to 30 seconds.
clusterState = client.getZkStateReader().getClusterState();
replica = clusterState.getReplica(collectionName, replicaName);
final DocCollection docCollection = clusterState.getCollectionOrNull(collectionName);
replica = (docCollection == null) ? null : docCollection.getReplica(replicaName);
if (replica == null) {
fail("Could not find collection/replica pair! " + collectionName + "/" + replicaName);
}

View File

@ -537,7 +537,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
private void incompleteOrOverlappingCustomRangeTest() throws Exception {
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
final DocRouter router = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getRouter();
Slice shard1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
Slice shard1 = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getSlice(SHARD1);
DocRouter.Range shard1Range = shard1.getRange() != null ? shard1.getRange() : router.fullRange();
List<DocRouter.Range> subRanges = new ArrayList<>();
@ -581,7 +581,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
private void splitByUniqueKeyTest() throws Exception {
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
final DocRouter router = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getRouter();
Slice shard1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD1);
Slice shard1 = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getSlice(SHARD1);
DocRouter.Range shard1Range = shard1.getRange() != null ? shard1.getRange() : router.fullRange();
List<DocRouter.Range> subRanges = new ArrayList<>();
if (usually()) {
@ -696,7 +696,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
final DocRouter router = clusterState.getCollection(collectionName).getRouter();
Slice shard1 = clusterState.getSlice(collectionName, SHARD1);
Slice shard1 = clusterState.getCollection(collectionName).getSlice(SHARD1);
DocRouter.Range shard1Range = shard1.getRange() != null ? shard1.getRange() : router.fullRange();
final List<DocRouter.Range> ranges = router.partitionRange(2, shard1Range);
final int[] docCounts = new int[ranges.size()];
@ -772,7 +772,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
final DocRouter router = clusterState.getCollection(collectionName).getRouter();
Slice shard1 = clusterState.getSlice(collectionName, SHARD1);
Slice shard1 = clusterState.getCollection(collectionName).getSlice(SHARD1);
DocRouter.Range shard1Range = shard1.getRange() != null ? shard1.getRange() : router.fullRange();
final List<DocRouter.Range> ranges = ((CompositeIdRouter) router).partitionRangeByKey(splitKey, shard1Range);
final int[] docCounts = new int[ranges.size()];
@ -835,8 +835,8 @@ public class ShardSplitTest extends BasicDistributedZkTest {
for (i = 0; i < 10; i++) {
ZkStateReader zkStateReader = cloudClient.getZkStateReader();
clusterState = zkStateReader.getClusterState();
slice1_0 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, "shard1_0");
slice1_1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, "shard1_1");
slice1_0 = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getSlice("shard1_0");
slice1_1 = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getSlice("shard1_1");
if (slice1_0.getState() == Slice.State.ACTIVE && slice1_1.getState() == Slice.State.ACTIVE) {
break;
}
@ -887,7 +887,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
query.set("distrib", false);
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
Slice slice = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, shard);
Slice slice = clusterState.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION).getSlice(shard);
long[] numFound = new long[slice.getReplicasMap().size()];
int c = 0;
for (Replica replica : slice.getReplicas()) {

View File

@ -388,7 +388,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
private void assertSingleReplicationAndShardSize(String collection, int numSlices) {
Collection<Slice> slices;
slices = cloudClient.getZkStateReader().getClusterState().getActiveSlices(collection);
slices = cloudClient.getZkStateReader().getClusterState().getCollection(collection).getActiveSlices();
assertEquals(numSlices, slices.size());
for (Slice slice : slices) {
assertEquals(1, slice.getReplicas().size());
@ -397,7 +397,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
private void assertSliceAndReplicaCount(String collection) {
Collection<Slice> slices;
slices = cloudClient.getZkStateReader().getClusterState().getActiveSlices(collection);
slices = cloudClient.getZkStateReader().getClusterState().getCollection(collection).getActiveSlices();
assertEquals(2, slices.size());
for (Slice slice : slices) {
assertEquals(2, slice.getReplicas().size());

View File

@ -53,6 +53,6 @@ public class SliceStateTest extends SolrTestCaseJ4 {
byte[] bytes = Utils.toJSON(clusterState);
ClusterState loadedClusterState = ClusterState.load(-1, bytes, liveNodes);
assertSame("Default state not set to active", Slice.State.ACTIVE, loadedClusterState.getSlice("collection1", "shard1").getState());
assertSame("Default state not set to active", Slice.State.ACTIVE, loadedClusterState.getCollection("collection1").getSlice("shard1").getState());
}
}

View File

@ -126,7 +126,7 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
urlMap.put(nodeKey, jettyURL.toString());
}
ClusterState clusterState = zkStateReader.getClusterState();
for (Slice slice : clusterState.getSlices(COLLECTION_NAME)) {
for (Slice slice : clusterState.getCollection(COLLECTION_NAME).getSlices()) {
String shardName = slice.getName();
Replica leader = slice.getLeader();
assertNotNull("slice has null leader: " + slice.toString(), leader);

View File

@ -37,6 +37,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrException;
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.params.CollectionParams;
@ -768,10 +769,11 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
client.getZkStateReader().forceUpdateCollection(collectionName);
ClusterState clusterState = client.getZkStateReader().getClusterState();
Replica replica = clusterState.getReplica(collectionName, replicaName);
if (replica == null) {
final DocCollection docCollection = clusterState.getCollectionOrNull(collectionName);
if (docCollection == null || docCollection.getReplica(replicaName) == null) {
fail("Could not find collection/replica pair! " + collectionName + "/" + replicaName);
}
Replica replica = docCollection.getReplica(replicaName);
Map<String, String> propMap = new HashMap<>();
for (String prop : props) {
propMap.put(prop, replica.getStr(prop));

View File

@ -98,7 +98,7 @@ public class TestLeaderElectionWithEmptyReplica extends SolrCloudTestCase {
(n, c) -> DocCollection.isFullyActive(n, c, 1, 2));
// now query each replica and check for consistency
assertConsistentReplicas(solrClient, solrClient.getZkStateReader().getClusterState().getSlice(COLLECTION_NAME, "shard1"));
assertConsistentReplicas(solrClient, solrClient.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlice("shard1"));
// sanity check that documents still exist
QueryResponse response = solrClient.query(new SolrQuery("*:*"));

View File

@ -57,7 +57,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
}
}
assertNotNull(notLeader);
Replica replica = cloudClient.getZkStateReader().getClusterState().getReplica(DEFAULT_COLLECTION, notLeader.coreNodeName);
Replica replica = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getReplica(notLeader.coreNodeName);
ZkCoreNodeProps replicaCoreNodeProps = new ZkCoreNodeProps(replica);
MockCoreDescriptor cd = new MockCoreDescriptor() {
@ -175,7 +175,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
timeOut = new TimeOut(30, TimeUnit.SECONDS);
while (!timeOut.hasTimedOut()) {
Replica r = cloudClient.getZkStateReader().getClusterState().getReplica(DEFAULT_COLLECTION, replica.getName());
Replica r = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getReplica(replica.getName());
if (r.getState() == Replica.State.DOWN) {
break;
}
@ -183,7 +183,7 @@ public class TestLeaderInitiatedRecoveryThread extends AbstractFullDistribZkTest
}
assertNull(zkController.getLeaderInitiatedRecoveryState(DEFAULT_COLLECTION, SHARD1, replica.getName()));
assertEquals(Replica.State.DOWN, cloudClient.getZkStateReader().getClusterState().getReplica(DEFAULT_COLLECTION, replica.getName()).getState());
assertEquals(Replica.State.DOWN, cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getReplica(replica.getName()).getState());
/*
6. Test that non-leader cannot set LIR nodes

View File

@ -0,0 +1,389 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.cloud;
import java.lang.invoke.MethodHandles;
import java.net.URL;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.embedded.JettyConfig;
import org.apache.solr.client.solrj.embedded.JettyConfig.Builder;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.index.TieredMergePolicyFactory;
import org.apache.solr.util.RevertDefaultThreadHandlerRule;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.RuleChain;
import org.junit.rules.TestRule;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Test of the MiniSolrCloudCluster functionality. Keep in mind,
* MiniSolrCloudCluster is designed to be used outside of the Lucene test
* hierarchy.
*/
@SuppressSysoutChecks(bugUrl = "Solr logs to JUL")
public class TestMiniSolrCloudCluster extends LuceneTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
protected int NUM_SERVERS = 5;
protected int NUM_SHARDS = 2;
protected int REPLICATION_FACTOR = 2;
public TestMiniSolrCloudCluster () {
NUM_SERVERS = 5;
NUM_SHARDS = 2;
REPLICATION_FACTOR = 2;
}
@BeforeClass
public static void setupHackNumerics() { // SOLR-10916
SolrTestCaseJ4.randomizeNumericTypesProperties();
}
@AfterClass
public static void clearHackNumerics() { // SOLR-10916
SolrTestCaseJ4.clearNumericTypesProperties();
}
@Rule
public TestRule solrTestRules = RuleChain
.outerRule(new SystemPropertiesRestoreRule());
@ClassRule
public static TestRule solrClassRules = RuleChain.outerRule(
new SystemPropertiesRestoreRule()).around(
new RevertDefaultThreadHandlerRule());
private MiniSolrCloudCluster createMiniSolrCloudCluster() throws Exception {
Builder jettyConfig = JettyConfig.builder();
jettyConfig.waitForLoadingCoresToFinish(null);
return new MiniSolrCloudCluster(NUM_SERVERS, createTempDir(), jettyConfig.build());
}
private void createCollection(MiniSolrCloudCluster miniCluster, String collectionName, String createNodeSet, String asyncId,
Boolean indexToPersist, Map<String,String> collectionProperties) throws Exception {
String configName = "solrCloudCollectionConfig";
miniCluster.uploadConfigSet(SolrTestCaseJ4.TEST_PATH().resolve("collection1").resolve("conf"), configName);
final boolean persistIndex = (indexToPersist != null ? indexToPersist.booleanValue() : random().nextBoolean());
if (collectionProperties == null) {
collectionProperties = new HashMap<>();
}
collectionProperties.putIfAbsent(CoreDescriptor.CORE_CONFIG, "solrconfig-tlog.xml");
collectionProperties.putIfAbsent("solr.tests.maxBufferedDocs", "100000");
collectionProperties.putIfAbsent("solr.tests.ramBufferSizeMB", "100");
// use non-test classes so RandomizedRunner isn't necessary
collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, TieredMergePolicyFactory.class.getName());
collectionProperties.putIfAbsent("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler");
collectionProperties.putIfAbsent("solr.directoryFactory", (persistIndex ? "solr.StandardDirectoryFactory" : "solr.RAMDirectoryFactory"));
if (asyncId == null) {
CollectionAdminRequest.createCollection(collectionName, configName, NUM_SHARDS, REPLICATION_FACTOR)
.setCreateNodeSet(createNodeSet)
.setProperties(collectionProperties)
.process(miniCluster.getSolrClient());
}
else {
CollectionAdminRequest.createCollection(collectionName, configName, NUM_SHARDS, REPLICATION_FACTOR)
.setCreateNodeSet(createNodeSet)
.setProperties(collectionProperties)
.processAndWait(miniCluster.getSolrClient(), 30);
}
}
@Test
public void testCollectionCreateSearchDelete() throws Exception {
final String collectionName = "testcollection";
MiniSolrCloudCluster miniCluster = createMiniSolrCloudCluster();
final CloudSolrClient cloudSolrClient = miniCluster.getSolrClient();
try {
assertNotNull(miniCluster.getZkServer());
List<JettySolrRunner> jettys = miniCluster.getJettySolrRunners();
assertEquals(NUM_SERVERS, jettys.size());
for (JettySolrRunner jetty : jettys) {
assertTrue(jetty.isRunning());
}
// shut down a server
log.info("#### Stopping a server");
JettySolrRunner stoppedServer = miniCluster.stopJettySolrRunner(0);
assertTrue(stoppedServer.isStopped());
assertEquals(NUM_SERVERS - 1, miniCluster.getJettySolrRunners().size());
// create a server
log.info("#### Starting a server");
JettySolrRunner startedServer = miniCluster.startJettySolrRunner();
assertTrue(startedServer.isRunning());
assertEquals(NUM_SERVERS, miniCluster.getJettySolrRunners().size());
// create collection
log.info("#### Creating a collection");
final String asyncId = (random().nextBoolean() ? null : "asyncId("+collectionName+".create)="+random().nextInt());
createCollection(miniCluster, collectionName, null, asyncId, null, null);
ZkStateReader zkStateReader = miniCluster.getSolrClient().getZkStateReader();
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
// modify/query collection
log.info("#### updating a querying collection");
cloudSolrClient.setDefaultCollection(collectionName);
SolrInputDocument doc = new SolrInputDocument();
doc.setField("id", "1");
cloudSolrClient.add(doc);
cloudSolrClient.commit();
SolrQuery query = new SolrQuery();
query.setQuery("*:*");
QueryResponse rsp = cloudSolrClient.query(query);
assertEquals(1, rsp.getResults().getNumFound());
// remove a server not hosting any replicas
zkStateReader.forceUpdateCollection(collectionName);
ClusterState clusterState = zkStateReader.getClusterState();
HashMap<String, JettySolrRunner> jettyMap = new HashMap<String, JettySolrRunner>();
for (JettySolrRunner jetty : miniCluster.getJettySolrRunners()) {
String key = jetty.getBaseUrl().toString().substring((jetty.getBaseUrl().getProtocol() + "://").length());
jettyMap.put(key, jetty);
}
Collection<Slice> slices = clusterState.getCollection(collectionName).getSlices();
// track the servers not host repliacs
for (Slice slice : slices) {
jettyMap.remove(slice.getLeader().getNodeName().replace("_solr", "/solr"));
for (Replica replica : slice.getReplicas()) {
jettyMap.remove(replica.getNodeName().replace("_solr", "/solr"));
}
}
assertTrue("Expected to find a node without a replica", jettyMap.size() > 0);
log.info("#### Stopping a server");
JettySolrRunner jettyToStop = jettyMap.entrySet().iterator().next().getValue();
jettys = miniCluster.getJettySolrRunners();
for (int i = 0; i < jettys.size(); ++i) {
if (jettys.get(i).equals(jettyToStop)) {
miniCluster.stopJettySolrRunner(i);
assertEquals(NUM_SERVERS - 1, miniCluster.getJettySolrRunners().size());
}
}
// re-create a server (to restore original NUM_SERVERS count)
log.info("#### Starting a server");
startedServer = miniCluster.startJettySolrRunner(jettyToStop);
assertTrue(startedServer.isRunning());
assertEquals(NUM_SERVERS, miniCluster.getJettySolrRunners().size());
CollectionAdminRequest.deleteCollection(collectionName).process(miniCluster.getSolrClient());
// create it again
String asyncId2 = (random().nextBoolean() ? null : "asyncId("+collectionName+".create)="+random().nextInt());
createCollection(miniCluster, collectionName, null, asyncId2, null, null);
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
// check that there's no left-over state
assertEquals(0, cloudSolrClient.query(new SolrQuery("*:*")).getResults().getNumFound());
cloudSolrClient.add(doc);
cloudSolrClient.commit();
assertEquals(1, cloudSolrClient.query(new SolrQuery("*:*")).getResults().getNumFound());
}
finally {
miniCluster.shutdown();
}
}
@Test
public void testCollectionCreateWithoutCoresThenDelete() throws Exception {
final String collectionName = "testSolrCloudCollectionWithoutCores";
final MiniSolrCloudCluster miniCluster = createMiniSolrCloudCluster();
final CloudSolrClient cloudSolrClient = miniCluster.getSolrClient();
try {
assertNotNull(miniCluster.getZkServer());
assertFalse(miniCluster.getJettySolrRunners().isEmpty());
// create collection
final String asyncId = (random().nextBoolean() ? null : "asyncId("+collectionName+".create)="+random().nextInt());
createCollection(miniCluster, collectionName, OverseerCollectionMessageHandler.CREATE_NODE_SET_EMPTY, asyncId, null, null);
try (SolrZkClient zkClient = new SolrZkClient
(miniCluster.getZkServer().getZkAddress(), AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
ZkStateReader zkStateReader = new ZkStateReader(zkClient)) {
zkStateReader.createClusterStateWatchersAndUpdate();
// wait for collection to appear
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
// check the collection's corelessness
{
int coreCount = 0;
for (Map.Entry<String,Slice> entry : zkStateReader.getClusterState()
.getCollection(collectionName).getSlicesMap().entrySet()) {
coreCount += entry.getValue().getReplicasMap().entrySet().size();
}
assertEquals(0, coreCount);
}
}
}
finally {
miniCluster.shutdown();
}
}
@Test
public void testStopAllStartAll() throws Exception {
final String collectionName = "testStopAllStartAllCollection";
final MiniSolrCloudCluster miniCluster = createMiniSolrCloudCluster();
try {
assertNotNull(miniCluster.getZkServer());
List<JettySolrRunner> jettys = miniCluster.getJettySolrRunners();
assertEquals(NUM_SERVERS, jettys.size());
for (JettySolrRunner jetty : jettys) {
assertTrue(jetty.isRunning());
}
createCollection(miniCluster, collectionName, null, null, Boolean.TRUE, null);
final CloudSolrClient cloudSolrClient = miniCluster.getSolrClient();
cloudSolrClient.setDefaultCollection(collectionName);
final SolrQuery query = new SolrQuery("*:*");
final SolrInputDocument doc = new SolrInputDocument();
try (SolrZkClient zkClient = new SolrZkClient
(miniCluster.getZkServer().getZkAddress(), AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
ZkStateReader zkStateReader = new ZkStateReader(zkClient)) {
zkStateReader.createClusterStateWatchersAndUpdate();
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
// modify collection
final int numDocs = 1 + random().nextInt(10);
for (int ii = 1; ii <= numDocs; ++ii) {
doc.setField("id", ""+ii);
cloudSolrClient.add(doc);
if (ii*2 == numDocs) cloudSolrClient.commit();
}
cloudSolrClient.commit();
// query collection
{
final QueryResponse rsp = cloudSolrClient.query(query);
assertEquals(numDocs, rsp.getResults().getNumFound());
}
// the test itself
zkStateReader.forceUpdateCollection(collectionName);
final ClusterState clusterState = zkStateReader.getClusterState();
final HashSet<Integer> leaderIndices = new HashSet<Integer>();
final HashSet<Integer> followerIndices = new HashSet<Integer>();
{
final HashMap<String,Boolean> shardLeaderMap = new HashMap<String,Boolean>();
for (final Slice slice : clusterState.getCollection(collectionName).getSlices()) {
for (final Replica replica : slice.getReplicas()) {
shardLeaderMap.put(replica.getNodeName().replace("_solr", "/solr"), Boolean.FALSE);
}
shardLeaderMap.put(slice.getLeader().getNodeName().replace("_solr", "/solr"), Boolean.TRUE);
}
for (int ii = 0; ii < jettys.size(); ++ii) {
final URL jettyBaseUrl = jettys.get(ii).getBaseUrl();
final String jettyBaseUrlString = jettyBaseUrl.toString().substring((jettyBaseUrl.getProtocol() + "://").length());
final Boolean isLeader = shardLeaderMap.get(jettyBaseUrlString);
if (Boolean.TRUE.equals(isLeader)) {
leaderIndices.add(new Integer(ii));
} else if (Boolean.FALSE.equals(isLeader)) {
followerIndices.add(new Integer(ii));
} // else neither leader nor follower i.e. node without a replica (for our collection)
}
}
final List<Integer> leaderIndicesList = new ArrayList<Integer>(leaderIndices);
final List<Integer> followerIndicesList = new ArrayList<Integer>(followerIndices);
// first stop the followers (in no particular order)
Collections.shuffle(followerIndicesList, random());
for (Integer ii : followerIndicesList) {
if (!leaderIndices.contains(ii)) {
miniCluster.stopJettySolrRunner(jettys.get(ii.intValue()));
}
}
// then stop the leaders (again in no particular order)
Collections.shuffle(leaderIndicesList, random());
for (Integer ii : leaderIndicesList) {
miniCluster.stopJettySolrRunner(jettys.get(ii.intValue()));
}
// calculate restart order
final List<Integer> restartIndicesList = new ArrayList<Integer>();
Collections.shuffle(leaderIndicesList, random());
restartIndicesList.addAll(leaderIndicesList);
Collections.shuffle(followerIndicesList, random());
restartIndicesList.addAll(followerIndicesList);
if (random().nextBoolean()) Collections.shuffle(restartIndicesList, random());
// and then restart jettys in that order
for (Integer ii : restartIndicesList) {
final JettySolrRunner jetty = jettys.get(ii.intValue());
if (!jetty.isRunning()) {
miniCluster.startJettySolrRunner(jetty);
assertTrue(jetty.isRunning());
}
}
AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
zkStateReader.forceUpdateCollection(collectionName);
// re-query collection
{
final QueryResponse rsp = cloudSolrClient.query(query);
assertEquals(numDocs, rsp.getResults().getNumFound());
}
}
}
finally {
miniCluster.shutdown();
}
}
}

View File

@ -159,7 +159,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
Replica leader = null;
Replica notLeader = null;
Collection<Replica> replicas = cloudClient.getZkStateReader().getClusterState().getSlice("football", "shard1").getReplicas();
Collection<Replica> replicas = cloudClient.getZkStateReader().getClusterState().getCollection("football").getSlice("shard1").getReplicas();
for (Replica replica : replicas) {
if (replica.getStr(ZkStateReader.LEADER_PROP) != null) {
leader = replica;

View File

@ -193,7 +193,7 @@ public class TestReplicaProperties extends ReplicaPropertiesBase {
for (int idx = 0; idx < 300; ++idx) { // Keep trying while Overseer writes the ZK state for up to 30 seconds.
lastFailMsg = "";
ClusterState clusterState = client.getZkStateReader().getClusterState();
for (Slice slice : clusterState.getSlices(collectionName)) {
for (Slice slice : clusterState.getCollection(collectionName).getSlices()) {
Boolean foundLeader = false;
Boolean foundPreferred = false;
for (Replica replica : slice.getReplicas()) {

View File

@ -45,7 +45,7 @@ public class TestShortCircuitedRequests extends AbstractFullDistribZkTestBase {
doQuery("a!doc1", "q", "*:*", ShardParams._ROUTE_, "a!"); // can go to any random node
// query shard3 directly with _route_=a! so that we trigger the short circuited request path
Replica shard3 = cloudClient.getZkStateReader().getClusterState().getLeader(DEFAULT_COLLECTION, "shard3");
Replica shard3 = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getLeader("shard3");
String nodeName = shard3.getNodeName();
SolrClient shard3Client = getClient(nodeName);
QueryResponse response = shard3Client.query(new SolrQuery("*:*").add(ShardParams._ROUTE_, "a!").add(ShardParams.SHARDS_INFO, "true"));

View File

@ -134,7 +134,7 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
}
zkStateReader.forceUpdateCollection(COLLECTION_NAME);
ClusterState clusterState = zkStateReader.getClusterState();
for (Slice slice : clusterState.getSlices(COLLECTION_NAME)) {
for (Slice slice : clusterState.getCollection(COLLECTION_NAME).getSlices()) {
String shardName = slice.getName();
Replica leader = slice.getLeader();
assertNotNull("slice has null leader: " + slice.toString(), leader);

View File

@ -25,6 +25,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
import org.apache.solr.common.SolrInputDocument;
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;
@ -75,7 +76,8 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS);
Boolean isPresent = null; // null meaning "don't know"
while (null == isPresent || shouldBePresent != isPresent.booleanValue()) {
final Collection<Slice> slices = getCommonCloudSolrClient().getZkStateReader().getClusterState().getSlices(collectionName);
final DocCollection docCollection = getCommonCloudSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName);
final Collection<Slice> slices = (docCollection != null) ? docCollection.getSlices() : null;
if (timeout.hasTimedOut()) {
printLayout();
fail("checkCoreNamePresenceAndSliceCount failed:"

View File

@ -33,6 +33,7 @@ import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.cloud.BasicDistributedZkTest;
import org.apache.solr.cloud.ChaosMonkey;
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.params.CollectionParams.CollectionAction;
@ -160,8 +161,10 @@ public class StressHdfsTest extends BasicDistributedZkTest {
// data dirs should be in zk, SOLR-8913
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
Slice slice = clusterState.getSlice(DELETE_DATA_DIR_COLLECTION, "shard1");
assertNotNull(clusterState.getSlices(DELETE_DATA_DIR_COLLECTION).toString(), slice);
final DocCollection docCollection = clusterState.getCollectionOrNull(DELETE_DATA_DIR_COLLECTION);
assertNotNull("Could not find :"+DELETE_DATA_DIR_COLLECTION, docCollection);
Slice slice = docCollection.getSlice("shard1");
assertNotNull(docCollection.getSlices().toString(), slice);
Collection<Replica> replicas = slice.getReplicas();
for (Replica replica : replicas) {
assertNotNull(replica.getProperties().toString(), replica.get("dataDir"));

View File

@ -0,0 +1,717 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.schema;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.util.BaseTestHarness;
import org.apache.solr.util.RestTestHarness;
import org.apache.zookeeper.data.Stat;
import org.eclipse.jetty.servlet.ServletHolder;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.restlet.ext.servlet.ServerServlet;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Ignore
public class TestCloudManagedSchemaConcurrent extends AbstractFullDistribZkTestBase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final String SUCCESS_XPATH = "/response/lst[@name='responseHeader']/int[@name='status'][.='0']";
private static final String PUT_DYNAMIC_FIELDNAME = "newdynamicfieldPut";
private static final String POST_DYNAMIC_FIELDNAME = "newdynamicfieldPost";
private static final String PUT_FIELDNAME = "newfieldPut";
private static final String POST_FIELDNAME = "newfieldPost";
private static final String PUT_FIELDTYPE = "newfieldtypePut";
private static final String POST_FIELDTYPE = "newfieldtypePost";
public TestCloudManagedSchemaConcurrent() {
super();
sliceCount = 4;
}
@BeforeClass
public static void initSysProperties() {
System.setProperty("managed.schema.mutable", "true");
System.setProperty("enable.update.log", "true");
}
@Override
public void distribTearDown() throws Exception {
super.distribTearDown();
for (RestTestHarness h : restTestHarnesses) {
h.close();
}
}
@Override
protected String getCloudSolrConfig() {
return "solrconfig-managed-schema.xml";
}
@Override
public SortedMap<ServletHolder,String> getExtraServlets() {
final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
extraServlets.put(solrRestApi, "/schema/*"); // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
return extraServlets;
}
private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
private void setupHarnesses() {
for (final SolrClient client : clients) {
RestTestHarness harness = new RestTestHarness(() -> ((HttpSolrClient)client).getBaseURL());
restTestHarnesses.add(harness);
}
}
private static void verifySuccess(String request, String response) throws Exception {
String result = BaseTestHarness.validateXPath(response, SUCCESS_XPATH);
if (null != result) {
String msg = "QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response;
log.error(msg);
fail(msg);
}
}
private static void addFieldPut(RestTestHarness publisher, String fieldName, int updateTimeoutSecs) throws Exception {
final String content = "{\"type\":\"text\",\"stored\":\"false\"}";
String request = "/schema/fields/" + fieldName + "?wt=xml";
if (updateTimeoutSecs > 0)
request += "&updateTimeoutSecs="+updateTimeoutSecs;
String response = publisher.put(request, content);
verifySuccess(request, response);
}
private static void addFieldPost(RestTestHarness publisher, String fieldName, int updateTimeoutSecs) throws Exception {
final String content = "[{\"name\":\""+fieldName+"\",\"type\":\"text\",\"stored\":\"false\"}]";
String request = "/schema/fields/?wt=xml";
if (updateTimeoutSecs > 0)
request += "&updateTimeoutSecs="+updateTimeoutSecs;
String response = publisher.post(request, content);
verifySuccess(request, response);
}
private static void addDynamicFieldPut(RestTestHarness publisher, String dynamicFieldPattern, int updateTimeoutSecs) throws Exception {
final String content = "{\"type\":\"text\",\"stored\":\"false\"}";
String request = "/schema/dynamicfields/" + dynamicFieldPattern + "?wt=xml";
if (updateTimeoutSecs > 0)
request += "&updateTimeoutSecs="+updateTimeoutSecs;
String response = publisher.put(request, content);
verifySuccess(request, response);
}
private static void addDynamicFieldPost(RestTestHarness publisher, String dynamicFieldPattern, int updateTimeoutSecs) throws Exception {
final String content = "[{\"name\":\""+dynamicFieldPattern+"\",\"type\":\"text\",\"stored\":\"false\"}]";
String request = "/schema/dynamicfields/?wt=xml";
if (updateTimeoutSecs > 0)
request += "&updateTimeoutSecs="+updateTimeoutSecs;
String response = publisher.post(request, content);
verifySuccess(request, response);
}
private static void copyField(RestTestHarness publisher, String source, String dest, int updateTimeoutSecs) throws Exception {
final String content = "[{\"source\":\""+source+"\",\"dest\":[\""+dest+"\"]}]";
String request = "/schema/copyfields/?wt=xml";
if (updateTimeoutSecs > 0)
request += "&updateTimeoutSecs="+updateTimeoutSecs;
String response = publisher.post(request, content);
verifySuccess(request, response);
}
private static void addFieldTypePut(RestTestHarness publisher, String typeName, int updateTimeoutSecs) throws Exception {
final String content = "{\"class\":\""+RANDOMIZED_NUMERIC_FIELDTYPES.get(Integer.class)+"\"}";
String request = "/schema/fieldtypes/" + typeName + "?wt=xml";
if (updateTimeoutSecs > 0)
request += "&updateTimeoutSecs="+updateTimeoutSecs;
String response = publisher.put(request, content);
verifySuccess(request, response);
}
private static void addFieldTypePost(RestTestHarness publisher, String typeName, int updateTimeoutSecs) throws Exception {
final String content = "[{\"name\":\""+typeName+"\",\"class\":\""+RANDOMIZED_NUMERIC_FIELDTYPES.get(Integer.class)+"\"}]";
String request = "/schema/fieldtypes/?wt=xml";
if (updateTimeoutSecs > 0)
request += "&updateTimeoutSecs="+updateTimeoutSecs;
String response = publisher.post(request, content);
verifySuccess(request, response);
}
private String[] getExpectedFieldResponses(Info info) {
String[] expectedAddFields = new String[1 + info.numAddFieldPuts + info.numAddFieldPosts];
expectedAddFields[0] = SUCCESS_XPATH;
for (int i = 0; i < info.numAddFieldPuts; ++i) {
String newFieldName = PUT_FIELDNAME + info.fieldNameSuffix + i;
expectedAddFields[1 + i]
= "/response/arr[@name='fields']/lst/str[@name='name'][.='" + newFieldName + "']";
}
for (int i = 0; i < info.numAddFieldPosts; ++i) {
String newFieldName = POST_FIELDNAME + info.fieldNameSuffix + i;
expectedAddFields[1 + info.numAddFieldPuts + i]
= "/response/arr[@name='fields']/lst/str[@name='name'][.='" + newFieldName + "']";
}
return expectedAddFields;
}
private String[] getExpectedDynamicFieldResponses(Info info) {
String[] expectedAddDynamicFields = new String[1 + info.numAddDynamicFieldPuts + info.numAddDynamicFieldPosts];
expectedAddDynamicFields[0] = SUCCESS_XPATH;
for (int i = 0; i < info.numAddDynamicFieldPuts; ++i) {
String newDynamicFieldPattern = PUT_DYNAMIC_FIELDNAME + info.fieldNameSuffix + i + "_*";
expectedAddDynamicFields[1 + i]
= "/response/arr[@name='dynamicFields']/lst/str[@name='name'][.='" + newDynamicFieldPattern + "']";
}
for (int i = 0; i < info.numAddDynamicFieldPosts; ++i) {
String newDynamicFieldPattern = POST_DYNAMIC_FIELDNAME + info.fieldNameSuffix + i + "_*";
expectedAddDynamicFields[1 + info.numAddDynamicFieldPuts + i]
= "/response/arr[@name='dynamicFields']/lst/str[@name='name'][.='" + newDynamicFieldPattern + "']";
}
return expectedAddDynamicFields;
}
private String[] getExpectedCopyFieldResponses(Info info) {
ArrayList<String> expectedCopyFields = new ArrayList<>();
expectedCopyFields.add(SUCCESS_XPATH);
for (CopyFieldInfo cpi : info.copyFields) {
String expectedSourceName = cpi.getSourceField();
expectedCopyFields.add
("/response/arr[@name='copyFields']/lst/str[@name='source'][.='" + expectedSourceName + "']");
String expectedDestName = cpi.getDestField();
expectedCopyFields.add
("/response/arr[@name='copyFields']/lst/str[@name='dest'][.='" + expectedDestName + "']");
}
return expectedCopyFields.toArray(new String[expectedCopyFields.size()]);
}
private String[] getExpectedFieldTypeResponses(Info info) {
String[] expectedAddFieldTypes = new String[1 + info.numAddFieldTypePuts + info.numAddFieldTypePosts];
expectedAddFieldTypes[0] = SUCCESS_XPATH;
for (int i = 0; i < info.numAddFieldTypePuts; ++i) {
String newFieldTypeName = PUT_FIELDTYPE + info.fieldNameSuffix + i;
expectedAddFieldTypes[1 + i]
= "/response/arr[@name='fieldTypes']/lst/str[@name='name'][.='" + newFieldTypeName + "']";
}
for (int i = 0; i < info.numAddFieldTypePosts; ++i) {
String newFieldTypeName = POST_FIELDTYPE + info.fieldNameSuffix + i;
expectedAddFieldTypes[1 + info.numAddFieldTypePuts + i]
= "/response/arr[@name='fieldTypes']/lst/str[@name='name'][.='" + newFieldTypeName + "']";
}
return expectedAddFieldTypes;
}
@Test
@ShardsFixed(num = 8)
public void test() throws Exception {
verifyWaitForSchemaUpdateToPropagate();
setupHarnesses();
concurrentOperationsTest();
schemaLockTest();
}
private static class Info {
int numAddFieldPuts = 0;
int numAddFieldPosts = 0;
int numAddDynamicFieldPuts = 0;
int numAddDynamicFieldPosts = 0;
int numAddFieldTypePuts = 0;
int numAddFieldTypePosts = 0;
public String fieldNameSuffix;
List<CopyFieldInfo> copyFields = new ArrayList<>();
public Info(String fieldNameSuffix) {
this.fieldNameSuffix = fieldNameSuffix;
}
}
private enum Operation {
PUT_AddField {
@Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
String fieldname = PUT_FIELDNAME + info.numAddFieldPuts++;
addFieldPut(publisher, fieldname, 15);
}
},
POST_AddField {
@Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
String fieldname = POST_FIELDNAME + info.numAddFieldPosts++;
addFieldPost(publisher, fieldname, 15);
}
},
PUT_AddDynamicField {
@Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
addDynamicFieldPut(publisher, PUT_DYNAMIC_FIELDNAME + info.numAddDynamicFieldPuts++ + "_*", 15);
}
},
POST_AddDynamicField {
@Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
addDynamicFieldPost(publisher, POST_DYNAMIC_FIELDNAME + info.numAddDynamicFieldPosts++ + "_*", 15);
}
},
POST_AddCopyField {
@Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
String sourceField = null;
String destField = null;
int sourceType = random().nextInt(3);
if (sourceType == 0) { // existing
sourceField = "name";
} else if (sourceType == 1) { // newly created
sourceField = "copySource" + fieldNum;
addFieldPut(publisher, sourceField, 15);
} else { // dynamic
sourceField = "*_dynamicSource" + fieldNum + "_t";
// * only supported if both src and dst use it
destField = "*_dynamicDest" + fieldNum + "_t";
}
if (destField == null) {
int destType = random().nextInt(2);
if (destType == 0) { // existing
destField = "title";
} else { // newly created
destField = "copyDest" + fieldNum;
addFieldPut(publisher, destField, 15);
}
}
copyField(publisher, sourceField, destField, 15);
info.copyFields.add(new CopyFieldInfo(sourceField, destField));
}
},
PUT_AddFieldType {
@Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
String typeName = PUT_FIELDTYPE + info.numAddFieldTypePuts++;
addFieldTypePut(publisher, typeName, 15);
}
},
POST_AddFieldType {
@Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
String typeName = POST_FIELDTYPE + info.numAddFieldTypePosts++;
addFieldTypePost(publisher, typeName, 15);
}
};
public abstract void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception;
private static final Operation[] VALUES = values();
public static Operation randomOperation() {
return VALUES[r.nextInt(VALUES.length)];
}
}
private void verifyWaitForSchemaUpdateToPropagate() throws Exception {
String testCollectionName = "collection1";
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
Replica shard1Leader = clusterState.getCollection(testCollectionName).getLeader("shard1");
final String coreUrl = (new ZkCoreNodeProps(shard1Leader)).getCoreUrl();
assertNotNull(coreUrl);
RestTestHarness harness = new RestTestHarness(() -> coreUrl.endsWith("/") ? coreUrl.substring(0, coreUrl.length()-1) : coreUrl);
try {
addFieldTypePut(harness, "fooInt", 15);
} finally {
harness.close();
}
// go into ZK to get the version of the managed schema after the update
SolrZkClient zkClient = cloudClient.getZkStateReader().getZkClient();
Stat stat = new Stat();
String znodePath = "/configs/conf1/managed-schema";
byte[] managedSchemaBytes = zkClient.getData(znodePath, null, stat, false);
int schemaZkVersion = stat.getVersion();
// now loop over all replicas and verify each has the same schema version
Replica randomReplicaNotLeader = null;
for (Slice slice : clusterState.getCollection(testCollectionName).getActiveSlices()) {
for (Replica replica : slice.getReplicas()) {
validateZkVersion(replica, schemaZkVersion, 0, false);
// save a random replica to test zk watcher behavior
if (randomReplicaNotLeader == null && !replica.getName().equals(shard1Leader.getName()))
randomReplicaNotLeader = replica;
}
}
assertNotNull(randomReplicaNotLeader);
// now update the data and then verify the znode watcher fires correctly
// before an after a zk session expiration (see SOLR-6249)
zkClient.setData(znodePath, managedSchemaBytes, schemaZkVersion, false);
stat = new Stat();
managedSchemaBytes = zkClient.getData(znodePath, null, stat, false);
int updatedSchemaZkVersion = stat.getVersion();
assertTrue(updatedSchemaZkVersion > schemaZkVersion);
validateZkVersion(randomReplicaNotLeader, updatedSchemaZkVersion, 2, true);
// ok - looks like the watcher fired correctly on the replica
// now, expire that replica's zk session and then verify the watcher fires again (after reconnect)
JettySolrRunner randomReplicaJetty =
getJettyOnPort(getReplicaPort(randomReplicaNotLeader));
assertNotNull(randomReplicaJetty);
chaosMonkey.expireSession(randomReplicaJetty);
// update the data again to cause watchers to fire
zkClient.setData(znodePath, managedSchemaBytes, updatedSchemaZkVersion, false);
stat = new Stat();
managedSchemaBytes = zkClient.getData(znodePath, null, stat, false);
updatedSchemaZkVersion = stat.getVersion();
// give up to 10 secs for the replica to recover after zk session loss and see the update
validateZkVersion(randomReplicaNotLeader, updatedSchemaZkVersion, 10, true);
}
/**
* Sends a GET request to get the zk schema version from a specific replica.
*/
protected void validateZkVersion(Replica replica, int schemaZkVersion, int waitSecs, boolean retry) throws Exception {
final String replicaUrl = (new ZkCoreNodeProps(replica)).getCoreUrl();
RestTestHarness testHarness = new RestTestHarness(() -> replicaUrl.endsWith("/") ? replicaUrl.substring(0, replicaUrl.length()-1) : replicaUrl);
try {
long waitMs = waitSecs * 1000L;
if (waitMs > 0) Thread.sleep(waitMs); // wait a moment for the zk watcher to fire
try {
testHarness.validateQuery("/schema/zkversion?wt=xml", "//zkversion=" + schemaZkVersion);
} catch (Exception exc) {
if (retry) {
// brief wait before retrying
Thread.sleep(waitMs > 0 ? waitMs : 2000L);
testHarness.validateQuery("/schema/zkversion?wt=xml", "//zkversion=" + schemaZkVersion);
} else {
throw exc;
}
}
} finally {
testHarness.close();
}
}
private void concurrentOperationsTest() throws Exception {
// First, add a bunch of fields and dynamic fields via PUT and POST, as well as copyFields,
// but do it fast enough and verify shards' schemas after all of them are added
int numFields = 100;
Info info = new Info("");
for (int fieldNum = 0; fieldNum <= numFields ; ++fieldNum) {
RestTestHarness publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
Operation.randomOperation().execute(publisher, fieldNum, info);
}
String[] expectedAddFields = getExpectedFieldResponses(info);
String[] expectedAddDynamicFields = getExpectedDynamicFieldResponses(info);
String[] expectedCopyFields = getExpectedCopyFieldResponses(info);
String[] expectedAddFieldTypes = getExpectedFieldTypeResponses(info);
boolean success = false;
long maxTimeoutMillis = 100000;
long startTime = System.nanoTime();
String request = null;
String response = null;
String result = null;
while ( ! success
&& TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutMillis) {
Thread.sleep(100);
for (RestTestHarness client : restTestHarnesses) {
// verify addFieldTypePuts and addFieldTypePosts
request = "/schema/fieldtypes?wt=xml";
response = client.query(request);
result = BaseTestHarness.validateXPath(response, expectedAddFieldTypes);
if (result != null) {
break;
}
// verify addFieldPuts and addFieldPosts
request = "/schema/fields?wt=xml";
response = client.query(request);
result = BaseTestHarness.validateXPath(response, expectedAddFields);
if (result != null) {
break;
}
// verify addDynamicFieldPuts and addDynamicFieldPosts
request = "/schema/dynamicfields?wt=xml";
response = client.query(request);
result = BaseTestHarness.validateXPath(response, expectedAddDynamicFields);
if (result != null) {
break;
}
// verify copyFields
request = "/schema/copyfields?wt=xml";
response = client.query(request);
result = BaseTestHarness.validateXPath(response, expectedCopyFields);
if (result != null) {
break;
}
}
success = (result == null);
}
if ( ! success) {
String msg = "QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response;
log.error(msg);
fail(msg);
}
}
private abstract class PutPostThread extends Thread {
RestTestHarness harness;
Info info;
public String fieldName;
public PutPostThread(RestTestHarness harness, Info info) {
this.harness = harness;
this.info = info;
}
public abstract void run();
}
private class PutFieldThread extends PutPostThread {
public PutFieldThread(RestTestHarness harness, Info info) {
super(harness, info);
fieldName = PUT_FIELDNAME + "Thread" + info.numAddFieldPuts++;
}
public void run() {
try {
// don't have the client side wait for all replicas to see the update or that defeats the purpose
// of testing the locking support on the server-side
addFieldPut(harness, fieldName, -1);
} catch (Exception e) {
// log.error("###ACTUAL FAILURE!");
throw new RuntimeException(e);
}
}
}
private class PostFieldThread extends PutPostThread {
public PostFieldThread(RestTestHarness harness, Info info) {
super(harness, info);
fieldName = POST_FIELDNAME + "Thread" + info.numAddFieldPosts++;
}
public void run() {
try {
addFieldPost(harness, fieldName, -1);
} catch (Exception e) {
// log.error("###ACTUAL FAILURE!");
throw new RuntimeException(e);
}
}
}
private class PutFieldTypeThread extends PutPostThread {
public PutFieldTypeThread(RestTestHarness harness, Info info) {
super(harness, info);
fieldName = PUT_FIELDTYPE + "Thread" + info.numAddFieldTypePuts++;
}
public void run() {
try {
addFieldTypePut(harness, fieldName, -1);
} catch (Exception e) {
// log.error("###ACTUAL FAILURE!");
throw new RuntimeException(e);
}
}
}
private class PostFieldTypeThread extends PutPostThread {
public PostFieldTypeThread(RestTestHarness harness, Info info) {
super(harness, info);
fieldName = POST_FIELDTYPE + "Thread" + info.numAddFieldTypePosts++;
}
public void run() {
try {
addFieldTypePost(harness, fieldName, -1);
} catch (Exception e) {
// log.error("###ACTUAL FAILURE!");
throw new RuntimeException(e);
}
}
}
private class PutDynamicFieldThread extends PutPostThread {
public PutDynamicFieldThread(RestTestHarness harness, Info info) {
super(harness, info);
fieldName = PUT_FIELDNAME + "Thread" + info.numAddFieldPuts++;
}
public void run() {
try {
addFieldPut(harness, fieldName, -1);
} catch (Exception e) {
// log.error("###ACTUAL FAILURE!");
throw new RuntimeException(e);
}
}
}
private class PostDynamicFieldThread extends PutPostThread {
public PostDynamicFieldThread(RestTestHarness harness, Info info) {
super(harness, info);
fieldName = POST_FIELDNAME + "Thread" + info.numAddFieldPosts++;
}
public void run() {
try {
addFieldPost(harness, fieldName, -1);
} catch (Exception e) {
// log.error("###ACTUAL FAILURE!");
throw new RuntimeException(e);
}
}
}
private void schemaLockTest() throws Exception {
// First, add a bunch of fields via PUT and POST, as well as copyFields,
// but do it fast enough and verify shards' schemas after all of them are added
int numFields = 5;
Info info = new Info("Thread");
for (int i = 0; i <= numFields ; ++i) {
// System.err.println("###ITERATION: " + i);
RestTestHarness publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
PostFieldThread postFieldThread = new PostFieldThread(publisher, info);
postFieldThread.start();
publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
PutFieldThread putFieldThread = new PutFieldThread(publisher, info);
putFieldThread.start();
publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
PostDynamicFieldThread postDynamicFieldThread = new PostDynamicFieldThread(publisher, info);
postDynamicFieldThread.start();
publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
PutDynamicFieldThread putDynamicFieldThread = new PutDynamicFieldThread(publisher, info);
putDynamicFieldThread.start();
publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
PostFieldTypeThread postFieldTypeThread = new PostFieldTypeThread(publisher, info);
postFieldTypeThread.start();
publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
PutFieldTypeThread putFieldTypeThread = new PutFieldTypeThread(publisher, info);
putFieldTypeThread.start();
postFieldThread.join();
putFieldThread.join();
postDynamicFieldThread.join();
putDynamicFieldThread.join();
postFieldTypeThread.join();
putFieldTypeThread.join();
String[] expectedAddFields = getExpectedFieldResponses(info);
String[] expectedAddFieldTypes = getExpectedFieldTypeResponses(info);
String[] expectedAddDynamicFields = getExpectedDynamicFieldResponses(info);
boolean success = false;
long maxTimeoutMillis = 100000;
long startTime = System.nanoTime();
String request = null;
String response = null;
String result = null;
while ( ! success
&& TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutMillis) {
Thread.sleep(10);
// int j = 0;
for (RestTestHarness client : restTestHarnesses) {
// System.err.println("###CHECKING HARNESS: " + j++ + " for iteration: " + i);
// verify addFieldPuts and addFieldPosts
request = "/schema/fields?wt=xml";
response = client.query(request);
//System.err.println("###RESPONSE: " + response);
result = BaseTestHarness.validateXPath(response, expectedAddFields);
if (result != null) {
// System.err.println("###FAILURE!");
break;
}
// verify addDynamicFieldPuts and addDynamicFieldPosts
request = "/schema/dynamicfields?wt=xml";
response = client.query(request);
//System.err.println("###RESPONSE: " + response);
result = BaseTestHarness.validateXPath(response, expectedAddDynamicFields);
if (result != null) {
// System.err.println("###FAILURE!");
break;
}
request = "/schema/fieldtypes?wt=xml";
response = client.query(request);
//System.err.println("###RESPONSE: " + response);
result = BaseTestHarness.validateXPath(response, expectedAddFieldTypes);
if (result != null) {
// System.err.println("###FAILURE!");
break;
}
}
success = (result == null);
}
if ( ! success) {
String msg = "QUERY FAILED: xpath=" + result + " request=" + request + " response=" + response;
log.error(msg);
fail(msg);
}
}
}
private static class CopyFieldInfo {
private String sourceField;
private String destField;
public CopyFieldInfo(String sourceField, String destField) {
this.sourceField = sourceField;
this.destField = destField;
}
public String getSourceField() { return sourceField; }
public String getDestField() { return destField; }
}
}

View File

@ -105,7 +105,7 @@ public class TestCloudSchemaless extends AbstractFullDistribZkTestBase {
// This tests that the replicas properly handle schema additions.
int slices = getCommonCloudSolrClient().getZkStateReader().getClusterState()
.getActiveSlices("collection1").size();
.getCollection("collection1").getActiveSlices().size();
int trials = 50;
// generate enough docs so that we can expect at least a doc per slice
int numDocsPerTrial = (int)(slices * (Math.log(slices) + 1));

View File

@ -89,20 +89,6 @@ public class ClusterState implements JSONWriter.Writable {
return result;
}
/**
* 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;
Slice slice = coll.getSlice(sliceName);
if (slice == null) return null;
return slice.getLeader();
}
/**
* Returns true if the specified collection name exists, false otherwise.
*
@ -115,77 +101,6 @@ public class ClusterState implements JSONWriter.Writable {
return getCollectionOrNull(collectionName) != null;
}
/**
* 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) {
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;
return coll.getActiveSlices();
}
/**
* Get the named DocCollection object, or throw an exception if it doesn't exist.
*/
@ -213,27 +128,6 @@ public class ClusterState implements JSONWriter.Writable {
return ref == null ? null : ref.get();
}
/**
* Get collection names.
*
* 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)}
*
* @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()) {
if (entry.getValue().get() != null) {
result.add(entry.getKey());
}
}
return result;
}
/**
* Get a map of collection name vs DocCollection objects
*
@ -397,6 +291,7 @@ public class ClusterState implements JSONWriter.Writable {
*
* @return null if ClusterState was created for publication, not consumption
* @deprecated true cluster state spans many ZK nodes, stop depending on the version number of the shared node!
* will be removed in 8.0
*/
@Deprecated
public Integer getZkClusterStateVersion() {

View File

@ -218,7 +218,7 @@ public class ClusterStateUtil {
public static int getLiveAndActiveReplicaCount(ZkStateReader zkStateReader, String collection) {
Collection<Slice> slices;
slices = zkStateReader.getClusterState().getActiveSlices(collection);
slices = zkStateReader.getClusterState().getCollection(collection).getActiveSlices();
int liveAndActive = 0;
for (Slice slice : slices) {
for (Replica replica : slice.getReplicas()) {

View File

@ -596,7 +596,7 @@ 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()} and {@link ClusterState#getCollectionsMap()} methods
* {@link ClusterState#getCollectionsMap()} methods
* have a safeguard against exposing wrong collection names to the users
*/
private void refreshCollectionList(Watcher watcher) throws KeeperException, InterruptedException {
@ -799,12 +799,13 @@ public class ZkStateReader implements Closeable {
if (clusterState == null) {
return null;
}
Map<String,Slice> slices = clusterState.getSlicesMap(collection);
if (slices == null) {
final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
if (docCollection == null || docCollection.getSlicesMap() == null) {
throw new ZooKeeperException(ErrorCode.BAD_REQUEST,
"Could not find collection in zk: " + collection);
}
Map<String,Slice> slices = docCollection.getSlicesMap();
Slice replicas = slices.get(shardId);
if (replicas == null) {
throw new ZooKeeperException(ErrorCode.BAD_REQUEST, "Could not find shardId in zk: " + shardId);

View File

@ -151,7 +151,9 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
if (verbose) System.out.println("-");
boolean sawLiveRecovering = false;
ClusterState clusterState = zkStateReader.getClusterState();
Map<String,Slice> slices = clusterState.getSlicesMap(collection);
final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
assertNotNull("Could not find collection:" + collection, docCollection);
Map<String,Slice> slices = docCollection.getSlicesMap();
assertNotNull("Could not find collection:" + collection, slices);
for (Map.Entry<String,Slice> entry : slices.entrySet()) {
Slice slice = entry.getValue();
@ -254,8 +256,9 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
int maxIterations = 100;
Replica.State coreState = null;
while(maxIterations-->0) {
Slice slice = reader.getClusterState().getSlice(collection, shard);
if(slice!=null) {
final DocCollection docCollection = reader.getClusterState().getCollectionOrNull(collection);
if(docCollection != null && docCollection.getSlice(shard)!=null) {
Slice slice = docCollection.getSlice(shard);
Replica replica = slice.getReplicasMap().get(coreNodeName);
if (replica != null) {
coreState = replica.getState();
@ -274,10 +277,12 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
zkStateReader.forceUpdateCollection(collection);
ClusterState clusterState = zkStateReader.getClusterState();
Map<String,Slice> slices = clusterState.getSlicesMap(collection);
if (slices == null) {
final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
if (docCollection == null || docCollection.getSlices() == null) {
throw new IllegalArgumentException("Cannot find collection:" + collection);
}
Map<String,Slice> slices = docCollection.getSlicesMap();
for (Map.Entry<String,Slice> entry : slices.entrySet()) {
Slice slice = entry.getValue();
if (slice.getState() != Slice.State.ACTIVE) {

View File

@ -351,11 +351,12 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
Thread.sleep(500);
}
cnt = 30;
while (reader.getClusterState().getSlices(collection).size() < slices) {
while (reader.getClusterState().getCollection(collection).getSlices().size() < slices) {
if (cnt == 0) {
throw new RuntimeException("timeout waiting for collection shards to come up: collection="+collection
+ ", slices.expected="+slices+ " slices.actual= " + reader.getClusterState().getSlices(collection).size()
+ " slices : "+ reader.getClusterState().getSlices(collection) );
+ ", slices.expected="+slices+ " slices.actual= " + reader.getClusterState().getCollection(collection).getSlices().size()
+ " slices : "+ reader.getClusterState().getCollection(collection).getSlices() );
}
cnt--;
Thread.sleep(500);
@ -874,11 +875,11 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
ZkNodeProps leader = clusterState.getLeader(collection, slice);
if (leader == null) {
throw new RuntimeException("Could not find leader:" + collection + " " + slice);
final DocCollection docCollection = clusterState.getCollectionOrNull(collection);
if (docCollection != null && docCollection.getLeader(slice) != null) {
return new ZkCoreNodeProps(docCollection.getLeader(slice));
}
return new ZkCoreNodeProps(leader);
throw new RuntimeException("Could not find leader:" + collection + " " + slice);
}
@Override
@ -1169,7 +1170,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
assertEquals(
"The client count does not match up with the shard count for slice:"
+ shard,
zkStateReader.getClusterState().getSlice(DEFAULT_COLLECTION, shard)
zkStateReader.getClusterState().getCollection(DEFAULT_COLLECTION).getSlice(shard)
.getReplicasMap().size(), solrJetties.size());
CloudJettyRunner lastJetty = null;
@ -1403,7 +1404,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
AbstractZkTestCase.TIMEOUT)) {
zk.createClusterStateWatchersAndUpdate();
clusterState = zk.getClusterState();
slices = clusterState.getSlicesMap(DEFAULT_COLLECTION);
final DocCollection docCollection = clusterState.getCollectionOrNull(DEFAULT_COLLECTION);
slices = (docCollection != null) ? docCollection.getSlicesMap() : null;
}
if (slices == null) {
@ -1949,7 +1951,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
zkr.forceUpdateCollection(testCollectionName); // force the state to be fresh
ClusterState cs = zkr.getClusterState();
Collection<Slice> slices = cs.getActiveSlices(testCollectionName);
Collection<Slice> slices = cs.getCollection(testCollectionName).getActiveSlices();
assertTrue(slices.size() == shards);
boolean allReplicasUp = false;
long waitMs = 0L;
@ -1958,7 +1960,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
while (waitMs < maxWaitMs && !allReplicasUp) {
cs = cloudClient.getZkStateReader().getClusterState();
assertNotNull(cs);
Slice shard = cs.getSlice(testCollectionName, shardId);
final DocCollection docCollection = cs.getCollectionOrNull(testCollectionName);
assertNotNull("No collection found for " + testCollectionName, docCollection);
Slice shard = docCollection.getSlice(shardId);
assertNotNull("No Slice for "+shardId, shard);
allReplicasUp = true; // assume true
Collection<Replica> replicas = shard.getReplicas();

View File

@ -333,7 +333,7 @@ public class ChaosMonkey {
}
private String getRandomSlice() {
Map<String,Slice> slices = zkStateReader.getClusterState().getSlicesMap(collection);
Map<String,Slice> slices = zkStateReader.getClusterState().getCollection(collection).getSlicesMap();
List<String> sliceKeyList = new ArrayList<>(slices.size());
sliceKeyList.addAll(slices.keySet());