mirror of https://github.com/apache/lucene.git
assume that sometimes the zk state can be stale vs our watcher based state when checking our state matches zk state
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1369189 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
cd7c8221f7
commit
66dac680fe
|
@ -540,18 +540,18 @@ public final class ZkController {
|
|||
String leaderUrl = getLeaderProps(collection, cloudDesc.getShardId()).getCoreUrl();
|
||||
|
||||
// now wait until our currently cloud state contains the latest leader
|
||||
String cloudStateLeader = zkStateReader.getLeaderUrl(collection, cloudDesc.getShardId(), 30000);
|
||||
String cloudStateLeader = zkStateReader.getLeaderUrl(collection, shardId, 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() + " our state says:" + leaderUrl + " but zookeeper says:" + cloudStateLeader);
|
||||
+ cloudDesc.getShardId() + " our state says:" + cloudStateLeader + " but zookeeper says:" + leaderUrl);
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
tries++;
|
||||
cloudStateLeader = zkStateReader.getLeaderUrl(collection,
|
||||
cloudDesc.getShardId(), 30000);
|
||||
cloudStateLeader = zkStateReader.getLeaderUrl(collection, shardId, 30000);
|
||||
leaderUrl = getLeaderProps(collection, cloudDesc.getShardId()).getCoreUrl();
|
||||
}
|
||||
|
||||
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
|
||||
|
@ -617,10 +617,10 @@ public final class ZkController {
|
|||
* @throws KeeperException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private ZkCoreNodeProps getLeaderProps(final String collection, final String slice)
|
||||
throws KeeperException, InterruptedException {
|
||||
private ZkCoreNodeProps getLeaderProps(final String collection,
|
||||
final String slice) throws KeeperException, InterruptedException {
|
||||
int iterCount = 60;
|
||||
while (iterCount-- > 0)
|
||||
while (iterCount-- > 0) {
|
||||
try {
|
||||
byte[] data = zkClient.getData(
|
||||
ZkStateReader.getShardLeadersPath(collection, slice), null, null,
|
||||
|
@ -631,6 +631,7 @@ public final class ZkController {
|
|||
} catch (NoNodeException e) {
|
||||
Thread.sleep(500);
|
||||
}
|
||||
}
|
||||
throw new RuntimeException("Could not get leader props");
|
||||
}
|
||||
|
||||
|
|
|
@ -32,12 +32,16 @@ import org.apache.solr.common.SolrException;
|
|||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.cloud.HashPartitioner.Range;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Immutable state of the cloud. Normally you can get the state by using
|
||||
* {@link ZkStateReader#getCloudState()}.
|
||||
*/
|
||||
public class CloudState implements JSONWriter.Writable {
|
||||
private static Logger log = LoggerFactory.getLogger(CloudState.class);
|
||||
|
||||
private final Map<String, Map<String,Slice>> collectionStates; // Map<collectionName, Map<sliceName,Slice>>
|
||||
private final Set<String> liveNodes;
|
||||
|
||||
|
@ -71,10 +75,10 @@ public class CloudState implements JSONWriter.Writable {
|
|||
Map<String,ZkNodeProps> leadersForCollection = leaders.get(collection.getKey());
|
||||
if (leadersForCollection == null) {
|
||||
leadersForCollection = new HashMap<String,ZkNodeProps>();
|
||||
|
||||
leaders.put(collection.getKey(), leadersForCollection);
|
||||
}
|
||||
leadersForCollection.put(sliceEntry.getKey(), props);
|
||||
break; // we found the leader for this shard
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -185,7 +185,7 @@ public class ZkStateReader {
|
|||
if (EventType.None.equals(event.getType())) {
|
||||
return;
|
||||
}
|
||||
log.info("A cluster state change has occurred");
|
||||
log.info("A cluster state change has occurred - updating...");
|
||||
try {
|
||||
|
||||
// delayed approach
|
||||
|
@ -411,9 +411,8 @@ public class ZkStateReader {
|
|||
public ZkNodeProps getLeaderProps(String collection, String shard, int timeout) throws InterruptedException {
|
||||
long timeoutAt = System.currentTimeMillis() + timeout;
|
||||
while (System.currentTimeMillis() < timeoutAt) {
|
||||
if (cloudState != null) {
|
||||
final CloudState currentState = cloudState;
|
||||
final ZkNodeProps nodeProps = currentState.getLeader(collection, shard);
|
||||
if (cloudState != null) {
|
||||
final ZkNodeProps nodeProps = cloudState.getLeader(collection, shard);
|
||||
if (nodeProps != null) {
|
||||
return nodeProps;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue