|
7 | 7 | import org.elasticsearch.hadoop.cfg.Settings;
|
8 | 8 | import org.elasticsearch.hadoop.rest.Transport;
|
9 | 9 | import org.elasticsearch.hadoop.rest.TransportFactory;
|
| 10 | +import org.elasticsearch.hadoop.util.SettingsUtils; |
10 | 11 |
|
11 |
| -import java.util.concurrent.ConcurrentHashMap; |
12 |
| -import java.util.concurrent.ConcurrentMap; |
13 |
| - |
14 |
| -import static org.elasticsearch.hadoop.cfg.InternalConfigurationOptions.INTERNAL_TRANSPORT_POOLING_KEY; |
| 12 | +import java.util.ArrayList; |
| 13 | +import java.util.HashMap; |
| 14 | +import java.util.List; |
| 15 | +import java.util.Map; |
15 | 16 |
|
16 | 17 | /**
|
17 | 18 | * Creates HTTP Transports that are backed by a pool of Transport objects for reuse.
|
18 | 19 | */
|
19 |
| -class PooledHttpTransportFactory implements TransportFactory { |
| 20 | +final class PooledHttpTransportFactory implements TransportFactory { |
20 | 21 |
|
21 | 22 | private final Log log = LogFactory.getLog(this.getClass());
|
22 |
| - private final ConcurrentMap<String, TransportPool> hostPools = new ConcurrentHashMap<String, TransportPool>(); |
| 23 | + private final Map<String, TransportPool> hostPools = new HashMap<String, TransportPool>(); |
23 | 24 | private final String jobKey;
|
24 | 25 |
|
25 | 26 | PooledHttpTransportFactory(String jobKey) {
|
26 | 27 | this.jobKey = jobKey;
|
27 | 28 | }
|
28 | 29 |
|
| 30 | + /** |
| 31 | + * {@inheritDoc} |
| 32 | + */ |
29 | 33 | @Override
|
30 |
| - public Transport create(Settings settings, String hostInfo) { |
| 34 | + public synchronized Transport create(Settings settings, String hostInfo) { |
31 | 35 | // Make sure that the caller's Settings has the correct job pool key.
|
32 |
| - if (!jobKey.equals(settings.getProperty(INTERNAL_TRANSPORT_POOLING_KEY))) { |
33 |
| - throw new EsHadoopIllegalArgumentException("Settings object passed does not have the same `"+INTERNAL_TRANSPORT_POOLING_KEY+"` property as when this pool was created. This could be a different job incorrectly polluting the TransportPool. Bailing out..."); |
| 36 | + assertCorrectJobId(settings); |
| 37 | + return borrowFrom(getOrCreateTransportPool(hostInfo, settings), hostInfo); |
| 38 | + } |
| 39 | + |
| 40 | + /** |
| 41 | + * Checks to ensure that the caller is using a settings object with the same job id |
| 42 | + * that this pool is responsible for. |
| 43 | + * @param settings To be checked |
| 44 | + */ |
| 45 | + private void assertCorrectJobId(Settings settings) { |
| 46 | + SettingsUtils.ensureJobTransportPoolingKey(settings); |
| 47 | + String requestingJobKey = SettingsUtils.getJobTransportPoolingKey(settings); |
| 48 | + if (!jobKey.equals(requestingJobKey)) { |
| 49 | + throw new EsHadoopIllegalArgumentException("Settings object passed does not have the same job " + |
| 50 | + "pooling key property as when this pool was created. Job key requested was [" + |
| 51 | + requestingJobKey + "] but this pool services job [" + jobKey + "]. This could be a " + |
| 52 | + "different job incorrectly polluting the TransportPool. Bailing out..."); |
34 | 53 | }
|
| 54 | + } |
35 | 55 |
|
36 |
| - TransportPool pool = hostPools.get(hostInfo); |
| 56 | + /** |
| 57 | + * Gets the transport pool for the given host info, or creates one if it is absent. |
| 58 | + * @param hostInfo To get a pool for |
| 59 | + * @param settings For creating the pool if it does not exist |
| 60 | + * @return A transport pool for the given host |
| 61 | + */ |
| 62 | + private TransportPool getOrCreateTransportPool(String hostInfo, Settings settings) { |
| 63 | + TransportPool pool; |
| 64 | + pool = hostPools.get(hostInfo); // Check again in case it was added while waiting for the lock |
37 | 65 | if (pool == null) {
|
38 |
| - synchronized (this) { |
39 |
| - pool = hostPools.get(hostInfo); // Check again in case it was added while waiting for the lock |
40 |
| - if (pool == null) { |
41 |
| - pool = new TransportPool(jobKey, hostInfo, settings); |
42 |
| - hostPools.put(hostInfo, pool); |
43 |
| - if (log.isDebugEnabled()) { |
44 |
| - log.debug("Creating new TransportPool for job ["+jobKey+"] for host ["+hostInfo+"]"); |
45 |
| - } |
46 |
| - } |
| 66 | + pool = new TransportPool(jobKey, hostInfo, settings); |
| 67 | + hostPools.put(hostInfo, pool); |
| 68 | + if (log.isDebugEnabled()) { |
| 69 | + log.debug("Creating new TransportPool for job ["+jobKey+"] for host ["+hostInfo+"]"); |
47 | 70 | }
|
48 | 71 | }
|
| 72 | + return pool; |
| 73 | + } |
49 | 74 |
|
| 75 | + /** |
| 76 | + * Creates a Transport using the given TransportPool. |
| 77 | + * @param pool Transport is borrowed from |
| 78 | + * @param hostInfo For logging purposes |
| 79 | + * @return A Transport backed by a pooled resource |
| 80 | + */ |
| 81 | + private Transport borrowFrom(TransportPool pool, String hostInfo) { |
50 | 82 | if (!pool.getJobPoolingKey().equals(jobKey)) {
|
51 | 83 | throw new EsHadoopIllegalArgumentException("PooledTransportFactory found a pool with a different owner than this job. This could be a different job incorrectly polluting the TransportPool. Bailing out...");
|
52 | 84 | }
|
53 |
| - |
54 |
| - Transport borrowed; |
55 | 85 | try {
|
56 |
| - borrowed = pool.borrowTransport(); |
| 86 | + return pool.borrowTransport(); |
57 | 87 | } catch (Exception e) {
|
58 | 88 | throw new EsHadoopException(String.format("Could not get a Transport from the Transport Pool for host [%s]", hostInfo));
|
59 | 89 | }
|
60 |
| - return borrowed; |
| 90 | + } |
| 91 | + |
| 92 | + /** |
| 93 | + * Iterates over the available host pools and asks each one to purge transports older than a certain age. |
| 94 | + * @return Total number of pooled connections still alive in this factory. |
| 95 | + */ |
| 96 | + synchronized int cleanPools() { |
| 97 | + int totalConnectionsRemaining = 0; |
| 98 | + List<String> hostsToRemove = new ArrayList<String>(); |
| 99 | + for (Map.Entry<String, TransportPool> hostPool : hostPools.entrySet()) { |
| 100 | + String host = hostPool.getKey(); |
| 101 | + TransportPool pool = hostPool.getValue(); |
| 102 | + |
| 103 | + int connectionsRemaining = pool.removeOldConnections(); |
| 104 | + if (connectionsRemaining == 0) { |
| 105 | + hostsToRemove.add(host); |
| 106 | + } else { |
| 107 | + totalConnectionsRemaining += connectionsRemaining; |
| 108 | + } |
| 109 | + } |
| 110 | + |
| 111 | + // Remove old pools that now have no connections. |
| 112 | + for (String hostToRemove : hostsToRemove) { |
| 113 | + hostPools.remove(hostToRemove); |
| 114 | + } |
| 115 | + |
| 116 | + return totalConnectionsRemaining; |
61 | 117 | }
|
62 | 118 | }
|
0 commit comments