mirror of https://github.com/apache/lucene.git
SOLR-4570: Even if an explicit shard id is used, ZkController#preRegister should still wait to see the shard id in it's current ClusterState.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1456731 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
4609e9017f
commit
841d757232
|
@ -105,6 +105,10 @@ Bug Fixes
|
|||
|
||||
* SOLR-4568: The lastPublished state check before becoming a leader is not
|
||||
working correctly. (Mark Miller)
|
||||
|
||||
* SOLR-4570: Even if an explicit shard id is used, ZkController#preRegister
|
||||
should still wait to see the shard id in it's current ClusterState.
|
||||
(Mark Miller)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
|
|
@ -836,20 +836,20 @@ public final class ZkController {
|
|||
.getCoreUrl();
|
||||
|
||||
// now wait until our currently cloud state contains the latest leader
|
||||
String clusterStateLeader = zkStateReader.getLeaderUrl(collection,
|
||||
String clusterStateLeaderUrl = zkStateReader.getLeaderUrl(collection,
|
||||
shardId, timeoutms * 2); // since we found it in zk, we are willing to
|
||||
// wait a while to find it in state
|
||||
int tries = 0;
|
||||
while (!leaderUrl.equals(clusterStateLeader)) {
|
||||
while (!leaderUrl.equals(clusterStateLeaderUrl)) {
|
||||
if (tries == 60) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"There is conflicting information about the leader of shard: "
|
||||
+ cloudDesc.getShardId() + " our state says:"
|
||||
+ clusterStateLeader + " but zookeeper says:" + leaderUrl);
|
||||
+ clusterStateLeaderUrl + " but zookeeper says:" + leaderUrl);
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
tries++;
|
||||
clusterStateLeader = zkStateReader.getLeaderUrl(collection, shardId,
|
||||
clusterStateLeaderUrl = zkStateReader.getLeaderUrl(collection, shardId,
|
||||
timeoutms);
|
||||
leaderUrl = getLeaderProps(collection, cloudDesc.getShardId(), timeoutms)
|
||||
.getCoreUrl();
|
||||
|
@ -858,7 +858,7 @@ public final class ZkController {
|
|||
} catch (Exception e) {
|
||||
log.error("Error getting leader from zk", e);
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Error getting leader from zk", e);
|
||||
"Error getting leader from zk for shard " + shardId, e);
|
||||
}
|
||||
return leaderUrl;
|
||||
}
|
||||
|
@ -1285,6 +1285,9 @@ public final class ZkController {
|
|||
String shardId;
|
||||
shardId = doGetShardIdProcess(cd.getName(), cd);
|
||||
cd.getCloudDescriptor().setShardId(shardId);
|
||||
} else {
|
||||
// still wait till we see us in local state
|
||||
doGetShardIdProcess(cd.getName(), cd);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue