mirror of https://github.com/apache/lucene.git
SOLR-5495: Hardening recovery scenarios after the leader receives an error trying to forward an update request to a replica.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1593312 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b1a81fe2bc
commit
eb3beb2d70
|
@ -82,6 +82,10 @@ Other Changes
|
|||
* SOLR-5868: HttpClient should be configured to use ALLOW_ALL_HOSTNAME hostname
|
||||
verifier to simplify SSL setup. (Steve Davids via Mark Miller)
|
||||
|
||||
* SOLR-5495: Recovery strategy for leader partitioned from replica case. Hardening
|
||||
recovery scenarios after the leader receives an error trying to forward an
|
||||
update request to a replica. (Timothy Potter)
|
||||
|
||||
================== 4.9.0 ==================
|
||||
|
||||
Versions of Major Components
|
||||
|
|
|
@ -95,6 +95,8 @@ public class JettySolrRunner {
|
|||
private LinkedList<FilterHolder> extraFilters;
|
||||
|
||||
private SSLConfig sslConfig;
|
||||
|
||||
private int proxyPort = -1;
|
||||
|
||||
public static class DebugFilter implements Filter {
|
||||
public int requestsToKeep = 10;
|
||||
|
@ -477,7 +479,7 @@ public class JettySolrRunner {
|
|||
if (0 == conns.length) {
|
||||
throw new RuntimeException("Jetty Server has no Connectors");
|
||||
}
|
||||
return conns[0].getLocalPort();
|
||||
return (proxyPort != -1) ? proxyPort : conns[0].getLocalPort();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -489,7 +491,16 @@ public class JettySolrRunner {
|
|||
if (lastPort == -1) {
|
||||
throw new IllegalStateException("You cannot get the port until this instance has started");
|
||||
}
|
||||
return lastPort;
|
||||
return (proxyPort != -1) ? proxyPort : lastPort;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the port of a local socket proxy that sits infront of this server; if set
|
||||
* then all client traffic will flow through the proxy, giving us the ability to
|
||||
* simulate network partitions very easily.
|
||||
*/
|
||||
public void setProxyPort(int proxyPort) {
|
||||
this.proxyPort = proxyPort;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -3,8 +3,6 @@ package org.apache.solr.cloud;
|
|||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.Slice;
|
||||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.common.cloud.ZkCmdExecutor;
|
||||
|
@ -26,7 +24,8 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/*
|
||||
|
@ -144,9 +143,8 @@ class ShardLeaderElectionContextBase extends ElectionContext {
|
|||
ZkStateReader.CORE_NAME_PROP,
|
||||
leaderProps.getProperties().get(ZkStateReader.CORE_NAME_PROP),
|
||||
ZkStateReader.STATE_PROP, ZkStateReader.ACTIVE);
|
||||
Overseer.getInQueue(zkClient).offer(ZkStateReader.toJSON(m));
|
||||
}
|
||||
|
||||
Overseer.getInQueue(zkClient).offer(ZkStateReader.toJSON(m));
|
||||
}
|
||||
}
|
||||
|
||||
// add core container and stop passing core around...
|
||||
|
@ -287,9 +285,11 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
core.getCoreDescriptor().getCloudDescriptor().setLeader(true);
|
||||
}
|
||||
|
||||
boolean isLeader = true;
|
||||
try {
|
||||
super.runLeaderProcess(weAreReplacement, 0);
|
||||
} catch (Exception e) {
|
||||
isLeader = false;
|
||||
SolrException.log(log, "There was a problem trying to register as the leader", e);
|
||||
|
||||
try (SolrCore core = cc.getCore(coreName)) {
|
||||
|
@ -305,7 +305,72 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
rejoinLeaderElection(leaderSeqPath, core);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
if (isLeader) {
|
||||
// check for any replicas in my shard that were set to down by the previous leader
|
||||
try {
|
||||
startLeaderInitiatedRecoveryOnReplicas(coreName);
|
||||
} catch (Exception exc) {
|
||||
// don't want leader election to fail because of
|
||||
// an error trying to tell others to recover
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void startLeaderInitiatedRecoveryOnReplicas(String coreName) throws Exception {
|
||||
try (SolrCore core = cc.getCore(coreName)) {
|
||||
CloudDescriptor cloudDesc = core.getCoreDescriptor().getCloudDescriptor();
|
||||
String coll = cloudDesc.getCollectionName();
|
||||
String shardId = cloudDesc.getShardId();
|
||||
String znodePath = zkController.getLeaderInitiatedRecoveryZnodePath(coll, shardId);
|
||||
List<String> replicas = null;
|
||||
try {
|
||||
replicas = zkClient.getChildren(znodePath, null, false);
|
||||
} catch (NoNodeException nne) {
|
||||
// this can be ignored
|
||||
}
|
||||
|
||||
if (replicas != null && replicas.size() > 0) {
|
||||
for (String replicaCore : replicas) {
|
||||
|
||||
if (coreName.equals(replicaCore))
|
||||
continue; // added safe-guard so we don't mark this core as down
|
||||
|
||||
String lirState = zkController.getLeaderInitiatedRecoveryState(coll, shardId, replicaCore);
|
||||
if (ZkStateReader.DOWN.equals(lirState) || ZkStateReader.RECOVERY_FAILED.equals(lirState)) {
|
||||
log.info("After "+coreName+" was elected leader, found "+
|
||||
replicaCore+" as "+lirState+" and needing recovery.");
|
||||
|
||||
List<ZkCoreNodeProps> replicaProps =
|
||||
zkController.getZkStateReader().getReplicaProps(
|
||||
collection, shardId, coreName, replicaCore, null, null);
|
||||
|
||||
if (replicaProps != null && replicaProps.size() > 0) {
|
||||
ZkCoreNodeProps coreNodeProps = null;
|
||||
for (ZkCoreNodeProps p : replicaProps) {
|
||||
if (p.getCoreName().equals(replicaCore)) {
|
||||
coreNodeProps = p;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
LeaderInitiatedRecoveryThread lirThread =
|
||||
new LeaderInitiatedRecoveryThread(zkController,
|
||||
cc,
|
||||
collection,
|
||||
shardId,
|
||||
coreNodeProps,
|
||||
120);
|
||||
zkController.ensureReplicaInLeaderInitiatedRecovery(
|
||||
collection, shardId, replicaCore, coreNodeProps, false);
|
||||
|
||||
ExecutorService executor = cc.getUpdateShardHandler().getUpdateExecutor();
|
||||
executor.execute(lirThread);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} // core gets closed automagically
|
||||
}
|
||||
|
||||
private void waitForReplicasToComeUp(boolean weAreReplacement, int timeoutms) throws InterruptedException {
|
||||
|
@ -373,7 +438,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
}
|
||||
|
||||
private boolean shouldIBeLeader(ZkNodeProps leaderProps, SolrCore core, boolean weAreReplacement) {
|
||||
log.info("Checking if I should try and be the leader.");
|
||||
log.info("Checking if I ("+core.getName()+") should try and be the leader.");
|
||||
|
||||
if (isClosed) {
|
||||
log.info("Bailing on leader process because we have been closed");
|
||||
|
@ -386,8 +451,18 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
return true;
|
||||
}
|
||||
|
||||
if (core.getCoreDescriptor().getCloudDescriptor().getLastPublished()
|
||||
.equals(ZkStateReader.ACTIVE)) {
|
||||
if (core.getCoreDescriptor().getCloudDescriptor().getLastPublished().equals(ZkStateReader.ACTIVE)) {
|
||||
|
||||
// maybe active but if the previous leader marked us as down and
|
||||
// we haven't recovered, then can't be leader
|
||||
String lirState = zkController.getLeaderInitiatedRecoveryState(collection, shardId, core.getName());
|
||||
if (ZkStateReader.DOWN.equals(lirState) || ZkStateReader.RECOVERING.equals(lirState)) {
|
||||
log.warn("Although my last published state is Active, the previous leader marked me "+core.getName()
|
||||
+ " as " + lirState
|
||||
+ " 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;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,234 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
import java.net.ConnectException;
|
||||
import java.net.SocketException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.http.NoHttpResponseException;
|
||||
import org.apache.http.conn.ConnectTimeoutException;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
|
||||
import org.apache.solr.core.CoreContainer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Background daemon thread that tries to send the REQUESTRECOVERY to a downed
|
||||
* replica; used by a shard leader to nag a replica into recovering after the
|
||||
* leader experiences an error trying to send an update request to the replica.
|
||||
*/
|
||||
public class LeaderInitiatedRecoveryThread extends Thread {
|
||||
|
||||
public final static Logger log = LoggerFactory.getLogger(LeaderInitiatedRecoveryThread.class);
|
||||
|
||||
protected ZkController zkController;
|
||||
protected CoreContainer coreContainer;
|
||||
protected String collection;
|
||||
protected String shardId;
|
||||
protected ZkCoreNodeProps nodeProps;
|
||||
protected int maxTries;
|
||||
|
||||
public LeaderInitiatedRecoveryThread(ZkController zkController,
|
||||
CoreContainer cc,
|
||||
String collection,
|
||||
String shardId,
|
||||
ZkCoreNodeProps nodeProps,
|
||||
int maxTries)
|
||||
{
|
||||
super("LeaderInitiatedRecoveryThread-"+nodeProps.getCoreName());
|
||||
this.zkController = zkController;
|
||||
this.coreContainer = cc;
|
||||
this.collection = collection;
|
||||
this.shardId = shardId;
|
||||
this.nodeProps = nodeProps;
|
||||
this.maxTries = maxTries;
|
||||
|
||||
setDaemon(true);
|
||||
}
|
||||
|
||||
public void run() {
|
||||
long startMs = System.currentTimeMillis();
|
||||
try {
|
||||
sendRecoveryCommandWithRetry();
|
||||
} catch (Exception exc) {
|
||||
log.error(getName()+" failed due to: "+exc, exc);
|
||||
if (exc instanceof SolrException) {
|
||||
throw (SolrException)exc;
|
||||
} else {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, exc);
|
||||
}
|
||||
}
|
||||
long diffMs = (System.currentTimeMillis() - startMs);
|
||||
log.info(getName()+" completed successfully after running for "+Math.round(diffMs/1000L)+" secs");
|
||||
}
|
||||
|
||||
protected void sendRecoveryCommandWithRetry() throws Exception {
|
||||
int tries = 0;
|
||||
long waitBetweenTriesMs = 5000L;
|
||||
boolean continueTrying = true;
|
||||
|
||||
String recoveryUrl = nodeProps.getBaseUrl();
|
||||
String replicaNodeName = nodeProps.getNodeName();
|
||||
String coreNeedingRecovery = nodeProps.getCoreName();
|
||||
String replicaUrl = nodeProps.getCoreUrl();
|
||||
|
||||
log.info(getName()+" started running to send REQUESTRECOVERY command to "+replicaUrl+
|
||||
"; will try for a max of "+(maxTries * (waitBetweenTriesMs/1000))+" secs");
|
||||
|
||||
RequestRecovery recoverRequestCmd = new RequestRecovery();
|
||||
recoverRequestCmd.setAction(CoreAdminAction.REQUESTRECOVERY);
|
||||
recoverRequestCmd.setCoreName(coreNeedingRecovery);
|
||||
|
||||
while (continueTrying && ++tries < maxTries) {
|
||||
if (tries > 1) {
|
||||
log.warn("Asking core "+coreNeedingRecovery+" on " + recoveryUrl +
|
||||
" to recover; unsuccessful after "+tries+" of "+maxTries+" attempts so far ...");
|
||||
} else {
|
||||
log.info("Asking core "+coreNeedingRecovery+" on " + recoveryUrl + " to recover");
|
||||
}
|
||||
|
||||
HttpSolrServer server = new HttpSolrServer(recoveryUrl);
|
||||
try {
|
||||
server.setSoTimeout(60000);
|
||||
server.setConnectionTimeout(15000);
|
||||
try {
|
||||
server.request(recoverRequestCmd);
|
||||
|
||||
log.info("Successfully sent "+CoreAdminAction.REQUESTRECOVERY+
|
||||
" command to core "+coreNeedingRecovery+" on "+recoveryUrl);
|
||||
|
||||
continueTrying = false; // succeeded, so stop looping
|
||||
} catch (Throwable t) {
|
||||
Throwable rootCause = SolrException.getRootCause(t);
|
||||
boolean wasCommError =
|
||||
(rootCause instanceof ConnectException ||
|
||||
rootCause instanceof ConnectTimeoutException ||
|
||||
rootCause instanceof NoHttpResponseException ||
|
||||
rootCause instanceof SocketException);
|
||||
|
||||
SolrException.log(log, recoveryUrl + ": Could not tell a replica to recover", t);
|
||||
|
||||
if (!wasCommError) {
|
||||
continueTrying = false;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
server.shutdown();
|
||||
}
|
||||
|
||||
// wait a few seconds
|
||||
if (continueTrying) {
|
||||
try {
|
||||
Thread.sleep(waitBetweenTriesMs);
|
||||
} catch (InterruptedException ignoreMe) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
|
||||
if (coreContainer.isShutDown()) {
|
||||
log.warn("Stop trying to send recovery command to downed replica "+coreNeedingRecovery+
|
||||
" on "+replicaNodeName+" because my core container is shutdown.");
|
||||
continueTrying = false;
|
||||
break;
|
||||
}
|
||||
|
||||
// see if the replica's node is still live, if not, no need to keep doing this loop
|
||||
ZkStateReader zkStateReader = zkController.getZkStateReader();
|
||||
try {
|
||||
zkStateReader.updateClusterState(true);
|
||||
} catch (Exception exc) {
|
||||
log.warn("Error when updating cluster state: "+exc);
|
||||
}
|
||||
|
||||
if (!zkStateReader.getClusterState().liveNodesContain(replicaNodeName)) {
|
||||
log.warn("Node "+replicaNodeName+" hosting core "+coreNeedingRecovery+
|
||||
" is no longer live. No need to keep trying to tell it to recover!");
|
||||
continueTrying = false;
|
||||
break;
|
||||
}
|
||||
|
||||
// additional safeguard against the replica trying to be in the active state
|
||||
// before acknowledging the leader initiated recovery command
|
||||
if (continueTrying && collection != null && shardId != null) {
|
||||
try {
|
||||
// call out to ZooKeeper to get the leader-initiated recovery state
|
||||
String lirState =
|
||||
zkController.getLeaderInitiatedRecoveryState(collection, shardId, coreNeedingRecovery);
|
||||
|
||||
if (lirState == null) {
|
||||
log.warn("Stop trying to send recovery command to downed replica "+coreNeedingRecovery+
|
||||
" on "+replicaNodeName+" because the znode no longer exists.");
|
||||
continueTrying = false;
|
||||
break;
|
||||
}
|
||||
|
||||
if (ZkStateReader.RECOVERING.equals(lirState)) {
|
||||
// replica has ack'd leader initiated recovery and entered the recovering state
|
||||
// so we don't need to keep looping to send the command
|
||||
continueTrying = false;
|
||||
log.info("Replica "+coreNeedingRecovery+
|
||||
" on node "+replicaNodeName+" ack'd the leader initiated recovery state, "
|
||||
+ "no need to keep trying to send recovery command");
|
||||
} else {
|
||||
String leaderCoreNodeName = zkStateReader.getLeaderRetry(collection, shardId, 5000).getName();
|
||||
List<ZkCoreNodeProps> replicaProps =
|
||||
zkStateReader.getReplicaProps(collection, shardId, leaderCoreNodeName, coreNeedingRecovery, null, null);
|
||||
if (replicaProps != null && replicaProps.size() > 0) {
|
||||
String replicaState = replicaProps.get(0).getState();
|
||||
if (ZkStateReader.ACTIVE.equals(replicaState)) {
|
||||
// replica published its state as "active",
|
||||
// which is bad if lirState is still "down"
|
||||
if (ZkStateReader.DOWN.equals(lirState)) {
|
||||
// OK, so the replica thinks it is active, but it never ack'd the leader initiated recovery
|
||||
// so its state cannot be trusted and it needs to be told to recover again ... and we keep looping here
|
||||
log.warn("Replica "+coreNeedingRecovery+" set to active but the leader thinks it should be in recovery;"
|
||||
+ " forcing it back to down state to re-run the leader-initiated recovery process; props: "+replicaProps.get(0));
|
||||
zkController.ensureReplicaInLeaderInitiatedRecovery(collection,
|
||||
shardId, replicaUrl, nodeProps, true); // force republish state to "down"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Exception ignoreMe) {
|
||||
log.warn("Failed to determine state of "+coreNeedingRecovery+" due to: "+ignoreMe);
|
||||
// eventually this loop will exhaust max tries and stop so we can just log this for now
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// replica is no longer in recovery on this node (may be handled on another node)
|
||||
zkController.removeReplicaFromLeaderInitiatedRecoveryHandling(replicaUrl);
|
||||
|
||||
if (continueTrying) {
|
||||
// ugh! this means the loop timed out before the recovery command could be delivered
|
||||
// how exotic do we want to get here?
|
||||
log.error("Timed out after waiting for "+(tries * (waitBetweenTriesMs/1000))+
|
||||
" secs to send the recovery request to: "+replicaUrl+"; not much more we can do here?");
|
||||
|
||||
// TODO: need to raise a JMX event to allow monitoring tools to take over from here
|
||||
|
||||
}
|
||||
}
|
||||
}
|
|
@ -351,6 +351,7 @@ public class RecoveryStrategy extends Thread implements ClosableThread {
|
|||
return;
|
||||
}
|
||||
|
||||
log.info("Publishing state of core "+core.getName()+" as recovering, leader is "+leaderUrl+" and I am "+ourUrl);
|
||||
zkController.publish(core.getCoreDescriptor(), ZkStateReader.RECOVERING);
|
||||
|
||||
|
||||
|
@ -608,6 +609,9 @@ public class RecoveryStrategy extends Thread implements ClosableThread {
|
|||
}
|
||||
HttpUriRequestResponse mrr = server.httpUriRequest(prepCmd);
|
||||
prevSendPreRecoveryHttpUriRequest = mrr.httpUriRequest;
|
||||
|
||||
log.info("Sending prep recovery command to {}; {}", leaderBaseUrl, prepCmd.toString());
|
||||
|
||||
mrr.future.get();
|
||||
} finally {
|
||||
server.shutdown();
|
||||
|
|
|
@ -17,6 +17,28 @@ package org.apache.solr.cloud;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.NetworkInterface;
|
||||
import java.net.URLEncoder;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Enumeration;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
|
@ -46,34 +68,13 @@ import org.apache.solr.handler.component.ShardHandler;
|
|||
import org.apache.solr.update.UpdateLog;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.KeeperException.ConnectionLossException;
|
||||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
import org.apache.zookeeper.KeeperException.SessionExpiredException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.NetworkInterface;
|
||||
import java.net.URLEncoder;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Enumeration;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
/**
|
||||
* Handle ZooKeeper interactions.
|
||||
*
|
||||
|
@ -174,6 +175,9 @@ public final class ZkController {
|
|||
|
||||
private volatile boolean isClosed;
|
||||
|
||||
// keeps track of replicas that have been asked to recover by leaders running on this node
|
||||
private Map<String,String> replicasInLeaderInitiatedRecovery = new HashMap<String,String>();
|
||||
|
||||
public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
|
||||
String localHostContext, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames, final CurrentCoreDescriptorProvider registerOnReconnect)
|
||||
throws InterruptedException, TimeoutException, IOException
|
||||
|
@ -807,6 +811,7 @@ public final class ZkController {
|
|||
Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler()
|
||||
.getUpdateLog().recoverFromLog();
|
||||
if (recoveryFuture != null) {
|
||||
log.info("Replaying tlog for "+ourUrl+" during startup... NOTE: This can take a while.");
|
||||
recoveryFuture.get(); // NOTE: this could potentially block for
|
||||
// minutes or more!
|
||||
// TODO: public as recovering in the mean time?
|
||||
|
@ -981,6 +986,14 @@ public final class ZkController {
|
|||
core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor());
|
||||
return true;
|
||||
}
|
||||
|
||||
// see if the leader told us to recover
|
||||
String lirState = getLeaderInitiatedRecoveryState(collection, shardId, coreName);
|
||||
if (ZkStateReader.DOWN.equals(lirState)) {
|
||||
log.info("Leader marked core "+core.getName()+" down; starting recovery process");
|
||||
core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor());
|
||||
return true;
|
||||
}
|
||||
} else {
|
||||
log.info("I am the leader, no recovery necessary");
|
||||
}
|
||||
|
@ -1023,7 +1036,30 @@ public final class ZkController {
|
|||
|
||||
assert collection != null && collection.length() > 0;
|
||||
|
||||
String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
|
||||
String shardId = cd.getCloudDescriptor().getShardId();
|
||||
String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
|
||||
// If the leader initiated recovery, then verify that this replica has performed
|
||||
// recovery as requested before becoming active; don't even look at lirState if going down
|
||||
if (!ZkStateReader.DOWN.equals(state)) {
|
||||
String lirState = getLeaderInitiatedRecoveryState(collection, shardId, cd.getName());
|
||||
if (lirState != null) {
|
||||
if ("active".equals(state)) {
|
||||
// trying to become active, so leader-initiated state must be recovering
|
||||
if (ZkStateReader.RECOVERING.equals(lirState)) {
|
||||
updateLeaderInitiatedRecoveryState(collection, shardId, cd.getName(), ZkStateReader.ACTIVE);
|
||||
} else if (ZkStateReader.DOWN.equals(lirState)) {
|
||||
throw new SolrException(ErrorCode.INVALID_STATE,
|
||||
"Cannot publish state of core '"+cd.getName()+"' as active without recovering first!");
|
||||
}
|
||||
} else if (ZkStateReader.RECOVERING.equals(state)) {
|
||||
// if it is currently DOWN, then trying to enter into recovering state is good
|
||||
if (ZkStateReader.DOWN.equals(lirState)) {
|
||||
updateLeaderInitiatedRecoveryState(collection, shardId, cd.getName(), ZkStateReader.RECOVERING);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//assert cd.getCloudDescriptor().getShardId() != null;
|
||||
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state",
|
||||
ZkStateReader.STATE_PROP, state,
|
||||
|
@ -1040,7 +1076,7 @@ public final class ZkController {
|
|||
}
|
||||
overseerJobQueue.offer(ZkStateReader.toJSON(m));
|
||||
}
|
||||
|
||||
|
||||
private boolean needsToBeAssignedShardId(final CoreDescriptor desc,
|
||||
final ClusterState state, final String coreNodeName) {
|
||||
|
||||
|
@ -1450,49 +1486,69 @@ public final class ZkController {
|
|||
String leaderBaseUrl = leaderProps.getBaseUrl();
|
||||
String leaderCoreName = leaderProps.getCoreName();
|
||||
|
||||
String ourUrl = ZkCoreNodeProps.getCoreUrl(getBaseUrl(),
|
||||
descriptor.getName());
|
||||
String myCoreName = descriptor.getName();
|
||||
String ourUrl = ZkCoreNodeProps.getCoreUrl(getBaseUrl(), myCoreName);
|
||||
|
||||
boolean isLeader = leaderProps.getCoreUrl().equals(ourUrl);
|
||||
if (!isLeader && !SKIP_AUTO_RECOVERY) {
|
||||
HttpSolrServer server = null;
|
||||
server = new HttpSolrServer(leaderBaseUrl);
|
||||
|
||||
// detect if this core is in leader-initiated recovery and if so,
|
||||
// then we don't need the leader to wait on seeing the down state
|
||||
String lirState = null;
|
||||
try {
|
||||
server.setConnectionTimeout(15000);
|
||||
server.setSoTimeout(120000);
|
||||
WaitForState prepCmd = new WaitForState();
|
||||
prepCmd.setCoreName(leaderCoreName);
|
||||
prepCmd.setNodeName(getNodeName());
|
||||
prepCmd.setCoreNodeName(coreZkNodeName);
|
||||
prepCmd.setState(ZkStateReader.DOWN);
|
||||
lirState = getLeaderInitiatedRecoveryState(collection, shard, myCoreName);
|
||||
} catch (Exception exc) {
|
||||
log.error("Failed to determine if replica "+myCoreName+
|
||||
" is in leader-initiated recovery due to: "+exc, exc);
|
||||
}
|
||||
|
||||
if (lirState != null) {
|
||||
log.info("Replica "+myCoreName+
|
||||
" is already in leader-initiated recovery, so not waiting for leader to see down state.");
|
||||
} else {
|
||||
|
||||
// let's retry a couple times - perhaps the leader just went down,
|
||||
// or perhaps he is just not quite ready for us yet
|
||||
retries = 6;
|
||||
for (int i = 0; i < retries; i++) {
|
||||
if (isClosed) {
|
||||
throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
|
||||
"We have been closed");
|
||||
}
|
||||
try {
|
||||
server.request(prepCmd);
|
||||
break;
|
||||
} catch (Exception e) {
|
||||
SolrException.log(log,
|
||||
"There was a problem making a request to the leader", e);
|
||||
log.info("Replica "+myCoreName+
|
||||
" NOT in leader-initiated recovery, need to wait for leader to see down state.");
|
||||
|
||||
HttpSolrServer server = null;
|
||||
server = new HttpSolrServer(leaderBaseUrl);
|
||||
try {
|
||||
server.setConnectionTimeout(15000);
|
||||
server.setSoTimeout(120000);
|
||||
WaitForState prepCmd = new WaitForState();
|
||||
prepCmd.setCoreName(leaderCoreName);
|
||||
prepCmd.setNodeName(getNodeName());
|
||||
prepCmd.setCoreNodeName(coreZkNodeName);
|
||||
prepCmd.setState(ZkStateReader.DOWN);
|
||||
|
||||
// let's retry a couple times - perhaps the leader just went down,
|
||||
// or perhaps he is just not quite ready for us yet
|
||||
retries = 6;
|
||||
for (int i = 0; i < retries; i++) {
|
||||
if (isClosed) {
|
||||
throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
|
||||
"We have been closed");
|
||||
}
|
||||
try {
|
||||
Thread.sleep(2000);
|
||||
} catch (InterruptedException e1) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
if (i == retries - 1) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"There was a problem making a request to the leader");
|
||||
server.request(prepCmd);
|
||||
break;
|
||||
} catch (Exception e) {
|
||||
SolrException.log(log,
|
||||
"There was a problem making a request to the leader", e);
|
||||
try {
|
||||
Thread.sleep(2000);
|
||||
} catch (InterruptedException e1) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
if (i == retries - 1) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"There was a problem making a request to the leader");
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
server.shutdown();
|
||||
}
|
||||
} finally {
|
||||
server.shutdown();
|
||||
}
|
||||
}
|
||||
return leaderProps;
|
||||
|
@ -1669,5 +1725,161 @@ public final class ZkController {
|
|||
CoreContainer getCoreContainer(){
|
||||
return cc;
|
||||
}
|
||||
|
||||
/**
|
||||
* When a leader receives a communication error when trying to send a request to a replica,
|
||||
* it calls this method to ensure the replica enters recovery when connectivity is restored.
|
||||
*
|
||||
* returns true if the node hosting the replica is still considered "live" by ZooKeeper;
|
||||
* false means the node is not live either, so no point in trying to send recovery commands
|
||||
* to it.
|
||||
*/
|
||||
public boolean ensureReplicaInLeaderInitiatedRecovery(final String collection,
|
||||
final String shardId, final String replicaUrl, final ZkCoreNodeProps replicaCoreProps, boolean forcePublishState)
|
||||
throws KeeperException, InterruptedException
|
||||
{
|
||||
|
||||
// First, determine if this replica is already in recovery handling
|
||||
// which is needed because there can be many concurrent errors flooding in
|
||||
// about the same replica having trouble and we only need to send the "needs"
|
||||
// recovery signal once
|
||||
boolean nodeIsLive = true;
|
||||
synchronized (replicasInLeaderInitiatedRecovery) {
|
||||
if (replicasInLeaderInitiatedRecovery.containsKey(replicaUrl)) {
|
||||
if (!forcePublishState) {
|
||||
return false; // already in this recovery process
|
||||
}
|
||||
}
|
||||
|
||||
// if the replica's state is not DOWN right now, make it so ...
|
||||
String replicaNodeName = replicaCoreProps.getNodeName();
|
||||
String replicaCoreName = replicaCoreProps.getCoreName();
|
||||
assert replicaCoreName != null : "No core name for replica "+replicaNodeName;
|
||||
|
||||
// we only really need to try to send the recovery command if the node itself is "live"
|
||||
if (getZkStateReader().getClusterState().liveNodesContain(replicaNodeName)) {
|
||||
replicasInLeaderInitiatedRecovery.put(replicaUrl,
|
||||
getLeaderInitiatedRecoveryZnodePath(collection, shardId, replicaCoreName));
|
||||
// create a znode that requires the replica needs to "ack" to verify it knows it was out-of-sync
|
||||
updateLeaderInitiatedRecoveryState(collection, shardId, replicaCoreName, ZkStateReader.DOWN);
|
||||
} else {
|
||||
nodeIsLive = false; // we really don't need to send the recovery request if the node is NOT live
|
||||
}
|
||||
}
|
||||
|
||||
String replicaCoreName = replicaCoreProps.getCoreName();
|
||||
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state",
|
||||
ZkStateReader.STATE_PROP, ZkStateReader.DOWN,
|
||||
ZkStateReader.BASE_URL_PROP, replicaCoreProps.getBaseUrl(),
|
||||
ZkStateReader.CORE_NAME_PROP, replicaCoreProps.getCoreName(),
|
||||
ZkStateReader.NODE_NAME_PROP, replicaCoreProps.getNodeName(),
|
||||
ZkStateReader.SHARD_ID_PROP, shardId,
|
||||
ZkStateReader.COLLECTION_PROP, collection);
|
||||
log.warn("Leader is publishing core={} state={} on behalf of un-reachable replica {}; forcePublishState? "+forcePublishState,
|
||||
replicaCoreName, ZkStateReader.DOWN, replicaUrl);
|
||||
overseerJobQueue.offer(ZkStateReader.toJSON(m));
|
||||
|
||||
return nodeIsLive;
|
||||
}
|
||||
|
||||
public boolean isReplicaInRecoveryHandling(String replicaUrl) {
|
||||
boolean exists = false;
|
||||
synchronized (replicasInLeaderInitiatedRecovery) {
|
||||
exists = replicasInLeaderInitiatedRecovery.containsKey(replicaUrl);
|
||||
}
|
||||
return exists;
|
||||
}
|
||||
|
||||
public void removeReplicaFromLeaderInitiatedRecoveryHandling(String replicaUrl) {
|
||||
synchronized(replicasInLeaderInitiatedRecovery) {
|
||||
replicasInLeaderInitiatedRecovery.remove(replicaUrl);
|
||||
}
|
||||
}
|
||||
|
||||
public String getLeaderInitiatedRecoveryState(String collection, String shardId, String coreName) {
|
||||
|
||||
if (collection == null || shardId == null || coreName == null)
|
||||
return null; // if we don't have complete data about a core in cloud mode, return null
|
||||
|
||||
String znodePath = getLeaderInitiatedRecoveryZnodePath(collection, shardId, coreName);
|
||||
String state = null;
|
||||
try {
|
||||
byte[] data = zkClient.getData(znodePath, null, new Stat(), false);
|
||||
if (data != null && data.length > 0)
|
||||
state = new String(data, "UTF-8");
|
||||
} catch (NoNodeException ignoreMe) {
|
||||
// safe to ignore as this znode will only exist if the leader initiated recovery
|
||||
} catch (ConnectionLossException cle) {
|
||||
// sort of safe to ignore ??? Usually these are seen when the core is going down
|
||||
// or there are bigger issues to deal with than reading this znode
|
||||
log.warn("Unable to read "+znodePath+" due to: "+cle);
|
||||
} catch (SessionExpiredException see) {
|
||||
// sort of safe to ignore ??? Usually these are seen when the core is going down
|
||||
// or there are bigger issues to deal with than reading this znode
|
||||
log.warn("Unable to read "+znodePath+" due to: "+see);
|
||||
} catch (UnsupportedEncodingException e) {
|
||||
throw new Error("JVM Does not seem to support UTF-8", e);
|
||||
} catch (Exception exc) {
|
||||
log.error("Failed to read data from znode "+znodePath+" due to: "+exc);
|
||||
if (exc instanceof SolrException) {
|
||||
throw (SolrException)exc;
|
||||
} else {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"Failed to read data from znodePath: "+znodePath, exc);
|
||||
}
|
||||
}
|
||||
return state;
|
||||
}
|
||||
|
||||
private void updateLeaderInitiatedRecoveryState(String collection, String shardId, String coreName, String state) {
|
||||
if (collection == null || shardId == null || coreName == null) {
|
||||
log.warn("Cannot set leader-initiated recovery state znode to "+state+" using: collection="+collection+
|
||||
"; shardId="+shardId+"; coreName="+coreName);
|
||||
return; // if we don't have complete data about a core in cloud mode, do nothing
|
||||
}
|
||||
|
||||
String znodePath = getLeaderInitiatedRecoveryZnodePath(collection, shardId, coreName);
|
||||
|
||||
if (ZkStateReader.ACTIVE.equals(state)) {
|
||||
// since we're marking it active, we don't need this znode anymore, so delete instead of update
|
||||
try {
|
||||
zkClient.delete(znodePath, -1, false);
|
||||
} catch (Exception justLogIt) {
|
||||
log.warn("Failed to delete znode "+znodePath+" due to: "+justLogIt);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
byte[] znodeData = null;
|
||||
try {
|
||||
znodeData = state.getBytes("UTF-8");
|
||||
} catch (UnsupportedEncodingException e) {
|
||||
throw new Error("JVM Does not seem to support UTF-8", e);
|
||||
}
|
||||
|
||||
boolean retryOnConnLoss = true; // be a little more robust when trying to write data
|
||||
try {
|
||||
if (zkClient.exists(znodePath, retryOnConnLoss)) {
|
||||
zkClient.setData(znodePath, znodeData, retryOnConnLoss);
|
||||
} else {
|
||||
zkClient.makePath(znodePath, znodeData, retryOnConnLoss);
|
||||
}
|
||||
log.info("Wrote "+state+" to "+znodePath);
|
||||
} catch (Exception exc) {
|
||||
if (exc instanceof SolrException) {
|
||||
throw (SolrException)exc;
|
||||
} else {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,
|
||||
"Failed to update data to "+state+" for znode: "+znodePath, exc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public String getLeaderInitiatedRecoveryZnodePath(String collection, String shardId) {
|
||||
return "/collections/"+collection+"/leader_initiated_recovery/"+shardId;
|
||||
}
|
||||
|
||||
public String getLeaderInitiatedRecoveryZnodePath(String collection, String shardId, String coreName) {
|
||||
return getLeaderInitiatedRecoveryZnodePath(collection, shardId)+"/"+coreName;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -845,7 +845,7 @@ public class CoreAdminHandler extends RequestHandlerBase {
|
|||
protected void handleRequestRecoveryAction(SolrQueryRequest req,
|
||||
SolrQueryResponse rsp) throws IOException {
|
||||
final SolrParams params = req.getParams();
|
||||
log.info("It has been requested that we recover");
|
||||
log.info("It has been requested that we recover: core="+params.get(CoreAdminParams.CORE));
|
||||
Thread thread = new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
|
@ -958,7 +958,8 @@ public class CoreAdminHandler extends RequestHandlerBase {
|
|||
Boolean onlyIfLeaderActive = params.getBool("onlyIfLeaderActive");
|
||||
|
||||
log.info("Going to wait for coreNodeName: " + coreNodeName + ", state: " + waitForState
|
||||
+ ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader);
|
||||
+ ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader
|
||||
+ ", onlyIfLeaderActive: "+onlyIfLeaderActive);
|
||||
|
||||
int maxTries = 0;
|
||||
String state = null;
|
||||
|
@ -1014,10 +1015,28 @@ public class CoreAdminHandler extends RequestHandlerBase {
|
|||
live = clusterState.liveNodesContain(nodeName);
|
||||
|
||||
String localState = cloudDescriptor.getLastPublished();
|
||||
|
||||
// TODO: This is funky but I've seen this in testing where the replica asks the
|
||||
// leader to be in recovery? Need to track down how that happens ... in the meantime,
|
||||
// this is a safeguard
|
||||
boolean leaderDoesNotNeedRecovery = (onlyIfLeader != null &&
|
||||
onlyIfLeader &&
|
||||
core.getName().equals(nodeProps.getStr("core")) &&
|
||||
ZkStateReader.RECOVERING.equals(waitForState) &&
|
||||
ZkStateReader.ACTIVE.equals(localState) &&
|
||||
ZkStateReader.ACTIVE.equals(state));
|
||||
|
||||
if (leaderDoesNotNeedRecovery) {
|
||||
log.warn("Leader "+core.getName()+" ignoring request to be in the recovering state because it is live and active.");
|
||||
}
|
||||
|
||||
boolean onlyIfActiveCheckResult = onlyIfLeaderActive != null && onlyIfLeaderActive && (localState == null || !localState.equals(ZkStateReader.ACTIVE));
|
||||
log.info("In WaitForState("+waitForState+"): collection="+collection+", shard="+slice.getName()+
|
||||
", isLeader? "+core.getCoreDescriptor().getCloudDescriptor().isLeader()+
|
||||
", live="+live+", currentState="+state+", localState="+localState+", nodeName="+nodeName+
|
||||
", coreNodeName="+coreNodeName+", onlyIfActiveCheckResult="+onlyIfActiveCheckResult+", nodeProps: "+nodeProps);
|
||||
|
||||
if (!onlyIfActiveCheckResult && nodeProps != null && state.equals(waitForState)) {
|
||||
if (!onlyIfActiveCheckResult && nodeProps != null && (state.equals(waitForState) || leaderDoesNotNeedRecovery)) {
|
||||
if (checkLive == null) {
|
||||
break;
|
||||
} else if (checkLive && live) {
|
||||
|
@ -1040,7 +1059,7 @@ public class CoreAdminHandler extends RequestHandlerBase {
|
|||
collection = cloudDescriptor.getCollectionName();
|
||||
shardId = cloudDescriptor.getShardId();
|
||||
leaderInfo = coreContainer.getZkController().
|
||||
getZkStateReader().getLeaderUrl(collection, shardId, 0);
|
||||
getZkStateReader().getLeaderUrl(collection, shardId, 5000);
|
||||
} catch (Exception exc) {
|
||||
leaderInfo = "Not available due to: " + exc;
|
||||
}
|
||||
|
|
|
@ -176,7 +176,7 @@ public class SolrCmdDistributor {
|
|||
for (Node node : nodes) {
|
||||
UpdateRequest uReq = new UpdateRequest();
|
||||
uReq.setParams(params);
|
||||
uReq.add(cmd.solrDoc, cmd.commitWithin, cmd.overwrite);
|
||||
uReq.add(cmd.solrDoc, cmd.commitWithin, cmd.overwrite);
|
||||
submit(new Req(cmd.toString(), node, uReq, synchronous));
|
||||
}
|
||||
|
||||
|
@ -230,6 +230,7 @@ public class SolrCmdDistributor {
|
|||
+ req.node.getUrl() + " retry:"
|
||||
+ req.retries + " " + req.cmdString + " params:" + req.uReq.getParams());
|
||||
}
|
||||
|
||||
try {
|
||||
SolrServer solrServer = servers.getSolrServer(req);
|
||||
NamedList<Object> rsp = solrServer.request(req.uReq);
|
||||
|
@ -258,6 +259,13 @@ public class SolrCmdDistributor {
|
|||
this.synchronous = synchronous;
|
||||
this.cmdString = cmdString;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("SolrCmdDistributor$Req: cmd=").append(String.valueOf(cmdString));
|
||||
sb.append("; node=").append(String.valueOf(node));
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -272,6 +280,14 @@ public class SolrCmdDistributor {
|
|||
public Exception e;
|
||||
public int statusCode = -1;
|
||||
public Req req;
|
||||
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("SolrCmdDistributor$Error: statusCode=").append(statusCode);
|
||||
sb.append("; exception=").append(String.valueOf(e));
|
||||
sb.append("; req=").append(String.valueOf(req));
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
public static abstract class Node {
|
||||
|
@ -284,11 +300,27 @@ public class SolrCmdDistributor {
|
|||
|
||||
public static class StdNode extends Node {
|
||||
protected ZkCoreNodeProps nodeProps;
|
||||
protected String collection;
|
||||
protected String shardId;
|
||||
|
||||
public StdNode(ZkCoreNodeProps nodeProps) {
|
||||
this.nodeProps = nodeProps;
|
||||
this(nodeProps, null, null);
|
||||
}
|
||||
|
||||
public StdNode(ZkCoreNodeProps nodeProps, String collection, String shardId) {
|
||||
this.nodeProps = nodeProps;
|
||||
this.collection = collection;
|
||||
this.shardId = shardId;
|
||||
}
|
||||
|
||||
public String getCollection() {
|
||||
return collection;
|
||||
}
|
||||
|
||||
public String getShardId() {
|
||||
return shardId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getUrl() {
|
||||
return nodeProps.getCoreUrl();
|
||||
|
@ -359,11 +391,9 @@ public class SolrCmdDistributor {
|
|||
public static class RetryNode extends StdNode {
|
||||
|
||||
private ZkStateReader zkStateReader;
|
||||
private String collection;
|
||||
private String shardId;
|
||||
|
||||
public RetryNode(ZkCoreNodeProps nodeProps, ZkStateReader zkStateReader, String collection, String shardId) {
|
||||
super(nodeProps);
|
||||
super(nodeProps, collection, shardId);
|
||||
this.zkStateReader = zkStateReader;
|
||||
this.collection = collection;
|
||||
this.shardId = shardId;
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.solr.update.processor;
|
|||
import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.ConnectException;
|
||||
import java.net.SocketException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -33,12 +35,15 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.http.NoHttpResponseException;
|
||||
import org.apache.http.conn.ConnectTimeoutException;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery;
|
||||
import org.apache.solr.cloud.CloudDescriptor;
|
||||
import org.apache.solr.cloud.DistributedQueue;
|
||||
import org.apache.solr.cloud.LeaderInitiatedRecoveryThread;
|
||||
import org.apache.solr.cloud.Overseer;
|
||||
import org.apache.solr.cloud.ZkController;
|
||||
import org.apache.solr.common.SolrException;
|
||||
|
@ -64,7 +69,9 @@ import org.apache.solr.common.params.SolrParams;
|
|||
import org.apache.solr.common.params.UpdateParams;
|
||||
import org.apache.solr.common.util.Hash;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.core.CoreContainer;
|
||||
import org.apache.solr.core.CoreDescriptor;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.handler.component.RealTimeGetComponent;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.request.SolrRequestInfo;
|
||||
|
@ -160,7 +167,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
private List<Node> nodes;
|
||||
|
||||
private UpdateCommand updateCommand; // the current command this processor is working on.
|
||||
|
||||
|
||||
public DistributedUpdateProcessor(SolrQueryRequest req,
|
||||
SolrQueryResponse rsp, UpdateRequestProcessor next) {
|
||||
super(next);
|
||||
|
@ -213,7 +220,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
}
|
||||
String coreName = req.getCore().getName();
|
||||
|
||||
ClusterState cstate = zkController.getClusterState();
|
||||
ClusterState cstate = zkController.getClusterState();
|
||||
DocCollection coll = cstate.getCollection(collection);
|
||||
Slice slice = coll.getRouter().getTargetSlice(id, doc, req.getParams(), coll);
|
||||
|
||||
|
@ -299,10 +306,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
boolean skip = skipListSet.contains(props.getCoreUrl());
|
||||
log.info("check url:" + props.getCoreUrl() + " against:" + skipListSet + " result:" + skip);
|
||||
if (!skip) {
|
||||
nodes.add(new StdNode(props));
|
||||
nodes.add(new StdNode(props, collection, shardId));
|
||||
}
|
||||
} else {
|
||||
nodes.add(new StdNode(props));
|
||||
nodes.add(new StdNode(props, collection, shardId));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -375,7 +382,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
if (sliceLeader != null && zkController.getClusterState().liveNodesContain(sliceLeader.getNodeName())) {
|
||||
if (nodes == null) nodes = new ArrayList<>();
|
||||
ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(sliceLeader);
|
||||
nodes.add(new StdNode(nodeProps));
|
||||
nodes.add(new StdNode(nodeProps, coll.getName(), shardId));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -471,6 +478,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
|
||||
String from = req.getParams().get(DISTRIB_FROM);
|
||||
ClusterState clusterState = zkController.getClusterState();
|
||||
|
||||
CloudDescriptor cloudDescriptor = req.getCore().getCoreDescriptor().getCloudDescriptor();
|
||||
Slice mySlice = clusterState.getSlice(collection, cloudDescriptor.getShardId());
|
||||
boolean localIsLeader = cloudDescriptor.isLeader();
|
||||
|
@ -528,7 +536,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
if (replicaProps != null) {
|
||||
nodes = new ArrayList<>(replicaProps.size());
|
||||
for (ZkCoreNodeProps props : replicaProps) {
|
||||
nodes.add(new StdNode(props));
|
||||
nodes.add(new StdNode(props, collection, shardId));
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -553,7 +561,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
}
|
||||
|
||||
boolean dropCmd = false;
|
||||
if (!forwardToLeader) {
|
||||
if (!forwardToLeader) {
|
||||
dropCmd = versionAdd(cmd);
|
||||
}
|
||||
|
||||
|
@ -582,6 +590,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
params.set(DISTRIB_UPDATE_PARAM, DistribPhase.FROMLEADER.toString());
|
||||
params.set(DISTRIB_FROM, ZkCoreNodeProps.getCoreUrl(
|
||||
zkController.getBaseUrl(), req.getCore().getName()));
|
||||
|
||||
params.set(DISTRIB_FROM_COLLECTION, req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName());
|
||||
params.set(DISTRIB_FROM_SHARD, req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId());
|
||||
for (Node nodesByRoutingRule : nodesByRoutingRules) {
|
||||
|
@ -600,7 +609,6 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
DistribPhase.TOLEADER.toString()));
|
||||
params.set(DISTRIB_FROM, ZkCoreNodeProps.getCoreUrl(
|
||||
zkController.getBaseUrl(), req.getCore().getName()));
|
||||
|
||||
cmdDistrib.distribAdd(cmd, nodes, params);
|
||||
}
|
||||
|
||||
|
@ -655,46 +663,66 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
// legit
|
||||
|
||||
for (final SolrCmdDistributor.Error error : errors) {
|
||||
|
||||
if (error.req.node instanceof RetryNode) {
|
||||
// we don't try to force a leader to recover
|
||||
// when we cannot forward to it
|
||||
continue;
|
||||
}
|
||||
// TODO: we should force their state to recovering ??
|
||||
// TODO: do retries??
|
||||
// TODO: what if its is already recovering? Right now recoveries queue up -
|
||||
// should they?
|
||||
final String recoveryUrl = error.req.node.getBaseUrl();
|
||||
|
||||
DistribPhase phase =
|
||||
DistribPhase.parseParam(error.req.uReq.getParams().get(DISTRIB_UPDATE_PARAM));
|
||||
if (phase != DistribPhase.FROMLEADER)
|
||||
continue; // don't have non-leaders try to recovery other nodes
|
||||
|
||||
Thread thread = new Thread() {
|
||||
{
|
||||
setDaemon(true);
|
||||
final String replicaUrl = error.req.node.getUrl();
|
||||
|
||||
int maxTries = 1;
|
||||
boolean sendRecoveryCommand = true;
|
||||
String collection = null;
|
||||
String shardId = null;
|
||||
|
||||
if (error.req.node instanceof StdNode) {
|
||||
StdNode stdNode = (StdNode)error.req.node;
|
||||
collection = stdNode.getCollection();
|
||||
shardId = stdNode.getShardId();
|
||||
try {
|
||||
// if false, then the node is probably not "live" anymore
|
||||
sendRecoveryCommand =
|
||||
zkController.ensureReplicaInLeaderInitiatedRecovery(collection,
|
||||
shardId,
|
||||
replicaUrl,
|
||||
stdNode.getNodeProps(),
|
||||
false);
|
||||
|
||||
// we want to try more than once, ~10 minutes
|
||||
if (sendRecoveryCommand) {
|
||||
maxTries = 120;
|
||||
} // else the node is no longer "live" so no need to send any recovery command
|
||||
|
||||
} catch (Exception e) {
|
||||
log.error("Leader failed to set replica "+
|
||||
error.req.node.getUrl()+" state to DOWN due to: "+e, e);
|
||||
}
|
||||
@Override
|
||||
public void run() {
|
||||
log.info("try and ask " + recoveryUrl + " to recover");
|
||||
HttpSolrServer server = new HttpSolrServer(recoveryUrl);
|
||||
try {
|
||||
server.setSoTimeout(60000);
|
||||
server.setConnectionTimeout(15000);
|
||||
} // else not a StdNode, recovery command still gets sent once
|
||||
|
||||
RequestRecovery recoverRequestCmd = new RequestRecovery();
|
||||
recoverRequestCmd.setAction(CoreAdminAction.REQUESTRECOVERY);
|
||||
recoverRequestCmd.setCoreName(error.req.node.getCoreName());
|
||||
try {
|
||||
server.request(recoverRequestCmd);
|
||||
} catch (Throwable t) {
|
||||
SolrException.log(log, recoveryUrl
|
||||
+ ": Could not tell a replica to recover", t);
|
||||
}
|
||||
} finally {
|
||||
server.shutdown();
|
||||
}
|
||||
}
|
||||
};
|
||||
ExecutorService executor = req.getCore().getCoreDescriptor().getCoreContainer().getUpdateShardHandler().getUpdateExecutor();
|
||||
executor.execute(thread);
|
||||
if (!sendRecoveryCommand)
|
||||
continue; // the replica is already in recovery handling or is not live
|
||||
|
||||
Throwable rootCause = SolrException.getRootCause(error.e);
|
||||
log.error("Setting up to try to start recovery on replica "+replicaUrl+" after: "+rootCause);
|
||||
|
||||
// try to send the recovery command to the downed replica in a background thread
|
||||
CoreContainer coreContainer = req.getCore().getCoreDescriptor().getCoreContainer();
|
||||
LeaderInitiatedRecoveryThread lirThread =
|
||||
new LeaderInitiatedRecoveryThread(zkController,
|
||||
coreContainer,
|
||||
collection,
|
||||
shardId,
|
||||
error.req.node.getNodeProps(),
|
||||
maxTries);
|
||||
ExecutorService executor = coreContainer.getUpdateShardHandler().getUpdateExecutor();
|
||||
executor.execute(lirThread);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1151,7 +1179,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
// don't forward to ourself
|
||||
leaderForAnyShard = true;
|
||||
} else {
|
||||
leaders.add(new StdNode(coreLeaderProps));
|
||||
leaders.add(new StdNode(coreLeaderProps, collection, sliceName));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1254,7 +1282,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
if (replicaProps != null) {
|
||||
List<Node> myReplicas = new ArrayList<>();
|
||||
for (ZkCoreNodeProps replicaProp : replicaProps) {
|
||||
myReplicas.add(new StdNode(replicaProp));
|
||||
myReplicas.add(new StdNode(replicaProp, collection, myShardId));
|
||||
}
|
||||
cmdDistrib.distribDelete(cmd, myReplicas, params);
|
||||
someReplicas = true;
|
||||
|
@ -1521,7 +1549,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
|
|||
for (Entry<String,Replica> entry : shardMap.entrySet()) {
|
||||
ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
|
||||
if (clusterState.liveNodesContain(nodeProps.getNodeName())) {
|
||||
urls.add(new StdNode(nodeProps));
|
||||
urls.add(new StdNode(nodeProps, collection, replicas.getName()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,545 @@
|
|||
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.io.File;
|
||||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.JSONTestUtil;
|
||||
import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
|
||||
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
import org.apache.solr.client.solrj.request.QueryRequest;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.apache.solr.common.cloud.ClusterState;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.cloud.Slice;
|
||||
import org.apache.solr.common.cloud.ZkCoreNodeProps;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Simulates HTTP partitions between a leader and replica but the replica does
|
||||
* not lose its ZooKeeper connection.
|
||||
*/
|
||||
@Slow
|
||||
@SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
|
||||
public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
|
||||
|
||||
private static final transient Logger log =
|
||||
LoggerFactory.getLogger(HttpPartitionTest.class);
|
||||
|
||||
// To prevent the test assertions firing too fast before cluster state
|
||||
// recognizes (and propagates) partitions
|
||||
private static final long sleepMsBeforeHealPartition = 1000L;
|
||||
|
||||
private Map<URI,SocketProxy> proxies = new HashMap<URI,SocketProxy>();
|
||||
private AtomicInteger portCounter = new AtomicInteger(0);
|
||||
private int basePort = 49900;
|
||||
|
||||
public HttpPartitionTest() {
|
||||
super();
|
||||
sliceCount = 2;
|
||||
shardCount = 2;
|
||||
}
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
System.setProperty("numShards", Integer.toString(sliceCount));
|
||||
}
|
||||
|
||||
@Override
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
if (!proxies.isEmpty()) {
|
||||
for (SocketProxy proxy : proxies.values()) {
|
||||
proxy.close();
|
||||
}
|
||||
}
|
||||
|
||||
System.clearProperty("numShards");
|
||||
|
||||
try {
|
||||
super.tearDown();
|
||||
} catch (Exception exc) {}
|
||||
|
||||
resetExceptionIgnores();
|
||||
}
|
||||
|
||||
/**
|
||||
* Overrides the parent implementation so that we can configure a socket proxy
|
||||
* to sit infront of each Jetty server, which gives us the ability to simulate
|
||||
* network partitions without having to fuss with IPTables (which is not very
|
||||
* cross platform friendly).
|
||||
*/
|
||||
@Override
|
||||
public JettySolrRunner createJetty(File solrHome, String dataDir,
|
||||
String shardList, String solrConfigOverride, String schemaOverride)
|
||||
throws Exception {
|
||||
|
||||
int jettyPort = basePort + portCounter.incrementAndGet();
|
||||
|
||||
JettySolrRunner jetty = new JettySolrRunner(solrHome.getPath(), context,
|
||||
jettyPort, solrConfigOverride, schemaOverride, false,
|
||||
getExtraServlets(), sslConfig, getExtraRequestFilters());
|
||||
jetty.setShards(shardList);
|
||||
jetty.setDataDir(getDataDir(dataDir));
|
||||
|
||||
// setup to proxy Http requests to this server unless it is the control
|
||||
// server
|
||||
int proxyPort = basePort + portCounter.incrementAndGet();
|
||||
jetty.setProxyPort(proxyPort);
|
||||
|
||||
jetty.start();
|
||||
|
||||
// create a socket proxy for the jetty server ...
|
||||
SocketProxy proxy = new SocketProxy(proxyPort, jetty.getBaseUrl().toURI());
|
||||
proxies.put(proxy.getUrl(), proxy);
|
||||
|
||||
return jetty;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doTest() throws Exception {
|
||||
waitForThingsToLevelOut(30000);
|
||||
|
||||
// test a 1x2 collection
|
||||
testRf2();
|
||||
|
||||
// now do similar for a 1x3 collection while taking 2 replicas on-and-off
|
||||
// each time
|
||||
testRf3();
|
||||
|
||||
// kill a leader and make sure recovery occurs as expected
|
||||
testRf3WithLeaderFailover();
|
||||
}
|
||||
|
||||
protected void testRf2() throws Exception {
|
||||
// create a collection that has 1 shard but 2 replicas
|
||||
String testCollectionName = "c8n_1x2";
|
||||
createCollection(testCollectionName, 1, 2, 1);
|
||||
cloudClient.setDefaultCollection(testCollectionName);
|
||||
|
||||
sendDoc(1);
|
||||
|
||||
Replica notLeader =
|
||||
ensureAllReplicasAreActive(testCollectionName, 2, 10).get(0);
|
||||
|
||||
// ok, now introduce a network partition between the leader and the replica
|
||||
SocketProxy proxy = getProxyForReplica(notLeader);
|
||||
|
||||
proxy.close();
|
||||
|
||||
// indexing during a partition
|
||||
sendDoc(2);
|
||||
|
||||
// Have the partition last at least 1 sec
|
||||
// While this gives the impression that recovery is timing related, this is
|
||||
// really only
|
||||
// to give time for the state to be written to ZK before the test completes.
|
||||
// In other words,
|
||||
// without a brief pause, the test finishes so quickly that it doesn't give
|
||||
// time for the recovery process to kick-in
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
|
||||
proxy.reopen();
|
||||
|
||||
List<Replica> notLeaders =
|
||||
ensureAllReplicasAreActive(testCollectionName, 2, 20); // shouldn't take 20 secs but just to be safe
|
||||
|
||||
sendDoc(3);
|
||||
|
||||
// sent 3 docs in so far, verify they are on the leader and replica
|
||||
assertDocsExistInAllReplicas(notLeaders, testCollectionName, 1, 3);
|
||||
|
||||
// now up the stakes and do more docs
|
||||
int numDocs = 1000;
|
||||
boolean hasPartition = false;
|
||||
for (int d = 0; d < numDocs; d++) {
|
||||
// create / restore partition every 100 docs
|
||||
if (d % 100 == 0) {
|
||||
if (hasPartition) {
|
||||
proxy.reopen();
|
||||
hasPartition = false;
|
||||
} else {
|
||||
if (d >= 100) {
|
||||
proxy.close();
|
||||
hasPartition = true;
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
}
|
||||
}
|
||||
}
|
||||
sendDoc(d + 4); // 4 is offset as we've already indexed 1-3
|
||||
}
|
||||
|
||||
// restore connectivity if lost
|
||||
if (hasPartition) {
|
||||
proxy.reopen();
|
||||
}
|
||||
|
||||
notLeaders = ensureAllReplicasAreActive(testCollectionName, 2, 20);
|
||||
|
||||
// verify all docs received
|
||||
assertDocsExistInAllReplicas(notLeaders, testCollectionName, 1, numDocs + 3);
|
||||
}
|
||||
|
||||
protected void testRf3() throws Exception {
|
||||
// create a collection that has 1 shard but 2 replicas
|
||||
String testCollectionName = "c8n_1x3";
|
||||
createCollection(testCollectionName, 1, 3, 1);
|
||||
cloudClient.setDefaultCollection(testCollectionName);
|
||||
|
||||
sendDoc(1);
|
||||
|
||||
List<Replica> notLeaders =
|
||||
ensureAllReplicasAreActive(testCollectionName, 3, 10);
|
||||
assertTrue("Expected 2 replicas for collection " + testCollectionName
|
||||
+ " but found " + notLeaders.size() + "; clusterState: "
|
||||
+ cloudClient.getZkStateReader().getClusterState(),
|
||||
notLeaders.size() == 2);
|
||||
|
||||
sendDoc(1);
|
||||
|
||||
// ok, now introduce a network partition between the leader and the replica
|
||||
SocketProxy proxy0 = getProxyForReplica(notLeaders.get(0));
|
||||
|
||||
proxy0.close();
|
||||
|
||||
// indexing during a partition
|
||||
sendDoc(2);
|
||||
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
|
||||
proxy0.reopen();
|
||||
|
||||
SocketProxy proxy1 = getProxyForReplica(notLeaders.get(1));
|
||||
|
||||
proxy1.close();
|
||||
|
||||
sendDoc(3);
|
||||
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
proxy1.reopen();
|
||||
|
||||
// sent 4 docs in so far, verify they are on the leader and replica
|
||||
notLeaders = ensureAllReplicasAreActive(testCollectionName, 3, 20);
|
||||
|
||||
sendDoc(4);
|
||||
|
||||
assertDocsExistInAllReplicas(notLeaders, testCollectionName, 1, 4);
|
||||
}
|
||||
|
||||
protected void testRf3WithLeaderFailover() throws Exception {
|
||||
// now let's create a partition in one of the replicas and outright
|
||||
// kill the leader ... see what happens
|
||||
// create a collection that has 1 shard but 2 replicas
|
||||
String testCollectionName = "c8n_1x3_lf"; // _lf is leader fails
|
||||
createCollection(testCollectionName, 1, 3, 1);
|
||||
cloudClient.setDefaultCollection(testCollectionName);
|
||||
|
||||
sendDoc(1);
|
||||
|
||||
List<Replica> notLeaders =
|
||||
ensureAllReplicasAreActive(testCollectionName, 3, 10);
|
||||
assertTrue("Expected 2 replicas for collection " + testCollectionName
|
||||
+ " but found " + notLeaders.size() + "; clusterState: "
|
||||
+ cloudClient.getZkStateReader().getClusterState(),
|
||||
notLeaders.size() == 2);
|
||||
|
||||
sendDoc(1);
|
||||
|
||||
// ok, now introduce a network partition between the leader and the replica
|
||||
SocketProxy proxy0 = null;
|
||||
proxy0 = getProxyForReplica(notLeaders.get(0));
|
||||
|
||||
proxy0.close();
|
||||
|
||||
// indexing during a partition
|
||||
sendDoc(2);
|
||||
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
|
||||
proxy0.reopen();
|
||||
|
||||
SocketProxy proxy1 = getProxyForReplica(notLeaders.get(1));
|
||||
|
||||
proxy1.close();
|
||||
|
||||
sendDoc(3);
|
||||
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
proxy1.reopen();
|
||||
|
||||
// sent 4 docs in so far, verify they are on the leader and replica
|
||||
notLeaders = ensureAllReplicasAreActive(testCollectionName, 3, 20);
|
||||
|
||||
sendDoc(4);
|
||||
|
||||
assertDocsExistInAllReplicas(notLeaders, testCollectionName, 1, 4);
|
||||
|
||||
Replica leader =
|
||||
cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
|
||||
String leaderNode = leader.getNodeName();
|
||||
assertNotNull("Could not find leader for shard1 of "+testCollectionName, leader);
|
||||
JettySolrRunner leaderJetty = getJettyOnPort(getReplicaPort(leader));
|
||||
|
||||
// since maxShardsPerNode is 1, we're safe to kill the leader
|
||||
notLeaders = ensureAllReplicasAreActive(testCollectionName, 3, 20);
|
||||
proxy0 = getProxyForReplica(notLeaders.get(0));
|
||||
proxy0.close();
|
||||
|
||||
// indexing during a partition
|
||||
// doc should be on leader and 1 replica
|
||||
sendDoc(5);
|
||||
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
|
||||
String shouldNotBeNewLeaderNode = notLeaders.get(0).getNodeName();
|
||||
|
||||
// kill the leader
|
||||
leaderJetty.stop();
|
||||
|
||||
if (leaderJetty.isRunning())
|
||||
fail("Failed to stop the leader on "+leaderNode);
|
||||
|
||||
SocketProxy oldLeaderProxy = getProxyForReplica(leader);
|
||||
if (oldLeaderProxy != null) {
|
||||
oldLeaderProxy.close();
|
||||
} else {
|
||||
log.warn("No SocketProxy found for old leader node "+leaderNode);
|
||||
}
|
||||
|
||||
Thread.sleep(sleepMsBeforeHealPartition);
|
||||
|
||||
Replica newLeader =
|
||||
cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1", 30000);
|
||||
|
||||
assertNotNull("No new leader was elected after 30 seconds", newLeader);
|
||||
|
||||
assertTrue("Expected node "+shouldNotBeNewLeaderNode+
|
||||
" to NOT be the new leader b/c it was out-of-sync with the old leader! ClusterState: "+
|
||||
cloudClient.getZkStateReader().getClusterState(),
|
||||
!shouldNotBeNewLeaderNode.equals(newLeader.getNodeName()));
|
||||
|
||||
proxy0.reopen();
|
||||
|
||||
Thread.sleep(10000L);
|
||||
|
||||
cloudClient.getZkStateReader().updateClusterState(true);
|
||||
|
||||
List<Replica> activeReps = getActiveOrRecoveringReplicas(testCollectionName, "shard1");
|
||||
assertTrue("Expected 2 of 3 replicas to be active but only found "+activeReps.size()+"; "+activeReps, activeReps.size() == 2);
|
||||
|
||||
sendDoc(6);
|
||||
|
||||
assertDocsExistInAllReplicas(activeReps, testCollectionName, 1, 6);
|
||||
}
|
||||
|
||||
protected List<Replica> getActiveOrRecoveringReplicas(String testCollectionName, String shardId) throws Exception {
|
||||
Map<String,Replica> activeReplicas = new HashMap<String,Replica>();
|
||||
ZkStateReader zkr = cloudClient.getZkStateReader();
|
||||
ClusterState cs = zkr.getClusterState();
|
||||
cs = zkr.getClusterState();
|
||||
assertNotNull(cs);
|
||||
for (Slice shard : cs.getActiveSlices(testCollectionName)) {
|
||||
if (shard.getName().equals(shardId)) {
|
||||
for (Replica replica : shard.getReplicas()) {
|
||||
String replicaState = replica.getStr(ZkStateReader.STATE_PROP);
|
||||
if (ZkStateReader.ACTIVE.equals(replicaState) || ZkStateReader.RECOVERING.equals(replicaState)) {
|
||||
activeReplicas.put(replica.getName(), replica);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
List<Replica> replicas = new ArrayList<Replica>();
|
||||
replicas.addAll(activeReplicas.values());
|
||||
return replicas;
|
||||
}
|
||||
|
||||
protected SocketProxy getProxyForReplica(Replica replica) throws Exception {
|
||||
String replicaBaseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
|
||||
assertNotNull(replicaBaseUrl);
|
||||
URL baseUrl = new URL(replicaBaseUrl);
|
||||
|
||||
SocketProxy proxy = proxies.get(baseUrl.toURI());
|
||||
if (proxy == null && !baseUrl.toExternalForm().endsWith("/")) {
|
||||
baseUrl = new URL(baseUrl.toExternalForm() + "/");
|
||||
proxy = proxies.get(baseUrl.toURI());
|
||||
}
|
||||
assertNotNull("No proxy found for " + baseUrl + "!", proxy);
|
||||
return proxy;
|
||||
}
|
||||
|
||||
protected void assertDocsExistInAllReplicas(List<Replica> notLeaders,
|
||||
String testCollectionName, int firstDocId, int lastDocId)
|
||||
throws Exception {
|
||||
Replica leader =
|
||||
cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1", 10000);
|
||||
HttpSolrServer leaderSolr = getHttpSolrServer(leader, testCollectionName);
|
||||
List<HttpSolrServer> replicas =
|
||||
new ArrayList<HttpSolrServer>(notLeaders.size());
|
||||
|
||||
for (Replica r : notLeaders) {
|
||||
replicas.add(getHttpSolrServer(r, testCollectionName));
|
||||
}
|
||||
try {
|
||||
for (int d = firstDocId; d <= lastDocId; d++) {
|
||||
String docId = String.valueOf(d);
|
||||
assertDocExists(leaderSolr, testCollectionName, docId);
|
||||
for (HttpSolrServer replicaSolr : replicas) {
|
||||
assertDocExists(replicaSolr, testCollectionName, docId);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (leaderSolr != null) {
|
||||
leaderSolr.shutdown();
|
||||
}
|
||||
for (HttpSolrServer replicaSolr : replicas) {
|
||||
replicaSolr.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected HttpSolrServer getHttpSolrServer(Replica replica, String coll) throws Exception {
|
||||
ZkCoreNodeProps zkProps = new ZkCoreNodeProps(replica);
|
||||
String url = zkProps.getBaseUrl() + "/" + coll;
|
||||
return new HttpSolrServer(url);
|
||||
}
|
||||
|
||||
protected void sendDoc(int docId) throws Exception {
|
||||
SolrInputDocument doc = new SolrInputDocument();
|
||||
doc.addField(id, String.valueOf(docId));
|
||||
doc.addField("a_t", "hello" + docId);
|
||||
cloudClient.add(doc);
|
||||
}
|
||||
|
||||
protected List<Replica> ensureAllReplicasAreActive(String testCollectionName, int rf, int maxWaitSecs) throws Exception {
|
||||
long startMs = System.currentTimeMillis();
|
||||
|
||||
Map<String,Replica> notLeaders = new HashMap<String,Replica>();
|
||||
|
||||
ZkStateReader zkr = cloudClient.getZkStateReader();
|
||||
ClusterState cs = zkr.getClusterState();
|
||||
Collection<Slice> slices = cs.getActiveSlices(testCollectionName);
|
||||
assertTrue(slices.size() == 1); // shards == 1
|
||||
boolean allReplicasUp = false;
|
||||
long waitMs = 0L;
|
||||
long maxWaitMs = maxWaitSecs * 1000L;
|
||||
Replica leader = null;
|
||||
while (waitMs < maxWaitMs && !allReplicasUp) {
|
||||
cs = zkr.getClusterState();
|
||||
assertNotNull(cs);
|
||||
for (Slice shard : cs.getActiveSlices(testCollectionName)) {
|
||||
allReplicasUp = true; // assume true
|
||||
Collection<Replica> replicas = shard.getReplicas();
|
||||
assertTrue(replicas.size() == rf);
|
||||
leader = shard.getLeader();
|
||||
assertNotNull(leader);
|
||||
|
||||
// ensure all replicas are "active" and identify the non-leader replica
|
||||
for (Replica replica : replicas) {
|
||||
String replicaState = replica.getStr(ZkStateReader.STATE_PROP);
|
||||
if (!ZkStateReader.ACTIVE.equals(replicaState)) {
|
||||
log.info("Replica " + replica.getName() + " is currently " + replicaState);
|
||||
allReplicasUp = false;
|
||||
}
|
||||
|
||||
if (!leader.equals(replica))
|
||||
notLeaders.put(replica.getName(), replica);
|
||||
}
|
||||
|
||||
if (!allReplicasUp) {
|
||||
try {
|
||||
Thread.sleep(500L);
|
||||
} catch (Exception ignoreMe) {}
|
||||
waitMs += 500L;
|
||||
}
|
||||
}
|
||||
} // end while
|
||||
|
||||
if (!allReplicasUp)
|
||||
fail("Didn't see all replicas come up within " + maxWaitMs + " ms! ClusterState: " + cs);
|
||||
|
||||
if (notLeaders.isEmpty())
|
||||
fail("Didn't isolate any replicas that are not the leader! ClusterState: " + cs);
|
||||
|
||||
long diffMs = (System.currentTimeMillis() - startMs);
|
||||
log.info("Took " + diffMs + " ms to see all replicas become active.");
|
||||
|
||||
List<Replica> replicas = new ArrayList<Replica>();
|
||||
replicas.addAll(notLeaders.values());
|
||||
return replicas;
|
||||
}
|
||||
|
||||
/**
|
||||
* Query the real-time get handler for a specific doc by ID to verify it
|
||||
* exists in the provided server.
|
||||
*/
|
||||
@SuppressWarnings("rawtypes")
|
||||
protected void assertDocExists(HttpSolrServer solr, String coll, String docId) throws Exception {
|
||||
QueryRequest qr = new QueryRequest(params("qt", "/get", "id", docId));
|
||||
NamedList rsp = solr.request(qr);
|
||||
String match =
|
||||
JSONTestUtil.matchObj("/id", rsp.get("doc"), new Integer(docId));
|
||||
assertTrue("Doc with id=" + docId + " not found in " + solr.getBaseURL()
|
||||
+ " due to: " + match, match == null);
|
||||
}
|
||||
|
||||
protected JettySolrRunner getJettyOnPort(int port) {
|
||||
JettySolrRunner theJetty = null;
|
||||
for (JettySolrRunner jetty : jettys) {
|
||||
if (port == jetty.getLocalPort()) {
|
||||
theJetty = jetty;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (theJetty == null) {
|
||||
if (controlJetty.getLocalPort() == port) {
|
||||
theJetty = controlJetty;
|
||||
}
|
||||
}
|
||||
|
||||
if (theJetty == null)
|
||||
fail("Not able to find JettySolrRunner for port: "+port);
|
||||
|
||||
return theJetty;
|
||||
}
|
||||
|
||||
protected int getReplicaPort(Replica replica) {
|
||||
String replicaNode = replica.getNodeName();
|
||||
String tmp = replicaNode.substring(replicaNode.indexOf(':')+1);
|
||||
if (tmp.indexOf('_') != -1)
|
||||
tmp = tmp.substring(0,tmp.indexOf('_'));
|
||||
return Integer.parseInt(tmp);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,406 @@
|
|||
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.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.ServerSocket;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketException;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import javax.net.ssl.SSLServerSocketFactory;
|
||||
import javax.net.ssl.SSLSocketFactory;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Kindly borrowed the idea and base implementation from the ActiveMQ project;
|
||||
* useful for blocking traffic on a specified port.
|
||||
*/
|
||||
public class SocketProxy {
|
||||
|
||||
private static final transient Logger log = LoggerFactory.getLogger(SocketProxy.class);
|
||||
|
||||
public static final int ACCEPT_TIMEOUT_MILLIS = 100;
|
||||
|
||||
private URI proxyUrl;
|
||||
private URI target;
|
||||
|
||||
private Acceptor acceptor;
|
||||
private ServerSocket serverSocket;
|
||||
|
||||
private CountDownLatch closed = new CountDownLatch(1);
|
||||
|
||||
public List<Bridge> connections = new LinkedList<Bridge>();
|
||||
|
||||
private int listenPort = 0;
|
||||
|
||||
private int receiveBufferSize = -1;
|
||||
|
||||
private boolean pauseAtStart = false;
|
||||
|
||||
private int acceptBacklog = 50;
|
||||
|
||||
public SocketProxy() throws Exception {}
|
||||
|
||||
public SocketProxy(URI uri) throws Exception {
|
||||
this(0, uri);
|
||||
}
|
||||
|
||||
public SocketProxy(int port, URI uri) throws Exception {
|
||||
listenPort = port;
|
||||
target = uri;
|
||||
open();
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return "SocketyProxy: port="+listenPort+"; target="+target;
|
||||
}
|
||||
|
||||
public void setReceiveBufferSize(int receiveBufferSize) {
|
||||
this.receiveBufferSize = receiveBufferSize;
|
||||
}
|
||||
|
||||
public void setTarget(URI tcpBrokerUri) {
|
||||
target = tcpBrokerUri;
|
||||
}
|
||||
|
||||
public void open() throws Exception {
|
||||
serverSocket = createServerSocket(target);
|
||||
serverSocket.setReuseAddress(true);
|
||||
if (receiveBufferSize > 0) {
|
||||
serverSocket.setReceiveBufferSize(receiveBufferSize);
|
||||
}
|
||||
if (proxyUrl == null) {
|
||||
serverSocket.bind(new InetSocketAddress(listenPort), acceptBacklog);
|
||||
proxyUrl = urlFromSocket(target, serverSocket);
|
||||
} else {
|
||||
serverSocket.bind(new InetSocketAddress(proxyUrl.getPort()));
|
||||
}
|
||||
acceptor = new Acceptor(serverSocket, target);
|
||||
if (pauseAtStart) {
|
||||
acceptor.pause();
|
||||
}
|
||||
new Thread(null, acceptor, "SocketProxy-Acceptor-"
|
||||
+ serverSocket.getLocalPort()).start();
|
||||
closed = new CountDownLatch(1);
|
||||
}
|
||||
|
||||
private boolean isSsl(URI target) {
|
||||
return "ssl".equals(target.getScheme());
|
||||
}
|
||||
|
||||
private ServerSocket createServerSocket(URI target) throws Exception {
|
||||
if (isSsl(target)) {
|
||||
return SSLServerSocketFactory.getDefault().createServerSocket();
|
||||
}
|
||||
return new ServerSocket();
|
||||
}
|
||||
|
||||
private Socket createSocket(URI target) throws Exception {
|
||||
if (isSsl(target)) {
|
||||
return SSLSocketFactory.getDefault().createSocket();
|
||||
}
|
||||
return new Socket();
|
||||
}
|
||||
|
||||
public URI getUrl() {
|
||||
return proxyUrl;
|
||||
}
|
||||
|
||||
/*
|
||||
* close all proxy connections and acceptor
|
||||
*/
|
||||
public void close() {
|
||||
List<Bridge> connections;
|
||||
synchronized (this.connections) {
|
||||
connections = new ArrayList<Bridge>(this.connections);
|
||||
}
|
||||
log.warn("Closing " + connections.size()+" connections to: "+getUrl());
|
||||
for (Bridge con : connections) {
|
||||
closeConnection(con);
|
||||
}
|
||||
acceptor.close();
|
||||
closed.countDown();
|
||||
}
|
||||
|
||||
/*
|
||||
* close all proxy receive connections, leaving acceptor open
|
||||
*/
|
||||
public void halfClose() {
|
||||
List<Bridge> connections;
|
||||
synchronized (this.connections) {
|
||||
connections = new ArrayList<Bridge>(this.connections);
|
||||
}
|
||||
log.info("halfClose, numConnections=" + connections.size());
|
||||
for (Bridge con : connections) {
|
||||
halfCloseConnection(con);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean waitUntilClosed(long timeoutSeconds)
|
||||
throws InterruptedException {
|
||||
return closed.await(timeoutSeconds, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
/*
|
||||
* called after a close to restart the acceptor on the same port
|
||||
*/
|
||||
public void reopen() {
|
||||
log.info("Re-opening connectivity to "+getUrl());
|
||||
try {
|
||||
open();
|
||||
} catch (Exception e) {
|
||||
log.debug("exception on reopen url:" + getUrl(), e);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* pause accepting new connections and data transfer through existing proxy
|
||||
* connections. All sockets remain open
|
||||
*/
|
||||
public void pause() {
|
||||
synchronized (connections) {
|
||||
log.info("pause, numConnections=" + connections.size());
|
||||
acceptor.pause();
|
||||
for (Bridge con : connections) {
|
||||
con.pause();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* continue after pause
|
||||
*/
|
||||
public void goOn() {
|
||||
synchronized (connections) {
|
||||
log.info("goOn, numConnections=" + connections.size());
|
||||
for (Bridge con : connections) {
|
||||
con.goOn();
|
||||
}
|
||||
}
|
||||
acceptor.goOn();
|
||||
}
|
||||
|
||||
private void closeConnection(Bridge c) {
|
||||
try {
|
||||
c.close();
|
||||
} catch (Exception e) {
|
||||
log.debug("exception on close of: " + c, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void halfCloseConnection(Bridge c) {
|
||||
try {
|
||||
c.halfClose();
|
||||
} catch (Exception e) {
|
||||
log.debug("exception on half close of: " + c, e);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isPauseAtStart() {
|
||||
return pauseAtStart;
|
||||
}
|
||||
|
||||
public void setPauseAtStart(boolean pauseAtStart) {
|
||||
this.pauseAtStart = pauseAtStart;
|
||||
}
|
||||
|
||||
public int getAcceptBacklog() {
|
||||
return acceptBacklog;
|
||||
}
|
||||
|
||||
public void setAcceptBacklog(int acceptBacklog) {
|
||||
this.acceptBacklog = acceptBacklog;
|
||||
}
|
||||
|
||||
private URI urlFromSocket(URI uri, ServerSocket serverSocket)
|
||||
throws Exception {
|
||||
int listenPort = serverSocket.getLocalPort();
|
||||
|
||||
return new URI(uri.getScheme(), uri.getUserInfo(), uri.getHost(),
|
||||
listenPort, uri.getPath(), uri.getQuery(), uri.getFragment());
|
||||
}
|
||||
|
||||
public class Bridge {
|
||||
|
||||
private Socket receiveSocket;
|
||||
private Socket sendSocket;
|
||||
private Pump requestThread;
|
||||
private Pump responseThread;
|
||||
|
||||
public Bridge(Socket socket, URI target) throws Exception {
|
||||
receiveSocket = socket;
|
||||
sendSocket = createSocket(target);
|
||||
if (receiveBufferSize > 0) {
|
||||
sendSocket.setReceiveBufferSize(receiveBufferSize);
|
||||
}
|
||||
sendSocket.connect(new InetSocketAddress(target.getHost(), target
|
||||
.getPort()));
|
||||
linkWithThreads(receiveSocket, sendSocket);
|
||||
log.info("proxy connection " + sendSocket + ", receiveBufferSize="
|
||||
+ sendSocket.getReceiveBufferSize());
|
||||
}
|
||||
|
||||
public void goOn() {
|
||||
responseThread.goOn();
|
||||
requestThread.goOn();
|
||||
}
|
||||
|
||||
public void pause() {
|
||||
requestThread.pause();
|
||||
responseThread.pause();
|
||||
}
|
||||
|
||||
public void close() throws Exception {
|
||||
synchronized (connections) {
|
||||
connections.remove(this);
|
||||
}
|
||||
receiveSocket.close();
|
||||
sendSocket.close();
|
||||
}
|
||||
|
||||
public void halfClose() throws Exception {
|
||||
receiveSocket.close();
|
||||
}
|
||||
|
||||
private void linkWithThreads(Socket source, Socket dest) {
|
||||
requestThread = new Pump(source, dest);
|
||||
requestThread.start();
|
||||
responseThread = new Pump(dest, source);
|
||||
responseThread.start();
|
||||
}
|
||||
|
||||
public class Pump extends Thread {
|
||||
|
||||
protected Socket src;
|
||||
private Socket destination;
|
||||
private AtomicReference<CountDownLatch> pause = new AtomicReference<CountDownLatch>();
|
||||
|
||||
public Pump(Socket source, Socket dest) {
|
||||
super("SocketProxy-DataTransfer-" + source.getPort() + ":"
|
||||
+ dest.getPort());
|
||||
src = source;
|
||||
destination = dest;
|
||||
pause.set(new CountDownLatch(0));
|
||||
}
|
||||
|
||||
public void pause() {
|
||||
pause.set(new CountDownLatch(1));
|
||||
}
|
||||
|
||||
public void goOn() {
|
||||
pause.get().countDown();
|
||||
}
|
||||
|
||||
public void run() {
|
||||
byte[] buf = new byte[1024];
|
||||
try {
|
||||
InputStream in = src.getInputStream();
|
||||
OutputStream out = destination.getOutputStream();
|
||||
while (true) {
|
||||
int len = in.read(buf);
|
||||
if (len == -1) {
|
||||
log.debug("read eof from:" + src);
|
||||
break;
|
||||
}
|
||||
pause.get().await();
|
||||
out.write(buf, 0, len);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.debug("read/write failed, reason: " + e.getLocalizedMessage());
|
||||
try {
|
||||
if (!receiveSocket.isClosed()) {
|
||||
// for halfClose, on read/write failure if we close the
|
||||
// remote end will see a close at the same time.
|
||||
close();
|
||||
}
|
||||
} catch (Exception ignore) {}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public class Acceptor implements Runnable {
|
||||
|
||||
private ServerSocket socket;
|
||||
private URI target;
|
||||
private AtomicReference<CountDownLatch> pause = new AtomicReference<CountDownLatch>();
|
||||
|
||||
public Acceptor(ServerSocket serverSocket, URI uri) {
|
||||
socket = serverSocket;
|
||||
target = uri;
|
||||
pause.set(new CountDownLatch(0));
|
||||
try {
|
||||
socket.setSoTimeout(ACCEPT_TIMEOUT_MILLIS);
|
||||
} catch (SocketException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
public void pause() {
|
||||
pause.set(new CountDownLatch(1));
|
||||
}
|
||||
|
||||
public void goOn() {
|
||||
pause.get().countDown();
|
||||
}
|
||||
|
||||
public void run() {
|
||||
try {
|
||||
while (!socket.isClosed()) {
|
||||
pause.get().await();
|
||||
try {
|
||||
Socket source = socket.accept();
|
||||
pause.get().await();
|
||||
if (receiveBufferSize > 0) {
|
||||
source.setReceiveBufferSize(receiveBufferSize);
|
||||
}
|
||||
log.info("accepted " + source + ", receiveBufferSize:"
|
||||
+ source.getReceiveBufferSize());
|
||||
synchronized (connections) {
|
||||
connections.add(new Bridge(source, target));
|
||||
}
|
||||
} catch (SocketTimeoutException expected) {}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.debug("acceptor: finished for reason: " + e.getLocalizedMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
try {
|
||||
socket.close();
|
||||
closed.countDown();
|
||||
goOn();
|
||||
} catch (IOException ignored) {}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -253,7 +253,14 @@ public class CoreAdminRequest extends SolrRequest
|
|||
|
||||
return params;
|
||||
}
|
||||
|
||||
|
||||
public String toString() {
|
||||
if (action != null) {
|
||||
return "WaitForState: "+getParams();
|
||||
} else {
|
||||
return super.toString();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class RequestRecovery extends CoreAdminRequest {
|
||||
|
|
Loading…
Reference in New Issue