refactor all Queue and BlockingQueue creations into a single factory method
This commit is contained in:
parent
e3a9271000
commit
08ecd9d772
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.action.search.type;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.dfs.DfsSearchResult;
|
||||
|
@ -36,19 +35,20 @@ import java.util.Queue;
|
|||
*/
|
||||
public class TransportSearchCache {
|
||||
|
||||
private final Queue<Collection<DfsSearchResult>> cacheDfsResults = new LinkedTransferQueue<Collection<DfsSearchResult>>();
|
||||
private final Queue<Collection<DfsSearchResult>> cacheDfsResults = ConcurrentCollections.newQueue();
|
||||
|
||||
private final Queue<Map<SearchShardTarget, QuerySearchResultProvider>> cacheQueryResults = new LinkedTransferQueue<Map<SearchShardTarget, QuerySearchResultProvider>>();
|
||||
private final Queue<Map<SearchShardTarget, QuerySearchResultProvider>> cacheQueryResults = ConcurrentCollections.newQueue();
|
||||
|
||||
private final Queue<Map<SearchShardTarget, FetchSearchResult>> cacheFetchResults = new LinkedTransferQueue<Map<SearchShardTarget, FetchSearchResult>>();
|
||||
private final Queue<Map<SearchShardTarget, FetchSearchResult>> cacheFetchResults = ConcurrentCollections.newQueue();
|
||||
|
||||
private final Queue<Map<SearchShardTarget, QueryFetchSearchResult>> cacheQueryFetchResults = new LinkedTransferQueue<Map<SearchShardTarget, QueryFetchSearchResult>>();
|
||||
private final Queue<Map<SearchShardTarget, QueryFetchSearchResult>> cacheQueryFetchResults = ConcurrentCollections.newQueue();
|
||||
|
||||
|
||||
public Collection<DfsSearchResult> obtainDfsResults() {
|
||||
Collection<DfsSearchResult> dfsSearchResults;
|
||||
while ((dfsSearchResults = cacheDfsResults.poll()) == null) {
|
||||
cacheDfsResults.offer(new LinkedTransferQueue<DfsSearchResult>());
|
||||
Queue<DfsSearchResult> offer = ConcurrentCollections.newQueue();
|
||||
cacheDfsResults.offer(offer);
|
||||
}
|
||||
return dfsSearchResults;
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.action.search.type;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.*;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
|
@ -29,6 +28,7 @@ import org.elasticsearch.common.collect.Tuple;
|
|||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.action.SearchServiceListener;
|
||||
import org.elasticsearch.search.action.SearchServiceTransportAction;
|
||||
|
@ -39,6 +39,7 @@ import org.elasticsearch.search.internal.InternalSearchResponse;
|
|||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.elasticsearch.action.search.type.TransportSearchHelper.internalScrollSearchRequest;
|
||||
|
@ -84,7 +85,7 @@ public class TransportSearchScrollQueryAndFetchAction extends AbstractComponent
|
|||
|
||||
private final DiscoveryNodes nodes;
|
||||
|
||||
private volatile LinkedTransferQueue<ShardSearchFailure> shardFailures;
|
||||
private volatile Queue<ShardSearchFailure> shardFailures;
|
||||
|
||||
private final Map<SearchShardTarget, QueryFetchSearchResult> queryFetchResults = searchCache.obtainQueryFetchResults();
|
||||
|
||||
|
@ -104,7 +105,7 @@ public class TransportSearchScrollQueryAndFetchAction extends AbstractComponent
|
|||
}
|
||||
|
||||
protected final ShardSearchFailure[] buildShardFailures() {
|
||||
LinkedTransferQueue<ShardSearchFailure> localFailures = shardFailures;
|
||||
Queue<ShardSearchFailure> localFailures = shardFailures;
|
||||
if (localFailures == null) {
|
||||
return ShardSearchFailure.EMPTY_ARRAY;
|
||||
}
|
||||
|
@ -115,7 +116,7 @@ public class TransportSearchScrollQueryAndFetchAction extends AbstractComponent
|
|||
// we simply try and return as much as possible
|
||||
protected final void addShardFailure(ShardSearchFailure failure) {
|
||||
if (shardFailures == null) {
|
||||
shardFailures = new LinkedTransferQueue<ShardSearchFailure>();
|
||||
shardFailures = ConcurrentCollections.newQueue();
|
||||
}
|
||||
shardFailures.add(failure);
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.action.search.type;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.*;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
|
@ -30,6 +29,7 @@ import org.elasticsearch.common.component.AbstractComponent;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.trove.ExtTIntArrayList;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.action.SearchServiceListener;
|
||||
import org.elasticsearch.search.action.SearchServiceTransportAction;
|
||||
|
@ -43,6 +43,7 @@ import org.elasticsearch.search.query.QuerySearchResultProvider;
|
|||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.elasticsearch.action.search.type.TransportSearchHelper.internalScrollSearchRequest;
|
||||
|
@ -88,7 +89,7 @@ public class TransportSearchScrollQueryThenFetchAction extends AbstractComponent
|
|||
|
||||
private final DiscoveryNodes nodes;
|
||||
|
||||
protected volatile LinkedTransferQueue<ShardSearchFailure> shardFailures;
|
||||
protected volatile Queue<ShardSearchFailure> shardFailures;
|
||||
|
||||
private final Map<SearchShardTarget, QuerySearchResultProvider> queryResults = searchCache.obtainQueryResults();
|
||||
|
||||
|
@ -109,7 +110,7 @@ public class TransportSearchScrollQueryThenFetchAction extends AbstractComponent
|
|||
}
|
||||
|
||||
protected final ShardSearchFailure[] buildShardFailures() {
|
||||
LinkedTransferQueue<ShardSearchFailure> localFailures = shardFailures;
|
||||
Queue<ShardSearchFailure> localFailures = shardFailures;
|
||||
if (localFailures == null) {
|
||||
return ShardSearchFailure.EMPTY_ARRAY;
|
||||
}
|
||||
|
@ -120,7 +121,7 @@ public class TransportSearchScrollQueryThenFetchAction extends AbstractComponent
|
|||
// we simply try and return as much as possible
|
||||
protected final void addShardFailure(ShardSearchFailure failure) {
|
||||
if (shardFailures == null) {
|
||||
shardFailures = new LinkedTransferQueue<ShardSearchFailure>();
|
||||
shardFailures = ConcurrentCollections.newQueue();
|
||||
}
|
||||
shardFailures.add(failure);
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.action.search.type;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.*;
|
||||
|
@ -30,6 +29,7 @@ import org.elasticsearch.common.collect.Tuple;
|
|||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.action.SearchServiceListener;
|
||||
import org.elasticsearch.search.action.SearchServiceTransportAction;
|
||||
|
@ -43,6 +43,7 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.elasticsearch.action.search.type.TransportSearchHelper.internalScrollSearchRequest;
|
||||
|
@ -88,7 +89,7 @@ public class TransportSearchScrollScanAction extends AbstractComponent {
|
|||
|
||||
private final DiscoveryNodes nodes;
|
||||
|
||||
protected volatile LinkedTransferQueue<ShardSearchFailure> shardFailures;
|
||||
protected volatile Queue<ShardSearchFailure> shardFailures;
|
||||
|
||||
private final Map<SearchShardTarget, QueryFetchSearchResult> queryFetchResults = searchCache.obtainQueryFetchResults();
|
||||
|
||||
|
@ -108,7 +109,7 @@ public class TransportSearchScrollScanAction extends AbstractComponent {
|
|||
}
|
||||
|
||||
protected final ShardSearchFailure[] buildShardFailures() {
|
||||
LinkedTransferQueue<ShardSearchFailure> localFailures = shardFailures;
|
||||
Queue<ShardSearchFailure> localFailures = shardFailures;
|
||||
if (localFailures == null) {
|
||||
return ShardSearchFailure.EMPTY_ARRAY;
|
||||
}
|
||||
|
@ -119,7 +120,7 @@ public class TransportSearchScrollScanAction extends AbstractComponent {
|
|||
// we simply try and return as much as possible
|
||||
protected final void addShardFailure(ShardSearchFailure failure) {
|
||||
if (shardFailures == null) {
|
||||
shardFailures = new LinkedTransferQueue<ShardSearchFailure>();
|
||||
shardFailures = ConcurrentCollections.newQueue();
|
||||
}
|
||||
shardFailures.add(failure);
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.action.search.type;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.*;
|
||||
import org.elasticsearch.action.support.TransportAction;
|
||||
|
@ -34,6 +33,7 @@ import org.elasticsearch.cluster.routing.ShardRouting;
|
|||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.trove.ExtTIntArrayList;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.action.SearchServiceListener;
|
||||
|
@ -46,6 +46,7 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
|
@ -92,7 +93,7 @@ public abstract class TransportSearchTypeAction extends TransportAction<SearchRe
|
|||
|
||||
private final AtomicInteger totalOps = new AtomicInteger();
|
||||
|
||||
private volatile LinkedTransferQueue<ShardSearchFailure> shardFailures;
|
||||
private volatile Queue<ShardSearchFailure> shardFailures;
|
||||
|
||||
protected volatile ShardDoc[] sortedShardList;
|
||||
|
||||
|
@ -308,7 +309,7 @@ public abstract class TransportSearchTypeAction extends TransportAction<SearchRe
|
|||
}
|
||||
|
||||
protected final ShardSearchFailure[] buildShardFailures() {
|
||||
LinkedTransferQueue<ShardSearchFailure> localFailures = shardFailures;
|
||||
Queue<ShardSearchFailure> localFailures = shardFailures;
|
||||
if (localFailures == null) {
|
||||
return ShardSearchFailure.EMPTY_ARRAY;
|
||||
}
|
||||
|
@ -319,7 +320,7 @@ public abstract class TransportSearchTypeAction extends TransportAction<SearchRe
|
|||
// we simply try and return as much as possible
|
||||
protected final void addShardFailure(ShardSearchFailure failure) {
|
||||
if (shardFailures == null) {
|
||||
shardFailures = new LinkedTransferQueue<ShardSearchFailure>();
|
||||
shardFailures = ConcurrentCollections.newQueue();
|
||||
}
|
||||
shardFailures.add(failure);
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.elasticsearch.client.transport;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
|
@ -38,13 +37,11 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.*;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -351,7 +348,7 @@ public class TransportClientNodesService extends AbstractComponent {
|
|||
}
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(nodesToPing.size());
|
||||
final LinkedTransferQueue<ClusterStateResponse> clusterStateResponses = new LinkedTransferQueue<ClusterStateResponse>();
|
||||
final Queue<ClusterStateResponse> clusterStateResponses = ConcurrentCollections.newQueue();
|
||||
for (final DiscoveryNode listedNode : nodesToPing) {
|
||||
threadPool.executor(ThreadPool.Names.MANAGEMENT).execute(new Runnable() {
|
||||
@Override
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.action.shard;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -38,6 +37,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
|
|||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
import org.elasticsearch.common.io.stream.VoidStreamable;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.*;
|
||||
|
||||
|
@ -62,7 +62,7 @@ public class ShardStateAction extends AbstractComponent {
|
|||
|
||||
private final ThreadPool threadPool;
|
||||
|
||||
private final BlockingQueue<ShardRouting> startedShardsQueue = new LinkedTransferQueue<ShardRouting>();
|
||||
private final BlockingQueue<ShardRouting> startedShardsQueue = ConcurrentCollections.newBlockingQueue();
|
||||
|
||||
@Inject
|
||||
public ShardStateAction(Settings settings, ClusterService clusterService, TransportService transportService,
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.cluster.service;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.ElasticSearchIllegalStateException;
|
||||
import org.elasticsearch.cluster.*;
|
||||
|
@ -34,6 +33,7 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoveryService;
|
||||
import org.elasticsearch.node.settings.NodeSettingsService;
|
||||
|
@ -76,7 +76,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
|
|||
private final List<ClusterStateListener> clusterStateListeners = new CopyOnWriteArrayList<ClusterStateListener>();
|
||||
private final List<ClusterStateListener> lastClusterStateListeners = new CopyOnWriteArrayList<ClusterStateListener>();
|
||||
|
||||
private final Queue<NotifyTimeout> onGoingTimeouts = new LinkedTransferQueue<NotifyTimeout>();
|
||||
private final Queue<NotifyTimeout> onGoingTimeouts = ConcurrentCollections.newQueue();
|
||||
|
||||
private volatile ClusterState clusterState = newClusterStateBuilder().build();
|
||||
|
||||
|
|
|
@ -20,10 +20,10 @@
|
|||
package org.elasticsearch.common;
|
||||
|
||||
import gnu.trove.map.hash.*;
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.common.trove.ExtTDoubleObjectHashMap;
|
||||
import org.elasticsearch.common.trove.ExtTHashMap;
|
||||
import org.elasticsearch.common.trove.ExtTLongObjectHashMap;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
|
||||
import java.lang.ref.SoftReference;
|
||||
import java.util.Arrays;
|
||||
|
@ -84,7 +84,7 @@ public class CacheRecycler {
|
|||
public static void pushHashMap(ExtTHashMap map) {
|
||||
Queue<ExtTHashMap> ref = hashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<ExtTHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
hashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -110,7 +110,7 @@ public class CacheRecycler {
|
|||
public static void pushDoubleObjectMap(ExtTDoubleObjectHashMap map) {
|
||||
Queue<ExtTDoubleObjectHashMap> ref = doubleObjectHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<ExtTDoubleObjectHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
doubleObjectHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -136,7 +136,7 @@ public class CacheRecycler {
|
|||
public static void pushLongObjectMap(ExtTLongObjectHashMap map) {
|
||||
Queue<ExtTLongObjectHashMap> ref = longObjectHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<ExtTLongObjectHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
longObjectHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -162,7 +162,7 @@ public class CacheRecycler {
|
|||
public static void pushLongLongMap(TLongLongHashMap map) {
|
||||
Queue<TLongLongHashMap> ref = longLongHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<TLongLongHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
longLongHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -189,7 +189,7 @@ public class CacheRecycler {
|
|||
public static void pushIntIntMap(TIntIntHashMap map) {
|
||||
Queue<TIntIntHashMap> ref = intIntHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<TIntIntHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
intIntHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -217,7 +217,7 @@ public class CacheRecycler {
|
|||
public static void pushFloatIntMap(TFloatIntHashMap map) {
|
||||
Queue<TFloatIntHashMap> ref = floatIntHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<TFloatIntHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
floatIntHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -245,7 +245,7 @@ public class CacheRecycler {
|
|||
public static void pushDoubleIntMap(TDoubleIntHashMap map) {
|
||||
Queue<TDoubleIntHashMap> ref = doubleIntHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<TDoubleIntHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
doubleIntHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -273,7 +273,7 @@ public class CacheRecycler {
|
|||
public static void pushByteIntMap(TByteIntHashMap map) {
|
||||
Queue<TByteIntHashMap> ref = byteIntHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<TByteIntHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
byteIntHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -300,7 +300,7 @@ public class CacheRecycler {
|
|||
public static void pushShortIntMap(TShortIntHashMap map) {
|
||||
Queue<TShortIntHashMap> ref = shortIntHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<TShortIntHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
shortIntHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -328,7 +328,7 @@ public class CacheRecycler {
|
|||
public static void pushLongIntMap(TLongIntHashMap map) {
|
||||
Queue<TLongIntHashMap> ref = longIntHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<TLongIntHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
longIntHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -356,7 +356,7 @@ public class CacheRecycler {
|
|||
public static <T> void pushObjectIntMap(TObjectIntHashMap<T> map) {
|
||||
Queue<TObjectIntHashMap> ref = objectIntHashMap.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<TObjectIntHashMap>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
objectIntHashMap.set(ref);
|
||||
}
|
||||
map.clear();
|
||||
|
@ -386,7 +386,7 @@ public class CacheRecycler {
|
|||
public static void pushObjectArray(Object[] objects) {
|
||||
Queue<Object[]> ref = objectArray.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<Object[]>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
objectArray.set(ref);
|
||||
}
|
||||
Arrays.fill(objects, null);
|
||||
|
@ -435,7 +435,7 @@ public class CacheRecycler {
|
|||
public static void pushIntArray(int[] ints, int sentinal) {
|
||||
Queue<int[]> ref = intArray.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<int[]>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
intArray.set(ref);
|
||||
}
|
||||
Arrays.fill(ints, sentinal);
|
||||
|
|
|
@ -19,10 +19,10 @@
|
|||
|
||||
package org.elasticsearch.common.io.stream;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.common.compress.Compressor;
|
||||
import org.elasticsearch.common.io.UTF8StreamWriter;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.monitor.jvm.JvmInfo;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -146,7 +146,7 @@ public class CachedStreamOutput {
|
|||
}
|
||||
Queue<Entry> ref = cache.get();
|
||||
if (ref == null) {
|
||||
ref = new LinkedTransferQueue<Entry>();
|
||||
ref = ConcurrentCollections.newQueue();
|
||||
counter.set(0);
|
||||
cache.set(ref);
|
||||
}
|
||||
|
|
|
@ -19,9 +19,12 @@
|
|||
|
||||
package org.elasticsearch.common.util.concurrent;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.common.collect.MapBackedSet;
|
||||
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
|
@ -53,6 +56,13 @@ public abstract class ConcurrentCollections {
|
|||
return new MapBackedSet<V>(new ConcurrentHashMap<V, Boolean>());
|
||||
}
|
||||
|
||||
public static <T> Queue<T> newQueue() {
|
||||
return new LinkedTransferQueue<T>();
|
||||
}
|
||||
|
||||
public static <T> BlockingQueue<T> newBlockingQueue() {
|
||||
return new LinkedTransferQueue<T>();
|
||||
}
|
||||
|
||||
private ConcurrentCollections() {
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.discovery.local;
|
||||
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.ElasticSearchIllegalStateException;
|
||||
import org.elasticsearch.cluster.*;
|
||||
|
@ -39,7 +38,6 @@ import org.elasticsearch.transport.TransportService;
|
|||
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
@ -300,7 +298,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
|
|||
|
||||
private class ClusterGroup {
|
||||
|
||||
private Queue<LocalDiscovery> members = new LinkedTransferQueue<LocalDiscovery>();
|
||||
private Queue<LocalDiscovery> members = ConcurrentCollections.newQueue();
|
||||
|
||||
Queue<LocalDiscovery> members() {
|
||||
return members;
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.elasticsearch.discovery.zen.ping.unicast;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.ElasticSearchIllegalArgumentException;
|
||||
import org.elasticsearch.ElasticSearchIllegalStateException;
|
||||
|
@ -78,7 +77,7 @@ public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implemen
|
|||
private final Map<Integer, ConcurrentMap<DiscoveryNode, PingResponse>> receivedResponses = newConcurrentMap();
|
||||
|
||||
// a list of temporal responses a node will return for a request (holds requests from other nodes)
|
||||
private final Queue<PingResponse> temporalResponses = new LinkedTransferQueue<PingResponse>();
|
||||
private final Queue<PingResponse> temporalResponses = ConcurrentCollections.newQueue();
|
||||
|
||||
private final CopyOnWriteArrayList<UnicastHostsProvider> hostsProviders = new CopyOnWriteArrayList<UnicastHostsProvider>();
|
||||
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.elasticsearch.threadpool;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import jsr166y.LinkedTransferQueue;
|
||||
import org.elasticsearch.ElasticSearchIllegalArgumentException;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
|
@ -35,10 +34,7 @@ import org.elasticsearch.common.settings.ImmutableSettings;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.SizeValue;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.EsAbortPolicy;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor;
|
||||
import org.elasticsearch.common.util.concurrent.XRejectedExecutionHandler;
|
||||
import org.elasticsearch.common.util.concurrent.*;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilderString;
|
||||
|
@ -249,7 +245,7 @@ public class ThreadPool extends AbstractComponent {
|
|||
String queueType = settings.get("queue_type", "linked");
|
||||
BlockingQueue<Runnable> workQueue;
|
||||
if (capacity == null) {
|
||||
workQueue = new LinkedTransferQueue<Runnable>();
|
||||
workQueue = ConcurrentCollections.newBlockingQueue();
|
||||
} else if ((int) capacity.singles() > 0) {
|
||||
if ("linked".equals(queueType)) {
|
||||
workQueue = new LinkedBlockingQueue<Runnable>((int) capacity.singles());
|
||||
|
|
Loading…
Reference in New Issue