SOLR-8075: Fix faulty implementation.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1714204 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2015-11-13 13:40:52 +00:00
parent 918476e0ac
commit 820eea1f3d
2 changed files with 74 additions and 34 deletions

View File

@ -393,14 +393,20 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
return;
}
log.info("I am the new leader: " + ZkCoreNodeProps.getCoreUrl(leaderProps) + " " + shardId);
core.getCoreDescriptor().getCloudDescriptor().setLeader(true);
}
boolean isLeader = true;
if (!isClosed) {
try {
// we must check LIR before registering as leader
checkLIR(coreName, allReplicasInLine);
super.runLeaderProcess(weAreReplacement, 0);
try (SolrCore core = cc.getCore(coreName)) {
core.getCoreDescriptor().getCloudDescriptor().setLeader(true);
}
log.info("I am the new leader: " + ZkCoreNodeProps.getCoreUrl(leaderProps) + " " + shardId);
} catch (Exception e) {
isLeader = false;
SolrException.log(log, "There was a problem trying to register as the leader", e);
@ -420,23 +426,6 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
}
if (isLeader) {
if (allReplicasInLine) {
// SOLR-8075: A bug may allow the proper leader to get marked as LIR DOWN and
// if we are marked as DOWN but were able to become the leader, we remove
// the DOWN entry here so that we don't fail publishing ACTIVE due to being in LIR.
// We only do this if all the replicas participated in the election just in case
// this was a valid LIR entry and the proper leader replica is missing.
try (SolrCore core = cc.getCore(coreName)) {
final Replica.State lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId,
core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
if (lirState == Replica.State.DOWN) {
zkController.updateLeaderInitiatedRecoveryState(collection, shardId,
leaderProps.getStr(ZkStateReader.CORE_NODE_NAME_PROP), Replica.State.ACTIVE, null, true);
}
}
}
// check for any replicas in my shard that were set to down by the previous leader
try {
startLeaderInitiatedRecoveryOnReplicas(coreName);
@ -453,6 +442,41 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
}
}
public void checkLIR(String coreName, boolean allReplicasInLine)
throws InterruptedException, KeeperException, IOException {
if (allReplicasInLine) {
// SOLR-8075: A bug may allow the proper leader to get marked as LIR DOWN and
// if we are marked as DOWN but were able to become the leader, we remove
// the DOWN entry here so that we don't fail publishing ACTIVE due to being in LIR.
// We only do this if all the replicas participated in the election just in case
// this was a valid LIR entry and the proper leader replica is missing.
try (SolrCore core = cc.getCore(coreName)) {
final Replica.State lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId,
core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
if (lirState == Replica.State.DOWN) {
// We can do this before registering as leader because only setting DOWN requires that
// we are already registered as leader, and here we are setting ACTIVE
// The fact that we just won the zk leader election provides a quasi lock on setting this state, but
// we should improve this: see SOLR-8075 discussion
zkController.updateLeaderInitiatedRecoveryState(collection, shardId,
leaderProps.getStr(ZkStateReader.CORE_NODE_NAME_PROP), Replica.State.ACTIVE, core.getCoreDescriptor(), true);
}
}
} else {
try (SolrCore core = cc.getCore(coreName)) {
final Replica.State lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId,
core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
if (lirState == Replica.State.DOWN || lirState == Replica.State.RECOVERING) {
log.warn("The previous leader marked me " + core.getName()
+ " as " + lirState.toString() + " and I haven't recovered yet, so I shouldn't be the leader.");
throw new SolrException(ErrorCode.SERVER_ERROR, "Leader Initiated Recovery prevented leadership");
}
}
}
}
private void startLeaderInitiatedRecoveryOnReplicas(String coreName) throws Exception {
try (SolrCore core = cc.getCore(coreName)) {
CloudDescriptor cloudDesc = core.getCoreDescriptor().getCloudDescriptor();
@ -594,18 +618,6 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
}
if (core.getCoreDescriptor().getCloudDescriptor().getLastPublished() == Replica.State.ACTIVE) {
// maybe active but if the previous leader marked us as down and
// we haven't recovered, then can't be leader
final Replica.State lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId,
core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
if (lirState == Replica.State.DOWN || lirState == Replica.State.RECOVERING) {
log.warn("Although my last published state is Active, the previous leader marked me "+core.getName()
+ " as " + lirState.toString()
+ " and I haven't recovered yet, so I shouldn't be the leader.");
return false;
}
log.info("My last published State was Active, it's okay to be the leader.");
return true;
}

View File

@ -26,6 +26,7 @@ import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -41,7 +42,7 @@ public class LeaderInitiatedRecoveryOnShardRestartTest extends AbstractFullDistr
public LeaderInitiatedRecoveryOnShardRestartTest() {
super();
sliceCount = 1;
fixShardCount(2);
fixShardCount(3);
}
@Test
@ -50,7 +51,7 @@ public class LeaderInitiatedRecoveryOnShardRestartTest extends AbstractFullDistr
String testCollectionName = "all_in_lir";
String shardId = "shard1";
createCollection(testCollectionName, 1, 2, 1);
createCollection(testCollectionName, 1, 3, 1);
cloudClient.setDefaultCollection(testCollectionName);
@ -64,8 +65,9 @@ public class LeaderInitiatedRecoveryOnShardRestartTest extends AbstractFullDistr
SolrZkClient zkClient = cloudClient.getZkStateReader().getZkClient();
zkClient.makePath("/collections/" + testCollectionName + "/leader_initiated_recovery/" + shardId + "/core_node1", znodeData, true);
zkClient.makePath("/collections/" + testCollectionName + "/leader_initiated_recovery/" + shardId + "/core_node2", znodeData, true);
zkClient.makePath("/collections/" + testCollectionName + "/leader_initiated_recovery/" + shardId + "/core_node3", znodeData, true);
printLayout();
// printLayout();
for (JettySolrRunner jetty : jettys) {
ChaosMonkey.stop(jetty);
@ -79,5 +81,31 @@ public class LeaderInitiatedRecoveryOnShardRestartTest extends AbstractFullDistr
// recoveries will not finish without SOLR-8075
waitForRecoveriesToFinish(testCollectionName, true);
// now expire each node
try {
zkClient.makePath("/collections/" + testCollectionName + "/leader_initiated_recovery/" + shardId + "/core_node1", znodeData, true);
} catch (NodeExistsException e) {
}
try {
zkClient.makePath("/collections/" + testCollectionName + "/leader_initiated_recovery/" + shardId + "/core_node2", znodeData, true);
} catch (NodeExistsException e) {
}
try {
zkClient.makePath("/collections/" + testCollectionName + "/leader_initiated_recovery/" + shardId + "/core_node3", znodeData, true);
} catch (NodeExistsException e) {
}
for (JettySolrRunner jetty : jettys) {
chaosMonkey.expireSession(jetty);
}
Thread.sleep(2000);
// recoveries will not finish without SOLR-8075
waitForRecoveriesToFinish(testCollectionName, true);
}
}