mirror of https://github.com/apache/lucene.git
SOLR-4933: Retry splitshard three times before giving up
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1498923 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
429bbe6950
commit
c100f376f0
|
@ -360,13 +360,17 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
|
|||
throw new SolrException(ErrorCode.BAD_REQUEST, "Sub-shard: " + subSlice + " exists in active state. Aborting split shard.");
|
||||
} else if (Slice.CONSTRUCTION.equals(oSlice.getState())) {
|
||||
for (Replica replica : oSlice.getReplicas()) {
|
||||
String core = replica.getStr("core");
|
||||
log.info("Unloading core: " + core + " from node: " + replica.getNodeName());
|
||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||
params.set(CoreAdminParams.ACTION, CoreAdminAction.UNLOAD.toString());
|
||||
params.set(CoreAdminParams.CORE, core);
|
||||
params.set(CoreAdminParams.DELETE_INDEX, "true");
|
||||
sendShardRequest(replica.getNodeName(), params);
|
||||
if (clusterState.liveNodesContain(replica.getNodeName())) {
|
||||
String core = replica.getStr("core");
|
||||
log.info("Unloading core: " + core + " from node: " + replica.getNodeName());
|
||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||
params.set(CoreAdminParams.ACTION, CoreAdminAction.UNLOAD.toString());
|
||||
params.set(CoreAdminParams.CORE, core);
|
||||
params.set(CoreAdminParams.DELETE_INDEX, "true");
|
||||
sendShardRequest(replica.getNodeName(), params);
|
||||
} else {
|
||||
log.warn("Replica {} exists in shard {} but is not live and cannot be unloaded", replica, oSlice);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -397,7 +401,12 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
|
|||
//params.set(ZkStateReader.NUM_SHARDS_PROP, numSlices); todo: is it necessary, we're not creating collections?
|
||||
|
||||
sendShardRequest(nodeName, params);
|
||||
}
|
||||
|
||||
collectShardResponses(results, true,
|
||||
"SPLTSHARD failed to create subshard leaders");
|
||||
|
||||
for (String subShardName : subShardNames) {
|
||||
// wait for parent leader to acknowledge the sub-shard core
|
||||
log.info("Asking parent leader to wait for: " + subShardName + " to be alive on: " + nodeName);
|
||||
String coreNodeName = waitForCoreNodeName(collection, zkStateReader.getZkClient().getBaseUrlForNodeName(nodeName), subShardName);
|
||||
|
@ -412,7 +421,7 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
|
|||
}
|
||||
|
||||
collectShardResponses(results, true,
|
||||
"SPLTSHARD failed to create subshard leaders or timed out waiting for them to come up");
|
||||
"SPLTSHARD timed out waiting for subshard leaders to come up");
|
||||
|
||||
log.info("Successfully created all sub-shards for collection "
|
||||
+ collectionName + " parent shard: " + slice + " on: " + parentShardLeader);
|
||||
|
@ -583,8 +592,13 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
|
|||
srsp = shardHandler.takeCompletedOrError();
|
||||
if (srsp != null) {
|
||||
processResponse(results, srsp);
|
||||
if (abortOnError && srsp.getException() != null) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, msgOnError, srsp.getException());
|
||||
Throwable exception = srsp.getException();
|
||||
if (abortOnError && exception != null) {
|
||||
// drain pending requests
|
||||
while (srsp != null) {
|
||||
srsp = shardHandler.takeCompletedOrError();
|
||||
}
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, msgOnError, exception);
|
||||
}
|
||||
}
|
||||
} while (srsp != null);
|
||||
|
|
|
@ -146,9 +146,22 @@ public class ShardSplitTest extends BasicDistributedZkTest {
|
|||
indexThread.start();
|
||||
|
||||
try {
|
||||
splitShard(SHARD1);
|
||||
log.info("Layout after split: \n");
|
||||
printLayout();
|
||||
for (int i = 0; i < 3; i++) {
|
||||
try {
|
||||
splitShard(SHARD1);
|
||||
log.info("Layout after split: \n");
|
||||
printLayout();
|
||||
break;
|
||||
} catch (HttpSolrServer.RemoteSolrException e) {
|
||||
if (e.code() != 500) {
|
||||
throw e;
|
||||
}
|
||||
log.error("SPLITSHARD failed. " + (i < 2 ? " Retring split" : ""), e);
|
||||
if (i == 2) {
|
||||
fail("SPLITSHARD was not successful even after three tries");
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
try {
|
||||
indexThread.join();
|
||||
|
@ -158,33 +171,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
|
|||
}
|
||||
|
||||
commit();
|
||||
|
||||
try {
|
||||
checkDocCountsAndShardStates(docCounts, numReplicas);
|
||||
} catch (HttpSolrServer.RemoteSolrException e) {
|
||||
if (e.code() != 500) {
|
||||
throw e;
|
||||
}
|
||||
|
||||
// if we get a 500 error, the split should be retried ... let's wait and see if it works...
|
||||
Slice slice1_0 = null, slice1_1 = null;
|
||||
int i = 0;
|
||||
for (i = 0; i < 60; i++) {
|
||||
ZkStateReader zkStateReader = cloudClient.getZkStateReader();
|
||||
zkStateReader.updateClusterState(true);
|
||||
clusterState = zkStateReader.getClusterState();
|
||||
slice1_0 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, "shard1_0");
|
||||
slice1_1 = clusterState.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, "shard1_1");
|
||||
if (slice1_0 != null && slice1_1 != null) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(500);
|
||||
}
|
||||
|
||||
if (slice1_0 == null || slice1_1 == null) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
checkDocCountsAndShardStates(docCounts, numReplicas);
|
||||
|
||||
// todo can't call waitForThingsToLevelOut because it looks for jettys of all shards
|
||||
// and the new sub-shards don't have any.
|
||||
|
|
Loading…
Reference in New Issue