SOLR-3939: Consider a sync attempt from leader to replica that fails due to 404 a success.

SOLR-3940: Rejoining the leader election incorrectly triggers the code path for a fresh cluster start rather than fail over.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1397665 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2012-10-12 18:25:44 +00:00
parent fcb7a55d3b
commit 28fe93b39b
8 changed files with 54 additions and 25 deletions

View File

@ -42,9 +42,7 @@ New Features
values of a multiValued field in their original order when highlighting.
(Joel Bernstein via yonik)
* SOLR-3929
support configuring IndexWriter max thread count in solrconfig
* SOLR-3929: Support configuring IndexWriter max thread count in solrconfig.
(phunt via Mark Miller)
Optimizations
@ -73,6 +71,12 @@ Bug Fixes
* SOLR-3917: Partial State on Schema-Browser UI is not defined for Dynamic
Fields & Types (steffkes)
* SOLR-3939: Consider a sync attempt from leader to replica that fails due
to 404 a success. (Mark Miller, Joel Bernstein)
* SOLR-3940: Rejoining the leader election incorrectly triggers the code path
for a fresh cluster start rather than fail over. (Mark Miller)
Other Changes
----------------------

View File

@ -324,7 +324,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
SolrException.log(log, "Error trying to start recovery", t);
}
leaderElector.joinElection(this);
leaderElector.joinElection(this, true);
}
private boolean shouldIBeLeader(ZkNodeProps leaderProps, SolrCore core) {

View File

@ -18,7 +18,6 @@ package org.apache.solr.cloud;
*/
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
@ -43,7 +42,7 @@ import org.slf4j.LoggerFactory;
* Leader Election process. This class contains the logic by which a
* leader is chosen. First call * {@link #setup(ElectionContext)} to ensure
* the election process is init'd. Next call
* {@link #joinElection(ElectionContext)} to start the leader election.
* {@link #joinElection(ElectionContext, boolean)} to start the leader election.
*
* The implementation follows the classic ZooKeeper recipe of creating an
* ephemeral, sequential node for each candidate and then looking at the set
@ -203,7 +202,7 @@ public class LeaderElector {
*
* @return sequential node number
*/
public int joinElection(ElectionContext context) throws KeeperException, InterruptedException, IOException {
public int joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
final String shardsElectZkPath = context.electionPath + LeaderElector.ELECTION_NODE;
long sessionId = zkClient.getSolrZooKeeper().getSessionId();
@ -259,7 +258,7 @@ public class LeaderElector {
}
}
int seq = getSeq(leaderSeqPath);
checkIfIamLeader(seq, context, false);
checkIfIamLeader(seq, context, replacement);
return seq;
}

View File

@ -191,7 +191,7 @@ public final class ZkController {
ZkController.this.overseer = new Overseer(shardHandler, adminPath, zkStateReader);
ElectionContext context = new OverseerElectionContext(zkClient, overseer, getNodeName());
overseerElector.joinElection(context);
overseerElector.joinElection(context, true);
zkStateReader.createClusterStateWatchersAndUpdate();
// cc.newCmdDistribExecutor();
@ -422,7 +422,7 @@ public final class ZkController {
this.overseer = new Overseer(shardHandler, adminPath, zkStateReader);
ElectionContext context = new OverseerElectionContext(zkClient, overseer, getNodeName());
overseerElector.setup(context);
overseerElector.joinElection(context);
overseerElector.joinElection(context, false);
zkStateReader.createClusterStateWatchersAndUpdate();
} catch (IOException e) {
@ -730,7 +730,7 @@ public final class ZkController {
leaderElector.setup(context);
electionContexts.put(coreZkNodeName, context);
leaderElector.joinElection(context);
leaderElector.joinElection(context, false);
}

View File

@ -312,6 +312,11 @@ public class PeerSync {
log.warn(msg() + " got a 503 from " + srsp.getShardAddress() + ", counting as success");
return true;
}
if (cantReachIsSuccess && sreq.purpose == 1 && srsp.getException() instanceof SolrException && ((SolrException) srsp.getException()).code() == 404) {
log.warn(msg() + " got a 404 from " + srsp.getShardAddress() + ", counting as success");
return true;
}
// TODO: at least log???
// srsp.getException().printStackTrace(System.out);

View File

@ -49,6 +49,7 @@ import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.response.CoreAdminResponse;
import org.apache.solr.client.solrj.response.QueryResponse;
@ -742,10 +743,10 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
0,
((HttpSolrServer) client).getBaseURL().length()
- DEFAULT_COLLECTION.length() - 1);
createCollection(oneInstanceCollection2, collectionClients, baseUrl, 1, "slice1");
createCollection(oneInstanceCollection2, collectionClients, baseUrl, 2, "slice2");
createCollection(oneInstanceCollection2, collectionClients, baseUrl, 3, "slice2");
createCollection(oneInstanceCollection2, collectionClients, baseUrl, 4, "slice1");
createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 1, "slice1");
createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 2, "slice2");
createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 3, "slice2");
createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 4, "slice1");
while (pending != null && pending.size() > 0) {
@ -764,7 +765,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
assertAllActive(oneInstanceCollection2, solrj.getZkStateReader());
printLayout();
//printLayout();
// TODO: enable when we don't falsely get slice1...
// solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
@ -803,6 +804,27 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
assertNotNull(slices);
String roles = slices.get("slice1").getReplicasMap().values().iterator().next().getStr(ZkStateReader.ROLES_PROP);
assertEquals("none", roles);
ZkCoreNodeProps props = new ZkCoreNodeProps(solrj.getZkStateReader().getClusterState().getLeader(oneInstanceCollection2, "slice1"));
// now test that unloading a core gets us a new leader
HttpSolrServer server = new HttpSolrServer(baseUrl);
Unload unloadCmd = new Unload(true);
unloadCmd.setCoreName(props.getCoreName());
String leader = props.getCoreUrl();
server.request(unloadCmd);
int tries = 50;
while (leader.equals(zkStateReader.getLeaderUrl(oneInstanceCollection2, "slice1", 10000))) {
Thread.sleep(100);
if (tries-- == 0) {
fail("Leader never changed");
}
}
}
private void testSearchByCollectionName() throws SolrServerException {
@ -875,10 +897,10 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
private void createCollection(String collection,
List<SolrServer> collectionClients, String baseUrl, int num) {
createCollection(collection, collectionClients, baseUrl, num, null);
createSolrCore(collection, collectionClients, baseUrl, num, null);
}
private void createCollection(final String collection,
private void createSolrCore(final String collection,
List<SolrServer> collectionClients, final String baseUrl, final int num,
final String shardId) {
Callable call = new Callable() {

View File

@ -40,7 +40,6 @@ import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
@Slow
@ -114,7 +113,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
elector, "shard1", "collection1", Integer.toString(nodeNumber),
props, zkStateReader);
elector.setup(context);
seq = elector.joinElection(context);
seq = elector.joinElection(context, false);
electionDone = true;
seqToThread.put(seq, this);
}
@ -175,7 +174,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
ElectionContext context = new ShardLeaderElectionContextBase(elector,
"shard2", "collection1", "dummynode1", props, zkStateReader);
elector.setup(context);
elector.joinElection(context);
elector.joinElection(context, false);
assertEquals("http://127.0.0.1/solr/",
getLeaderUrl("collection1", "shard2"));
}
@ -188,7 +187,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
ElectionContext firstContext = new ShardLeaderElectionContextBase(first,
"slice1", "collection2", "dummynode1", props, zkStateReader);
first.setup(firstContext);
first.joinElection(firstContext);
first.joinElection(firstContext, false);
Thread.sleep(1000);
assertEquals("original leader was not registered", "http://127.0.0.1/solr/1/", getLeaderUrl("collection2", "slice1"));
@ -199,7 +198,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
ElectionContext context = new ShardLeaderElectionContextBase(second,
"slice1", "collection2", "dummynode1", props, zkStateReader);
second.setup(context);
second.joinElection(context);
second.joinElection(context, false);
Thread.sleep(1000);
assertEquals("original leader should have stayed leader", "http://127.0.0.1/solr/1/", getLeaderUrl("collection2", "slice1"));
firstContext.cancelElection();

View File

@ -139,7 +139,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
ShardLeaderElectionContextBase ctx = new ShardLeaderElectionContextBase(
elector, shardId, collection, nodeName + "_" + coreName, props,
zkStateReader);
elector.joinElection(ctx);
elector.joinElection(ctx, false);
return shardId;
}
Thread.sleep(500);
@ -876,7 +876,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
new HttpShardHandlerFactory().getShardHandler(), "/admin/cores", reader);
ElectionContext ec = new OverseerElectionContext(zkClient, overseer, address.replaceAll("/", "_"));
overseerElector.setup(ec);
overseerElector.joinElection(ec);
overseerElector.joinElection(ec, false);
return zkClient;
}