You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/02/12 03:45:06 UTC
svn commit: r1243195 -
/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
Author: markrmiller
Date: Sun Feb 12 02:45:06 2012
New Revision: 1243195
URL: http://svn.apache.org/viewvc?rev=1243195&view=rev
Log:
SOLR-3125: When a SolrCore registers in zk there may be stale leader state in the clusterstate.json.
Modified:
lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1243195&r1=1243194&r2=1243195&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java Sun Feb 12 02:45:06 2012
@@ -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;