SOLR-10456: Deprecate timeout related setters from SolrClients, and replace with Builder based implementation

This commit is contained in:
Anshum Gupta 2017-07-03 21:44:18 -07:00
parent 8218a5b2c6
commit 6c163658bb
49 changed files with 502 additions and 209 deletions

View File

@ -397,6 +397,9 @@ Other Changes
(added leading '/' in request handlers). Switch all tests referring to "standard" request handler to
instead refer to "/select" with SearchHandler. Deprecated the old StandardRequestHandler. (David Smiley)
* SOLR-10456: Deprecate timeout related setters from SolrClients, and replace with Builder based implementation
(Jason Gerlowski, Anshum Gupta)
================== 6.7.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -305,9 +305,7 @@ public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTe
sidl.add(sd);
}
try (HttpSolrClient solrServer = getHttpSolrClient(getSourceUrl())) {
solrServer.setConnectionTimeout(15000);
solrServer.setSoTimeout(30000);
try (HttpSolrClient solrServer = getHttpSolrClient(getSourceUrl(), 15000, 30000)) {
solrServer.add(sidl);
solrServer.commit(true, true);
}

View File

@ -508,6 +508,13 @@ public class JettySolrRunner {
public SolrClient newClient() {
return new HttpSolrClient.Builder(getBaseUrl().toString()).build();
}
public SolrClient newClient(int connectionTimeoutMillis, int socketTimeoutMillis) {
return new HttpSolrClient.Builder(getBaseUrl().toString())
.withConnectionTimeout(connectionTimeoutMillis)
.withSocketTimeout(socketTimeoutMillis)
.build();
}
public DebugFilter getDebugFilter() {
return (DebugFilter)debugFilter.getFilter();

View File

@ -201,9 +201,10 @@ public class LeaderInitiatedRecoveryThread extends Thread {
log.info("Asking core={} coreNodeName={} on " + recoveryUrl + " to recover", coreNeedingRecovery, replicaCoreNodeName);
}
try (HttpSolrClient client = new HttpSolrClient.Builder(recoveryUrl).build()) {
client.setSoTimeout(60000);
client.setConnectionTimeout(15000);
try (HttpSolrClient client = new HttpSolrClient.Builder(recoveryUrl)
.withConnectionTimeout(15000)
.withSocketTimeout(60000)
.build()) {
try {
client.request(recoverRequestCmd);

View File

@ -443,10 +443,11 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
final String createUrl, final String dataDir, final String ulogDir,
final String coreNodeName, final String coreName, final String shardId) {
try (HttpSolrClient client = new HttpSolrClient.Builder(createUrl).build()) {
try (HttpSolrClient client = new HttpSolrClient.Builder(createUrl)
.withConnectionTimeout(30000)
.withSocketTimeout(60000)
.build()) {
log.debug("create url={}", createUrl);
client.setConnectionTimeout(30000);
client.setSoTimeout(60000);
Create createCmd = new Create();
createCmd.setCollection(collection);
createCmd.setCoreNodeName(coreNodeName);

View File

@ -503,9 +503,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
static UpdateResponse softCommit(String url) throws SolrServerException, IOException {
try (HttpSolrClient client = new HttpSolrClient.Builder(url).build()) {
client.setConnectionTimeout(30000);
client.setSoTimeout(120000);
try (HttpSolrClient client = new HttpSolrClient.Builder(url)
.withConnectionTimeout(30000)
.withSocketTimeout(120000)
.build()) {
UpdateRequest ureq = new UpdateRequest();
ureq.setParams(new ModifiableSolrParams());
ureq.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, true, true);

View File

@ -255,8 +255,9 @@ public class RecoveryStrategy implements Runnable, Closeable {
final private void commitOnLeader(String leaderUrl) throws SolrServerException,
IOException {
try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl).build()) {
client.setConnectionTimeout(30000);
try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl)
.withConnectionTimeout(30000)
.build()) {
UpdateRequest ureq = new UpdateRequest();
ureq.setParams(new ModifiableSolrParams());
ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true);

View File

@ -292,9 +292,11 @@ public class SyncStrategy {
recoverRequestCmd.setAction(CoreAdminAction.REQUESTRECOVERY);
recoverRequestCmd.setCoreName(coreName);
try (HttpSolrClient client = new HttpSolrClient.Builder(baseUrl).withHttpClient(SyncStrategy.this.client).build()) {
client.setConnectionTimeout(30000);
client.setSoTimeout(120000);
try (HttpSolrClient client = new HttpSolrClient.Builder(baseUrl)
.withHttpClient(SyncStrategy.this.client)
.withConnectionTimeout(30000)
.withSocketTimeout(120000)
.build()) {
client.request(recoverRequestCmd);
} catch (Throwable t) {
SolrException.log(log, ZkCoreNodeProps.getCoreUrl(leaderProps) + ": Could not tell a replica to recover", t);

View File

@ -1605,9 +1605,10 @@ public class ZkController {
log.info("Replica " + myCoreNodeName +
" NOT in leader-initiated recovery, need to wait for leader to see down state.");
try (HttpSolrClient client = new Builder(leaderBaseUrl).build()) {
client.setConnectionTimeout(15000);
client.setSoTimeout(120000);
try (HttpSolrClient client = new Builder(leaderBaseUrl)
.withConnectionTimeout(15000)
.withSocketTimeout(120000)
.build()) {
WaitForState prepCmd = new WaitForState();
prepCmd.setCoreName(leaderCoreName);
prepCmd.setNodeName(getNodeName());

View File

@ -789,8 +789,9 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
private void commitOnLeader(String leaderUrl) throws SolrServerException,
IOException {
try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl).build()) {
client.setConnectionTimeout(30000);
try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl)
.withConnectionTimeout(30000)
.build()) {
UpdateRequest ureq = new UpdateRequest();
ureq.setParams(new ModifiableSolrParams());
ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
@ -827,9 +828,10 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
@Override
public Long call() throws Exception {
try (HttpSolrClient server = new HttpSolrClient.Builder(baseUrl).build()) {
server.setConnectionTimeout(15000);
server.setSoTimeout(60000);
try (HttpSolrClient server = new HttpSolrClient.Builder(baseUrl)
.withConnectionTimeout(15000)
.withSocketTimeout(60000)
.build()) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CommonParams.ACTION, CdcrParams.CdcrAction.SHARDCHECKPOINT.toString());

View File

@ -131,9 +131,10 @@ class CdcrUpdateLogSynchronizer implements CdcrStateManager.CdcrStateObserver {
return;
}
HttpSolrClient server = new HttpSolrClient.Builder(leaderUrl).build();
server.setConnectionTimeout(15000);
server.setSoTimeout(60000);
HttpSolrClient server = new HttpSolrClient.Builder(leaderUrl)
.withConnectionTimeout(15000)
.withSocketTimeout(60000)
.build();
ModifiableSolrParams params = new ModifiableSolrParams();
params.set(CommonParams.ACTION, CdcrParams.CdcrAction.LASTPROCESSEDVERSION.toString());

View File

@ -276,9 +276,11 @@ public class IndexFetcher {
QueryRequest req = new QueryRequest(params);
// TODO modify to use shardhandler
try (HttpSolrClient client = new Builder(masterUrl).withHttpClient(myHttpClient).build()) {
client.setSoTimeout(soTimeout);
client.setConnectionTimeout(connTimeout);
try (HttpSolrClient client = new Builder(masterUrl)
.withHttpClient(myHttpClient)
.withConnectionTimeout(connTimeout)
.withSocketTimeout(soTimeout)
.build()) {
return client.request(req);
} catch (SolrServerException e) {
@ -298,9 +300,11 @@ public class IndexFetcher {
QueryRequest req = new QueryRequest(params);
// TODO modify to use shardhandler
try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl).withHttpClient(myHttpClient).build()) {
client.setSoTimeout(soTimeout);
client.setConnectionTimeout(connTimeout);
try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl)
.withHttpClient(myHttpClient)
.withConnectionTimeout(connTimeout)
.withSocketTimeout(soTimeout)
.build()) {
NamedList response = client.request(req);
List<Map<String, Object>> files = (List<Map<String,Object>>) response.get(CMD_GET_FILE_LIST);
@ -1691,9 +1695,9 @@ public class IndexFetcher {
try (HttpSolrClient client = new Builder(masterUrl)
.withHttpClient(myHttpClient)
.withResponseParser(null)
.withConnectionTimeout(connTimeout)
.withSocketTimeout(soTimeout)
.build()) {
client.setSoTimeout(soTimeout);
client.setConnectionTimeout(connTimeout);
QueryRequest req = new QueryRequest(params);
response = client.request(req);
is = (InputStream) response.get("stream");
@ -1800,9 +1804,11 @@ public class IndexFetcher {
params.set(CommonParams.QT, ReplicationHandler.PATH);
// TODO use shardhandler
try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl).withHttpClient(myHttpClient).build()) {
client.setSoTimeout(soTimeout);
client.setConnectionTimeout(connTimeout);
try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl)
.withHttpClient(myHttpClient)
.withConnectionTimeout(connTimeout)
.withSocketTimeout(soTimeout)
.build()) {
QueryRequest request = new QueryRequest(params);
return client.request(request);
}

View File

@ -442,9 +442,10 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
ZkNodeProps leaderProps = docCollection.getLeader(shard);
ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
try (HttpSolrClient client = new Builder(nodeProps.getBaseUrl()).build()) {
client.setConnectionTimeout(15000);
client.setSoTimeout(60000);
try (HttpSolrClient client = new Builder(nodeProps.getBaseUrl())
.withConnectionTimeout(15000)
.withSocketTimeout(60000)
.build()) {
RequestSyncShard reqSyncShard = new RequestSyncShard();
reqSyncShard.setCollection(collection);
reqSyncShard.setShard(shard);

View File

@ -224,9 +224,9 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
protected LBHttpSolrClient createLoadbalancer(HttpClient httpClient){
LBHttpSolrClient client = new Builder()
.withHttpClient(httpClient)
.withConnectionTimeout(connectionTimeout)
.withSocketTimeout(soTimeout)
.build();
client.setConnectionTimeout(connectionTimeout);
client.setSoTimeout(soTimeout);
return client;
}

View File

@ -76,11 +76,9 @@ public class ConnectionReuseTest extends SolrCloudTestCase {
case 1:
return getHttpSolrClient(url.toString() + "/" + COLLECTION, httpClient);
case 2:
CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress(), random().nextBoolean(), httpClient);
CloudSolrClient client = getCloudSolrClient(cluster.getZkServer().getZkAddress(), random().nextBoolean(), httpClient, 30000, 60000);
client.setParallelUpdates(random().nextBoolean());
client.setDefaultCollection(COLLECTION);
client.getLbClient().setConnectionTimeout(30000);
client.getLbClient().setSoTimeout(60000);
return client;
}
throw new RuntimeException("impossible");

View File

@ -782,8 +782,7 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase {
protected static SolrClient createNewSolrServer(String baseUrl) {
try {
// setup the server...
HttpSolrClient s = getHttpSolrClient(baseUrl);
s.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
HttpSolrClient s = getHttpSolrClient(baseUrl, DEFAULT_CONNECTION_TIMEOUT);
return s;
} catch (Exception ex) {
throw new RuntimeException(ex);

View File

@ -536,9 +536,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
private void testStopAndStartCoresInOneInstance() throws Exception {
JettySolrRunner jetty = jettys.get(0);
try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient()) {
httpSolrClient.setConnectionTimeout(15000);
httpSolrClient.setSoTimeout(60000);
try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
ThreadPoolExecutor executor = null;
try {
executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
@ -772,9 +770,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
String collection = elements[elements.length - 1];
String urlString = url.toString();
urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
try (HttpSolrClient client = getHttpSolrClient(urlString)) {
client.setConnectionTimeout(15000);
client.setSoTimeout(60000);
try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
ModifiableSolrParams params = new ModifiableSolrParams();
//params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
params.set("qt", "/admin/metrics");
@ -860,9 +856,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState().getLeader(oneInstanceCollection2, "shard1"));
// now test that unloading a core gets us a new leader
try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString())) {
unloadClient.setConnectionTimeout(15000);
unloadClient.setSoTimeout(60000);
try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {
Unload unloadCmd = new Unload(true);
unloadCmd.setCoreName(props.getCoreName());
@ -1133,6 +1127,18 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
throw new RuntimeException(ex);
}
}
protected SolrClient createNewSolrClient(String collection, String baseUrl, int connectionTimeoutMillis, int socketTimeoutMillis) {
try {
// setup the server...
HttpSolrClient client = getHttpSolrClient(baseUrl + "/" + collection, connectionTimeoutMillis, socketTimeoutMillis);
return client;
}
catch (Exception ex) {
throw new RuntimeException(ex);
}
}
@Override
protected QueryResponse queryServer(ModifiableSolrParams params) throws SolrServerException, IOException {

View File

@ -59,7 +59,7 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
protected static final String[] fieldNames = new String[]{"f_i", "f_f", "f_d", "f_l", "f_dt"};
protected static final RandVal[] randVals = new RandVal[]{rint, rfloat, rdouble, rlong, rdate};
private int clientSoTimeout;
private int clientSoTimeout = 60000;
public String[] getFieldNames() {
return fieldNames;
@ -92,9 +92,7 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
}
fixShardCount(numShards);
// None of the operations used here are particularly costly, so this should work.
// Using this low timeout will also help us catch index stalling.
clientSoTimeout = 5000;
}
@Override
@ -102,9 +100,20 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
return onlyLeaderIndexes;
}
@Override
protected CloudSolrClient createCloudClient(String defaultCollection) {
CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, clientSoTimeout);
client.setParallelUpdates(random().nextBoolean());
if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
return client;
}
@Test
public void test() throws Exception {
cloudClient.setSoTimeout(clientSoTimeout);
// None of the operations used here are particularly costly, so this should work.
// Using this low timeout will also help us catch index stalling.
clientSoTimeout = 5000;
cloudClient = createCloudClient(DEFAULT_COLLECTION);
boolean testSuccessful = false;
try {
handle.clear();

View File

@ -84,7 +84,7 @@ public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDi
protected static final String[] fieldNames = new String[]{"f_i", "f_f", "f_d", "f_l", "f_dt"};
protected static final RandVal[] randVals = new RandVal[]{rint, rfloat, rdouble, rlong, rdate};
private int clientSoTimeout;
private int clientSoTimeout = 60000;
public String[] getFieldNames() {
return fieldNames;
@ -115,19 +115,29 @@ public class ChaosMonkeyNothingIsSafeWithPullReplicasTest extends AbstractFullDi
fixShardCount(numNodes);
log.info("Starting ChaosMonkey test with {} shards and {} nodes", sliceCount, numNodes);
// None of the operations used here are particularly costly, so this should work.
// Using this low timeout will also help us catch index stalling.
clientSoTimeout = 5000;
}
@Override
protected boolean useTlogReplicas() {
return useTlogReplicas;
}
@Override
protected CloudSolrClient createCloudClient(String defaultCollection) {
CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, clientSoTimeout);
client.setParallelUpdates(random().nextBoolean());
if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
return client;
}
@Test
public void test() throws Exception {
cloudClient.setSoTimeout(clientSoTimeout);
// None of the operations used here are particularly costly, so this should work.
// Using this low timeout will also help us catch index stalling.
clientSoTimeout = 5000;
cloudClient = createCloudClient(DEFAULT_COLLECTION);
DocCollection docCollection = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION);
assertEquals(this.sliceCount, docCollection.getSlices().size());
Slice s = docCollection.getSlice("shard1");

View File

@ -691,8 +691,7 @@ public class FullSolrCloudDistribCmdsTest extends AbstractFullDistribZkTestBase
long beforeCount = results.getResults().getNumFound();
int cnt = TEST_NIGHTLY ? 2933 : 313;
try (ConcurrentUpdateSolrClient concurrentClient = getConcurrentUpdateSolrClient(
((HttpSolrClient) clients.get(0)).getBaseURL(), 10, 2)) {
concurrentClient.setConnectionTimeout(120000);
((HttpSolrClient) clients.get(0)).getBaseURL(), 10, 2, 120000)) {
for (int i = 0; i < cnt; i++) {
index_specific(concurrentClient, id, docId++, "text_t", "some text so that it not's negligent work to parse this doc, even though it's still a pretty short doc");
}

View File

@ -58,9 +58,9 @@ class FullThrottleStoppableIndexingThread extends StoppableIndexingThread {
.withHttpClient(httpClient)
.withQueueSize(8)
.withThreadCount(2)
.withConnectionTimeout(10000)
.withSocketTimeout(clientSoTimeout)
.build();
cusc.setConnectionTimeout(10000);
cusc.setSoTimeout(clientSoTimeout);
}
@Override

View File

@ -96,11 +96,11 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
CloudSolrClient client = new CloudSolrClient.Builder()
.withZkHost(zkServer.getZkAddress())
.sendDirectUpdatesToAnyShardReplica()
.withConnectionTimeout(30000)
.withSocketTimeout(60000)
.build();
client.setParallelUpdates(random().nextBoolean());
if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
client.getLbClient().setConnectionTimeout(30000);
client.getLbClient().setSoTimeout(60000);
return client;
}

View File

@ -80,13 +80,7 @@ public class MigrateRouteKeyTest extends SolrCloudTestCase {
}
protected void invokeCollectionMigration(CollectionAdminRequest.AsyncCollectionAdminRequest request) throws IOException, SolrServerException, InterruptedException {
if (random().nextBoolean()) {
cluster.getSolrClient().setSoTimeout(60000); // can take a while
request.process(cluster.getSolrClient());
}
else {
request.processAndWait(cluster.getSolrClient(), 60000);
}
request.processAndWait(cluster.getSolrClient(), 60000);
}
@Test

View File

@ -932,9 +932,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
.getBaseURL();
baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl)) {
baseServer.setConnectionTimeout(30000);
baseServer.setSoTimeout(60000 * 5);
try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl, 30000, 60000 * 5)) {
baseServer.request(request);
}
}
@ -1007,15 +1005,13 @@ public class ShardSplitTest extends BasicDistributedZkTest {
@Override
protected SolrClient createNewSolrClient(String collection, String baseUrl) {
HttpSolrClient client = (HttpSolrClient) super.createNewSolrClient(collection, baseUrl);
client.setSoTimeout(5 * 60 * 1000);
HttpSolrClient client = (HttpSolrClient) super.createNewSolrClient(collection, baseUrl, DEFAULT_CONNECTION_TIMEOUT, 5 * 60 * 1000);
return client;
}
@Override
protected SolrClient createNewSolrClient(int port) {
HttpSolrClient client = (HttpSolrClient) super.createNewSolrClient(port);
client.setSoTimeout(5 * 60 * 1000);
HttpSolrClient client = (HttpSolrClient) super.createNewSolrClient(port, DEFAULT_CONNECTION_TIMEOUT, 5 * 60 * 1000);
return client;
}

View File

@ -108,9 +108,8 @@ public class SyncSliceTest extends AbstractFullDistribZkTestBase {
.getBaseURL();
baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
try (HttpSolrClient baseClient = getHttpSolrClient(baseUrl)) {
// we only set the connect timeout, not so timeout
baseClient.setConnectionTimeout(30000);
// we only set the connect timeout, not so timeout
try (HttpSolrClient baseClient = getHttpSolrClient(baseUrl, 30000)) {
baseClient.request(request);
}

View File

@ -92,9 +92,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
assertEquals(1, replicas.size());
String baseUrl = replicas.iterator().next().getStr(ZkStateReader.BASE_URL_PROP);
if (!baseUrl.endsWith("/")) baseUrl += "/";
try (HttpSolrClient client = getHttpSolrClient(baseUrl + "a1x2")) {
client.setSoTimeout(5000);
client.setConnectionTimeout(2000);
try (HttpSolrClient client = getHttpSolrClient(baseUrl + "a1x2", 2000, 5000)) {
log.info("Making requests to " + baseUrl + "a1x2");
for (int i = 0; i < 10; i++) {
@ -170,9 +168,7 @@ public class TestRandomRequestDistribution extends AbstractFullDistribZkTestBase
if (!baseUrl.endsWith("/")) baseUrl += "/";
String path = baseUrl + "football";
log.info("Firing queries against path=" + path);
try (HttpSolrClient client = getHttpSolrClient(path)) {
client.setSoTimeout(5000);
client.setConnectionTimeout(2000);
try (HttpSolrClient client = getHttpSolrClient(path, 2000, 5000)) {
SolrCore leaderCore = null;
for (JettySolrRunner jetty : jettys) {

View File

@ -189,8 +189,7 @@ public class TestRequestStatusCollectionAPI extends BasicDistributedZkTest {
String baseUrl = ((HttpSolrClient) shardToJetty.get(SHARD1).get(0).client.solrClient).getBaseURL();
baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl)) {
baseServer.setConnectionTimeout(15000);
try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl, 15000)) {
return baseServer.request(request);
}

View File

@ -221,8 +221,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
// so that we start with some versions when we reload...
DirectUpdateHandler2.commitOnClose = false;
try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3")) {
addClient.setConnectionTimeout(30000);
try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3", 30000)) {
// add a few docs
for (int x = 20; x < 100; x++) {
@ -235,9 +234,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
//collectionClient.commit();
// unload the leader
try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getBaseUrl())) {
collectionClient.setConnectionTimeout(15000);
collectionClient.setSoTimeout(30000);
try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getBaseUrl(), 15000, 30000)) {
Unload unloadCmd = new Unload(false);
unloadCmd.setCoreName(leaderProps.getCoreName());
@ -259,9 +256,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
// ensure there is a leader
zkStateReader.getLeaderRetry("unloadcollection", "shard1", 15000);
try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(1).getBaseUrl() + "/unloadcollection_shard1_replica2")) {
addClient.setConnectionTimeout(30000);
addClient.setSoTimeout(90000);
try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(1).getBaseUrl() + "/unloadcollection_shard1_replica2", 30000, 90000)) {
// add a few docs while the leader is down
for (int x = 101; x < 200; x++) {
@ -281,9 +276,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
// unload the leader again
leaderProps = getLeaderUrlFromZk("unloadcollection", "shard1");
try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getBaseUrl())) {
collectionClient.setConnectionTimeout(15000);
collectionClient.setSoTimeout(30000);
try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getBaseUrl(), 15000, 30000)) {
Unload unloadCmd = new Unload(false);
unloadCmd.setCoreName(leaderProps.getCoreName());
@ -313,27 +306,21 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
long found1, found3;
try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(1).getBaseUrl() + "/unloadcollection_shard1_replica2")) {
adminClient.setConnectionTimeout(15000);
adminClient.setSoTimeout(30000);
try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(1).getBaseUrl() + "/unloadcollection_shard1_replica2", 15000, 30000)) {
adminClient.commit();
SolrQuery q = new SolrQuery("*:*");
q.set("distrib", false);
found1 = adminClient.query(q).getResults().getNumFound();
}
try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3")) {
adminClient.setConnectionTimeout(15000);
adminClient.setSoTimeout(30000);
try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3", 15000, 30000)) {
adminClient.commit();
SolrQuery q = new SolrQuery("*:*");
q.set("distrib", false);
found3 = adminClient.query(q).getResults().getNumFound();
}
try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(3).getBaseUrl() + "/unloadcollection_shard1_replica4")) {
adminClient.setConnectionTimeout(15000);
adminClient.setSoTimeout(30000);
try (HttpSolrClient adminClient = getHttpSolrClient(jettys.get(3).getBaseUrl() + "/unloadcollection_shard1_replica4", 15000, 30000)) {
adminClient.commit();
SolrQuery q = new SolrQuery("*:*");
q.set("distrib", false);
@ -348,9 +335,7 @@ public class UnloadDistributedZkTest extends BasicDistributedZkTest {
private void testUnloadLotsOfCores() throws Exception {
JettySolrRunner jetty = jettys.get(0);
try (final HttpSolrClient adminClient = (HttpSolrClient) jetty.newClient()) {
adminClient.setConnectionTimeout(15000);
adminClient.setSoTimeout(60000);
try (final HttpSolrClient adminClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
int numReplicas = atLeast(3);
ThreadPoolExecutor executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),

View File

@ -180,7 +180,7 @@ public class StressHdfsTest extends BasicDistributedZkTest {
int i = 0;
for (SolrClient client : clients) {
try (HttpSolrClient c = getHttpSolrClient(getBaseUrl(client) + "/" + DELETE_DATA_DIR_COLLECTION)) {
try (HttpSolrClient c = getHttpSolrClient(getBaseUrl(client) + "/" + DELETE_DATA_DIR_COLLECTION, 30000)) {
int docCnt = random().nextInt(1000) + 1;
for (int j = 0; j < docCnt; j++) {
c.add(getDoc("id", i++, "txt_t", "just some random text for a doc"));
@ -192,7 +192,6 @@ public class StressHdfsTest extends BasicDistributedZkTest {
c.commit(true, true, true);
}
c.setConnectionTimeout(30000);
NamedList<Object> response = c.query(
new SolrQuery().setRequestHandler("/admin/system")).getResponse();
NamedList<Object> coreInfo = (NamedList<Object>) response.get("core");

View File

@ -136,15 +136,11 @@ public class OpenCloseCoreStressTest extends SolrTestCaseJ4 {
// Mostly to keep annoying logging messages from being sent out all the time.
for (int idx = 0; idx < indexingThreads; ++idx) {
HttpSolrClient client = getHttpSolrClient(url);
client.setConnectionTimeout(30000);
client.setSoTimeout(60000);
HttpSolrClient client = getHttpSolrClient(url, 30000, 60000);
indexingClients.add(client);
}
for (int idx = 0; idx < queryThreads; ++idx) {
HttpSolrClient client = getHttpSolrClient(url);
client.setConnectionTimeout(30000);
client.setSoTimeout(30000);
HttpSolrClient client = getHttpSolrClient(url, 30000, 30000);
queryingClients.add(client);
}

View File

@ -164,9 +164,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
try {
// setup the client...
final String baseUrl = buildUrl(port) + "/" + DEFAULT_TEST_CORENAME;
HttpSolrClient client = getHttpSolrClient(baseUrl);
client.setConnectionTimeout(15000);
client.setSoTimeout(90000);
HttpSolrClient client = getHttpSolrClient(baseUrl, 15000, 90000);
return client;
}
catch (Exception ex) {

View File

@ -83,9 +83,7 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
try {
// setup the client...
final String baseUrl = buildUrl(port, context);
HttpSolrClient client = getHttpSolrClient(baseUrl);
client.setConnectionTimeout(15000);
client.setSoTimeout(60000);
HttpSolrClient client = getHttpSolrClient(baseUrl, 15000, 60000);
return client;
}
catch (Exception ex) {

View File

@ -71,9 +71,7 @@ public class TestRestoreCore extends SolrJettyTestBase {
try {
// setup the client...
final String baseUrl = buildUrl(port, context);
HttpSolrClient client = getHttpSolrClient(baseUrl);
client.setConnectionTimeout(15000);
client.setSoTimeout(60000);
HttpSolrClient client = getHttpSolrClient(baseUrl, 15000, 60000);
return client;
}
catch (Exception ex) {

View File

@ -259,18 +259,14 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
JettySolrRunner runner = new JettySolrRunner(solrHomeDirectory.getAbsolutePath(), buildJettyConfig("/solr"));
runner.start();
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
SolrInputDocument doc = new SolrInputDocument();
doc.addField("id", "123");
client.add(doc);
client.commit();
}
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString())) {
client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString(), DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
CoreAdminRequest.Unload req = new CoreAdminRequest.Unload(false);
req.setDeleteInstanceDir(true);
req.setCoreName("corex");
@ -295,25 +291,22 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
JettySolrRunner runner = new JettySolrRunner(solrHomeDirectory.getAbsolutePath(), buildJettyConfig("/solr"));
runner.start();
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT,
DEFAULT_CONNECTION_TIMEOUT)) {
SolrInputDocument doc = new SolrInputDocument();
doc.addField("id", "123");
client.add(doc);
client.commit();
}
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT,
DEFAULT_CONNECTION_TIMEOUT)) {
QueryResponse result = client.query(new SolrQuery("id:*"));
assertEquals(1,result.getResults().getNumFound());
}
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString())) {
client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString(), DEFAULT_CONNECTION_TIMEOUT,
DEFAULT_CONNECTION_TIMEOUT)) {
CoreAdminRequest.Unload req = new CoreAdminRequest.Unload(false);
req.setDeleteInstanceDir(false);//random().nextBoolean());
req.setCoreName("corex");
@ -321,9 +314,8 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
}
HttpSolrClient.RemoteSolrException rse = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT * 1000);
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT,
DEFAULT_CONNECTION_TIMEOUT * 1000)) {
client.query(new SolrQuery("id:*"));
} finally {
runner.stop();
@ -344,9 +336,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
JettySolrRunner runner = new JettySolrRunner(solrHomeDirectory.getAbsolutePath(), buildJettyConfig("/solr"));
runner.start();
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex")) {
client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl() + "/corex", DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
SolrInputDocument doc = new SolrInputDocument();
doc.addField("id", "123");
client.add(doc);
@ -365,9 +355,7 @@ public class CoreAdminHandlerTest extends SolrTestCaseJ4 {
String top = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
FileUtils.copyFile(new File(top, "bad-error-solrconfig.xml"), new File(subHome, "solrconfig.xml"));
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString())) {
client.setConnectionTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
client.setSoTimeout(SolrTestCaseJ4.DEFAULT_CONNECTION_TIMEOUT);
try (HttpSolrClient client = getHttpSolrClient(runner.getBaseUrl().toString(), DEFAULT_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT)) {
try {
CoreAdminRequest.reloadCore("corex", client);
} catch (Exception e) {

View File

@ -195,6 +195,10 @@ public class CloudSolrClient extends SolrClient {
this.retryExpiryTime = TimeUnit.NANOSECONDS.convert(secs, TimeUnit.SECONDS);
}
/**
* @deprecated since 7.0 Use {@link Builder} methods instead.
*/
@Deprecated
public void setSoTimeout(int timeout) {
lbClient.setSoTimeout(timeout);
}
@ -264,14 +268,29 @@ public class CloudSolrClient extends SolrClient {
}
this.clientIsInternal = builder.httpClient == null;
this.shutdownLBHttpSolrServer = builder.loadBalancedSolrClient == null;
if(builder.lbClientBuilder != null) builder.loadBalancedSolrClient = builder.lbClientBuilder.build();
if(builder.lbClientBuilder != null) {
propagateLBClientConfigOptions(builder);
builder.loadBalancedSolrClient = builder.lbClientBuilder.build();
}
if(builder.loadBalancedSolrClient != null) builder.httpClient = builder.loadBalancedSolrClient.getHttpClient();
this.myClient = (builder.httpClient == null) ? HttpClientUtil.createClient(null) : builder.httpClient;
if (builder.loadBalancedSolrClient == null) builder.loadBalancedSolrClient = createLBHttpSolrClient(myClient);
if (builder.loadBalancedSolrClient == null) builder.loadBalancedSolrClient = createLBHttpSolrClient(builder, myClient);
this.lbClient = builder.loadBalancedSolrClient;
this.updatesToLeaders = builder.shardLeadersOnly;
this.directUpdatesToLeadersOnly = builder.directUpdatesToLeadersOnly;
}
private void propagateLBClientConfigOptions(Builder builder) {
final LBHttpSolrClient.Builder lbBuilder = builder.lbClientBuilder;
if (builder.connectionTimeoutMillis != null) {
lbBuilder.withConnectionTimeout(builder.connectionTimeoutMillis);
}
if (builder.socketTimeoutMillis != null) {
lbBuilder.withSocketTimeout(builder.socketTimeoutMillis);
}
}
/**Sets the cache ttl for DocCollection Objects cached . This is only applicable for collections which are persisted outside of clusterstate.json
* @param seconds ttl value in seconds
@ -1292,6 +1311,10 @@ public class CloudSolrClient extends SolrClient {
return results;
}
/**
* @deprecated since 7.0 Use {@link Builder} methods instead.
*/
@Deprecated
public void setConnectionTimeout(int timeout) {
this.lbClient.setConnectionTimeout(timeout);
}
@ -1325,10 +1348,16 @@ public class CloudSolrClient extends SolrClient {
return true;
}
private static LBHttpSolrClient createLBHttpSolrClient(HttpClient httpClient) {
final LBHttpSolrClient lbClient = new LBHttpSolrClient.Builder()
.withHttpClient(httpClient)
.build();
private static LBHttpSolrClient createLBHttpSolrClient(Builder cloudSolrClientBuilder, HttpClient httpClient) {
final LBHttpSolrClient.Builder lbBuilder = new LBHttpSolrClient.Builder();
lbBuilder.withHttpClient(httpClient);
if (cloudSolrClientBuilder.connectionTimeoutMillis != null) {
lbBuilder.withConnectionTimeout(cloudSolrClientBuilder.connectionTimeoutMillis);
}
if (cloudSolrClientBuilder.socketTimeoutMillis != null) {
lbBuilder.withSocketTimeout(cloudSolrClientBuilder.socketTimeoutMillis);
}
final LBHttpSolrClient lbClient = lbBuilder.build();
lbClient.setRequestWriter(new BinaryRequestWriter());
lbClient.setParser(new BinaryResponseParser());
@ -1348,6 +1377,8 @@ public class CloudSolrClient extends SolrClient {
protected boolean shardLeadersOnly;
protected boolean directUpdatesToLeadersOnly;
protected ClusterStateProvider stateProvider;
protected Integer connectionTimeoutMillis;
protected Integer socketTimeoutMillis;
public Builder() {
@ -1483,6 +1514,30 @@ public class CloudSolrClient extends SolrClient {
this.stateProvider = stateProvider;
return this;
}
/**
* Tells {@link Builder} that created clients should obey the following timeout when connecting to Solr servers.
*/
public Builder withConnectionTimeout(int connectionTimeoutMillis) {
if (connectionTimeoutMillis <= 0) {
throw new IllegalArgumentException("connectionTimeoutMillis must be a positive integer.");
}
this.connectionTimeoutMillis = connectionTimeoutMillis;
return this;
}
/**
* Tells {@link Builder} that created clients should set the following read timeout on all sockets.
*/
public Builder withSocketTimeout(int socketTimeoutMillis) {
if (socketTimeoutMillis <= 0) {
throw new IllegalArgumentException("socketTimeoutMillis must be a positive integer.");
}
this.socketTimeoutMillis = socketTimeoutMillis;
return this;
}
/**
* Create a {@link CloudSolrClient} based on the provided configuration.

View File

@ -129,6 +129,8 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
this.threadCount = builder.threadCount;
this.runners = new LinkedList<>();
this.streamDeletes = builder.streamDeletes;
this.connectionTimeout = builder.connectionTimeoutMillis;
this.soTimeout = builder.socketTimeoutMillis;
if (builder.executorService != null) {
this.scheduler = builder.executorService;
@ -703,6 +705,10 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
}
}
/**
* @deprecated since 7.0 Use {@link Builder} methods instead.
*/
@Deprecated
public void setConnectionTimeout(int timeout) {
this.connectionTimeout = timeout;
}
@ -710,7 +716,10 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
/**
* set soTimeout (read timeout) on the underlying HttpConnectionManager. This is desirable for queries, but probably
* not for indexing.
*
* @deprecated since 7.0 Use {@link Builder} methods instead.
*/
@Deprecated
public void setSoTimeout(int timeout) {
this.soTimeout = timeout;
}
@ -768,6 +777,8 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
protected int threadCount;
protected ExecutorService executorService;
protected boolean streamDeletes;
protected Integer connectionTimeoutMillis;
protected Integer socketTimeoutMillis;
/**
* Create a Builder object, based on the provided Solr URL.
@ -849,6 +860,31 @@ public class ConcurrentUpdateSolrClient extends SolrClient {
return this;
}
/**
* Tells {@link Builder} that created clients should obey the following timeout when connecting to Solr servers.
*/
public Builder withConnectionTimeout(int connectionTimeoutMillis) {
if (connectionTimeoutMillis <= 0) {
throw new IllegalArgumentException("connectionTimeoutMillis must be a positive integer.");
}
this.connectionTimeoutMillis = connectionTimeoutMillis;
return this;
}
/**
* Tells {@link Builder} that created clients should set the following read timeout on all sockets.
*/
public Builder withSocketTimeout(int socketTimeoutMillis) {
if (socketTimeoutMillis <= 0) {
throw new IllegalArgumentException("socketTimeoutMillis must be a positive integer.");
}
this.socketTimeoutMillis = socketTimeoutMillis;
return this;
}
/**
* Create a {@link ConcurrentUpdateSolrClient} based on the provided configuration options.
*/

View File

@ -199,6 +199,8 @@ public class HttpSolrClient extends SolrClient {
this.parser = builder.responseParser;
this.invariantParams = builder.invariantParams;
this.connectionTimeout = builder.connectionTimeoutMillis;
this.soTimeout = builder.socketTimeoutMillis;
}
public Set<String> getQueryParams() {
@ -713,7 +715,10 @@ public class HttpSolrClient extends SolrClient {
*
* @param timeout
* Timeout in milliseconds
**/
*
* @deprecated since 7.0 Use {@link Builder} methods instead.
*/
@Deprecated
public void setConnectionTimeout(int timeout) {
this.connectionTimeout = timeout;
}
@ -724,7 +729,10 @@ public class HttpSolrClient extends SolrClient {
*
* @param timeout
* Timeout in milliseconds
**/
*
s * @deprecated since 7.0 Use {@link Builder} methods instead.
*/
@Deprecated
public void setSoTimeout(int timeout) {
this.soTimeout = timeout;
}
@ -825,6 +833,8 @@ public class HttpSolrClient extends SolrClient {
protected ResponseParser responseParser;
protected boolean compression;
protected ModifiableSolrParams invariantParams = new ModifiableSolrParams();
protected Integer connectionTimeoutMillis;
protected Integer socketTimeoutMillis;
public Builder() {
this.responseParser = new BinaryResponseParser();
@ -931,6 +941,30 @@ public class HttpSolrClient extends SolrClient {
this.invariantParams.add(params);
return this;
}
/**
* Tells {@link Builder} that created clients should obey the following timeout when connecting to Solr servers.
*/
public Builder withConnectionTimeout(int connectionTimeoutMillis) {
if (connectionTimeoutMillis <= 0) {
throw new IllegalArgumentException("connectionTimeoutMillis must be a positive integer.");
}
this.connectionTimeoutMillis = connectionTimeoutMillis;
return this;
}
/**
* Tells {@link Builder} that created clients should set the following read timeout on all sockets.
*/
public Builder withSocketTimeout(int socketTimeoutMillis) {
if (socketTimeoutMillis <= 0) {
throw new IllegalArgumentException("socketTimeoutMillis must be a positive integer.");
}
this.socketTimeoutMillis = socketTimeoutMillis;
return this;
}
/**
* Create a {@link HttpSolrClient} based on provided configuration.

View File

@ -268,7 +268,8 @@ public class LBHttpSolrClient extends SolrClient {
this.clientIsInternal = builder.httpClient == null;
this.httpSolrClientBuilder = builder.httpSolrClientBuilder;
this.httpClient = builder.httpClient == null ? constructClient(builder.baseSolrUrls.toArray(new String[builder.baseSolrUrls.size()])) : builder.httpClient;
this.connectionTimeout = builder.connectionTimeoutMillis;
this.soTimeout = builder.socketTimeoutMillis;
this.parser = builder.responseParser;
if (! builder.baseSolrUrls.isEmpty()) {
@ -316,16 +317,28 @@ public class LBHttpSolrClient extends SolrClient {
HttpSolrClient client;
if (httpSolrClientBuilder != null) {
synchronized (this) {
client = httpSolrClientBuilder
httpSolrClientBuilder
.withBaseSolrUrl(server)
.withHttpClient(httpClient)
.build();
.withHttpClient(httpClient);
if (connectionTimeout != null) {
httpSolrClientBuilder.withConnectionTimeout(connectionTimeout);
}
if (soTimeout != null) {
httpSolrClientBuilder.withSocketTimeout(soTimeout);
}
client = httpSolrClientBuilder.build();
}
} else {
client = new HttpSolrClient.Builder(server)
final HttpSolrClient.Builder clientBuilder = new HttpSolrClient.Builder(server)
.withHttpClient(httpClient)
.withResponseParser(parser)
.build();
.withResponseParser(parser);
if (connectionTimeout != null) {
clientBuilder.withConnectionTimeout(connectionTimeout);
}
if (soTimeout != null) {
clientBuilder.withSocketTimeout(soTimeout);
}
client = clientBuilder.build();
}
if (requestWriter != null) {
client.setRequestWriter(requestWriter);
@ -558,6 +571,10 @@ public class LBHttpSolrClient extends SolrClient {
return null;
}
/**
* @deprecated since 7.0 Use {@link Builder} methods instead.
*/
@Deprecated
public void setConnectionTimeout(int timeout) {
this.connectionTimeout = timeout;
synchronized (aliveServers) {
@ -575,7 +592,10 @@ public class LBHttpSolrClient extends SolrClient {
/**
* set soTimeout (read timeout) on the underlying HttpConnectionManager. This is desirable for queries, but probably
* not for indexing.
*
* @deprecated since 7.0 Use {@link Builder} methods instead.
*/
@Deprecated
public void setSoTimeout(int timeout) {
this.soTimeout = timeout;
synchronized (aliveServers) {
@ -866,6 +886,8 @@ public class LBHttpSolrClient extends SolrClient {
protected HttpClient httpClient;
protected ResponseParser responseParser;
protected HttpSolrClient.Builder httpSolrClientBuilder;
protected Integer connectionTimeoutMillis;
protected Integer socketTimeoutMillis;
public Builder() {
this.baseSolrUrls = new ArrayList<>();
@ -958,6 +980,30 @@ public class LBHttpSolrClient extends SolrClient {
this.httpSolrClientBuilder = builder;
return this;
}
/**
* Tells {@link Builder} that created clients should obey the following timeout when connecting to Solr servers.
*/
public Builder withConnectionTimeout(int connectionTimeoutMillis) {
if (connectionTimeoutMillis <= 0) {
throw new IllegalArgumentException("connectionTimeoutMillis must be a positive integer.");
}
this.connectionTimeoutMillis = connectionTimeoutMillis;
return this;
}
/**
* Tells {@link Builder} that created clients should set the following read timeout on all sockets.
*/
public Builder withSocketTimeout(int socketTimeoutMillis) {
if (socketTimeoutMillis <= 0) {
throw new IllegalArgumentException("socketTimeoutMillis must be a positive integer.");
}
this.socketTimeoutMillis = socketTimeoutMillis;
return this;
}
/**
* Create a {@link HttpSolrClient} based on provided configuration.

View File

@ -40,8 +40,7 @@ public class SolrExampleBinaryTest extends SolrExampleTests {
try {
// setup the server...
String url = jetty.getBaseUrl().toString() + "/collection1";
HttpSolrClient client = getHttpSolrClient(url);
client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT);
client.setUseMultiPartPost(random().nextBoolean());
// where the magic happens

View File

@ -37,9 +37,8 @@ public class SolrExampleXMLTest extends SolrExampleTests {
public SolrClient createNewSolrClient() {
try {
String url = jetty.getBaseUrl().toString() + "/collection1";
HttpSolrClient client = getHttpSolrClient(url);
HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT);
client.setUseMultiPartPost(random().nextBoolean());
client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
client.setParser(new XMLResponseParser());
client.setRequestWriter(new RequestWriter());
return client;

View File

@ -40,8 +40,7 @@ public class SolrExceptionTest extends LuceneTestCase {
// switched to a local address to avoid going out on the net, ns lookup issues, etc.
// set a 1ms timeout to let the connection fail faster.
httpClient = HttpClientUtil.createClient(null);
try (HttpSolrClient client = getHttpSolrClient("http://[ff01::114]:11235/solr/", httpClient)) {
client.setConnectionTimeout(1);
try (HttpSolrClient client = getHttpSolrClient("http://[ff01::114]:11235/solr/", httpClient, 1)) {
SolrQuery query = new SolrQuery("test123");
client.query(query);
}

View File

@ -133,8 +133,7 @@ public class SolrSchemalessExampleTest extends SolrExampleTestsBase {
try {
// setup the server...
String url = jetty.getBaseUrl().toString() + "/collection1";
HttpSolrClient client = getHttpSolrClient(url);
client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT);
client.setUseMultiPartPost(random().nextBoolean());
if (random().nextBoolean()) {

View File

@ -206,9 +206,7 @@ public class TestLBHttpSolrClient extends SolrTestCaseJ4 {
CloseableHttpClient myHttpClient = HttpClientUtil.createClient(null);
try {
LBHttpSolrClient client = getLBHttpSolrClient(myHttpClient, s);
client.setConnectionTimeout(500);
client.setSoTimeout(500);
LBHttpSolrClient client = getLBHttpSolrClient(myHttpClient, 500, 500, s);
client.setAliveCheckInterval(500);
// Kill a server and test again

View File

@ -209,8 +209,7 @@ public class BasicHttpSolrClientTest extends SolrJettyTestBase {
public void testTimeout() throws Exception {
SolrQuery q = new SolrQuery("*:*");
try(HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/slow/foo")) {
client.setSoTimeout(2000);
try(HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/slow/foo", DEFAULT_CONNECTION_TIMEOUT, 2000)) {
client.query(q, METHOD.GET);
fail("No exception thrown.");
} catch (SolrServerException e) {

View File

@ -463,9 +463,7 @@ public class CloudSolrClientTest extends SolrCloudTestCase {
SolrServerException, IOException {
NamedList<Object> resp;
try (HttpSolrClient client = getHttpSolrClient(baseUrl + "/"+ collectionName)) {
client.setConnectionTimeout(15000);
client.setSoTimeout(60000);
try (HttpSolrClient client = getHttpSolrClient(baseUrl + "/"+ collectionName, 15000, 60000)) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set("qt", "/admin/mbeans");
params.set("stats", "true");

View File

@ -70,8 +70,7 @@ public class HttpSolrClientConPoolTest extends SolrJettyTestBase {
fooUrl = jetty.getBaseUrl().toString() + "/" + "collection1";
CloseableHttpClient httpClient = HttpClientUtil.createClient(new ModifiableSolrParams(), pool,
false /* let client shutdown it*/);
client1 = getHttpSolrClient(fooUrl, httpClient);
client1.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
client1 = getHttpSolrClient(fooUrl, httpClient, DEFAULT_CONNECTION_TIMEOUT);
}
final String barUrl = yetty.getBaseUrl().toString() + "/" + "collection1";

View File

@ -152,8 +152,7 @@ abstract public class SolrJettyTestBase extends SolrTestCaseJ4
try {
// setup the client...
String url = jetty.getBaseUrl().toString() + "/" + "collection1";
HttpSolrClient client = getHttpSolrClient(url);
client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT);
return client;
}
catch( Exception ex ) {

View File

@ -2295,6 +2295,50 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
* {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
*/
public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly, int socketTimeoutMillis) {
if (shardLeadersOnly) {
return new CloudSolrClientBuilder()
.withZkHost(zkHost)
.sendUpdatesOnlyToShardLeaders()
.withSocketTimeout(socketTimeoutMillis)
.build();
}
return new CloudSolrClientBuilder()
.withZkHost(zkHost)
.sendUpdatesToAllReplicasInShard()
.withSocketTimeout(socketTimeoutMillis)
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
* {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
*/
public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly, int connectionTimeoutMillis, int socketTimeoutMillis) {
if (shardLeadersOnly) {
return new CloudSolrClientBuilder()
.withZkHost(zkHost)
.sendUpdatesOnlyToShardLeaders()
.withConnectionTimeout(connectionTimeoutMillis)
.withSocketTimeout(socketTimeoutMillis)
.build();
}
return new CloudSolrClientBuilder()
.withZkHost(zkHost)
.sendUpdatesToAllReplicasInShard()
.withConnectionTimeout(connectionTimeoutMillis)
.withSocketTimeout(socketTimeoutMillis)
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
@ -2315,6 +2359,31 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
* {@link org.apache.solr.client.solrj.impl.CloudSolrClient.Builder} class directly
*/
public static CloudSolrClient getCloudSolrClient(String zkHost, boolean shardLeadersOnly, HttpClient httpClient,
int connectionTimeoutMillis, int socketTimeoutMillis) {
if (shardLeadersOnly) {
return new CloudSolrClientBuilder()
.withZkHost(zkHost)
.withHttpClient(httpClient)
.sendUpdatesOnlyToShardLeaders()
.withConnectionTimeout(connectionTimeoutMillis)
.withSocketTimeout(socketTimeoutMillis)
.build();
}
return new CloudSolrClientBuilder()
.withZkHost(zkHost)
.withHttpClient(httpClient)
.sendUpdatesToAllReplicasInShard()
.withConnectionTimeout(connectionTimeoutMillis)
.withSocketTimeout(socketTimeoutMillis)
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
@ -2327,6 +2396,19 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
* {@link org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient.Builder} class directly
*/
public static ConcurrentUpdateSolrClient getConcurrentUpdateSolrClient(String baseSolrUrl, int queueSize, int threadCount, int connectionTimeoutMillis) {
return new ConcurrentUpdateSolrClient.Builder(baseSolrUrl)
.withQueueSize(queueSize)
.withThreadCount(threadCount)
.withConnectionTimeout(connectionTimeoutMillis)
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
@ -2352,6 +2434,21 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
* {@link org.apache.solr.client.solrj.impl.LBHttpSolrClient.Builder} class directly
*/
public static LBHttpSolrClient getLBHttpSolrClient(HttpClient client, int connectionTimeoutMillis,
int socketTimeoutMillis, String... solrUrls) {
return new LBHttpSolrClient.Builder()
.withHttpClient(client)
.withBaseSolrUrls(solrUrls)
.withConnectionTimeout(connectionTimeoutMillis)
.withSocketTimeout(socketTimeoutMillis)
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
@ -2398,6 +2495,18 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
.withHttpClient(httpClient)
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
* {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
*/
public static HttpSolrClient getHttpSolrClient(String url, HttpClient httpClient, int connectionTimeoutMillis) {
return new Builder(url)
.withHttpClient(httpClient)
.withConnectionTimeout(connectionTimeoutMillis)
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
@ -2408,6 +2517,29 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
return new Builder(url)
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
* {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
*/
public static HttpSolrClient getHttpSolrClient(String url, int connectionTimeoutMillis) {
return new Builder(url)
.withConnectionTimeout(connectionTimeoutMillis)
.build();
}
/**
* This method <i>may</i> randomize unspecified aspects of the resulting SolrClient.
* Tests that do not wish to have any randomized behavior should use the
* {@link org.apache.solr.client.solrj.impl.HttpSolrClient.Builder} class directly
*/
public static HttpSolrClient getHttpSolrClient(String url, int connectionTimeoutMillis, int socketTimeoutMillis) {
return new Builder(url)
.withConnectionTimeout(connectionTimeoutMillis)
.withSocketTimeout(socketTimeoutMillis)
.build();
}
/**
* Returns a randomly generated Date in the appropriate Solr external (input) format

View File

@ -295,11 +295,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
}
protected CloudSolrClient createCloudClient(String defaultCollection) {
CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean());
CloudSolrClient client = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean(), 30000, 60000);
client.setParallelUpdates(random().nextBoolean());
if (defaultCollection != null) client.setDefaultCollection(defaultCollection);
client.getLbClient().setConnectionTimeout(30000);
client.getLbClient().setSoTimeout(60000);
return client;
}
@ -1708,15 +1706,29 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
protected SolrClient createNewSolrClient(int port) {
return createNewSolrClient(DEFAULT_COLLECTION, port);
}
protected SolrClient createNewSolrClient(int port, int connectionTimeoutMillis, int socketTimeoutMillis) {
return createNewSolrClient(DEFAULT_COLLECTION, port, connectionTimeoutMillis, socketTimeoutMillis);
}
protected SolrClient createNewSolrClient(String coreName, int port) {
try {
// setup the server...
String baseUrl = buildUrl(port);
String url = baseUrl + (baseUrl.endsWith("/") ? "" : "/") + coreName;
HttpSolrClient client = getHttpSolrClient(url);
client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
client.setSoTimeout(60000);
HttpSolrClient client = getHttpSolrClient(url, DEFAULT_CONNECTION_TIMEOUT, 60000);
return client;
} catch (Exception ex) {
throw new RuntimeException(ex);
}
}
protected SolrClient createNewSolrClient(String coreName, int port, int connectionTimeoutMillis, int socketTimeoutMillis) {
try {
// setup the server...
String baseUrl = buildUrl(port);
String url = baseUrl + (baseUrl.endsWith("/") ? "" : "/") + coreName;
HttpSolrClient client = getHttpSolrClient(url, connectionTimeoutMillis, socketTimeoutMillis);
return client;
} catch (Exception ex) {
throw new RuntimeException(ex);
@ -1726,8 +1738,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
protected SolrClient createNewSolrClient(String collection, String baseUrl) {
try {
// setup the server...
HttpSolrClient client = getHttpSolrClient(baseUrl + "/" + collection);
client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
HttpSolrClient client = getHttpSolrClient(baseUrl + "/" + collection, DEFAULT_CONNECTION_TIMEOUT);
return client;
}
catch (Exception ex) {
@ -1811,9 +1822,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
boolean updatesToLeaders = random().nextBoolean();
boolean parallelUpdates = random().nextBoolean();
commonCloudSolrClient = getCloudSolrClient(zkServer.getZkAddress(),
updatesToLeaders);
commonCloudSolrClient.getLbClient().setConnectionTimeout(5000);
commonCloudSolrClient.getLbClient().setSoTimeout(120000);
updatesToLeaders, 5000, 120000);
commonCloudSolrClient.setParallelUpdates(parallelUpdates);
commonCloudSolrClient.setDefaultCollection(DEFAULT_COLLECTION);
commonCloudSolrClient.connect();