SOLR-9439: Shard split clean up logic for older failed splits is faulty

This commit is contained in:
Shalin Shekhar Mangar 2016-08-27 09:08:53 +05:30
parent ae40929f0b
commit 7d2f42e543
5 changed files with 130 additions and 15 deletions

View File

@ -81,6 +81,8 @@ Bug Fixes
* SOLR-9445: Admin requests are retried by CloudSolrClient and LBHttpSolrClient on failure. (shalin)
* SOLR-9439: Shard split clean up logic for older failed splits is faulty. (shalin)
Optimizations
----------------------

View File

@ -46,6 +46,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.util.TestInjection;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -79,6 +80,7 @@ public class SplitShardCmd implements Cmd {
log.info("Split shard invoked");
ZkStateReader zkStateReader = ocmh.zkStateReader;
zkStateReader.forceUpdateCollection(collectionName);
String splitKey = message.getStr("split.key");
ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler();
@ -197,7 +199,10 @@ public class SplitShardCmd implements Cmd {
subSlices.add(subSlice);
String subShardName = collectionName + "_" + subSlice + "_replica1";
subShardNames.add(subShardName);
}
boolean oldShardsDeleted = false;
for (String subSlice : subSlices) {
Slice oSlice = collection.getSlice(subSlice);
if (oSlice != null) {
final Slice.State state = oSlice.getState();
@ -206,24 +211,33 @@ public class SplitShardCmd implements Cmd {
"Sub-shard: " + subSlice + " exists in active state. Aborting split shard.");
} else if (state == Slice.State.CONSTRUCTION || state == Slice.State.RECOVERY) {
// delete the shards
for (String sub : subSlices) {
log.info("Sub-shard: {} already exists therefore requesting its deletion", sub);
Map<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, "deleteshard");
propMap.put(COLLECTION_PROP, collectionName);
propMap.put(SHARD_ID_PROP, sub);
ZkNodeProps m = new ZkNodeProps(propMap);
try {
ocmh.commandMap.get(DELETESHARD).call(clusterState, m, new NamedList());
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to delete already existing sub shard: " + sub,
e);
}
log.info("Sub-shard: {} already exists therefore requesting its deletion", subSlice);
Map<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, "deleteshard");
propMap.put(COLLECTION_PROP, collectionName);
propMap.put(SHARD_ID_PROP, subSlice);
ZkNodeProps m = new ZkNodeProps(propMap);
try {
ocmh.commandMap.get(DELETESHARD).call(clusterState, m, new NamedList());
} catch (SolrException e) {
throwIfNotNonExistentCoreException(subSlice, e);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to delete already existing sub shard: " + subSlice,
e);
}
oldShardsDeleted = true;
}
}
}
if (oldShardsDeleted) {
// refresh the locally cached cluster state
zkStateReader.forceUpdateCollection(collectionName);
clusterState = zkStateReader.getClusterState();
collection = clusterState.getCollection(collectionName);
}
final String asyncId = message.getStr(ASYNC);
Map<String, String> requestMap = new HashMap<>();
@ -406,6 +420,8 @@ public class SplitShardCmd implements Cmd {
replicas.add(propMap);
}
assert TestInjection.injectSplitFailureBeforeReplicaCreation();
// we must set the slice state into recovery before actually creating the replica cores
// this ensures that the logic inside Overseer to update sub-shard state to 'active'
// always gets a chance to execute. See SOLR-7673
@ -455,4 +471,24 @@ public class SplitShardCmd implements Cmd {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, e);
}
}
private void throwIfNotNonExistentCoreException(String subSlice, SolrException e) {
Throwable t = e;
String cause = null;
while (t != null) {
if (t instanceof SolrException) {
SolrException solrException = (SolrException) t;
cause = solrException.getMetadata("cause");
if (cause != null && !"NonExistentCore".equals(cause)) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to delete already existing sub shard: " + subSlice,
e);
}
}
t = t.getCause();
}
if (!"NonExistentCore".equals(cause)) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to delete already existing sub shard: " + subSlice,
e);
}
}
}

View File

