SOLR-3125: When a SolrCore registers in zk there may be stale leader state in the clusterstate.json.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1243195 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2012-02-12 02:45:06 +00:00
parent 4cffbd71b5
commit 83fa9d80fa
1 changed files with 48 additions and 7 deletions

View File

@ -50,6 +50,7 @@ import org.apache.solr.core.SolrCore;
import org.apache.solr.update.UpdateLog;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -464,11 +465,6 @@ public final class ZkController {
final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
final String collection = cloudDesc.getCollectionName();
log.info("Attempting to update " + ZkStateReader.CLUSTER_STATE + " version "
+ null);
CloudState state = CloudState.load(zkClient, zkStateReader.getCloudState().getLiveNodes());
final String coreZkNodeName = getNodeName() + "_" + coreName;
@ -495,8 +491,25 @@ public final class ZkController {
joinElection(collection, coreZkNodeName, shardId, leaderProps);
String leaderUrl = zkStateReader.getLeaderUrl(collection,
cloudDesc.getShardId(), 30000);
// rather than look in the cluster state file, we go straight to the zknodes
// here, because on cluster restart there could be stale leader info in the
// cluster state node that won't be updated for a moment
String leaderUrl = getLeaderUrl(collection, cloudDesc.getShardId());
// now wait until our currently cloud state contains the latest leader
String cloudStateLeader = zkStateReader.getLeaderUrl(collection, cloudDesc.getShardId(), 30000);
int tries = 0;
while (!leaderUrl.equals(cloudStateLeader)) {
if (tries == 60) {
throw new SolrException(ErrorCode.SERVER_ERROR,
"There is conflicting information about the leader of shard: "
+ cloudDesc.getShardId());
}
Thread.sleep(1000);
tries++;
cloudStateLeader = zkStateReader.getLeaderUrl(collection,
cloudDesc.getShardId(), 30000);
}
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
log.info("We are " + ourUrl + " and leader is " + leaderUrl);
@ -551,6 +564,32 @@ public final class ZkController {
return shardId;
}
/**
* Get leader URL directly from zk nodes.
*
* @param collection
* @param slice
* @return
* @throws KeeperException
* @throws InterruptedException
*/
private String getLeaderUrl(final String collection, final String slice)
throws KeeperException, InterruptedException {
int iterCount = 60;
while (iterCount-- > 0)
try {
byte[] data = zkClient.getData(
ZkStateReader.getShardLeadersPath(collection, slice), null, null,
true);
ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(
ZkNodeProps.load(data));
return leaderProps.getCoreUrl();
} catch (NoNodeException e) {
Thread.sleep(500);
}
throw new RuntimeException("Could not get leader props");
}
private void joinElection(final String collection,
@ -600,6 +639,8 @@ public final class ZkController {
core.getUpdateHandler().getSolrCoreState().doRecovery(core);
return true;
}
} else {
log.info("I am the leader, no recovery necessary");
}
return false;