SOLR-10879: Make sure we don't lose single replicas when deleting a node.

This commit is contained in:
Andrzej Bialecki 2017-07-04 20:19:08 +02:00
parent ddfa074214
commit cb23fa9b4e
3 changed files with 96 additions and 11 deletions

View File

@ -304,6 +304,9 @@ Bug Fixes
* SOLR-10878: MOVEREPLICA command may lose data when replicationFactor is 1. (ab, shalin)
* SOLR-10879: DELETEREPLICA and DELETENODE commands should prevent data loss when
replicationFactor is 1. (ab)
Optimizations
----------------------

View File

@ -19,6 +19,7 @@ package org.apache.solr.cloud;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.CountDownLatch;
@ -26,7 +27,11 @@ import java.util.concurrent.TimeUnit;
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.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.NamedList;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
@ -54,7 +59,40 @@ public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Source Node: " + node + " is not live");
}
List<ZkNodeProps> sourceReplicas = ReplaceNodeCmd.getReplicasOfNode(node, state);
cleanupReplicas(results, state, sourceReplicas, ocmh, node, message.getStr(ASYNC));
List<String> singleReplicas = verifyReplicaAvailability(sourceReplicas, state);
if (!singleReplicas.isEmpty()) {
results.add("failure", "Can't delete the only existing non-PULL replica(s) on node " + node + ": " + singleReplicas.toString());
} else {
cleanupReplicas(results, state, sourceReplicas, ocmh, node, message.getStr(ASYNC));
}
}
// collect names of replicas that cannot be deleted
static List<String> verifyReplicaAvailability(List<ZkNodeProps> sourceReplicas, ClusterState state) {
List<String> res = new ArrayList<>();
for (ZkNodeProps sourceReplica : sourceReplicas) {
String coll = sourceReplica.getStr(COLLECTION_PROP);
String shard = sourceReplica.getStr(SHARD_ID_PROP);
String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
DocCollection collection = state.getCollection(coll);
Slice slice = collection.getSlice(shard);
if (slice.getReplicas().size() < 2) {
// can't delete the only replica in existence
res.add(coll + "/" + shard + "/" + replicaName + ", type=" + sourceReplica.getStr(ZkStateReader.REPLICA_TYPE));
} else { // check replica types
int otherNonPullReplicas = 0;
for (Replica r : slice.getReplicas()) {
if (!r.getName().equals(replicaName) && !r.getType().equals(Replica.Type.PULL)) {
otherNonPullReplicas++;
}
}
// can't delete - there are no other non-pull replicas
if (otherNonPullReplicas == 0) {
res.add(coll + "/" + shard + "/" + replicaName + ", type=" + sourceReplica.getStr(ZkStateReader.REPLICA_TYPE));
}
}
}
return res;
}
static void cleanupReplicas(NamedList results,
@ -67,7 +105,8 @@ public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
for (ZkNodeProps sourceReplica : sourceReplicas) {
String coll = sourceReplica.getStr(COLLECTION_PROP);
String shard = sourceReplica.getStr(SHARD_ID_PROP);
log.info("Deleting replica for collection={} shard={} on node={}", coll, shard, node);
String type = sourceReplica.getStr(ZkStateReader.REPLICA_TYPE);
log.info("Deleting replica type={} for collection={} shard={} on node={}", type, coll, shard, node);
NamedList deleteResult = new NamedList();
try {
if (async != null) sourceReplica = sourceReplica.plus(ASYNC, async);

View File

@ -18,19 +18,29 @@
package org.apache.solr.cloud;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.RequestStatusState;
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;
import org.apache.solr.common.util.StrUtils;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DeleteNodeTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(6)
@ -47,28 +57,61 @@ public class DeleteNodeTest extends SolrCloudTestCase {
cluster.waitForAllNodes(5000);
CloudSolrClient cloudClient = cluster.getSolrClient();
String coll = "deletenodetest_coll";
Set<String> liveNodes = cloudClient.getZkStateReader().getClusterState().getLiveNodes();
ClusterState state = cloudClient.getZkStateReader().getClusterState();
Set<String> liveNodes = state.getLiveNodes();
ArrayList<String> l = new ArrayList<>(liveNodes);
Collections.shuffle(l, random());
CollectionAdminRequest.Create create = pickRandom(
CollectionAdminRequest.createCollection(coll, "conf1", 5, 2, 0, 0),
CollectionAdminRequest.createCollection(coll, "conf1", 5, 1, 1, 0),
CollectionAdminRequest.createCollection(coll, "conf1", 5, 0, 1, 1));
CollectionAdminRequest.createCollection(coll, "conf1", 5, 0, 1, 1),
// check RF=1
CollectionAdminRequest.createCollection(coll, "conf1", 5, 1, 0, 0),
CollectionAdminRequest.createCollection(coll, "conf1", 5, 0, 1, 0)
);
create.setCreateNodeSet(StrUtils.join(l, ',')).setMaxShardsPerNode(3);
cloudClient.request(create);
state = cloudClient.getZkStateReader().getClusterState();
String node2bdecommissioned = l.get(0);
// check what replicas are on the node, and whether the call should fail
boolean shouldFail = false;
DocCollection docColl = state.getCollection(coll);
log.info("#### DocCollection: " + docColl);
List<Replica> replicas = docColl.getReplicas(node2bdecommissioned);
if (replicas != null) {
for (Replica replica : replicas) {
String shard = docColl.getShardId(node2bdecommissioned, replica.getStr(ZkStateReader.CORE_NAME_PROP));
Slice slice = docColl.getSlice(shard);
boolean hasOtherNonPullReplicas = false;
for (Replica r: slice.getReplicas()) {
if (!r.getName().equals(replica.getName()) &&
!r.getNodeName().equals(node2bdecommissioned) &&
r.getType() != Replica.Type.PULL) {
hasOtherNonPullReplicas = true;
break;
}
}
if (!hasOtherNonPullReplicas) {
shouldFail = true;
break;
}
}
}
new CollectionAdminRequest.DeleteNode(node2bdecommissioned).processAsync("003", cloudClient);
CollectionAdminRequest.RequestStatus requestStatus = CollectionAdminRequest.requestStatus("003");
boolean success = false;
CollectionAdminRequest.RequestStatusResponse rsp = null;
for (int i = 0; i < 200; i++) {
CollectionAdminRequest.RequestStatusResponse rsp = requestStatus.process(cloudClient);
if (rsp.getRequestStatus() == RequestStatusState.COMPLETED) {
success = true;
rsp = requestStatus.process(cloudClient);
if (rsp.getRequestStatus() == RequestStatusState.FAILED || rsp.getRequestStatus() == RequestStatusState.COMPLETED) {
break;
}
assertFalse(rsp.getRequestStatus() == RequestStatusState.FAILED);
Thread.sleep(50);
}
assertTrue(success);
log.info("####### DocCollection after: " + cloudClient.getZkStateReader().getClusterState().getCollection(coll));
if (shouldFail) {
assertTrue(String.valueOf(rsp), rsp.getRequestStatus() == RequestStatusState.FAILED);
} else {
assertFalse(String.valueOf(rsp), rsp.getRequestStatus() == RequestStatusState.FAILED);
}
}
}