You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/07/02 15:27:12 UTC
svn commit: r1498928 - in /lucene/dev/branches/branch_4x: ./ solr/ solr/core/
solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
Author: shalin
Date: Tue Jul 2 13:27:12 2013
New Revision: 1498928
URL: http://svn.apache.org/r1498928
Log:
SOLR-4933: Retry splitshard three times before giving up
Modified:
lucene/dev/branches/branch_4x/ (props changed)
lucene/dev/branches/branch_4x/solr/ (props changed)
lucene/dev/branches/branch_4x/solr/core/ (props changed)
lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java?rev=1498928&r1=1498927&r2=1498928&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java Tue Jul 2 13:27:12 2013
@@ -360,13 +360,17 @@ public class OverseerCollectionProcessor
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
//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
}
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
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);
Modified: lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java?rev=1498928&r1=1498927&r2=1498928&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ShardSplitTest.java Tue Jul 2 13:27:12 2013
@@ -146,9 +146,22 @@ public class ShardSplitTest extends Basi
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 Basi
}
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.