@ -1018,8 +1018,11 @@ public class CoreContainer {
}
CoreDescriptor cd = solrCores.getCoreDescriptor(name);
if (cd == null)
throw new SolrException(ErrorCode.BAD_REQUEST, "Cannot unload non-existent core [" + name + "]");
if (cd == null) {
SolrException solrException = new SolrException(ErrorCode.BAD_REQUEST, "Cannot unload non-existent core [" + name + "]");
solrException.setMetadata("cause", "NonExistentCore");
throw solrException;
}
boolean close = solrCores.isLoadedNotPendingClose(name);
SolrCore core = solrCores.remove(name);

View File

@ -113,6 +113,8 @@ public class TestInjection {
public static String randomDelayInCoreCreation = null;
public static int randomDelayMaxInCoreCreationInSec = 10;
public static String splitFailureBeforeReplicaCreation = null;
private static Set<Timer> timers = Collections.synchronizedSet(new HashSet<Timer>());
@ -124,6 +126,7 @@ public class TestInjection {
updateLogReplayRandomPause = null;
updateRandomPause = null;
randomDelayInCoreCreation = null;
splitFailureBeforeReplicaCreation = null;
for (Timer timer : timers) {
timer.cancel();
@ -285,6 +288,23 @@ public class TestInjection {
return true;
}
public static boolean injectSplitFailureBeforeReplicaCreation() {
if (splitFailureBeforeReplicaCreation != null) {
Random rand = random();
if (null == rand) return true;
Pair<Boolean,Integer> pair = parseValue(splitFailureBeforeReplicaCreation);
boolean enabled = pair.first();
int chanceIn100 = pair.second();
if (enabled && rand.nextInt(100) >= (100 - chanceIn100)) {
log.info("Injecting failure in creating replica for sub-shard");
throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to create replica");
}
}
return true;
}
private static Pair<Boolean,Integer> parseValue(String raw) {
Matcher m = ENABLED_PERCENT.matcher(raw);

View File

@ -40,6 +40,7 @@ import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.CompositeIdRouter;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.HashBasedRouter;
import org.apache.solr.common.cloud.Replica;
@ -49,6 +50,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TestInjection;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -89,6 +91,58 @@ public class ShardSplitTest extends BasicDistributedZkTest {
//waitForThingsToLevelOut(15);
}
/**
* Used to test that we can split a shard when a previous split event
* left sub-shards in construction or recovery state.
*
* See SOLR-9439
*/
@Test
public void testSplitAfterFailedSplit() throws Exception {
waitForThingsToLevelOut(15);
TestInjection.splitFailureBeforeReplicaCreation = "true:100"; // we definitely want split to fail
try {
try {
CollectionAdminRequest.SplitShard splitShard = CollectionAdminRequest.splitShard(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
splitShard.setShardName(SHARD1);
splitShard.process(cloudClient);
fail("Shard split was not supposed to succeed after failure injection!");
} catch (Exception e) {
// expected
}
// assert that sub-shards cores exist and sub-shard is in construction state
ZkStateReader zkStateReader = cloudClient.getZkStateReader();
zkStateReader.forceUpdateCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
ClusterState state = zkStateReader.getClusterState();
DocCollection collection = state.getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
Slice shard10 = collection.getSlice(SHARD1_0);
assertEquals(Slice.State.CONSTRUCTION, shard10.getState());
assertEquals(1, shard10.getReplicas().size());
Slice shard11 = collection.getSlice(SHARD1_1);
assertEquals(Slice.State.CONSTRUCTION, shard11.getState());
assertEquals(1, shard11.getReplicas().size());
// lets retry the split
TestInjection.reset(); // let the split succeed
try {
CollectionAdminRequest.SplitShard splitShard = CollectionAdminRequest.splitShard(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
splitShard.setShardName(SHARD1);
splitShard.process(cloudClient);
// Yay!
} catch (Exception e) {
log.error("Shard split failed", e);
fail("Shard split did not succeed after a previous failed split attempt left sub-shards in construction state");
}
} finally {
TestInjection.reset();
}
}
@Test
public void testSplitShardWithRule() throws Exception {
waitForThingsToLevelOut(15);