Merge pull request #13463 from jasontedor/sets-be-gone

Remove and forbid use of com.google.common.collect.Sets
This commit is contained in:
Jason Tedor 2015-09-10 11:22:38 -04:00
commit 6885ab9680
76 changed files with 524 additions and 234 deletions

View File

@ -36,11 +36,10 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static com.google.common.collect.Sets.newHashSet;
/**
*/
public class TransportClusterSearchShardsAction extends TransportMasterNodeReadAction<ClusterSearchShardsRequest, ClusterSearchShardsResponse> {
@ -72,7 +71,7 @@ public class TransportClusterSearchShardsAction extends TransportMasterNodeReadA
ClusterState clusterState = clusterService.state();
String[] concreteIndices = indexNameExpressionResolver.concreteIndices(clusterState, request);
Map<String, Set<String>> routingMap = indexNameExpressionResolver.resolveSearchRouting(state, request.routing(), request.indices());
Set<String> nodeIds = newHashSet();
Set<String> nodeIds = new HashSet<>();
GroupShardsIterator groupShardsIterator = clusterService.operationRouting().searchShards(clusterState, concreteIndices, routingMap, request.preference());
ShardRouting shard;
ClusterSearchShardsGroup[] groupResponses = new ClusterSearchShardsGroup[groupShardsIterator.size()];

View File

@ -33,12 +33,11 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static com.google.common.collect.Sets.newHashSet;
/**
* Status of a snapshot
*/
@ -103,7 +102,7 @@ public class SnapshotStatus implements ToXContent, Streamable {
ImmutableMap.Builder<String, SnapshotIndexStatus> indicesStatus = ImmutableMap.builder();
Set<String> indices = newHashSet();
Set<String> indices = new HashSet<>();
for (SnapshotIndexShardStatus shard : shards) {
indices.add(shard.getIndex());
}

View File

@ -44,12 +44,11 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static com.google.common.collect.Sets.newHashSet;
/**
*/
public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<SnapshotsStatusRequest, SnapshotsStatusResponse> {
@ -94,7 +93,7 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
return;
}
Set<String> nodesIds = newHashSet();
Set<String> nodesIds = new HashSet<>();
for (SnapshotsInProgress.Entry entry : currentSnapshots) {
for (SnapshotsInProgress.ShardSnapshotStatus status : entry.shards().values()) {
if (status.nodeId() != null) {
@ -140,7 +139,7 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses) throws IOException {
// First process snapshot that are currently processed
List<SnapshotStatus> builder = new ArrayList<>();
Set<SnapshotId> currentSnapshotIds = newHashSet();
Set<SnapshotId> currentSnapshotIds = new HashSet<>();
if (!currentSnapshots.isEmpty()) {
Map<String, TransportNodesSnapshotsStatus.NodeSnapshotStatus> nodeSnapshotStatusMap;
if (nodeSnapshotStatuses != null) {

View File

@ -19,7 +19,6 @@
package org.elasticsearch.action.admin.indices.alias;
import com.google.common.collect.Sets;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions;
import org.elasticsearch.action.support.ActionFilters;
@ -38,7 +37,11 @@ import org.elasticsearch.rest.action.admin.indices.alias.delete.AliasesNotFoundE
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.*;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
/**
* Add/remove aliases action
@ -68,7 +71,7 @@ public class TransportIndicesAliasesAction extends TransportMasterNodeAction<Ind
@Override
protected ClusterBlockException checkBlock(IndicesAliasesRequest request, ClusterState state) {
Set<String> indices = Sets.newHashSet();
Set<String> indices = new HashSet<>();
for (AliasActions aliasAction : request.aliasActions()) {
for (String index : aliasAction.indices()) {
indices.add(index);

View File

@ -19,7 +19,6 @@
package org.elasticsearch.action.admin.indices.create;
import com.google.common.collect.Sets;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.cluster.ack.ClusterStateUpdateRequest;
import org.elasticsearch.cluster.block.ClusterBlock;
@ -28,6 +27,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.transport.TransportMessage;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@ -47,11 +47,11 @@ public class CreateIndexClusterStateUpdateRequest extends ClusterStateUpdateRequ
private final Map<String, String> mappings = new HashMap<>();
private final Set<Alias> aliases = Sets.newHashSet();
private final Set<Alias> aliases = new HashSet<>();
private final Map<String, IndexMetaData.Custom> customs = new HashMap<>();
private final Set<ClusterBlock> blocks = Sets.newHashSet();
private final Set<ClusterBlock> blocks = new HashSet<>();
CreateIndexClusterStateUpdateRequest(TransportMessage originalMessage, String cause, String index, boolean updateAllTypes) {

View File

@ -20,7 +20,6 @@
package org.elasticsearch.action.admin.indices.create;
import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.action.ActionRequest;
@ -45,6 +44,7 @@ import org.elasticsearch.common.xcontent.XContentType;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@ -72,7 +72,7 @@ public class CreateIndexRequest extends AcknowledgedRequest<CreateIndexRequest>
private final Map<String, String> mappings = new HashMap<>();
private final Set<Alias> aliases = Sets.newHashSet();
private final Set<Alias> aliases = new HashSet<>();
private final Map<String, IndexMetaData.Custom> customs = new HashMap<>();

View File

@ -19,7 +19,6 @@
package org.elasticsearch.action.admin.indices.segments;
import com.google.common.collect.Sets;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
@ -35,6 +34,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -60,7 +60,7 @@ public class IndicesSegmentResponse extends BroadcastResponse implements ToXCont
}
Map<String, IndexSegments> indicesSegments = new HashMap<>();
Set<String> indices = Sets.newHashSet();
Set<String> indices = new HashSet<>();
for (ShardSegments shard : shards) {
indices.add(shard.getShardRouting().getIndex());
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.action.admin.indices.stats;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.routing.ShardRouting;
@ -34,6 +33,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -87,7 +87,7 @@ public class IndicesStatsResponse extends BroadcastResponse implements ToXConten
}
Map<String, IndexStats> indicesStats = new HashMap<>();
Set<String> indices = Sets.newHashSet();
Set<String> indices = new HashSet<>();
for (ShardStats shard : shards) {
indices.add(shard.getShardRouting().getIndex());
}

View File

@ -42,10 +42,10 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.action.ValidateActions.addValidationError;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.common.settings.Settings.readSettingsFromStream;
@ -70,7 +70,7 @@ public class PutIndexTemplateRequest extends MasterNodeRequest<PutIndexTemplateR
private Map<String, String> mappings = new HashMap<>();
private final Set<Alias> aliases = newHashSet();
private final Set<Alias> aliases = new HashSet<>();
private Map<String, IndexMetaData.Custom> customs = new HashMap<>();

View File

@ -19,7 +19,6 @@
package org.elasticsearch.action.admin.indices.upgrade.get;
import com.google.common.collect.Sets;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.common.io.stream.StreamInput;
@ -31,6 +30,7 @@ import org.elasticsearch.common.xcontent.XContentBuilderString;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -54,7 +54,7 @@ public class UpgradeStatusResponse extends BroadcastResponse implements ToXConte
}
Map<String, IndexUpgradeStatus> indicesUpgradeStats = new HashMap<>();
Set<String> indices = Sets.newHashSet();
Set<String> indices = new HashSet<>();
for (ShardUpgradeStatus shard : shards) {
indices.add(shard.getIndex());
}

View File

@ -46,12 +46,11 @@ import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static com.google.common.collect.Sets.newHashSet;
/**
* Upgrade index/indices action.
*/
@ -158,7 +157,7 @@ public class TransportUpgradeAction extends TransportBroadcastByNodeAction<Upgra
* Finds all indices that have not all primaries available
*/
private Set<String> indicesWithMissingPrimaries(ClusterState clusterState, String[] concreteIndices) {
Set<String> indices = newHashSet();
Set<String> indices = new HashSet<>();
RoutingTable routingTable = clusterState.routingTable();
for (String index : concreteIndices) {
IndexRoutingTable indexRoutingTable = routingTable.index(index);

View File

@ -19,7 +19,6 @@
package org.elasticsearch.action.termvectors;
import com.google.common.collect.Sets;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.action.ActionRequestValidationException;
@ -33,6 +32,7 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.VersionType;

View File

@ -20,7 +20,6 @@
package org.elasticsearch.client.transport;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.google.common.collect.Sets;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
@ -396,7 +395,7 @@ public class TransportClientNodesService extends AbstractComponent {
protected void doSample() {
// the nodes we are going to ping include the core listed nodes that were added
// and the last round of discovered nodes
Set<DiscoveryNode> nodesToPing = Sets.newHashSet();
Set<DiscoveryNode> nodesToPing = new HashSet<>();
for (DiscoveryNode node : listedNodes) {
nodesToPing.add(node);
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.cluster.block;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
@ -31,6 +30,7 @@ import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@ -270,7 +270,7 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
public static class Builder {
private Set<ClusterBlock> global = Sets.newHashSet();
private Set<ClusterBlock> global = new HashSet<>();
private Map<String, Set<ClusterBlock>> indices = new HashMap<>();
@ -281,7 +281,7 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
global.addAll(blocks.global());
for (Map.Entry<String, ImmutableSet<ClusterBlock>> entry : blocks.indices().entrySet()) {
if (!indices.containsKey(entry.getKey())) {
indices.put(entry.getKey(), Sets.<ClusterBlock>newHashSet());
indices.put(entry.getKey(), new HashSet<>());
}
indices.get(entry.getKey()).addAll(entry.getValue());
}
@ -319,7 +319,7 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
public Builder addIndexBlock(String index, ClusterBlock block) {
if (!indices.containsKey(index)) {
indices.put(index, Sets.<ClusterBlock>newHashSet());
indices.put(index, new HashSet<>());
}
indices.get(index).add(block);
return this;

View File

@ -20,7 +20,6 @@ package org.elasticsearch.cluster.metadata;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import com.google.common.collect.Sets;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.MapBuilder;
@ -29,6 +28,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;

View File

@ -19,7 +19,6 @@
package org.elasticsearch.cluster.metadata;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.google.common.collect.Sets;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.support.master.MasterNodeRequest;
import org.elasticsearch.cluster.ClusterService;
@ -39,6 +38,7 @@ import org.elasticsearch.indices.InvalidIndexTemplateException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
@ -76,7 +76,7 @@ public class MetaDataIndexTemplateService extends AbstractComponent {
@Override
public ClusterState execute(ClusterState currentState) {
Set<String> templateNames = Sets.newHashSet();
Set<String> templateNames = new HashSet<>();
for (ObjectCursor<String> cursor : currentState.metaData().templates().keys()) {
String templateName = cursor.value;
if (Regex.simpleMatch(request.name, templateName)) {

View File

@ -20,8 +20,6 @@
package org.elasticsearch.cluster.metadata;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.google.common.collect.Sets;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingClusterStateUpdateRequest;
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
@ -49,6 +47,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -182,7 +181,7 @@ public class MetaDataMappingService extends AbstractComponent {
// we need to create the index here, and add the current mapping to it, so we can merge
indexService = indicesService.createIndex(indexMetaData.index(), indexMetaData.settings(), currentState.nodes().localNode().id());
removeIndex = true;
Set<String> typesToIntroduce = Sets.newHashSet();
Set<String> typesToIntroduce = new HashSet<>();
for (MappingTask task : tasks) {
if (task instanceof UpdateTask) {
typesToIntroduce.add(((UpdateTask) task).type);
@ -223,7 +222,7 @@ public class MetaDataMappingService extends AbstractComponent {
boolean dirty = false;
String index = indexService.index().name();
// keep track of what we already refreshed, no need to refresh it again...
Set<String> processedRefreshes = Sets.newHashSet();
Set<String> processedRefreshes = new HashSet<>();
for (MappingTask task : tasks) {
if (task instanceof RefreshTask) {
RefreshTask refreshTask = (RefreshTask) task;

View File

@ -19,13 +19,16 @@
package org.elasticsearch.cluster.metadata;
import com.google.common.collect.Sets;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsClusterStateUpdateRequest;
import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeSettingsClusterStateUpdateRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.*;
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.routing.RoutingTable;
@ -41,7 +44,13 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.settings.IndexDynamicSettings;
import java.util.*;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -185,8 +194,8 @@ public class MetaDataUpdateSettingsService extends AbstractComponent implements
final Settings closeSettings = updatedSettingsBuilder.build();
final Set<String> removedSettings = Sets.newHashSet();
final Set<String> errors = Sets.newHashSet();
final Set<String> removedSettings = new HashSet<>();
final Set<String> errors = new HashSet<>();
for (Map.Entry<String, String> setting : updatedSettingsBuilder.internalMap().entrySet()) {
if (!dynamicSettings.hasDynamicSetting(setting.getKey())) {
removedSettings.add(setting.getKey());
@ -225,8 +234,8 @@ public class MetaDataUpdateSettingsService extends AbstractComponent implements
// allow to change any settings to a close index, and only allow dynamic settings to be changed
// on an open index
Set<String> openIndices = Sets.newHashSet();
Set<String> closeIndices = Sets.newHashSet();
Set<String> openIndices = new HashSet<>();
Set<String> closeIndices = new HashSet<>();
for (String index : actualIndices) {
if (currentState.metaData().index(index).state() == IndexMetaData.State.OPEN) {
openIndices.add(index);

View File

@ -22,7 +22,6 @@ package org.elasticsearch.cluster.routing;
import com.carrotsearch.hppc.IntSet;
import com.carrotsearch.hppc.cursors.IntCursor;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
import com.google.common.collect.Sets;
import com.google.common.collect.UnmodifiableIterator;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.cluster.AbstractDiffable;
@ -37,6 +36,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
@ -137,7 +137,7 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
// check the number of shards
if (indexMetaData.numberOfShards() != shards().size()) {
Set<Integer> expected = Sets.newHashSet();
Set<Integer> expected = new HashSet<>();
for (int i = 0; i < indexMetaData.numberOfShards(); i++) {
expected.add(i);
}
@ -176,7 +176,7 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
* @return number of distinct nodes this index has at least one shard allocated on
*/
public int numberOfNodesShardsAreAllocatedOn(String... excludedNodes) {
Set<String> nodes = Sets.newHashSet();
Set<String> nodes = new HashSet<>();
for (IndexShardRoutingTable shardRoutingTable : this) {
for (ShardRouting shardRouting : shardRoutingTable) {
if (shardRouting.assignedToNode()) {

View File

@ -20,12 +20,12 @@
package org.elasticsearch.cluster.routing;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Sets;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;

View File

@ -35,14 +35,13 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Predicate;
import static com.google.common.collect.Sets.newHashSet;
/**
* {@link RoutingNodes} represents a copy the routing information contained in
* the {@link ClusterState cluster state}.
@ -704,7 +703,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
int inactivePrimaryCount = 0;
int inactiveShardCount = 0;
int relocating = 0;
final Set<ShardId> seenShards = newHashSet();
final Set<ShardId> seenShards = new HashSet<>();
Map<String, Integer> indicesAndShards = new HashMap<>();
for (RoutingNode node : routingNodes) {
for (ShardRouting shard : node) {

View File

@ -19,7 +19,6 @@
package org.elasticsearch.cluster.routing.allocation.decider;
import com.google.common.collect.Sets;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterInfo;
@ -36,6 +35,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.RatioValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.Map;

View File

@ -19,11 +19,16 @@
package org.elasticsearch.common.geo.builders;
import com.google.common.collect.Sets;
import com.spatial4j.core.exception.InvalidShapeException;
import com.spatial4j.core.shape.Shape;
import com.vividsolutions.jts.geom.*;
import com.vividsolutions.jts.geom.Coordinate;
import com.vividsolutions.jts.geom.Geometry;
import com.vividsolutions.jts.geom.GeometryFactory;
import com.vividsolutions.jts.geom.LinearRing;
import com.vividsolutions.jts.geom.MultiPolygon;
import com.vividsolutions.jts.geom.Polygon;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;

View File

@ -19,12 +19,12 @@
package org.elasticsearch.common.inject;
import com.google.common.collect.Sets;
import org.elasticsearch.common.inject.matcher.Matcher;
import org.elasticsearch.common.inject.name.Names;
import org.elasticsearch.common.inject.spi.Message;
import java.lang.reflect.Type;
import java.util.HashSet;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
@ -69,7 +69,7 @@ public class Injectors {
* @return a set of objects returned from this injector
*/
public static <T> Set<T> getInstancesOf(Injector injector, Class<T> baseClass) {
Set<T> answer = Sets.newHashSet();
Set<T> answer = new HashSet<>();
Set<Entry<Key<?>, Binding<?>>> entries = injector.getBindings().entrySet();
for (Entry<Key<?>, Binding<?>> entry : entries) {
Key<?> key = entry.getKey();
@ -93,7 +93,7 @@ public class Injectors {
* @return a set of objects returned from this injector
*/
public static <T> Set<T> getInstancesOf(Injector injector, Matcher<Class> matcher) {
Set<T> answer = Sets.newHashSet();
Set<T> answer = new HashSet<>();
Set<Entry<Key<?>, Binding<?>>> entries = injector.getBindings().entrySet();
for (Entry<Key<?>, Binding<?>> entry : entries) {
Key<?> key = entry.getKey();
@ -114,7 +114,7 @@ public class Injectors {
* @return a set of objects returned from this injector
*/
public static <T> Set<Provider<T>> getProvidersOf(Injector injector, Matcher<Class> matcher) {
Set<Provider<T>> answer = Sets.newHashSet();
Set<Provider<T>> answer = new HashSet<>();
Set<Entry<Key<?>, Binding<?>>> entries = injector.getBindings().entrySet();
for (Entry<Key<?>, Binding<?>> entry : entries) {
Key<?> key = entry.getKey();
@ -135,7 +135,7 @@ public class Injectors {
* @return a set of objects returned from this injector
*/
public static <T> Set<Provider<T>> getProvidersOf(Injector injector, Class<T> baseClass) {
Set<Provider<T>> answer = Sets.newHashSet();
Set<Provider<T>> answer = new HashSet<>();
Set<Entry<Key<?>, Binding<?>>> entries = injector.getBindings().entrySet();
for (Entry<Key<?>, Binding<?>> entry : entries) {
Key<?> key = entry.getKey();
@ -186,7 +186,7 @@ public class Injectors {
* @return a set of objects returned from this injector
*/
public static Set<Binding<?>> getBindingsOf(Injector injector, Matcher<Class> matcher) {
Set<Binding<?>> answer = Sets.newHashSet();
Set<Binding<?>> answer = new HashSet<>();
Set<Entry<Key<?>, Binding<?>>> entries = injector.getBindings().entrySet();
for (Entry<Key<?>, Binding<?>> entry : entries) {
Key<?> key = entry.getKey();
@ -205,7 +205,7 @@ public class Injectors {
* @return a set of objects returned from this injector
*/
public static Set<Binding<?>> getBindingsOf(Injector injector, Class<?> baseClass) {
Set<Binding<?>> answer = Sets.newHashSet();
Set<Binding<?>> answer = new HashSet<>();
Set<Entry<Key<?>, Binding<?>>> entries = injector.getBindings().entrySet();
for (Entry<Key<?>, Binding<?>> entry : entries) {
Key<?> key = entry.getKey();

View File

@ -16,8 +16,7 @@
package org.elasticsearch.common.inject;
import com.google.common.collect.Sets;
import java.util.HashSet;
import java.util.Set;
/**
@ -34,7 +33,7 @@ final class WeakKeySet {
* keys whose class names are equal but class loaders are different. This shouldn't be an issue
* in practice.
*/
private Set<String> backingSet = Sets.newHashSet();
private Set<String> backingSet = new HashSet<>();
public boolean add(Key<?> key) {
return backingSet.add(key.toString());

View File

@ -16,7 +16,6 @@
package org.elasticsearch.common.inject.spi;
import com.google.common.collect.Sets;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Binder;
import org.elasticsearch.common.inject.Binding;
@ -49,6 +48,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -141,7 +141,7 @@ public final class Elements {
private RecordingBinder(Stage stage) {
this.stage = stage;
this.modules = Sets.newHashSet();
this.modules = new HashSet<>();
this.elements = new ArrayList<>();
this.source = null;
this.sourceProvider = new SourceProvider().plusSkippedClasses(
@ -172,7 +172,7 @@ public final class Elements {
*/
private RecordingBinder(RecordingBinder parent, PrivateElementsImpl privateElements) {
this.stage = parent.stage;
this.modules = Sets.newHashSet();
this.modules = new HashSet<>();
this.elements = privateElements.getElementsMutable();
this.source = parent.source;
this.sourceProvider = parent.sourceProvider;

View File

@ -17,7 +17,6 @@
package org.elasticsearch.common.inject.util;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Binder;
import org.elasticsearch.common.inject.Binding;
@ -36,6 +35,7 @@ import java.lang.annotation.Annotation;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -151,8 +151,8 @@ public final class Modules {
final List<Element> elements = Elements.getElements(baseModules);
final List<Element> overrideElements = Elements.getElements(overrides);
final Set<Key> overriddenKeys = Sets.newHashSet();
final Set<Class<? extends Annotation>> overridesScopeAnnotations = Sets.newHashSet();
final Set<Key> overriddenKeys = new HashSet<>();
final Set<Class<? extends Annotation>> overridesScopeAnnotations = new HashSet<>();
// execute the overrides module, keeping track of which keys and scopes are bound
new ModuleWriter(binder()) {
@ -201,7 +201,7 @@ public final class Modules {
PrivateBinder privateBinder = binder.withSource(privateElements.getSource())
.newPrivateBinder();
Set<Key<?>> skippedExposes = Sets.newHashSet();
Set<Key<?>> skippedExposes = new HashSet<>();
for (Key<?> key : privateElements.getExposedKeys()) {
if (overriddenKeys.remove(key)) {

View File

@ -25,12 +25,11 @@ import org.elasticsearch.common.io.FastStringReader;
import java.io.IOException;
import java.io.Reader;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import static com.google.common.collect.Sets.newHashSet;
/**
*
*/
@ -136,7 +135,7 @@ public class AllEntries extends Reader {
}
public Set<String> fields() {
Set<String> fields = newHashSet();
Set<String> fields = new HashSet<>();
for (Entry entry : entries) {
fields.add(entry.name());
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.common.util;
import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.IndexWriter;
@ -38,13 +37,25 @@ import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock;
import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.shard.ShardStateMetaData;
import java.io.IOException;
import java.io.PrintStream;
import java.nio.file.*;
import java.nio.file.DirectoryStream;
import java.nio.file.FileStore;
import java.nio.file.FileVisitResult;
import java.nio.file.FileVisitor;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardCopyOption;
import java.nio.file.attribute.BasicFileAttributes;
import java.util.*;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
/**
*/
@ -343,7 +354,7 @@ public class MultiDataPathUpgrader {
}
private static Set<ShardId> findAllShardIds(Path... locations) throws IOException {
final Set<ShardId> shardIds = Sets.newHashSet();
final Set<ShardId> shardIds = new HashSet<>();
for (final Path location : locations) {
if (Files.isDirectory(location)) {
shardIds.addAll(findAllShardsForIndex(location));

View File

@ -19,6 +19,7 @@
package org.elasticsearch.common.util.concurrent;
import java.util.Collections;
import java.util.Deque;
import java.util.Queue;
import java.util.Set;
@ -29,7 +30,6 @@ import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.LinkedTransferQueue;
import com.google.common.collect.Sets;
/**
*
@ -67,7 +67,7 @@ public abstract class ConcurrentCollections {
}
public static <V> Set<V> newConcurrentSet() {
return Sets.newSetFromMap(ConcurrentCollections.<V, Boolean>newConcurrentMap());
return Collections.newSetFromMap(ConcurrentCollections.<V, Boolean>newConcurrentMap());
}
public static <T> Queue<T> newQueue() {

View File

@ -0,0 +1,79 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.common.util.set;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
public final class Sets {
private Sets() {
}
public static <T> HashSet<T> newHashSet(Iterator<T> iterator) {
Objects.requireNonNull(iterator);
HashSet<T> set = new HashSet<>();
while (iterator.hasNext()) {
set.add(iterator.next());
}
return set;
}
public static <T> HashSet<T> newHashSet(Iterable<T> iterable) {
Objects.requireNonNull(iterable);
return iterable instanceof Collection ? new HashSet<>((Collection)iterable) : newHashSet(iterable.iterator());
}
public static <T> HashSet<T> newHashSet(T... elements) {
Objects.requireNonNull(elements);
HashSet<T> set = new HashSet<>(elements.length);
Collections.addAll(set, elements);
return set;
}
public static <T> Set<T> newConcurrentHashSet() {
return Collections.newSetFromMap(new ConcurrentHashMap<>());
}
public static <T> boolean haveEmptyIntersection(Set<T> left, Set<T> right) {
Objects.requireNonNull(left);
Objects.requireNonNull(right);
return !left.stream().anyMatch(k -> right.contains(k));
}
public static <T> Set<T> difference(Set<T> left, Set<T> right) {
Objects.requireNonNull(left);
Objects.requireNonNull(right);
return left.stream().filter(k -> !right.contains(k)).collect(Collectors.toSet());
}
public static <T> Set<T> union(Set<T> left, Set<T> right) {
Objects.requireNonNull(left);
Objects.requireNonNull(right);
Set<T> union = new HashSet<>(left);
union.addAll(right);
return union;
}
}

View File

@ -20,13 +20,19 @@
package org.elasticsearch.discovery.local;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.*;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateNonMasterUpdateTask;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.IncompatibleClusterStateVersionException;
import org.elasticsearch.cluster.ProcessedClusterStateNonMasterUpdateTask;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodeService;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.RoutingService;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
@ -36,7 +42,12 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.discovery.*;
import org.elasticsearch.discovery.AckClusterStatePublishResponseHandler;
import org.elasticsearch.discovery.BlockingClusterStatePublishResponseHandler;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoveryService;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.InitialStateDiscoveryListener;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.transport.TransportService;
@ -48,7 +59,6 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicBoolean;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.cluster.ClusterState.Builder;
/**
@ -227,7 +237,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
firstMaster.master = true;
}
final Set<String> newMembers = newHashSet();
final Set<String> newMembers = new HashSet<>();
for (LocalDiscovery discovery : clusterGroup.members()) {
newMembers.add(discovery.localNode.id());
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.discovery.zen;
import com.google.common.collect.Sets;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent;
@ -73,6 +72,7 @@ import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Queue;
@ -948,9 +948,9 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
}
// nodes discovered during pinging
Set<DiscoveryNode> activeNodes = Sets.newHashSet();
Set<DiscoveryNode> activeNodes = new HashSet<>();
// nodes discovered who has previously been part of the cluster and do not ping for the very first time
Set<DiscoveryNode> joinedOnceActiveNodes = Sets.newHashSet();
Set<DiscoveryNode> joinedOnceActiveNodes = new HashSet<>();
if (localNode.masterNode()) {
activeNodes.add(localNode);
long joinsCounter = clusterJoinsCounter.get();

View File

@ -20,12 +20,15 @@
package org.elasticsearch.env;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.*;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.store.NativeFSLockFactory;
import org.apache.lucene.store.SimpleFSDirectory;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -34,7 +37,6 @@ import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
@ -46,8 +48,21 @@ import org.elasticsearch.monitor.fs.FsProbe;
import java.io.Closeable;
import java.io.IOException;
import java.nio.file.*;
import java.util.*;
import java.nio.file.AtomicMoveNotSupportedException;
import java.nio.file.DirectoryStream;
import java.nio.file.FileStore;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@ -643,7 +658,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
throw new IllegalStateException("node is not configured to store local location");
}
assert assertEnvIsLocked();
Set<String> indices = Sets.newHashSet();
Set<String> indices = new HashSet<>();
for (NodePath nodePath : nodePaths) {
Path indicesLocation = nodePath.indicesPath;
if (Files.isDirectory(indicesLocation)) {
@ -673,7 +688,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
throw new IllegalStateException("node is not configured to store local location");
}
assert assertEnvIsLocked();
final Set<ShardId> shardIds = Sets.newHashSet();
final Set<ShardId> shardIds = new HashSet<>();
String indexName = index.name();
for (final NodePath nodePath : nodePaths) {
Path location = nodePath.indicesPath;

View File

@ -19,7 +19,6 @@
package org.elasticsearch.index.get;
import com.google.common.collect.Sets;
import org.apache.lucene.index.Term;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.Nullable;
@ -28,6 +27,7 @@ import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.metrics.MeanMetric;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentType;

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.mapper;
import com.google.common.base.Function;
import com.google.common.collect.Collections2;
import com.google.common.collect.Sets;
import org.apache.lucene.analysis.Analyzer;
import org.elasticsearch.common.collect.CopyOnWriteHashMap;
import org.elasticsearch.common.regex.Regex;
@ -30,6 +29,7 @@ import org.elasticsearch.index.analysis.FieldNameAnalyzer;
import java.util.AbstractMap;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
@ -92,7 +92,7 @@ public final class DocumentFieldMappers implements Iterable<FieldMapper> {
}
public Collection<String> simpleMatchToFullName(String pattern) {
Set<String> fields = Sets.newHashSet();
Set<String> fields = new HashSet<>();
for (FieldMapper fieldMapper : this) {
if (Regex.simpleMatch(pattern, fieldMapper.fieldType().names().fullName())) {
fields.add(fieldMapper.fieldType().names().fullName());

View File

@ -19,8 +19,6 @@
package org.elasticsearch.index.mapper;
import com.google.common.collect.Sets;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
@ -34,20 +32,20 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.core.DateFieldMapper.DateFieldType;
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
import org.elasticsearch.index.mapper.core.DateFieldMapper.DateFieldType;
import org.elasticsearch.index.mapper.core.StringFieldMapper.StringFieldType;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.object.ArrayValueMapperParser;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.mapper.object.RootObjectMapper;
import org.elasticsearch.percolator.PercolatorService;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -169,7 +167,7 @@ class DocumentParser implements Closeable {
}
// apply doc boost
if (context.docBoost() != 1.0f) {
Set<String> encounteredFields = Sets.newHashSet();
Set<String> encounteredFields = new HashSet<>();
for (ParseContext.Document doc : context.docs()) {
encounteredFields.clear();
for (IndexableField field : doc) {

View File

@ -21,12 +21,12 @@ package org.elasticsearch.index.mapper;
import com.google.common.base.Function;
import com.google.common.collect.Iterators;
import com.google.common.collect.Sets;
import org.elasticsearch.common.collect.CopyOnWriteHashMap;
import org.elasticsearch.common.regex.Regex;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
@ -152,7 +152,7 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
* Returns a list of the index names of a simple match regex like pattern against full name and index name.
*/
public Collection<String> simpleMatchToIndexNames(String pattern) {
Set<String> fields = Sets.newHashSet();
Set<String> fields = new HashSet<>();
for (MappedFieldType fieldType : this) {
if (Regex.simpleMatch(pattern, fieldType.names().fullName())) {
fields.add(fieldType.names().indexName());
@ -167,7 +167,7 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
* Returns a list of the full names of a simple match regex like pattern against full name and index name.
*/
public Collection<String> simpleMatchToFullName(String pattern) {
Set<String> fields = Sets.newHashSet();
Set<String> fields = new HashSet<>();
for (MappedFieldType fieldType : this) {
if (Regex.simpleMatch(pattern, fieldType.names().fullName())) {
fields.add(fieldType.names().fullName());

View File

@ -18,7 +18,6 @@
*/
package org.elasticsearch.index.mapper.core;
import com.google.common.collect.Sets;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.codecs.PostingsFormat;
@ -29,6 +28,7 @@ import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;

View File

@ -19,7 +19,6 @@
package org.elasticsearch.index.mapper.object;
import com.google.common.collect.Sets;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
@ -38,6 +37,7 @@ import org.elasticsearch.index.settings.IndexSettings;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -66,7 +66,7 @@ public class RootObjectMapper extends ObjectMapper {
protected final List<DynamicTemplate> dynamicTemplates = new ArrayList<>();
// we use this to filter out seen date formats, because we might get duplicates during merging
protected Set<String> seenDateFormats = Sets.newHashSet();
protected Set<String> seenDateFormats = new HashSet<>();
protected List<FormatDateTimeFormatter> dynamicDateTimeFormatters = new ArrayList<>();
protected boolean dateDetection = Defaults.DATE_DETECTION;

View File

@ -19,7 +19,6 @@
package org.elasticsearch.index.query;
import com.google.common.collect.Sets;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.BooleanClause;
@ -41,7 +40,13 @@ import org.elasticsearch.index.search.morelikethis.MoreLikeThisFetchService;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.*;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import static org.elasticsearch.index.mapper.Uid.createUidAsBytes;
@ -185,7 +190,7 @@ public class MoreLikeThisQueryParser implements QueryParser {
likeItems.add(Item.parse(parser, parseContext.parseFieldMatcher(), new Item()));
}
} else if (parseContext.parseFieldMatcher().match(currentFieldName, Field.STOP_WORDS)) {
Set<String> stopWords = Sets.newHashSet();
Set<String> stopWords = new HashSet<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
stopWords.add(parser.text());
}

View File

@ -19,12 +19,22 @@
package org.elasticsearch.index.store;
import com.google.common.collect.Sets;
import org.apache.lucene.store.*;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.FileSwitchDirectory;
import org.apache.lucene.store.LockFactory;
import org.apache.lucene.store.MMapDirectory;
import org.apache.lucene.store.NIOFSDirectory;
import org.apache.lucene.store.NativeFSLockFactory;
import org.apache.lucene.store.RateLimitedFSDirectory;
import org.apache.lucene.store.SimpleFSDirectory;
import org.apache.lucene.store.SimpleFSLockFactory;
import org.apache.lucene.store.StoreRateLimiting;
import org.apache.lucene.util.Constants;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.ShardPath;
@ -32,7 +42,6 @@ import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Collections;
import java.util.Locale;
import java.util.Set;
/**

View File

@ -20,7 +20,6 @@
package org.elasticsearch.node.internal;
import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.Strings;
@ -37,6 +36,7 @@ import java.io.InputStreamReader;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -105,7 +105,7 @@ public class InternalSettingsPreparer {
}
if (loadFromEnv) {
boolean settingsFileFound = false;
Set<String> foundSuffixes = Sets.newHashSet();
Set<String> foundSuffixes = new HashSet<>();
for (String allowedSuffix : ALLOWED_SUFFIXES) {
Path path = environment.configFile().resolve("elasticsearch" + allowedSuffix);
if (Files.exists(path)) {

View File

@ -19,7 +19,6 @@
package org.elasticsearch.rest.action.admin.cluster.node.info;
import com.google.common.collect.Sets;
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest;
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
import org.elasticsearch.client.Client;
@ -27,8 +26,15 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.rest.*;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.BytesRestResponse;
import org.elasticsearch.rest.RestChannel;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.RestResponse;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.rest.action.support.RestBuilderListener;
import java.util.Set;

View File

@ -19,7 +19,6 @@
package org.elasticsearch.search.highlight;
import com.google.common.collect.Sets;
import org.apache.lucene.search.vectorhighlight.SimpleBoundaryScanner;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.xcontent.XContentParser;
@ -30,6 +29,7 @@ import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -211,7 +211,7 @@ public class HighlighterParseElement implements SearchParseElement {
}
fieldOptionsBuilder.postTags(postTagsList.toArray(new String[postTagsList.size()]));
} else if ("matched_fields".equals(fieldName) || "matchedFields".equals(fieldName)) {
Set<String> matchedFields = Sets.newHashSet();
Set<String> matchedFields = new HashSet<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
matchedFields.add(parser.text());
}

View File

@ -81,6 +81,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -89,7 +90,6 @@ import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CopyOnWriteArrayList;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_CREATION_DATE;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_INDEX_UUID;
@ -229,7 +229,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
RoutingTable.Builder rtBuilder = RoutingTable.builder(currentState.routingTable());
final ImmutableMap<ShardId, RestoreInProgress.ShardRestoreStatus> shards;
Set<String> aliases = newHashSet();
Set<String> aliases = new HashSet<>();
if (!renamedIndices.isEmpty()) {
// We have some indices to restore
ImmutableMap.Builder<ShardId, RestoreInProgress.ShardRestoreStatus> shardsBuilder = ImmutableMap.builder();

View File

@ -64,13 +64,13 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.cluster.SnapshotsInProgress.completed;
/**
@ -139,7 +139,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* @return list of snapshots
*/
public List<Snapshot> snapshots(String repositoryName) {
Set<Snapshot> snapshotSet = newHashSet();
Set<Snapshot> snapshotSet = new HashSet<>();
List<SnapshotsInProgress.Entry> entries = currentSnapshots(repositoryName, null);
for (SnapshotsInProgress.Entry entry : entries) {
snapshotSet.add(inProgressSnapshot(entry));
@ -736,8 +736,8 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* @return list of failed and closed indices
*/
private Tuple<Set<String>, Set<String>> indicesWithMissingShards(ImmutableMap<ShardId, SnapshotsInProgress.ShardSnapshotStatus> shards, MetaData metaData) {
Set<String> missing = newHashSet();
Set<String> closed = newHashSet();
Set<String> missing = new HashSet<>();
Set<String> closed = new HashSet<>();
for (ImmutableMap.Entry<ShardId, SnapshotsInProgress.ShardSnapshotStatus> entry : shards.entrySet()) {
if (entry.getValue().state() == State.MISSING) {
if (metaData.hasIndex(entry.getKey().getIndex()) && metaData.index(entry.getKey().getIndex()).getState() == IndexMetaData.State.CLOSE) {

View File

@ -21,7 +21,6 @@ package org.elasticsearch;
import com.fasterxml.jackson.core.JsonLocation;
import com.fasterxml.jackson.core.JsonParseException;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import org.codehaus.groovy.runtime.typehandling.GroovyCastException;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.RoutingMissingException;
@ -32,14 +31,26 @@ import org.elasticsearch.client.AbstractClientHeadersTestCase;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.SnapshotId;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.cluster.routing.IllegalShardRoutingStateException;
import org.elasticsearch.cluster.routing.RoutingTableValidation;
import org.elasticsearch.cluster.routing.RoutingValidationException;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.io.stream.*;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NotSerializableExceptionWrapper;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.transport.LocalTransportAddress;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.CancellableThreadsTests;
import org.elasticsearch.common.xcontent.*;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentLocation;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.index.AlreadyExpiredException;
import org.elasticsearch.index.Index;
@ -48,7 +59,10 @@ import org.elasticsearch.index.engine.IndexFailedEngineException;
import org.elasticsearch.index.engine.RecoveryEngineException;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.index.query.QueryParsingException;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.indices.IndexTemplateAlreadyExistsException;
import org.elasticsearch.indices.IndexTemplateMissingException;
import org.elasticsearch.indices.InvalidIndexTemplateException;

View File

@ -19,10 +19,8 @@
package org.elasticsearch.action.admin.indices.template.put;
import com.google.common.collect.Sets;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexTemplateFilter;
import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService;
import org.elasticsearch.cluster.metadata.MetaDataIndexTemplateService;
import org.elasticsearch.cluster.metadata.MetaDataIndexTemplateService.PutRequest;
@ -33,6 +31,7 @@ import org.junit.Test;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@ -82,7 +81,7 @@ public class MetaDataIndexTemplateServiceTests extends ESTestCase {
null,
Version.CURRENT,
null,
Sets.<IndexTemplateFilter>newHashSet(),
new HashSet<>(),
null,
null
);

View File

@ -19,7 +19,6 @@
package org.elasticsearch.aliases;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequestBuilder;
@ -34,7 +33,6 @@ import org.elasticsearch.cluster.metadata.AliasMetaData;
import org.elasticsearch.cluster.metadata.AliasOrIndex;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.StopWatch;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
@ -49,12 +47,12 @@ import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Test;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.client.Requests.createIndexRequest;
import static org.elasticsearch.client.Requests.indexRequest;
import static org.elasticsearch.cluster.metadata.IndexMetaData.INDEX_METADATA_BLOCK;
@ -1050,7 +1048,7 @@ public class IndexAliasesIT extends ESIntegTestCase {
private void assertHits(SearchHits hits, String... ids) {
assertThat(hits.totalHits(), equalTo((long) ids.length));
Set<String> hitIds = newHashSet();
Set<String> hitIds = new HashSet<>();
for (SearchHit hit : hits.getHits()) {
hitIds.add(hit.id());
}

View File

@ -19,10 +19,10 @@
package org.elasticsearch.cache.recycler;
import com.google.common.collect.Sets;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.recycler.Recycler.V;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.threadpool.ThreadPool;
@ -46,7 +46,7 @@ public class MockPageCacheRecycler extends PageCacheRecycler {
// and releasing pages, lets make sure that after a reasonable timeout, all master
// copy (snapshot) have been released
boolean success =
ESTestCase.awaitBusy(() -> Sets.intersection(masterCopy.keySet(), ACQUIRED_PAGES.keySet()).isEmpty());
ESTestCase.awaitBusy(() -> Sets.haveEmptyIntersection(masterCopy.keySet(), ACQUIRED_PAGES.keySet()));
if (!success) {
masterCopy.keySet().retainAll(ACQUIRED_PAGES.keySet());
ACQUIRED_PAGES.keySet().removeAll(masterCopy.keySet()); // remove all existing master copy we will report on

View File

@ -26,7 +26,6 @@ import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.junit.Test;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.test.VersionUtils.randomVersion;
public class HumanReadableIndexSettingsTests extends ESTestCase {

View File

@ -19,15 +19,14 @@
package org.elasticsearch.cluster.metadata;
import com.google.common.collect.Sets;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData.State;
import org.elasticsearch.common.Strings;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.indices.IndexClosedException;
import org.elasticsearch.test.ESTestCase;
import org.junit.Test;
@ -36,8 +35,14 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import static com.google.common.collect.Sets.newHashSet;
import static org.hamcrest.Matchers.*;
import static org.elasticsearch.common.util.set.Sets.newHashSet;
import static org.hamcrest.Matchers.arrayContaining;
import static org.hamcrest.Matchers.arrayContainingInAnyOrder;
import static org.hamcrest.Matchers.arrayWithSize;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.emptyArray;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
/**
*/
@ -562,7 +567,7 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
.put(indexBuilder("kuku"));
ClusterState state = ClusterState.builder(new ClusterName("_name")).metaData(mdBuilder).build();
IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen());
assertThat(newHashSet(indexNameExpressionResolver.concreteIndices(context, new String[]{})), equalTo(Sets.newHashSet("kuku", "testXXX")));
assertThat(newHashSet(indexNameExpressionResolver.concreteIndices(context, new String[]{})), equalTo(newHashSet("kuku", "testXXX")));
}
@Test

View File

@ -28,7 +28,7 @@ import org.junit.Test;
import java.util.Arrays;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.common.util.set.Sets.newHashSet;
import static org.hamcrest.Matchers.equalTo;
public class WildcardExpressionResolverTests extends ESTestCase {

View File

@ -36,14 +36,22 @@ import org.elasticsearch.test.ESAllocationTestCase;
import org.junit.Test;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.cluster.routing.ShardRoutingState.*;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED;
import static org.elasticsearch.cluster.routing.allocation.RoutingNodesUtils.numberOfShardsOfType;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.nullValue;
/**
*
@ -262,7 +270,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
assertThat(nodeIndex, lessThan(25));
}
RoutingNodes routingNodes = clusterState.getRoutingNodes();
Set<String> encounteredIndices = newHashSet();
Set<String> encounteredIndices = new HashSet<>();
for (RoutingNode routingNode : routingNodes) {
assertThat(routingNode.numberOfShardsWithState(STARTED), equalTo(0));
assertThat(routingNode.size(), equalTo(2));

View File

@ -20,10 +20,10 @@
package org.elasticsearch.common.cli;
import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import com.google.common.jimfs.Configuration;
import com.google.common.jimfs.Jimfs;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.env.Environment;
import org.elasticsearch.test.ESTestCase;
import org.junit.Test;
@ -32,10 +32,18 @@ import java.io.IOException;
import java.nio.file.FileSystem;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.attribute.*;
import java.nio.file.attribute.GroupPrincipal;
import java.nio.file.attribute.PosixFileAttributeView;
import java.nio.file.attribute.PosixFileAttributes;
import java.nio.file.attribute.PosixFilePermission;
import java.nio.file.attribute.UserPrincipal;
import java.util.Set;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.is;
/**
*

View File

@ -19,7 +19,6 @@
package org.elasticsearch.common.lucene.index;
import com.google.common.collect.Sets;
import org.apache.lucene.analysis.core.KeywordAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -50,6 +49,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -123,7 +123,7 @@ public class FreqTermsEnumTests extends ESTestCase {
}
}
Set<String> deletedIds = Sets.newHashSet();
Set<String> deletedIds = new HashSet<>();
for (int i = 0; i < docs.length; i++) {
Document doc = docs[i];
if (randomInt(5) == 2) {
@ -157,7 +157,7 @@ public class FreqTermsEnumTests extends ESTestCase {
}
private void addFreqs(Document doc, Map<String, FreqHolder> reference) {
Set<String> addedDocFreq = Sets.newHashSet();
Set<String> addedDocFreq = new HashSet<>();
for (IndexableField field : doc.getFields("field")) {
String term = field.stringValue();
FreqHolder freqHolder = reference.get(term);

View File

@ -21,12 +21,12 @@ package org.elasticsearch.common.util;
import com.carrotsearch.randomizedtesting.RandomizedContext;
import com.carrotsearch.randomizedtesting.SeedUtils;
import com.google.common.collect.Sets;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.test.ESTestCase;
@ -58,7 +58,7 @@ public class MockBigArrays extends BigArrays {
// not empty, we might be executing on a shared cluster that keeps on obtaining
// and releasing arrays, lets make sure that after a reasonable timeout, all master
// copy (snapshot) have been released
boolean success = ESTestCase.awaitBusy(() -> Sets.intersection(masterCopy.keySet(), ACQUIRED_ARRAYS.keySet()).isEmpty());
boolean success = ESTestCase.awaitBusy(() -> Sets.haveEmptyIntersection(masterCopy.keySet(), ACQUIRED_ARRAYS.keySet()));
if (!success) {
masterCopy.keySet().retainAll(ACQUIRED_ARRAYS.keySet());
ACQUIRED_ARRAYS.keySet().removeAll(masterCopy.keySet()); // remove all existing master copy we will report on

View File

@ -19,7 +19,6 @@
package org.elasticsearch.common.util;
import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@ -28,6 +27,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.shard.ShardId;
@ -39,8 +39,16 @@ import java.io.BufferedWriter;
import java.io.IOException;
import java.io.InputStream;
import java.net.URISyntaxException;
import java.nio.file.*;
import java.util.*;
import java.nio.file.DirectoryStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
/**
*/

View File

@ -19,15 +19,13 @@
package org.elasticsearch.index.deletionpolicy;
import com.google.common.collect.Sets;
import org.elasticsearch.common.util.set.Sets;
import org.hamcrest.Description;
import org.hamcrest.Matcher;
import org.hamcrest.TypeSafeMatcher;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
/**
*

View File

@ -19,16 +19,41 @@
package org.elasticsearch.index.query;
import com.google.common.collect.Sets;
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
import org.apache.lucene.index.*;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.queries.BoostingQuery;
import org.apache.lucene.queries.ExtendedCommonTermsQuery;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.*;
import org.apache.lucene.search.spans.*;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.FuzzyQuery;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.NumericRangeQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.search.RegexpQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TermRangeQuery;
import org.apache.lucene.search.WildcardQuery;
import org.apache.lucene.search.spans.FieldMaskingSpanQuery;
import org.apache.lucene.search.spans.SpanContainingQuery;
import org.apache.lucene.search.spans.SpanFirstQuery;
import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
import org.apache.lucene.search.spans.SpanNearQuery;
import org.apache.lucene.search.spans.SpanNotQuery;
import org.apache.lucene.search.spans.SpanOrQuery;
import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.search.spans.SpanWithinQuery;
import org.apache.lucene.spatial.prefix.IntersectsPrefixTreeFilter;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@ -50,6 +75,7 @@ import org.elasticsearch.common.lucene.search.function.WeightFactorFunction;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
@ -69,14 +95,50 @@ import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.*;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
import static org.elasticsearch.index.query.QueryBuilders.boostingQuery;
import static org.elasticsearch.index.query.QueryBuilders.commonTermsQuery;
import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.disMaxQuery;
import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.fuzzyQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.moreLikeThisQuery;
import static org.elasticsearch.index.query.QueryBuilders.multiMatchQuery;
import static org.elasticsearch.index.query.QueryBuilders.notQuery;
import static org.elasticsearch.index.query.QueryBuilders.prefixQuery;
import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery;
import static org.elasticsearch.index.query.QueryBuilders.rangeQuery;
import static org.elasticsearch.index.query.QueryBuilders.regexpQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanContainingQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanFirstQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanNearQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanNotQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanOrQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanTermQuery;
import static org.elasticsearch.index.query.QueryBuilders.spanWithinQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.index.query.QueryBuilders.termsQuery;
import static org.elasticsearch.index.query.QueryBuilders.wildcardQuery;
import static org.elasticsearch.test.StreamsUtils.copyToBytesFromClasspath;
import static org.elasticsearch.test.StreamsUtils.copyToStringFromClasspath;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertBooleanSubQuery;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.sameInstance;
public class SimpleIndexQueryParserTests extends ESSingleNodeTestCase {

View File

@ -21,7 +21,6 @@ package org.elasticsearch.indexlifecycle;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
@ -31,6 +30,7 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
@ -43,10 +43,16 @@ import static org.elasticsearch.client.Requests.clusterHealthRequest;
import static org.elasticsearch.client.Requests.createIndexRequest;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.cluster.routing.ShardRoutingState.*;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.test.ESIntegTestCase.Scope;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.nullValue;
/**

View File

@ -18,11 +18,11 @@
*/
package org.elasticsearch.indices.recovery;
import com.google.common.collect.Sets;
import org.apache.lucene.store.IndexOutput;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.transport.LocalTransportAddress;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.store.StoreFileMetaData;

View File

@ -18,7 +18,6 @@
*/
package org.elasticsearch.indices.template;
import com.google.common.collect.Sets;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.alias.Alias;
@ -42,12 +41,24 @@ import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.*;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertThrows;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
/**
*
@ -379,7 +390,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase {
searchResponse = client().prepareSearch("complex_filtered_alias").get();
assertHitCount(searchResponse, 3l);
Set<String> types = Sets.newHashSet();
Set<String> types = new HashSet<>();
for (SearchHit searchHit : searchResponse.getHits().getHits()) {
types.add(searchHit.getType());
}

View File

@ -29,8 +29,8 @@ import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.cluster.metadata.AliasAction.newAddAliasAction;
import static org.elasticsearch.common.util.set.Sets.newHashSet;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.nullValue;

View File

@ -21,7 +21,6 @@ package org.elasticsearch.script;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.script.ScriptService.ScriptType;
@ -121,7 +120,7 @@ public class ScriptModesTests extends ESTestCase {
@Test
public void testScriptTypeGenericSettings() {
int randomInt = randomIntBetween(1, ScriptType.values().length - 1);
Set<ScriptType> randomScriptTypesSet = Sets.newHashSet();
Set<ScriptType> randomScriptTypesSet = new HashSet<>();
ScriptMode[] randomScriptModes = new ScriptMode[randomInt];
for (int i = 0; i < randomInt; i++) {
boolean added = false;
@ -154,7 +153,7 @@ public class ScriptModesTests extends ESTestCase {
@Test
public void testScriptContextGenericSettings() {
int randomInt = randomIntBetween(1, scriptContexts.length - 1);
Set<ScriptContext> randomScriptContextsSet = Sets.newHashSet();
Set<ScriptContext> randomScriptContextsSet = new HashSet<>();
ScriptMode[] randomScriptModes = new ScriptMode[randomInt];
for (int i = 0; i < randomInt; i++) {
boolean added = false;

View File

@ -18,8 +18,6 @@
*/
package org.elasticsearch.search.aggregations.bucket;
import com.google.common.collect.Sets;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.unit.DistanceUnit;
@ -35,6 +33,7 @@ import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -360,7 +359,7 @@ public class GeoDistanceIT extends ESIntegTestCase {
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
Terms cities = bucket.getAggregations().get("cities");
assertThat(cities, Matchers.notNullValue());
Set<String> names = Sets.newHashSet();
Set<String> names = new HashSet<>();
for (Terms.Bucket city : cities.getBuckets()) {
names.add(city.getKeyAsString());
}
@ -380,7 +379,7 @@ public class GeoDistanceIT extends ESIntegTestCase {
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
cities = bucket.getAggregations().get("cities");
assertThat(cities, Matchers.notNullValue());
names = Sets.newHashSet();
names = new HashSet<>();
for (Terms.Bucket city : cities.getBuckets()) {
names.add(city.getKeyAsString());
}
@ -400,7 +399,7 @@ public class GeoDistanceIT extends ESIntegTestCase {
assertThat(bucket.getAggregations().asList().isEmpty(), is(false));
cities = bucket.getAggregations().get("cities");
assertThat(cities, Matchers.notNullValue());
names = Sets.newHashSet();
names = new HashSet<>();
for (Terms.Bucket city : cities.getBuckets()) {
names.add(city.getKeyAsString());
}

View File

@ -20,18 +20,14 @@
package org.elasticsearch.search.basic;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.client.Requests;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.functionscore.script.ScriptScoreFunctionBuilder;
@ -48,20 +44,29 @@ import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Test;
import java.io.IOException;
import java.util.HashSet;
import java.util.Set;
import java.util.TreeSet;
import static org.elasticsearch.action.search.SearchType.*;
import static org.elasticsearch.client.Requests.*;
import static org.elasticsearch.action.search.SearchType.DFS_QUERY_AND_FETCH;
import static org.elasticsearch.action.search.SearchType.DFS_QUERY_THEN_FETCH;
import static org.elasticsearch.action.search.SearchType.QUERY_AND_FETCH;
import static org.elasticsearch.action.search.SearchType.QUERY_THEN_FETCH;
import static org.elasticsearch.client.Requests.createIndexRequest;
import static org.elasticsearch.client.Requests.searchRequest;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
public class TransportTwoNodesSearchIT extends ESIntegTestCase {
@ -75,7 +80,7 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase {
}
private Set<String> prepareData(int numShards) throws Exception {
Set<String> fullExpectedIds = Sets.newTreeSet();
Set<String> fullExpectedIds = new TreeSet<>();
Settings.Builder settingsBuilder = settingsBuilder()
.put(indexSettings())
@ -214,7 +219,7 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase {
.query(matchAllQuery())
.explain(true);
Set<String> collectedIds = Sets.newTreeSet();
Set<String> collectedIds = new TreeSet<>();
SearchResponse searchResponse = client().search(searchRequest("test").source(source.from(0).size(60)).searchType(QUERY_THEN_FETCH)).actionGet();
assertNoFailures(searchResponse);
@ -268,7 +273,7 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase {
.query(termQuery("multi", "test"))
.from(0).size(20).explain(true);
Set<String> expectedIds = Sets.newHashSet();
Set<String> expectedIds = new HashSet<>();
for (int i = 0; i < 100; i++) {
expectedIds.add(Integer.toString(i));
}
@ -308,7 +313,7 @@ public class TransportTwoNodesSearchIT extends ESIntegTestCase {
.query(termQuery("multi", "test"))
.from(0).size(20).explain(true);
Set<String> expectedIds = Sets.newHashSet();
Set<String> expectedIds = new HashSet<>();
for (int i = 0; i < 100; i++) {
expectedIds.add(Integer.toString(i));
}

View File

@ -19,11 +19,10 @@
package org.elasticsearch.search.query;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.google.common.collect.Sets;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.MatchQueryBuilder;
@ -47,9 +46,28 @@ import java.util.concurrent.ExecutionException;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.*;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
import static org.elasticsearch.index.query.QueryBuilders.disMaxQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchPhrasePrefixQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchPhraseQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
import static org.elasticsearch.index.query.QueryBuilders.multiMatchQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFirstHit;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSecondHit;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.hasId;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThan;
public class MultiMatchQueryIT extends ESIntegTestCase {

View File

@ -18,8 +18,7 @@
*/
package org.elasticsearch.search.suggest;
import com.google.common.collect.Sets;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.suggest.SuggestRequest;
import org.elasticsearch.action.suggest.SuggestRequestBuilder;
@ -27,6 +26,7 @@ import org.elasticsearch.action.suggest.SuggestResponse;
import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.search.suggest.Suggest.Suggestion;
@ -38,15 +38,21 @@ import org.elasticsearch.search.suggest.completion.CompletionSuggestionFuzzyBuil
import org.elasticsearch.search.suggest.context.ContextBuilder;
import org.elasticsearch.search.suggest.context.ContextMapping;
import org.elasticsearch.test.ESIntegTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.hamcrest.Matchers;
import org.junit.Test;
import java.io.IOException;
import java.util.*;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSuggestion;
import static org.elasticsearch.test.hamcrest.ElasticsearchGeoAssertions.assertDistance;
import static org.hamcrest.Matchers.containsString;

View File

@ -25,7 +25,6 @@ import com.carrotsearch.randomizedtesting.generators.RandomInts;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.carrotsearch.randomizedtesting.generators.RandomStrings;
import com.google.common.collect.Iterators;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
@ -65,6 +64,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.http.HttpServerTransport;
import org.elasticsearch.index.IndexService;

View File

@ -19,7 +19,6 @@
package org.elasticsearch.test.rest.client;
import com.carrotsearch.randomizedtesting.RandomizedTest;
import com.google.common.collect.Sets;
import org.apache.http.config.Registry;
import org.apache.http.config.RegistryBuilder;
import org.apache.http.conn.socket.ConnectionSocketFactory;
@ -38,6 +37,7 @@ import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkAddress;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.test.rest.client.http.HttpRequestBuilder;
import org.elasticsearch.test.rest.client.http.HttpResponse;
import org.elasticsearch.test.rest.spec.RestApi;

View File

@ -18,11 +18,10 @@
*/
package org.elasticsearch.test.rest.section;
import com.google.common.collect.Sets;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
/**
* Holds a REST test suite loaded from a specific yaml file.
@ -35,7 +34,7 @@ public class RestTestSuite {
private SetupSection setupSection;
private Set<TestSection> testSections = Sets.newTreeSet();
private Set<TestSection> testSections = new TreeSet<>();
public RestTestSuite(String api, String name) {
this.api = api;

View File

@ -18,7 +18,6 @@
*/
package org.elasticsearch.test.rest.support;
import com.google.common.collect.Sets;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.PathUtils;
@ -156,7 +155,7 @@ public final class FileUtils {
String groupName = file.toAbsolutePath().getParent().getFileName().toString();
Set<Path> filesSet = files.get(groupName);
if (filesSet == null) {
filesSet = Sets.newHashSet();
filesSet = new HashSet<>();
files.put(groupName, filesSet);
}
filesSet.add(file);

View File

@ -19,7 +19,6 @@
package org.elasticsearch.threadpool;
import com.google.common.collect.Sets;
import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
import org.elasticsearch.action.index.IndexRequestBuilder;
@ -44,16 +43,24 @@ import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.lang.management.ThreadInfo;
import java.lang.management.ThreadMXBean;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.*;
import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.Executor;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.ESIntegTestCase.Scope;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.sameInstance;
/**
*/
@ -69,7 +76,7 @@ public class SimpleThreadPoolIT extends ESIntegTestCase {
public void verifyThreadNames() throws Exception {
ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
Set<String> preNodeStartThreadNames = Sets.newHashSet();
Set<String> preNodeStartThreadNames = new HashSet<>();
for (long l : threadBean.getAllThreadIds()) {
ThreadInfo threadInfo = threadBean.getThreadInfo(l);
if (threadInfo != null) {
@ -98,7 +105,7 @@ public class SimpleThreadPoolIT extends ESIntegTestCase {
assertNoFailures(client().prepareSearch("idx").setQuery(QueryBuilders.termQuery("str_value", "s" + i)).get());
assertNoFailures(client().prepareSearch("idx").setQuery(QueryBuilders.termQuery("l_value", i)).get());
}
Set<String> threadNames = Sets.newHashSet();
Set<String> threadNames = new HashSet<>();
for (long l : threadBean.getAllThreadIds()) {
ThreadInfo threadInfo = threadBean.getThreadInfo(l);
if (threadInfo != null) {

View File

@ -93,3 +93,4 @@ com.google.common.base.Predicates
com.google.common.base.Strings
com.google.common.base.Throwables
com.google.common.collect.Maps
com.google.common.collect.Sets