Skip to content

Commit 201c83e

Browse files
authored
Correctly handling EsHadoopException in TransportPool.validate() (#2150)
1 parent 35982db commit 201c83e

File tree

2 files changed

+84
-0
lines changed

2 files changed

+84
-0
lines changed

mr/src/itest/java/org/elasticsearch/hadoop/rest/pooling/AbstractTransportPoolTest.java

Lines changed: 76 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
package org.elasticsearch.hadoop.rest.pooling;
2121

2222
import org.elasticsearch.hadoop.cfg.Settings;
23+
import org.elasticsearch.hadoop.mr.security.HadoopUserProvider;
2324
import org.elasticsearch.hadoop.rest.Transport;
2425
import org.elasticsearch.hadoop.security.SecureSettings;
2526
import org.elasticsearch.hadoop.util.SettingsUtils;
@@ -29,6 +30,7 @@
2930
import java.util.UUID;
3031

3132
import static org.elasticsearch.hadoop.cfg.ConfigurationOptions.ES_NET_TRANSPORT_POOLING_EXPIRATION_TIMEOUT;
33+
import static org.elasticsearch.hadoop.cfg.ConfigurationOptions.ES_SECURITY_USER_PROVIDER_CLASS;
3234

3335
public class AbstractTransportPoolTest {
3436

@@ -77,4 +79,78 @@ public void removeOldConnections() throws Exception {
7779
}
7880
}
7981

82+
83+
@Test
84+
public void testValidationWithNoUserProvider() throws Exception {
85+
/*
86+
* In this test, we make sure that validating PooledTransports does not throw an exception if the resource is bad. Instead, the
87+
* PooledTransport is replaced.
88+
*/
89+
Settings settings = new TestSettings();
90+
settings.setProperty(ES_NET_TRANSPORT_POOLING_EXPIRATION_TIMEOUT, "1s");
91+
String host = SettingsUtils.discoveredOrDeclaredNodes(settings).get(0);
92+
93+
TransportPool pool = new TransportPool(UUID.randomUUID().toString(), host, settings, new SecureSettings(settings));
94+
95+
Transport transport1 = null;
96+
97+
try {
98+
transport1 = pool.borrowTransport();
99+
transport1.close();
100+
101+
// Wait the amount of time to close.
102+
Thread.sleep(settings.getTransportPoolingExpirationTimeout().millis() + 1000L);
103+
104+
transport1 = pool.borrowTransport();
105+
transport1.close();
106+
107+
// Will need to remove 2 connections at this point
108+
pool.removeOldConnections();
109+
110+
} finally {
111+
// Close everything
112+
if (transport1 != null) {
113+
transport1.close();
114+
}
115+
}
116+
117+
}
118+
119+
@Test
120+
public void testValidationWithUserProvider() throws Exception {
121+
/*
122+
* In this test, we make sure that validating PooledTransports does not throw an exception if the resource is bad when we have set
123+
* a ES_SECURITY_USER_PROVIDER_CLASS. Instead, the PooledTransport is replaced. Previously this would incorrectly throw an
124+
* exception.
125+
*/
126+
Settings settings = new TestSettings();
127+
settings.setProperty(ES_NET_TRANSPORT_POOLING_EXPIRATION_TIMEOUT, "1s");
128+
settings.setProperty(ES_SECURITY_USER_PROVIDER_CLASS, HadoopUserProvider.class.getName());
129+
String badHost = "127.0.0.1:11111"; //intentionally not a real host so that validation fails
130+
131+
TransportPool pool = new TransportPool(UUID.randomUUID().toString(), badHost, settings, new SecureSettings(settings));
132+
133+
Transport transport1 = null;
134+
135+
try {
136+
transport1 = pool.borrowTransport();
137+
transport1.close();
138+
139+
// Wait the amount of time to close.
140+
Thread.sleep(settings.getTransportPoolingExpirationTimeout().millis() + 1000L);
141+
142+
// The following will throw an exception reported in https://github.com/elastic/elasticsearch-hadoop/issues/1362 without the fix
143+
transport1 = pool.borrowTransport();
144+
transport1.close();
145+
146+
// Will need to remove 2 connections at this point
147+
pool.removeOldConnections();
148+
149+
} finally {
150+
// Close everything
151+
if (transport1 != null) {
152+
transport1.close();
153+
}
154+
}
155+
}
80156
}

mr/src/main/java/org/elasticsearch/hadoop/rest/pooling/TransportPool.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020

2121
import org.apache.commons.logging.Log;
2222
import org.apache.commons.logging.LogFactory;
23+
import org.elasticsearch.hadoop.EsHadoopException;
2324
import org.elasticsearch.hadoop.EsHadoopIllegalStateException;
2425
import org.elasticsearch.hadoop.cfg.Settings;
2526
import org.elasticsearch.hadoop.rest.Request;
@@ -101,6 +102,13 @@ private boolean validate(PooledTransport transport) {
101102
try {
102103
Response response = transport.execute(validationRequest);
103104
return response.hasSucceeded();
105+
} catch (EsHadoopException e) {
106+
if (e.getCause() instanceof IOException) {
107+
log.warn("Could not validate pooled connection on lease. Releasing pooled connection and trying again...", e.getCause());
108+
return false;
109+
} else {
110+
throw e;
111+
}
104112
} catch (IOException ioe) {
105113
log.warn("Could not validate pooled connection on lease. Releasing pooled connection and trying again...", ioe);
106114
return false;

0 commit comments

Comments
 (0)