SOLR-6931: We should do a limited retry when using HttpClient.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1651767 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2015-01-14 18:38:20 +00:00
parent 4930a38e84
commit a6b2647c92
6 changed files with 56 additions and 21 deletions

View File

@ -447,6 +447,9 @@ Bug Fixes
* SOLR-6764: Field types need to be re-informed after reloading a managed schema from ZK
(Timothy Potter)
* SOLR-6931: We should do a limited retry when using HttpClient.
(Mark Miller, Hrishikesh Gadre, Gregory Chanan)
Optimizations
----------------------

View File

@ -16,8 +16,22 @@ package org.apache.solr.handler.component;
* limitations under the License.
*/
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang.StringUtils;
import org.apache.http.client.HttpClient;
import org.apache.http.impl.client.DefaultHttpClient;
import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
@ -33,18 +47,6 @@ import org.apache.solr.util.DefaultSolrThreadFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.apache.solr.util.plugin.PluginInfoInitialized {
protected static Logger log = LoggerFactory.getLogger(HttpShardHandlerFactory.class);
@ -76,6 +78,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
int keepAliveTime = 5;
int queueSize = -1;
boolean accessPolicy = false;
boolean useRetries = false;
private String scheme = null;
@ -98,6 +101,10 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
// Configure if the threadpool favours fairness over throughput
static final String INIT_FAIRNESS_POLICY = "fairnessPolicy";
// Turn on retries for certain IOExceptions, many of which can happen
// due to connection pooling limitations / races
static final String USE_RETRIES = "useRetries";
/**
* Get {@link ShardHandler} that uses the default http client.
@ -130,6 +137,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
this.keepAliveTime = getParameter(args, MAX_THREAD_IDLE_TIME, keepAliveTime);
this.queueSize = getParameter(args, INIT_SIZE_OF_QUEUE, queueSize);
this.accessPolicy = getParameter(args, INIT_FAIRNESS_POLICY, accessPolicy);
this.useRetries = getParameter(args, USE_RETRIES, useRetries);
// magic sysprop to make tests reproducible: set by SolrTestCaseJ4.
String v = System.getProperty("tests.shardhandler.randomSeed");
@ -154,8 +162,18 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
clientParams.set(HttpClientUtil.PROP_MAX_CONNECTIONS, maxConnections);
clientParams.set(HttpClientUtil.PROP_SO_TIMEOUT, soTimeout);
clientParams.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT, connectionTimeout);
clientParams.set(HttpClientUtil.PROP_USE_RETRY, false);
if (!useRetries) {
clientParams.set(HttpClientUtil.PROP_USE_RETRY, false);
}
this.defaultClient = HttpClientUtil.createClient(clientParams);
// must come after createClient
if (useRetries) {
// our default retry handler will never retry on IOException if the request has been sent already,
// but for these read only requests we can use the standard DefaultHttpRequestRetryHandler rules
((DefaultHttpClient) this.defaultClient).setHttpRequestRetryHandler(new DefaultHttpRequestRetryHandler());
}
this.loadbalancer = createLoadbalancer(defaultClient);
}

View File

@ -61,7 +61,10 @@ public class UpdateShardHandler {
params.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT,
cfg.getDistributedConnectionTimeout());
}
params.set(HttpClientUtil.PROP_USE_RETRY, false);
// in the update case, we want to do retries, and to use
// the default Solr retry handler that createClient will
// give us
params.set(HttpClientUtil.PROP_USE_RETRY, true);
log.info("Creating UpdateShardHandler HTTP client with params: {}", params);
client = HttpClientUtil.createClient(params, clientConnectionManager);
}

View File

@ -51,16 +51,15 @@ public class HttpClientConfigurer {
config.getInt(HttpClientUtil.PROP_SO_TIMEOUT));
}
if (config.get(HttpClientUtil.PROP_USE_RETRY) != null) {
HttpClientUtil.setUseRetry(httpClient,
config.getBool(HttpClientUtil.PROP_USE_RETRY));
}
if (config.get(HttpClientUtil.PROP_FOLLOW_REDIRECTS) != null) {
HttpClientUtil.setFollowRedirects(httpClient,
config.getBool(HttpClientUtil.PROP_FOLLOW_REDIRECTS));
}
// always call setUseRetry, whether it is in config or not
HttpClientUtil.setUseRetry(httpClient,
config.getBool(HttpClientUtil.PROP_USE_RETRY, true));
final String basicAuthUser = config
.get(HttpClientUtil.PROP_BASIC_AUTH_USER);
final String basicAuthPass = config

View File

@ -241,7 +241,14 @@ public class HttpClientUtil {
if (!useRetry) {
httpClient.setHttpRequestRetryHandler(NO_RETRY);
} else {
httpClient.setHttpRequestRetryHandler(new DefaultHttpRequestRetryHandler());
// if the request is not fully sent, we retry
// streaming updates are not a problem, because they are not retryable
httpClient.setHttpRequestRetryHandler(new DefaultHttpRequestRetryHandler(){
@Override
protected boolean handleAsIdempotent(final HttpRequest request) {
return false; // we can't tell if a Solr request is idempotent
}
});
}
}

View File

@ -223,7 +223,12 @@ public class LBHttpSolrClient extends SolrClient {
this.parser = parser;
if (httpClient == null) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(HttpClientUtil.PROP_USE_RETRY, false);
if (solrServerUrl.length > 1) {
// we prefer retrying another server
params.set(HttpClientUtil.PROP_USE_RETRY, false);
} else {
params.set(HttpClientUtil.PROP_USE_RETRY, true);
}
this.httpClient = HttpClientUtil.createClient(params);
} else {
this.httpClient = httpClient;