SOLR-11011: Fix backward compatibility problem + audit the usage of old Assign.buildCoreName(String collectionName, String shard, Replica.Type type, int replicaNum)

This commit is contained in:
Cao Manh Dat 2017-07-25 15:04:15 +07:00
parent 56ad1a7a9b
commit f087c5acae
15 changed files with 210 additions and 44 deletions

View File

@ -104,7 +104,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live"); throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
} }
if (coreName == null) { if (coreName == null) {
coreName = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(), coll.getName(), shard, replicaType); coreName = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(), coll, shard, replicaType);
} else if (!skipCreateReplicaInClusterState) { } else if (!skipCreateReplicaInClusterState) {
//Validate that the core name is unique in that collection //Validate that the core name is unique in that collection
for (Slice slice : coll.getSlices()) { for (Slice slice : coll.getSlices()) {

View File

@ -62,13 +62,14 @@ import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET_SHUFFLE; import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET_SHUFFLE;
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET_SHUFFLE_DEFAULT; import static org.apache.solr.cloud.OverseerCollectionMessageHandler.CREATE_NODE_SET_SHUFFLE_DEFAULT;
import static org.apache.solr.common.cloud.DocCollection.SNITCH; import static org.apache.solr.common.cloud.DocCollection.SNITCH;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE; import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH; import static org.apache.solr.common.cloud.ZkStateReader.SOLR_AUTOSCALING_CONF_PATH;
public class Assign { public class Assign {
public static int incAndGetId(SolrZkClient zkClient, String collection) { public static int incAndGetId(SolrZkClient zkClient, String collection, int defaultValue) {
String path = "/collections/"+collection; String path = "/collections/"+collection;
try { try {
if (!zkClient.exists(path, true)) { if (!zkClient.exists(path, true)) {
@ -81,7 +82,7 @@ public class Assign {
path += "/counter"; path += "/counter";
if (!zkClient.exists(path, true)) { if (!zkClient.exists(path, true)) {
try { try {
zkClient.create(path, NumberUtils.intToBytes(0), CreateMode.PERSISTENT, true); zkClient.create(path, NumberUtils.intToBytes(defaultValue), CreateMode.PERSISTENT, true);
} catch (KeeperException.NodeExistsException e) { } catch (KeeperException.NodeExistsException e) {
// it's okay if another beats us creating the node // it's okay if another beats us creating the node
} }
@ -112,8 +113,16 @@ public class Assign {
} }
} }
public static String assignNode(SolrZkClient client, String collection) { public static String assignNode(SolrZkClient client, DocCollection collection) {
return "core_node" + incAndGetId(client, collection); // for backward compatibility;
int numReplicas = collection.getReplicas().size();
String coreNodeName = "core_node" + incAndGetId(client, collection.getName(), numReplicas * 20);
while (collection.getReplica(coreNodeName) != null) {
// there is wee chance that, the new coreNodeName id not totally unique,
// but this will be guaranteed unique for new collections
coreNodeName = "core_node" + incAndGetId(client, collection.getName(), numReplicas * 20);
}
return coreNodeName;
} }
/** /**
@ -160,15 +169,33 @@ public class Assign {
return returnShardId; return returnShardId;
} }
public static String buildCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) { private static String buildCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
// TODO: Adding the suffix is great for debugging, but may be an issue if at some point we want to support a way to change replica type // TODO: Adding the suffix is great for debugging, but may be an issue if at some point we want to support a way to change replica type
return String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum); return String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum);
} }
public static String buildCoreName(SolrZkClient zkClient, String collection, String shard, Replica.Type type) { public static String buildCoreName(SolrZkClient zkClient, DocCollection collection, String shard, Replica.Type type) {
int replicaNum = incAndGetId(zkClient, collection); Slice slice = collection.getSlice(shard);
return buildCoreName(collection, shard, type, replicaNum); int numReplicas = collection.getReplicas().size();
int replicaNum = incAndGetId(zkClient, collection.getName(), numReplicas * 20);
String coreName = buildCoreName(collection.getName(), shard, type, replicaNum);
while (existCoreName(coreName, slice)) {
replicaNum = incAndGetId(zkClient, collection.getName(), numReplicas * 20);
coreName = buildCoreName(collection.getName(), shard, type, replicaNum);
}
return coreName;
} }
private static boolean existCoreName(String coreName, Slice slice) {
if (slice == null) return false;
for (Replica replica : slice.getReplicas()) {
if (coreName.equals(replica.getStr(CORE_NAME_PROP))) {
return true;
}
}
return false;
}
public static List<String> getLiveOrLiveAndCreateNodeSetList(final Set<String> liveNodes, final ZkNodeProps message, final Random random) { public static List<String> getLiveOrLiveAndCreateNodeSetList(final Set<String> liveNodes, final ZkNodeProps message, final Random random) {
// TODO: add smarter options that look at the current number of cores per // TODO: add smarter options that look at the current number of cores per
// node? // node?

View File

@ -211,7 +211,8 @@ public class CreateCollectionCmd implements Cmd {
Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>(); Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
for (ReplicaPosition replicaPosition : replicaPositions) { for (ReplicaPosition replicaPosition : replicaPositions) {
String nodeName = replicaPosition.node; String nodeName = replicaPosition.node;
String coreName = Assign.buildCoreName(collectionName, replicaPosition.shard, replicaPosition.type, replicaPosition.index + 1); String coreName = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(), zkStateReader.getClusterState().getCollection(collectionName),
replicaPosition.shard, replicaPosition.type);
log.debug(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}" log.debug(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}"
, coreName, replicaPosition.shard, collectionName, nodeName)); , coreName, replicaPosition.shard, collectionName, nodeName));

View File

@ -88,23 +88,19 @@ public class CreateShardCmd implements Cmd {
int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0; int createdNrtReplicas = 0, createdTlogReplicas = 0, createdPullReplicas = 0;
CountDownLatch countDownLatch = new CountDownLatch(totalReplicas); CountDownLatch countDownLatch = new CountDownLatch(totalReplicas);
for (int j = 1; j <= totalReplicas; j++) { for (int j = 1; j <= totalReplicas; j++) {
int coreNameNumber;
Replica.Type typeToCreate; Replica.Type typeToCreate;
if (createdNrtReplicas < numNrtReplicas) { if (createdNrtReplicas < numNrtReplicas) {
createdNrtReplicas++; createdNrtReplicas++;
coreNameNumber = createdNrtReplicas;
typeToCreate = Replica.Type.NRT; typeToCreate = Replica.Type.NRT;
} else if (createdTlogReplicas < numTlogReplicas) { } else if (createdTlogReplicas < numTlogReplicas) {
createdTlogReplicas++; createdTlogReplicas++;
coreNameNumber = createdTlogReplicas;
typeToCreate = Replica.Type.TLOG; typeToCreate = Replica.Type.TLOG;
} else { } else {
createdPullReplicas++; createdPullReplicas++;
coreNameNumber = createdPullReplicas;
typeToCreate = Replica.Type.PULL; typeToCreate = Replica.Type.PULL;
} }
String nodeName = sortedNodeList.get(((j - 1)) % sortedNodeList.size()).nodeName; String nodeName = sortedNodeList.get(((j - 1)) % sortedNodeList.size()).nodeName;
String coreName = Assign.buildCoreName(collectionName, sliceName, typeToCreate, coreNameNumber); String coreName = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(), collection, sliceName, typeToCreate);
// String coreName = collectionName + "_" + sliceName + "_replica" + j; // String coreName = collectionName + "_" + sliceName + "_replica" + j;
log.info("Creating replica " + coreName + " as part of slice " + sliceName + " of collection " + collectionName log.info("Creating replica " + coreName + " as part of slice " + sliceName + " of collection " + collectionName
+ " on " + nodeName); + " on " + nodeName);

View File

@ -224,7 +224,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
Slice tempSourceSlice = clusterState.getCollection(tempSourceCollectionName).getSlices().iterator().next(); Slice tempSourceSlice = clusterState.getCollection(tempSourceCollectionName).getSlices().iterator().next();
Replica tempSourceLeader = zkStateReader.getLeaderRetry(tempSourceCollectionName, tempSourceSlice.getName(), 120000); Replica tempSourceLeader = zkStateReader.getLeaderRetry(tempSourceCollectionName, tempSourceSlice.getName(), 120000);
String tempCollectionReplica1 = Assign.buildCoreName(tempSourceCollectionName, tempSourceSlice.getName(), Replica.Type.NRT, 1); String tempCollectionReplica1 = tempSourceLeader.getCoreName();
String coreNodeName = ocmh.waitForCoreNodeName(tempSourceCollectionName, String coreNodeName = ocmh.waitForCoreNodeName(tempSourceCollectionName,
sourceLeader.getNodeName(), tempCollectionReplica1); sourceLeader.getNodeName(), tempCollectionReplica1);
// wait for the replicas to be seen as active on temp source leader // wait for the replicas to be seen as active on temp source leader
@ -257,7 +257,8 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
log.info("Creating a replica of temporary collection: {} on the target leader node: {}", log.info("Creating a replica of temporary collection: {} on the target leader node: {}",
tempSourceCollectionName, targetLeader.getNodeName()); tempSourceCollectionName, targetLeader.getNodeName());
String tempCollectionReplica2 = Assign.buildCoreName(tempSourceCollectionName, tempSourceSlice.getName(), Replica.Type.NRT, 2); String tempCollectionReplica2 = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(),
zkStateReader.getClusterState().getCollection(tempSourceCollectionName), tempSourceSlice.getName(), Replica.Type.NRT);
props = new HashMap<>(); props = new HashMap<>();
props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower()); props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());
props.put(COLLECTION_PROP, tempSourceCollectionName); props.put(COLLECTION_PROP, tempSourceCollectionName);

View File

@ -184,7 +184,7 @@ public class MoveReplicaCmd implements Cmd{
private void moveNormalReplica(ClusterState clusterState, NamedList results, String targetNode, String async, private void moveNormalReplica(ClusterState clusterState, NamedList results, String targetNode, String async,
DocCollection coll, Replica replica, Slice slice, int timeout) throws Exception { DocCollection coll, Replica replica, Slice slice, int timeout) throws Exception {
String newCoreName = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(), coll.getName(), slice.getName(), replica.getType()); String newCoreName = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(), coll, slice.getName(), replica.getType());
ZkNodeProps addReplicasProps = new ZkNodeProps( ZkNodeProps addReplicasProps = new ZkNodeProps(
COLLECTION_PROP, coll.getName(), COLLECTION_PROP, coll.getName(),
SHARD_ID_PROP, slice.getName(), SHARD_ID_PROP, slice.getName(),

View File

@ -205,7 +205,7 @@ public class SplitShardCmd implements Cmd {
for (int i = 0; i < subRanges.size(); i++) { for (int i = 0; i < subRanges.size(); i++) {
String subSlice = slice + "_" + i; String subSlice = slice + "_" + i;
subSlices.add(subSlice); subSlices.add(subSlice);
String subShardName = Assign.buildCoreName(collectionName, subSlice, Replica.Type.NRT, 1); String subShardName = Assign.buildCoreName(ocmh.zkStateReader.getZkClient(), collection, subSlice, Replica.Type.NRT);
subShardNames.add(subShardName); subShardNames.add(subShardName);
} }

View File

@ -240,7 +240,7 @@ public class ReplicaMutator {
log.debug("node=" + coreNodeName + " is already registered"); log.debug("node=" + coreNodeName + " is already registered");
} else { } else {
// if coreNodeName is null, auto assign one // if coreNodeName is null, auto assign one
coreNodeName = Assign.assignNode(zkStateReader.getZkClient(), collection.getName()); coreNodeName = Assign.assignNode(zkStateReader.getZkClient(), collection);
} }
message.getProperties().put(ZkStateReader.CORE_NODE_NAME_PROP, message.getProperties().put(ZkStateReader.CORE_NODE_NAME_PROP,
coreNodeName); coreNodeName);

View File

@ -70,7 +70,7 @@ public class SliceMutator {
if (message.getStr(ZkStateReader.CORE_NODE_NAME_PROP) != null) { if (message.getStr(ZkStateReader.CORE_NODE_NAME_PROP) != null) {
coreNodeName = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP); coreNodeName = message.getStr(ZkStateReader.CORE_NODE_NAME_PROP);
} else { } else {
coreNodeName = Assign.assignNode(zkStateReader.getZkClient(), collection.getName()); coreNodeName = Assign.assignNode(zkStateReader.getZkClient(), collection);
} }
Replica replica = new Replica(coreNodeName, Replica replica = new Replica(coreNodeName,
makeMap( makeMap(

View File

@ -0,0 +1,101 @@
/*
* 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.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.HashSet;
import java.util.Set;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.common.cloud.ClusterStateUtil;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.zookeeper.KeeperException;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Test for backward compatibility when users update from 6.x or 7.0 to 7.1,
* then the counter of collection does not exist in Zk
* TODO Remove in Solr 9.0
*/
public class AssignBackwardCompatibilityTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private static final String COLLECTION = "collection1";
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(4)
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-dynamic").resolve("conf"))
.configure();
CollectionAdminRequest.createCollection(COLLECTION, 1, 4)
.setMaxShardsPerNode(1000)
.process(cluster.getSolrClient());
}
@Test
public void test() throws IOException, SolrServerException, KeeperException, InterruptedException {
Set<String> coreNames = new HashSet<>();
Set<String> coreNodeNames = new HashSet<>();
int numOperations = random().nextInt(4 * 15);
int numLiveReplicas = 4;
boolean clearedCounter = false;
for (int i = 0; i < numOperations; i++) {
boolean deleteReplica = random().nextBoolean() && numLiveReplicas > 1;
// No need to clear counter more than one time
if (random().nextInt(30) < 5 && !clearedCounter) {
// clear counter
cluster.getZkClient().delete("/collections/"+COLLECTION+"/counter", -1, true);
clearedCounter = true;
}
if (deleteReplica) {
assertTrue(ClusterStateUtil.waitForLiveAndActiveReplicaCount(
cluster.getSolrClient().getZkStateReader(), COLLECTION, numLiveReplicas, 30000));
DocCollection dc = getCollectionState(COLLECTION);
Replica replica = getRandomReplica(dc.getSlice("shard1"), (r) -> r.getState() == Replica.State.ACTIVE);
CollectionAdminRequest.deleteReplica(COLLECTION, "shard1", replica.getName()).process(cluster.getSolrClient());
numLiveReplicas--;
} else {
CollectionAdminResponse response = CollectionAdminRequest.addReplicaToShard(COLLECTION, "shard1")
.process(cluster.getSolrClient());
assertTrue(response.isSuccess());
String coreName = response.getCollectionCoresStatus()
.keySet().iterator().next();
assertFalse("Core name is not unique coreName=" + coreName + " " + coreNames, coreNames.contains(coreName));
coreNames.add(coreName);
Replica newReplica = getCollectionState(COLLECTION).getReplicas().stream()
.filter(r -> r.getCoreName().equals(coreName))
.findAny().get();
String coreNodeName = newReplica.getName();
assertFalse("Core node name is not unique", coreNodeNames.contains(coreName));
coreNodeNames.add(coreNodeName);
numLiveReplicas++;
}
}
}
}

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.solr.cloud; package org.apache.solr.cloud;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
@ -25,9 +26,13 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.Replica; 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.SolrZkClient;
import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.ExecutorUtil;
import org.apache.zookeeper.KeeperException;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -65,11 +70,11 @@ public class AssignTest extends SolrTestCaseJ4 {
); );
when(zkClient.getData(anyString(), any(), any(), anyBoolean())).then(invocation -> when(zkClient.getData(anyString(), any(), any(), anyBoolean())).then(invocation ->
zkClientData.get(invocation.getArgument(0))); zkClientData.get(invocation.getArgument(0)));
String nodeName = Assign.assignNode(zkClient, "collection1"); String nodeName = Assign.assignNode(zkClient, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
assertEquals("core_node1", nodeName); assertEquals("core_node1", nodeName);
nodeName = Assign.assignNode(zkClient, "collection2"); nodeName = Assign.assignNode(zkClient, new DocCollection("collection2", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
assertEquals("core_node1", nodeName); assertEquals("core_node1", nodeName);
nodeName = Assign.assignNode(zkClient, "collection1"); nodeName = Assign.assignNode(zkClient, new DocCollection("collection1", new HashMap<>(), new HashMap<>(), DocRouter.DEFAULT));
assertEquals("core_node2", nodeName); assertEquals("core_node2", nodeName);
} }
@ -98,7 +103,7 @@ public class AssignTest extends SolrTestCaseJ4 {
for (int i = 0; i < 1000; i++) { for (int i = 0; i < 1000; i++) {
futures.add(executor.submit(() -> { futures.add(executor.submit(() -> {
String collection = collections[random().nextInt(collections.length)]; String collection = collections[random().nextInt(collections.length)];
int id = Assign.incAndGetId(zkClient, collection); int id = Assign.incAndGetId(zkClient, collection, 0);
Object val = collectionUniqueIds.get(collection).put(id, fixedValue); Object val = collectionUniqueIds.get(collection).put(id, fixedValue);
if (val != null) { if (val != null) {
fail("ZkController do not generate unique id for " + collection); fail("ZkController do not generate unique id for " + collection);
@ -120,9 +125,22 @@ public class AssignTest extends SolrTestCaseJ4 {
@Test @Test
public void testBuildCoreName() { public void testBuildCoreName() throws IOException, InterruptedException, KeeperException {
assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildCoreName("collection1", "shard1", Replica.Type.NRT, 1)); String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildCoreName("collection1", "shard2", Replica.Type.PULL,2)); ZkTestServer server = new ZkTestServer(zkDir);
server.run();
try (SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), 10000)) {
zkClient.makePath("/", true);
Map<String, Slice> slices = new HashMap<>();
slices.put("shard1", new Slice("shard1", new HashMap<>(), null));
slices.put("shard2", new Slice("shard2", new HashMap<>(), null));
DocCollection docCollection = new DocCollection("collection1", slices, null, DocRouter.DEFAULT);
assertEquals("Core name pattern changed", "collection1_shard1_replica_n1", Assign.buildCoreName(zkClient, docCollection, "shard1", Replica.Type.NRT));
assertEquals("Core name pattern changed", "collection1_shard2_replica_p2", Assign.buildCoreName(zkClient, docCollection, "shard2", Replica.Type.PULL));
} finally {
server.shutdown();
}
} }
} }

View File

@ -95,7 +95,7 @@ public class ClusterStateUpdateTest extends SolrCloudTestCase {
assertEquals(1, shards.size()); assertEquals(1, shards.size());
// assert this is core of container1 // assert this is core of container1
Replica zkProps = shards.get("core_node1"); Replica zkProps = shards.values().iterator().next();
assertNotNull(zkProps); assertNotNull(zkProps);

View File

@ -284,7 +284,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
.process(cluster.getSolrClient()).getStatus()); .process(cluster.getSolrClient()).getStatus());
assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker", "shard1") assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker", "shard1")
.setNode(cluster.getJettySolrRunner(0).getNodeName()) .setNode(cluster.getJettySolrRunner(0).getNodeName())
.setCoreName(Assign.buildCoreName("halfcollection", "shard1", Replica.Type.NRT, 1)) .setCoreName("halfcollection_shard1_replica_n1")
.process(cluster.getSolrClient()).isSuccess()); .process(cluster.getSolrClient()).isSuccess());
assertEquals(0, CollectionAdminRequest.createCollection("halfcollectionblocker2", "conf",1, 1) assertEquals(0, CollectionAdminRequest.createCollection("halfcollectionblocker2", "conf",1, 1)
@ -292,7 +292,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
.process(cluster.getSolrClient()).getStatus()); .process(cluster.getSolrClient()).getStatus());
assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker2", "shard1") assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker2", "shard1")
.setNode(cluster.getJettySolrRunner(1).getNodeName()) .setNode(cluster.getJettySolrRunner(1).getNodeName())
.setCoreName(Assign.buildCoreName("halfcollection", "shard1", Replica.Type.NRT, 1)) .setCoreName("halfcollection_shard1_replica_n1")
.process(cluster.getSolrClient()).isSuccess()); .process(cluster.getSolrClient()).isSuccess());
String nn1 = cluster.getJettySolrRunner(0).getNodeName(); String nn1 = cluster.getJettySolrRunner(0).getNodeName();

View File

@ -70,8 +70,8 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
assertTrue(response.isSuccess()); assertTrue(response.isSuccess());
Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus(); Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
assertEquals(4, coresStatus.size()); assertEquals(4, coresStatus.size());
for (int i=0; i<4; i++) { for (String coreName : coresStatus.keySet()) {
NamedList<Integer> status = coresStatus.get(Assign.buildCoreName(collectionName, "shard" + (i/2+1), Replica.Type.NRT, (i%2+1))); NamedList<Integer> status = coresStatus.get(coreName);
assertEquals(0, (int)status.get("status")); assertEquals(0, (int)status.get("status"));
assertTrue(status.get("QTime") > 0); assertTrue(status.get("QTime") > 0);
} }
@ -98,8 +98,8 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
assertTrue(response.isSuccess()); assertTrue(response.isSuccess());
Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus(); Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
assertEquals(4, coresStatus.size()); assertEquals(4, coresStatus.size());
for (int i=0; i<4; i++) { for (String coreName : coresStatus.keySet()) {
NamedList<Integer> status = coresStatus.get(Assign.buildCoreName(collectionName, "shard" + (i/2+1), Replica.Type.NRT, (i%2+1))); NamedList<Integer> status = coresStatus.get(coreName);
assertEquals(0, (int)status.get("status")); assertEquals(0, (int)status.get("status"));
assertTrue(status.get("QTime") > 0); assertTrue(status.get("QTime") > 0);
} }
@ -168,9 +168,18 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
assertTrue(response.isSuccess()); assertTrue(response.isSuccess());
coresStatus = response.getCollectionCoresStatus(); coresStatus = response.getCollectionCoresStatus();
assertEquals(3, coresStatus.size()); assertEquals(3, coresStatus.size());
assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName, "shardC", Replica.Type.NRT, 1)).get("status")); int replicaTlog = 0;
assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName, "shardC", Replica.Type.TLOG, 1)).get("status")); int replicaNrt = 0;
assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName, "shardC", Replica.Type.PULL, 1)).get("status")); int replicaPull = 0;
for (String coreName : coresStatus.keySet()) {
assertEquals(0, (int) coresStatus.get(coreName).get("status"));
if (coreName.contains("shardC_replica_t")) replicaTlog++;
else if (coreName.contains("shardC_replica_n")) replicaNrt++;
else replicaPull++;
}
assertEquals(1, replicaNrt);
assertEquals(1, replicaTlog);
assertEquals(1, replicaPull);
response = CollectionAdminRequest.deleteShard(collectionName, "shardC").process(cluster.getSolrClient()); response = CollectionAdminRequest.deleteShard(collectionName, "shardC").process(cluster.getSolrClient());
@ -208,8 +217,15 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
assertEquals(0, response.getStatus()); assertEquals(0, response.getStatus());
assertTrue(response.isSuccess()); assertTrue(response.isSuccess());
Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus(); Map<String, NamedList<Integer>> coresStatus = response.getCollectionCoresStatus();
assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName, "shard1_0" , Replica.Type.NRT, 1)).get("status")); int shard10 = 0;
assertEquals(0, (int) coresStatus.get(Assign.buildCoreName(collectionName, "shard1_1" , Replica.Type.NRT, 1)).get("status")); int shard11 = 0;
for (String coreName : coresStatus.keySet()) {
assertEquals(0, (int) coresStatus.get(coreName).get("status"));
if (coreName.contains("_shard1_0")) shard10++;
else shard11++;
}
assertEquals(1, shard10);
assertEquals(1, shard11);
waitForState("Expected all shards to be active and parent shard to be removed", collectionName, (n, c) -> { waitForState("Expected all shards to be active and parent shard to be removed", collectionName, (n, c) -> {
if (c.getSlice("shard1").getState() == Slice.State.ACTIVE) if (c.getSlice("shard1").getState() == Slice.State.ACTIVE)
@ -254,7 +270,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
DocCollection testCollection = getCollectionState(collectionName); DocCollection testCollection = getCollectionState(collectionName);
Replica replica1 = testCollection.getReplica("core_node1"); Replica replica1 = testCollection.getReplicas().iterator().next();
CoreStatus coreStatus = getCoreStatus(replica1); CoreStatus coreStatus = getCoreStatus(replica1);
assertEquals(Paths.get(coreStatus.getDataDirectory()).toString(), dataDir.toString()); assertEquals(Paths.get(coreStatus.getDataDirectory()).toString(), dataDir.toString());

View File

@ -237,6 +237,14 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
}); });
when(clusterStateMock.getLiveNodes()).thenReturn(liveNodes); when(clusterStateMock.getLiveNodes()).thenReturn(liveNodes);
Map<String, byte[]> zkClientData = new HashMap<>();
when(solrZkClientMock.setData(anyString(), any(), anyInt(), anyBoolean())).then(invocation -> {
zkClientData.put(invocation.getArgument(0), invocation.getArgument(1));
return null;
}
);
when(solrZkClientMock.getData(anyString(), any(), any(), anyBoolean())).then(invocation ->
zkClientData.get(invocation.getArgument(0)));
when(solrZkClientMock.create(any(), any(), any(), anyBoolean())).thenAnswer(invocation -> { when(solrZkClientMock.create(any(), any(), any(), anyBoolean())).thenAnswer(invocation -> {
String key = invocation.getArgument(0); String key = invocation.getArgument(0);
zkMap.put(key, null); zkMap.put(key, null);
@ -376,9 +384,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 {
assertEquals(numberOfSlices * numberOfReplica, coreNames.size()); assertEquals(numberOfSlices * numberOfReplica, coreNames.size());
for (int i = 1; i <= numberOfSlices; i++) { for (int i = 1; i <= numberOfSlices; i++) {
for (int j = 1; j <= numberOfReplica; j++) { for (int j = 1; j <= numberOfReplica; j++) {
String coreName = Assign.buildCoreName(COLLECTION_NAME, "shard" + i, Replica.Type.NRT, j); String coreName = coreNames.get((i-1) * numberOfReplica + (j-1));
assertTrue("Shard " + coreName + " was not created",
coreNames.contains(coreName));
if (dontShuffleCreateNodeSet) { if (dontShuffleCreateNodeSet) {
final String expectedNodeName = nodeUrlWithoutProtocolPartForLiveNodes.get((numberOfReplica * (i - 1) + (j - 1)) % nodeUrlWithoutProtocolPartForLiveNodes.size()); final String expectedNodeName = nodeUrlWithoutProtocolPartForLiveNodes.get((numberOfReplica * (i - 1) + (j - 1)) % nodeUrlWithoutProtocolPartForLiveNodes.size());