mirror of https://github.com/apache/lucene.git
SOLR-8075: Leader Initiated Recovery should not stop a leader that participated in an election with all of it's replicas from becoming a valid leader.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1706699 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
66ab2d013f
commit
f4a599904f
|
@ -228,6 +228,9 @@ Bug Fixes
|
|||
* SOLR-8094: HdfsUpdateLog should not replay buffered documents as a replacement to dropping them.
|
||||
(Mark Miller)
|
||||
|
||||
* SOLR-8075: Leader Initiated Recovery should not stop a leader that participated in an election with all
|
||||
of it's replicas from becoming a valid leader. (Mark Miller)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -287,8 +287,9 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
Overseer.getInQueue(zkClient).offer(Utils.toJSON(m));
|
||||
|
||||
int leaderVoteWait = cc.getZkController().getLeaderVoteWait();
|
||||
boolean allReplicasInLine = false;
|
||||
if (!weAreReplacement) {
|
||||
waitForReplicasToComeUp(leaderVoteWait);
|
||||
allReplicasInLine = waitForReplicasToComeUp(leaderVoteWait);
|
||||
}
|
||||
|
||||
if (isClosed) {
|
||||
|
@ -407,6 +408,23 @@ 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);
|
||||
|
@ -476,7 +494,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
} // core gets closed automagically
|
||||
}
|
||||
|
||||
private void waitForReplicasToComeUp(int timeoutms) throws InterruptedException {
|
||||
// returns true if all replicas are found to be up, false if not
|
||||
private boolean waitForReplicasToComeUp(int timeoutms) throws InterruptedException {
|
||||
long timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutms, TimeUnit.MILLISECONDS);
|
||||
final String shardsElectZkPath = electionPath + LeaderElector.ELECTION_NODE;
|
||||
|
||||
|
@ -502,7 +521,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
// on startup and after connection timeout, wait for all known shards
|
||||
if (found >= slices.getReplicasMap().size()) {
|
||||
log.info("Enough replicas found to continue.");
|
||||
return;
|
||||
return true;
|
||||
} else {
|
||||
if (cnt % 40 == 0) {
|
||||
log.info("Waiting until we see more replicas up for shard {}: total={}"
|
||||
|
@ -515,12 +534,12 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
|
||||
if (System.nanoTime() > timeoutAt) {
|
||||
log.info("Was waiting for replicas to come up, but they are taking too long - assuming they won't come back till later");
|
||||
return;
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
log.warn("Shard not found: " + shardId + " for collection " + collection);
|
||||
|
||||
return;
|
||||
return false;
|
||||
|
||||
}
|
||||
|
||||
|
@ -528,6 +547,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
slices = zkController.getClusterState().getSlice(collection, shardId);
|
||||
cnt++;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private void rejoinLeaderElection(SolrCore core)
|
||||
|
|
|
@ -37,7 +37,7 @@
|
|||
<str name="hostContext">${hostContext:solr}</str>
|
||||
<int name="zkClientTimeout">${solr.zkclienttimeout:30000}</int>
|
||||
<bool name="genericCoreNodeNames">${genericCoreNodeNames:true}</bool>
|
||||
<int name="leaderVoteWait">0</int>
|
||||
<int name="leaderVoteWait">10000</int>
|
||||
<int name="distribUpdateConnTimeout">${distribUpdateConnTimeout:45000}</int>
|
||||
<int name="distribUpdateSoTimeout">${distribUpdateSoTimeout:340000}</int>
|
||||
</solrcloud>
|
||||
|
|
|
@ -0,0 +1,83 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.lucene.util.LuceneTestCase.Nightly;
|
||||
import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
|
||||
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.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@Slow
|
||||
@SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
|
||||
@Nightly
|
||||
public class LeaderInitiatedRecoveryOnShardRestartTest extends AbstractFullDistribZkTestBase {
|
||||
|
||||
protected static final transient Logger log = LoggerFactory.getLogger(LeaderInitiatedRecoveryOnShardRestartTest.class);
|
||||
|
||||
|
||||
public LeaderInitiatedRecoveryOnShardRestartTest() {
|
||||
super();
|
||||
sliceCount = 1;
|
||||
fixShardCount(2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRestartWithAllInLIR() throws Exception {
|
||||
waitForThingsToLevelOut(30000);
|
||||
|
||||
String testCollectionName = "all_in_lir";
|
||||
String shardId = "shard1";
|
||||
createCollection(testCollectionName, 1, 2, 1);
|
||||
|
||||
cloudClient.setDefaultCollection(testCollectionName);
|
||||
|
||||
Map<String,Object> stateObj = Utils.makeMap();
|
||||
stateObj.put(ZkStateReader.STATE_PROP, "down");
|
||||
stateObj.put("createdByNodeName", "test");
|
||||
stateObj.put("createdByCoreNodeName", "test");
|
||||
|
||||
byte[] znodeData = Utils.toJSON(stateObj);
|
||||
|
||||
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);
|
||||
|
||||
printLayout();
|
||||
|
||||
for (JettySolrRunner jetty : jettys) {
|
||||
ChaosMonkey.stop(jetty);
|
||||
}
|
||||
|
||||
Thread.sleep(2000);
|
||||
|
||||
for (JettySolrRunner jetty : jettys) {
|
||||
ChaosMonkey.start(jetty);
|
||||
}
|
||||
|
||||
// recoveries will not finish without SOLR-8075
|
||||
waitForRecoveriesToFinish(testCollectionName, true);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue