SOLR-5859 improved logging, and fix a potential bug

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1584271 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Noble Paul 2014-04-03 07:07:50 +00:00
parent b790367aa6
commit 77f452b45b
3 changed files with 16 additions and 8 deletions

View File

@ -68,6 +68,7 @@ public abstract class ElectionContext {
public void cancelElection() throws InterruptedException, KeeperException { public void cancelElection() throws InterruptedException, KeeperException {
try { try {
log.info("canceling election {}",leaderSeqPath );
zkClient.delete(leaderSeqPath, -1, true); zkClient.delete(leaderSeqPath, -1, true);
} catch (NoNodeException e) { } catch (NoNodeException e) {
// fine // fine

View File

@ -283,6 +283,7 @@ public class Overseer {
} }
} finally { } finally {
log.info("Overseer Loop exiting : {}", LeaderElector.getNodeName(myId));
new Thread("OverseerExitThread"){ new Thread("OverseerExitThread"){
//do this in a separate thread because any wait is interrupted in this main thread //do this in a separate thread because any wait is interrupted in this main thread
@Override @Override
@ -372,7 +373,7 @@ public class Overseer {
} else if(CLUSTERPROP.isEqual(operation)){ } else if(CLUSTERPROP.isEqual(operation)){
handleProp(message); handleProp(message);
} else if( QUIT.equals(operation)){ } else if( QUIT.equals(operation)){
log.info("################Quit command receive"); log.info("Quit command received {}", LeaderElector.getNodeName(myId));
overseerCollectionProcessor.close(); overseerCollectionProcessor.close();
close(); close();
} else{ } else{

View File

@ -289,7 +289,7 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
} }
private void prioritizeOverseerNodes() throws KeeperException, InterruptedException { private void prioritizeOverseerNodes() throws KeeperException, InterruptedException {
log.info("prioritizing overseer nodes"); log.info("prioritizing overseer nodes at {}", LeaderElector.getNodeName(myId));
SolrZkClient zk = zkStateReader.getZkClient(); SolrZkClient zk = zkStateReader.getZkClient();
if(!zk.exists(ZkStateReader.ROLES,true))return; if(!zk.exists(ZkStateReader.ROLES,true))return;
Map m = (Map) ZkStateReader.fromJSON(zk.getData(ZkStateReader.ROLES, null, new Stat(), true)); Map m = (Map) ZkStateReader.fromJSON(zk.getData(ZkStateReader.ROLES, null, new Stat(), true));
@ -301,6 +301,7 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
List<String> nodeNames = getSortedOverseerNodeNames(zk); List<String> nodeNames = getSortedOverseerNodeNames(zk);
if(nodeNames.size()<2) return; if(nodeNames.size()<2) return;
boolean designateIsInFront = overseerDesignates.contains( nodeNames.get(0));
// //
ArrayList<String> nodesTobePushedBack = new ArrayList<>(); ArrayList<String> nodesTobePushedBack = new ArrayList<>();
@ -322,7 +323,7 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
} }
} }
if(availableDesignates.size()>1) break; if(availableDesignates.size()>1) break;//we don't need to line up more than 2 designates
} }
if(!availableDesignates.isEmpty()){ if(!availableDesignates.isEmpty()){
@ -360,17 +361,22 @@ public class OverseerCollectionProcessor implements Runnable, ClosableThread {
log.warn("available designates and current state {} {} ", availableDesignates, getSortedOverseerNodeNames(zk)); log.warn("available designates and current state {} {} ", availableDesignates, getSortedOverseerNodeNames(zk));
} }
} else { } else if(!designateIsInFront) {
log.warn("No overseer designates are available, overseerDesignates: {}, live nodes : {}",overseerDesignates,nodeNames); log.warn("No overseer designates are available, overseerDesignates: {}, live nodes : {}",overseerDesignates,nodeNames);
return; return;
} }
String leaderNode = getLeaderNode(zkStateReader.getZkClient()); String leaderNode = getLeaderNode(zkStateReader.getZkClient());
if(leaderNode ==null) return; if(leaderNode ==null) return;
if(!overseerDesignates.contains(leaderNode) && !availableDesignates.isEmpty()){ if(!overseerDesignates.contains(leaderNode) ){
//this means there are designated Overseer nodes and I am not one of them , kill myself List<String> sortedNodes = getSortedOverseerNodeNames(zk);
log.info("I am not an overseer designate , forcing myself out {} ", leaderNode);
Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(new ZkNodeProps( Overseer.QUEUE_OPERATION, Overseer.QUIT))); if(leaderNode.equals(sortedNodes.get(0)) || // I am leader and I am in front of the queue
overseerDesignates.contains(sortedNodes.get(0))) {// I am leader but somebody else is in the front , Screwed up leader election
//this means there are I am not a designate and the next guy is lined up to become the leader, kill myself
log.info("I am not an overseer designate , forcing myself out {} ", leaderNode);
Overseer.getInQueue(zkStateReader.getZkClient()).offer(ZkStateReader.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, Overseer.QUIT)));
}
} }
} }