You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/01/19 19:50:09 UTC
svn commit: r1233525 - in
/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr:
cloud/ElectionContext.java cloud/SyncStrategy.java
update/processor/DistributedUpdateProcessor.java
Author: markrmiller
Date: Thu Jan 19 18:50:09 2012
New Revision: 1233525
URL: http://svn.apache.org/viewvc?rev=1233525&view=rev
Log: (empty)
Modified:
lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java?rev=1233525&r1=1233524&r2=1233525&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java Thu Jan 19 18:50:09 2012
@@ -50,8 +50,6 @@ public abstract class ElectionContext {
abstract void runLeaderProcess(String leaderSeqPath, boolean weAreReplacement) throws KeeperException, InterruptedException, IOException;
}
-
-
class ShardLeaderElectionContextBase extends ElectionContext {
protected final SolrZkClient zkClient;
@@ -114,6 +112,7 @@ final class ShardLeaderElectionContext e
try {
// should I be leader?
if (weAreReplacement && !shouldIBeLeader(leaderProps)) {
+ System.out.println("there is a better leader candidate it appears");
rejoinLeaderElection(leaderSeqPath, core);
return;
}
@@ -159,10 +158,6 @@ final class ShardLeaderElectionContext e
leaderElector.joinElection(this);
}
-
-
-
-
private boolean shouldIBeLeader(ZkNodeProps leaderProps) {
CloudState cloudState = zkController.getZkStateReader().getCloudState();
Map<String,Slice> slices = cloudState.getSlices(this.collection);
@@ -195,7 +190,6 @@ final class ShardLeaderElectionContext e
return !foundSomeoneElseActive;
}
-
}
final class OverseerElectionContext extends ElectionContext {
Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java?rev=1233525&r1=1233524&r2=1233525&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java Thu Jan 19 18:50:09 2012
@@ -27,18 +27,24 @@ import java.util.Map;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery;
import org.apache.solr.common.cloud.CloudState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.SolrCore;
import org.apache.solr.update.PeerSync;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class SyncStrategy {
+ protected final Logger log = LoggerFactory.getLogger(getClass());
+
public boolean sync(ZkController zkController, SolrCore core,
ZkNodeProps leaderProps) {
zkController.publish(core, ZkStateReader.SYNC);
@@ -168,7 +174,10 @@ public class SyncStrategy {
List<String> syncWith = new ArrayList<String>();
for (ZkCoreNodeProps node : nodes) {
- syncWith.add(node.getCoreUrl());
+ // if we see a leader, must be stale state, and this is the guy that went down
+ if (!node.getNodeProps().keySet().contains(ZkStateReader.LEADER_PROP)) {
+ syncWith.add(node.getCoreUrl());
+ }
}
@@ -194,18 +203,33 @@ public class SyncStrategy {
for (ZkCoreNodeProps node : nodes) {
try {
// TODO: do we first everyone register as sync phase? get the overseer
- // to
- // do
- // it?
+ // to do it?
+ // TODO: this should be done in parallel
QueryRequest qr = new QueryRequest(params("qt", "/get", "getVersions",
- Integer.toString(1000), "sync", StrUtils.join(
- Collections.singletonList(zkLeader.getCoreUrl()), ',')));
- CommonsHttpSolrServer server = null;
-
- server = new CommonsHttpSolrServer(node.getCoreUrl());
+ Integer.toString(1000), "sync", zkLeader.getCoreUrl(), "distrib", "false"));
+ CommonsHttpSolrServer server = new CommonsHttpSolrServer(node.getCoreUrl());
NamedList rsp = server.request(qr);
System.out.println("response about syncing to leader:" + rsp);
+ boolean success = (Boolean) rsp.get("sync");
+ System.out.println("success:" + success);
+ if (!success) {
+ System.out.println("try and ask " + node.getCoreUrl() + " to recover");
+ log.info("try and ask " + node.getCoreUrl() + " to recover");
+ try {
+ server = new CommonsHttpSolrServer(node.getBaseUrl());
+ server.setSoTimeout(5000);
+ server.setConnectionTimeout(5000);
+
+ RequestRecovery recoverRequestCmd = new RequestRecovery();
+ recoverRequestCmd.setAction(CoreAdminAction.REQUESTRECOVERY);
+ recoverRequestCmd.setCoreName(node.getCoreName());
+
+ server.request(recoverRequestCmd);
+ } catch (Exception e) {
+ log.info("Could not tell a replica to recover", e);
+ }
+ }
} catch (Exception e) {
// nocommit
e.printStackTrace();
Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1233525&r1=1233524&r2=1233525&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Thu Jan 19 18:50:09 2012
@@ -283,7 +283,7 @@ public class DistributedUpdateProcessor
continue;
}
// TODO: we should force their state to recovering ??
-
+ // TODO: could be sent in parallel
// TODO: do retries??
// TODO: what if its is already recovering? Right now recoveries queue up -
// should they?