Pārlūkot izejas kodu

Ban ImmutableMap$Builder in core's main

Almost there!
Nik Everett 10 gadi atpakaļ
vecāks
revīzija
380dbbfb23
37 mainītis faili ar 376 papildinājumiem un 301 dzēšanām
  1. 18 10
      core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java
  2. 5 4
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java
  3. 4 2
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java
  4. 4 3
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java
  5. 6 4
      core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexStatus.java
  6. 6 4
      core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStatus.java
  7. 9 9
      core/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsResponse.java
  8. 3 3
      core/src/main/java/org/elasticsearch/cluster/ClusterState.java
  9. 16 17
      core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java
  10. 51 17
      core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java
  11. 11 4
      core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java
  12. 7 8
      core/src/main/java/org/elasticsearch/common/inject/assistedinject/FactoryProvider2.java
  13. 16 14
      core/src/main/java/org/elasticsearch/common/inject/internal/MoreTypes.java
  14. 11 10
      core/src/main/java/org/elasticsearch/index/analysis/NGramTokenizerFactory.java
  15. 6 6
      core/src/main/java/org/elasticsearch/index/mapper/Mapping.java
  16. 15 15
      core/src/main/java/org/elasticsearch/index/store/Store.java
  17. 2 4
      core/src/main/java/org/elasticsearch/indices/flush/ShardsSyncedFlushResult.java
  18. 9 4
      core/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java
  19. 7 7
      core/src/main/java/org/elasticsearch/monitor/jvm/DeadlockAnalyzer.java
  20. 5 6
      core/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java
  21. 10 8
      core/src/main/java/org/elasticsearch/script/ScriptService.java
  22. 13 15
      core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java
  23. 3 3
      core/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java
  24. 6 7
      core/src/main/java/org/elasticsearch/search/fetch/fielddata/FieldDataFieldsFetchSubPhase.java
  25. 9 11
      core/src/main/java/org/elasticsearch/search/fetch/script/ScriptFieldsFetchSubPhase.java
  26. 4 11
      core/src/main/java/org/elasticsearch/search/fetch/source/FetchSourceSubPhase.java
  27. 16 12
      core/src/main/java/org/elasticsearch/search/lookup/IndexLookup.java
  28. 2 10
      core/src/main/java/org/elasticsearch/search/lookup/SearchLookup.java
  29. 46 26
      core/src/main/java/org/elasticsearch/search/query/QueryPhase.java
  30. 5 6
      core/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java
  31. 13 13
      core/src/main/java/org/elasticsearch/search/sort/SortParseElement.java
  32. 5 5
      core/src/main/java/org/elasticsearch/search/suggest/SuggestPhase.java
  33. 7 13
      core/src/main/java/org/elasticsearch/search/suggest/completion/Completion090PostingsFormat.java
  34. 8 6
      core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java
  35. 6 1
      core/src/main/java/org/elasticsearch/tribe/TribeService.java
  36. 1 0
      dev-tools/src/main/resources/forbidden/all-signatures.txt
  37. 11 3
      plugins/discovery-multicast/src/main/java/org/elasticsearch/plugin/discovery/multicast/MulticastZenPing.java

+ 18 - 10
core/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java

@@ -19,14 +19,19 @@
 
 package org.apache.lucene.queryparser.classic;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.*;
-import org.apache.lucene.util.automaton.RegExp;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.DisjunctionMaxQuery;
+import org.apache.lucene.search.FuzzyQuery;
+import org.apache.lucene.search.MatchNoDocsQuery;
+import org.apache.lucene.search.MultiPhraseQuery;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.automaton.RegExp;
 import org.elasticsearch.common.lucene.search.Queries;
 import org.elasticsearch.common.unit.Fuzziness;
 import org.elasticsearch.index.mapper.MappedFieldType;
@@ -38,9 +43,12 @@ import org.elasticsearch.index.query.support.QueryParsers;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 
+import static java.util.Collections.unmodifiableMap;
 import static org.elasticsearch.common.lucene.search.Queries.fixNegativeQueryIfNeeded;
 
 /**
@@ -52,13 +60,13 @@ import static org.elasticsearch.common.lucene.search.Queries.fixNegativeQueryIfN
  */
 public class MapperQueryParser extends QueryParser {
 
-    public static final ImmutableMap<String, FieldQueryExtension> fieldQueryExtensions;
+    public static final Map<String, FieldQueryExtension> FIELD_QUERY_EXTENSIONS;
 
     static {
-        fieldQueryExtensions = ImmutableMap.<String, FieldQueryExtension>builder()
-                .put(ExistsFieldQueryExtension.NAME, new ExistsFieldQueryExtension())
-                .put(MissingFieldQueryExtension.NAME, new MissingFieldQueryExtension())
-                .build();
+        Map<String, FieldQueryExtension> fieldQueryExtensions = new HashMap<>();
+        fieldQueryExtensions.put(ExistsFieldQueryExtension.NAME, new ExistsFieldQueryExtension());
+        fieldQueryExtensions.put(MissingFieldQueryExtension.NAME, new MissingFieldQueryExtension());
+        FIELD_QUERY_EXTENSIONS = unmodifiableMap(fieldQueryExtensions);
     }
 
     private final QueryShardContext context;
@@ -124,7 +132,7 @@ public class MapperQueryParser extends QueryParser {
 
     @Override
     public Query getFieldQuery(String field, String queryText, boolean quoted) throws ParseException {
-        FieldQueryExtension fieldQueryExtension = fieldQueryExtensions.get(field);
+        FieldQueryExtension fieldQueryExtension = FIELD_QUERY_EXTENSIONS.get(field);
         if (fieldQueryExtension != null) {
             return fieldQueryExtension.query(context, queryText);
         }
@@ -540,7 +548,7 @@ public class MapperQueryParser extends QueryParser {
                     return newMatchAllDocsQuery();
                 }
                 // effectively, we check if a field exists or not
-                return fieldQueryExtensions.get(ExistsFieldQueryExtension.NAME).query(context, actualField);
+                return FIELD_QUERY_EXTENSIONS.get(ExistsFieldQueryExtension.NAME).query(context, actualField);
             }
         }
         if (lowercaseExpandedTerms) {

+ 5 - 4
core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java

@@ -19,8 +19,6 @@
 
 package org.elasticsearch.action.admin.cluster.node.info;
 
-import com.google.common.collect.ImmutableMap;
-
 import org.elasticsearch.Build;
 import org.elasticsearch.Version;
 import org.elasticsearch.action.support.nodes.BaseNodeResponse;
@@ -37,8 +35,11 @@ import org.elasticsearch.threadpool.ThreadPoolInfo;
 import org.elasticsearch.transport.TransportInfo;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  * Node information (static, does not change over time).
  */
@@ -187,12 +188,12 @@ public class NodeInfo extends BaseNodeResponse {
         version = Version.readVersion(in);
         build = Build.readBuild(in);
         if (in.readBoolean()) {
-            ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
+            Map<String, String> builder = new HashMap<>();
             int size = in.readVInt();
             for (int i = 0; i < size; i++) {
                 builder.put(in.readString(), in.readString());
             }
-            serviceAttributes = builder.build();
+            serviceAttributes = unmodifiableMap(builder);
         }
         if (in.readBoolean()) {
             settings = Settings.readSettingsFromStream(in);

+ 4 - 2
core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java

@@ -19,6 +19,8 @@
 
 package org.elasticsearch.action.admin.cluster.node.info;
 
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
 import org.elasticsearch.action.support.nodes.BaseNodesResponse;
 import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.common.io.stream.StreamInput;
@@ -85,8 +87,8 @@ public class NodesInfoResponse extends BaseNodesResponse<NodeInfo> implements To
 
             if (!nodeInfo.getNode().attributes().isEmpty()) {
                 builder.startObject("attributes");
-                for (Map.Entry<String, String> attr : nodeInfo.getNode().attributes().entrySet()) {
-                    builder.field(attr.getKey(), attr.getValue(), XContentBuilder.FieldCaseConversion.NONE);
+                for (ObjectObjectCursor<String, String> attr : nodeInfo.getNode().attributes()) {
+                    builder.field(attr.key, attr.value, XContentBuilder.FieldCaseConversion.NONE);
                 }
                 builder.endObject();
             }

+ 4 - 3
core/src/main/java/org/elasticsearch/action/admin/cluster/node/stats/NodeStats.java

@@ -19,6 +19,8 @@
 
 package org.elasticsearch.action.admin.cluster.node.stats;
 
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
 import org.elasticsearch.action.support.nodes.BaseNodeResponse;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.Nullable;
@@ -38,7 +40,6 @@ import org.elasticsearch.threadpool.ThreadPoolStats;
 import org.elasticsearch.transport.TransportStats;
 
 import java.io.IOException;
-import java.util.Map;
 
 /**
  * Node statistics (dynamic, changes depending on when created).
@@ -281,8 +282,8 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
 
             if (!getNode().attributes().isEmpty()) {
                 builder.startObject("attributes");
-                for (Map.Entry<String, String> attr : getNode().attributes().entrySet()) {
-                    builder.field(attr.getKey(), attr.getValue(), XContentBuilder.FieldCaseConversion.NONE);
+                for (ObjectObjectCursor<String, String> attr : getNode().attributes()) {
+                    builder.field(attr.key, attr.value, XContentBuilder.FieldCaseConversion.NONE);
                 }
                 builder.endObject();
             }

+ 6 - 4
core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotIndexStatus.java

@@ -19,16 +19,18 @@
 
 package org.elasticsearch.action.admin.cluster.snapshots.status;
 
-import com.google.common.collect.ImmutableMap;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentBuilderString;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  * Represents snapshot status of all shards in the index
  */
@@ -45,14 +47,14 @@ public class SnapshotIndexStatus implements Iterable<SnapshotIndexShardStatus>,
     SnapshotIndexStatus(String index, Collection<SnapshotIndexShardStatus> shards) {
         this.index = index;
 
-        ImmutableMap.Builder<Integer, SnapshotIndexShardStatus> builder = ImmutableMap.builder();
+        Map<Integer, SnapshotIndexShardStatus> indexShards = new HashMap<>();
         stats = new SnapshotStats();
         for (SnapshotIndexShardStatus shard : shards) {
-            builder.put(shard.getShardId(), shard);
+            indexShards.put(shard.getShardId(), shard);
             stats.add(shard.getStats());
         }
         shardsStats = new SnapshotShardsStats(shards);
-        indexShards = builder.build();
+        this.indexShards = unmodifiableMap(indexShards);
     }
 
     /**

+ 6 - 4
core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStatus.java

@@ -19,7 +19,6 @@
 
 package org.elasticsearch.action.admin.cluster.snapshots.status;
 
-import com.google.common.collect.ImmutableMap;
 import org.elasticsearch.cluster.SnapshotsInProgress.State;
 import org.elasticsearch.cluster.metadata.SnapshotId;
 import org.elasticsearch.common.io.stream.StreamInput;
@@ -33,11 +32,14 @@ import org.elasticsearch.common.xcontent.XContentFactory;
 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 java.util.Collections.unmodifiableMap;
+
 /**
  * Status of a snapshot
  */
@@ -49,7 +51,7 @@ public class SnapshotStatus implements ToXContent, Streamable {
 
     private List<SnapshotIndexShardStatus> shards;
 
-    private ImmutableMap<String, SnapshotIndexStatus> indicesStatus;
+    private Map<String, SnapshotIndexStatus> indicesStatus;
 
     private SnapshotShardsStats shardsStats;
 
@@ -100,7 +102,7 @@ public class SnapshotStatus implements ToXContent, Streamable {
             return this.indicesStatus;
         }
 
-        ImmutableMap.Builder<String, SnapshotIndexStatus> indicesStatus = ImmutableMap.builder();
+        Map<String, SnapshotIndexStatus> indicesStatus = new HashMap<>();
 
         Set<String> indices = new HashSet<>();
         for (SnapshotIndexShardStatus shard : shards) {
@@ -116,7 +118,7 @@ public class SnapshotStatus implements ToXContent, Streamable {
             }
             indicesStatus.put(index, new SnapshotIndexStatus(index, shards));
         }
-        this.indicesStatus = indicesStatus.build();
+        this.indicesStatus = unmodifiableMap(indicesStatus);
         return this.indicesStatus;
 
     }

+ 9 - 9
core/src/main/java/org/elasticsearch/action/admin/indices/stats/IndicesStatsResponse.java

@@ -19,7 +19,6 @@
 
 package org.elasticsearch.action.admin.indices.stats;
 
-import com.google.common.collect.ImmutableMap;
 import org.elasticsearch.action.ShardOperationFailedException;
 import org.elasticsearch.action.support.broadcast.BroadcastResponse;
 import org.elasticsearch.cluster.routing.ShardRouting;
@@ -38,13 +37,15 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  */
 public class IndicesStatsResponse extends BroadcastResponse implements ToXContent {
 
     private ShardStats[] shards;
 
-    private ImmutableMap<ShardRouting, CommonStats> shardStatsMap;
+    private Map<ShardRouting, CommonStats> shardStatsMap;
 
     IndicesStatsResponse() {
 
@@ -55,16 +56,15 @@ public class IndicesStatsResponse extends BroadcastResponse implements ToXConten
         this.shards = shards;
     }
 
-    public ImmutableMap<ShardRouting, CommonStats> asMap() {
-        if (shardStatsMap == null) {
-            ImmutableMap.Builder<ShardRouting, CommonStats> mb = ImmutableMap.builder();
+    public Map<ShardRouting, CommonStats> asMap() {
+        if (this.shardStatsMap == null) {
+            Map<ShardRouting, CommonStats> shardStatsMap = new HashMap<>();
             for (ShardStats ss : shards) {
-                mb.put(ss.getShardRouting(), ss.getStats());
+                shardStatsMap.put(ss.getShardRouting(), ss.getStats());
             }
-
-            shardStatsMap = mb.build();
+            this.shardStatsMap = unmodifiableMap(shardStatsMap);
         }
-        return shardStatsMap;
+        return this.shardStatsMap;
     }
 
     public ShardStats[] getShards() {

+ 3 - 3
core/src/main/java/org/elasticsearch/cluster/ClusterState.java

@@ -389,9 +389,9 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
 
             if (!blocks().indices().isEmpty()) {
                 builder.startObject("indices");
-                for (Map.Entry<String, Set<ClusterBlock>> entry : blocks().indices().entrySet()) {
-                    builder.startObject(entry.getKey());
-                    for (ClusterBlock block : entry.getValue()) {
+                for (ObjectObjectCursor<String, Set<ClusterBlock>> entry : blocks().indices()) {
+                    builder.startObject(entry.key);
+                    for (ClusterBlock block : entry.value) {
                         block.toXContent(builder, params);
                     }
                     builder.endObject();

+ 16 - 17
core/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java

@@ -19,7 +19,7 @@
 
 package org.elasticsearch.cluster.block;
 
-import com.google.common.collect.ImmutableMap;
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
 
 import org.elasticsearch.cluster.AbstractDiffable;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
@@ -38,7 +38,6 @@ import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.stream.Stream;
 
-import static java.util.Collections.emptyMap;
 import static java.util.Collections.emptySet;
 import static java.util.Collections.unmodifiableSet;
 import static java.util.stream.Collectors.toSet;
@@ -48,17 +47,17 @@ import static java.util.stream.Stream.concat;
  * Represents current cluster level blocks to block dirty operations done against the cluster.
  */
 public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
-    public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(emptySet(), emptyMap());
+    public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(emptySet(), ImmutableOpenMap.of());
 
     public static final ClusterBlocks PROTO = EMPTY_CLUSTER_BLOCK;
 
     private final Set<ClusterBlock> global;
 
-    private final Map<String, Set<ClusterBlock>> indicesBlocks;
+    private final ImmutableOpenMap<String, Set<ClusterBlock>> indicesBlocks;
 
     private final ImmutableLevelHolder[] levelHolders;
 
-    ClusterBlocks(Set<ClusterBlock> global, Map<String, Set<ClusterBlock>> indicesBlocks) {
+    ClusterBlocks(Set<ClusterBlock> global, ImmutableOpenMap<String, Set<ClusterBlock>> indicesBlocks) {
         this.global = global;
         this.indicesBlocks = indicesBlocks;
 
@@ -70,8 +69,8 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
                     .collect(toSet()));
 
             ImmutableOpenMap.Builder<String, Set<ClusterBlock>> indicesBuilder = ImmutableOpenMap.builder();
-            for (Map.Entry<String, Set<ClusterBlock>> entry : indicesBlocks.entrySet()) {
-                indicesBuilder.put(entry.getKey(), unmodifiableSet(entry.getValue().stream()
+            for (ObjectObjectCursor<String, Set<ClusterBlock>> entry : indicesBlocks) {
+                indicesBuilder.put(entry.key, unmodifiableSet(entry.value.stream()
                         .filter(containsLevel)
                         .collect(toSet())));
             }
@@ -84,7 +83,7 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
         return global;
     }
 
-    public Map<String, Set<ClusterBlock>> indices() {
+    public ImmutableOpenMap<String, Set<ClusterBlock>> indices() {
         return indicesBlocks;
     }
 
@@ -204,9 +203,9 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
     public void writeTo(StreamOutput out) throws IOException {
         writeBlockSet(global, out);
         out.writeVInt(indicesBlocks.size());
-        for (Map.Entry<String, Set<ClusterBlock>> entry : indicesBlocks.entrySet()) {
-            out.writeString(entry.getKey());
-            writeBlockSet(entry.getValue(), out);
+        for (ObjectObjectCursor<String, Set<ClusterBlock>> entry : indicesBlocks) {
+            out.writeString(entry.key);
+            writeBlockSet(entry.value, out);
         }
     }
 
@@ -220,8 +219,8 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
     @Override
     public ClusterBlocks readFrom(StreamInput in) throws IOException {
         Set<ClusterBlock> global = readBlockSet(in);
-        ImmutableMap.Builder<String, Set<ClusterBlock>> indicesBuilder = ImmutableMap.builder();
         int size = in.readVInt();
+        ImmutableOpenMap.Builder<String, Set<ClusterBlock>> indicesBuilder = ImmutableOpenMap.builder(size);
         for (int j = 0; j < size; j++) {
             indicesBuilder.put(in.readString().intern(), readBlockSet(in));
         }
@@ -273,11 +272,11 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
 
         public Builder blocks(ClusterBlocks blocks) {
             global.addAll(blocks.global());
-            for (Map.Entry<String, Set<ClusterBlock>> entry : blocks.indices().entrySet()) {
-                if (!indices.containsKey(entry.getKey())) {
-                    indices.put(entry.getKey(), new HashSet<>());
+            for (ObjectObjectCursor<String, Set<ClusterBlock>> entry : blocks.indices()) {
+                if (!indices.containsKey(entry.key)) {
+                    indices.put(entry.key, new HashSet<>());
                 }
-                indices.get(entry.getKey()).addAll(entry.getValue());
+                indices.get(entry.key).addAll(entry.value);
             }
             return this;
         }
@@ -340,7 +339,7 @@ public class ClusterBlocks extends AbstractDiffable<ClusterBlocks> {
 
         public ClusterBlocks build() {
             // We copy the block sets here in case of the builder is modified after build is called
-            ImmutableMap.Builder<String, Set<ClusterBlock>> indicesBuilder = ImmutableMap.builder();
+            ImmutableOpenMap.Builder<String, Set<ClusterBlock>> indicesBuilder = ImmutableOpenMap.builder(indices.size());
             for (Map.Entry<String, Set<ClusterBlock>> entry : indices.entrySet()) {
                 indicesBuilder.put(entry.getKey(), unmodifiableSet(new HashSet<>(entry.getValue())));
             }

+ 51 - 17
core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java

@@ -19,13 +19,15 @@
 
 package org.elasticsearch.cluster.node;
 
-import com.google.common.collect.ImmutableMap;
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
 
 import org.elasticsearch.Version;
 import org.elasticsearch.common.Booleans;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.io.stream.*;
-import org.elasticsearch.common.network.NetworkUtils;
+import org.elasticsearch.common.collect.ImmutableOpenMap;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.Streamable;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.TransportAddress;
 import org.elasticsearch.common.transport.TransportAddressSerializers;
@@ -33,7 +35,6 @@ import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -100,7 +101,7 @@ public class DiscoveryNode implements Streamable, ToXContent {
     private String hostName;
     private String hostAddress;
     private TransportAddress address;
-    private Map<String, String> attributes;
+    private ImmutableOpenMap<String, String> attributes;
     private Version version = Version.CURRENT;
 
     DiscoveryNode() {
@@ -143,7 +144,7 @@ public class DiscoveryNode implements Streamable, ToXContent {
     }
 
     /**
-     * Creates a new {@link DiscoveryNode}
+     * Creates a new {@link DiscoveryNode}.
      * <p>
      * <b>Note:</b> if the version of the node is unknown {@link #MINIMUM_DISCOVERY_NODE_VERSION} should be used.
      * it corresponds to the minimum version this elasticsearch version can communicate with. If a higher version is used
@@ -163,7 +164,7 @@ public class DiscoveryNode implements Streamable, ToXContent {
         if (nodeName != null) {
             this.nodeName = nodeName.intern();
         }
-        ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
+        ImmutableOpenMap.Builder<String, String> builder = ImmutableOpenMap.builder();
         for (Map.Entry<String, String> entry : attributes.entrySet()) {
             builder.put(entry.getKey().intern(), entry.getValue().intern());
         }
@@ -175,6 +176,39 @@ public class DiscoveryNode implements Streamable, ToXContent {
         this.version = version;
     }
 
+    /**
+     * Creates a new {@link DiscoveryNode}.
+     * <p>
+     * <b>Note:</b> if the version of the node is unknown {@link #MINIMUM_DISCOVERY_NODE_VERSION} should be used.
+     * it corresponds to the minimum version this elasticsearch version can communicate with. If a higher version is used
+     * the node might not be able to communicate with the remove node. After initial handshakes node versions will be discovered
+     * and updated.
+     * </p>
+     *
+     * @param nodeName    the nodes name
+     * @param nodeId      the nodes unique id.
+     * @param hostName    the nodes hostname
+     * @param hostAddress the nodes host address
+     * @param address     the nodes transport address
+     * @param attributes  node attributes
+     * @param version     the version of the node.
+     */
+    public DiscoveryNode(String nodeName, String nodeId, String hostName, String hostAddress, TransportAddress address, ImmutableOpenMap<String, String> attributes, Version version) {
+        if (nodeName != null) {
+            this.nodeName = nodeName.intern();
+        }
+        ImmutableOpenMap.Builder<String, String> builder = ImmutableOpenMap.builder();
+        for (ObjectObjectCursor<String, String> entry : attributes) {
+            builder.put(entry.key.intern(), entry.value.intern());
+        }
+        this.attributes = builder.build();
+        this.nodeId = nodeId.intern();
+        this.hostName = hostName.intern();
+        this.hostAddress = hostAddress.intern();
+        this.address = address;
+        this.version = version;
+    }
+
     /**
      * Should this node form a connection to the provided node.
      */
@@ -230,14 +264,14 @@ public class DiscoveryNode implements Streamable, ToXContent {
     /**
      * The node attributes.
      */
-    public Map<String, String> attributes() {
+    public ImmutableOpenMap<String, String> attributes() {
         return this.attributes;
     }
 
     /**
      * The node attributes.
      */
-    public Map<String, String> getAttributes() {
+    public ImmutableOpenMap<String, String> getAttributes() {
         return attributes();
     }
 
@@ -319,11 +353,11 @@ public class DiscoveryNode implements Streamable, ToXContent {
         hostAddress = in.readString().intern();
         address = TransportAddressSerializers.addressFromStream(in);
         int size = in.readVInt();
-        ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
+        ImmutableOpenMap.Builder<String, String> attributes = ImmutableOpenMap.builder(size);
         for (int i = 0; i < size; i++) {
-            builder.put(in.readString().intern(), in.readString().intern());
+            attributes.put(in.readString().intern(), in.readString().intern());
         }
-        attributes = builder.build();
+        this.attributes = attributes.build();
         version = Version.readVersion(in);
     }
 
@@ -335,9 +369,9 @@ public class DiscoveryNode implements Streamable, ToXContent {
         out.writeString(hostAddress);
         addressToStream(out, address);
         out.writeVInt(attributes.size());
-        for (Map.Entry<String, String> entry : attributes.entrySet()) {
-            out.writeString(entry.getKey());
-            out.writeString(entry.getValue());
+        for (ObjectObjectCursor<String, String> entry : attributes) {
+            out.writeString(entry.key);
+            out.writeString(entry.value);
         }
         Version.writeVersion(version, out);
     }
@@ -385,8 +419,8 @@ public class DiscoveryNode implements Streamable, ToXContent {
         builder.field("transport_address", address().toString());
 
         builder.startObject("attributes");
-        for (Map.Entry<String, String> attr : attributes().entrySet()) {
-            builder.field(attr.getKey(), attr.getValue());
+        for (ObjectObjectCursor<String, String> attr : attributes) {
+            builder.field(attr.key, attr.value);
         }
         builder.endObject();
 

+ 11 - 4
core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java

@@ -22,6 +22,7 @@ package org.elasticsearch.cluster.node;
 import com.carrotsearch.hppc.ObjectHashSet;
 import com.carrotsearch.hppc.cursors.ObjectCursor;
 import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.AbstractDiffable;
 import org.elasticsearch.common.Booleans;
@@ -33,7 +34,12 @@ import org.elasticsearch.common.regex.Regex;
 import org.elasticsearch.common.transport.TransportAddress;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * This class holds all {@link DiscoveryNode} in the cluster and provides convenience methods to
@@ -374,9 +380,9 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
                             }
                         } else {
                             for (DiscoveryNode node : this) {
-                                for (Map.Entry<String, String> entry : node.attributes().entrySet()) {
-                                    String attrName = entry.getKey();
-                                    String attrValue = entry.getValue();
+                                for (ObjectObjectCursor<String, String> entry : node.attributes()) {
+                                    String attrName = entry.key;
+                                    String attrValue = entry.value;
                                     if (Regex.simpleMatch(matchAttrName, attrName) && Regex.simpleMatch(matchAttrValue, attrValue)) {
                                         resolvedNodesIds.add(node.id());
                                     }
@@ -563,6 +569,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
         }
     }
 
+    @Override
     public void writeTo(StreamOutput out) throws IOException {
         if (masterNodeId == null) {
             out.writeBoolean(false);

+ 7 - 8
core/src/main/java/org/elasticsearch/common/inject/assistedinject/FactoryProvider2.java

@@ -16,8 +16,6 @@
 
 package org.elasticsearch.common.inject.assistedinject;
 
-import com.google.common.collect.ImmutableMap;
-
 import org.elasticsearch.common.inject.AbstractModule;
 import org.elasticsearch.common.inject.Binder;
 import org.elasticsearch.common.inject.Binding;
@@ -41,9 +39,11 @@ import java.lang.reflect.Proxy;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static java.util.Collections.unmodifiableMap;
 import static org.elasticsearch.common.inject.internal.Annotations.getKey;
 
 /**
@@ -91,7 +91,7 @@ public final class FactoryProvider2<F> implements InvocationHandler, Provider<F>
      */
     private final Key<?> producedType;
     private final Map<Method, Key<?>> returnTypesByMethod;
-    private final ImmutableMap<Method, List<Key<?>>> paramTypes;
+    private final Map<Method, List<Key<?>>> paramTypes;
 
     /**
      * the hosting injector, or null if we haven't been initialized yet
@@ -117,9 +117,8 @@ public final class FactoryProvider2<F> implements InvocationHandler, Provider<F>
                 Class<F> factoryRawType = (Class) factoryType.getRawType();
 
         try {
-            ImmutableMap.Builder<Method, Key<?>> returnTypesBuilder = ImmutableMap.builder();
-            ImmutableMap.Builder<Method, List<Key<?>>> paramTypesBuilder
-                    = ImmutableMap.builder();
+            Map<Method, Key<?>> returnTypesBuilder = new HashMap<>();
+            Map<Method, List<Key<?>>> paramTypesBuilder = new HashMap<>();
             // TODO: also grab methods from superinterfaces
             for (Method method : factoryRawType.getMethods()) {
                 Key<?> returnType = getKey(
@@ -135,8 +134,8 @@ public final class FactoryProvider2<F> implements InvocationHandler, Provider<F>
                 }
                 paramTypesBuilder.put(method, Collections.unmodifiableList(keys));
             }
-            returnTypesByMethod = returnTypesBuilder.build();
-            paramTypes = paramTypesBuilder.build();
+            returnTypesByMethod = unmodifiableMap(returnTypesBuilder);
+            paramTypes = unmodifiableMap(paramTypesBuilder);
         } catch (ErrorsException e) {
             throw new ConfigurationException(e.getErrors().getMessages());
         }

+ 16 - 14
core/src/main/java/org/elasticsearch/common/inject/internal/MoreTypes.java

@@ -17,8 +17,6 @@
 
 package org.elasticsearch.common.inject.internal;
 
-import com.google.common.collect.ImmutableMap;
-
 import org.elasticsearch.common.inject.ConfigurationException;
 import org.elasticsearch.common.inject.TypeLiteral;
 import org.elasticsearch.common.inject.spi.Message;
@@ -35,11 +33,13 @@ import java.lang.reflect.Type;
 import java.lang.reflect.TypeVariable;
 import java.lang.reflect.WildcardType;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Objects;
 
 import static java.util.Collections.singleton;
+import static java.util.Collections.unmodifiableMap;
 
 /**
  * Static methods for working with types that we aren't publishing in the
@@ -54,18 +54,20 @@ public class MoreTypes {
     private MoreTypes() {
     }
 
-    private static final Map<TypeLiteral<?>, TypeLiteral<?>> PRIMITIVE_TO_WRAPPER
-            = new ImmutableMap.Builder<TypeLiteral<?>, TypeLiteral<?>>()
-            .put(TypeLiteral.get(boolean.class), TypeLiteral.get(Boolean.class))
-            .put(TypeLiteral.get(byte.class), TypeLiteral.get(Byte.class))
-            .put(TypeLiteral.get(short.class), TypeLiteral.get(Short.class))
-            .put(TypeLiteral.get(int.class), TypeLiteral.get(Integer.class))
-            .put(TypeLiteral.get(long.class), TypeLiteral.get(Long.class))
-            .put(TypeLiteral.get(float.class), TypeLiteral.get(Float.class))
-            .put(TypeLiteral.get(double.class), TypeLiteral.get(Double.class))
-            .put(TypeLiteral.get(char.class), TypeLiteral.get(Character.class))
-            .put(TypeLiteral.get(void.class), TypeLiteral.get(Void.class))
-            .build();
+    private static final Map<TypeLiteral<?>, TypeLiteral<?>> PRIMITIVE_TO_WRAPPER;
+    static {
+        Map<TypeLiteral<?>, TypeLiteral<?>> primitiveToWrapper = new HashMap<>();
+        primitiveToWrapper.put(TypeLiteral.get(boolean.class), TypeLiteral.get(Boolean.class));
+        primitiveToWrapper.put(TypeLiteral.get(byte.class), TypeLiteral.get(Byte.class));
+        primitiveToWrapper.put(TypeLiteral.get(short.class), TypeLiteral.get(Short.class));
+        primitiveToWrapper.put(TypeLiteral.get(int.class), TypeLiteral.get(Integer.class));
+        primitiveToWrapper.put(TypeLiteral.get(long.class), TypeLiteral.get(Long.class));
+        primitiveToWrapper.put(TypeLiteral.get(float.class), TypeLiteral.get(Float.class));
+        primitiveToWrapper.put(TypeLiteral.get(double.class), TypeLiteral.get(Double.class));
+        primitiveToWrapper.put(TypeLiteral.get(char.class), TypeLiteral.get(Character.class));
+        primitiveToWrapper.put(TypeLiteral.get(void.class), TypeLiteral.get(Void.class));
+        PRIMITIVE_TO_WRAPPER = unmodifiableMap(primitiveToWrapper);
+    }
 
     /**
      * Returns an equivalent type that's safe for use in a key. The returned type will be free of

+ 11 - 10
core/src/main/java/org/elasticsearch/index/analysis/NGramTokenizerFactory.java

@@ -19,12 +19,10 @@
 
 package org.elasticsearch.index.analysis;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.ngram.Lucene43NGramTokenizer;
 import org.apache.lucene.analysis.ngram.NGramTokenizer;
 import org.apache.lucene.util.Version;
-
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.inject.assistedinject.Assisted;
 import org.elasticsearch.common.settings.Settings;
@@ -33,9 +31,12 @@ import org.elasticsearch.index.settings.IndexSettings;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
+import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  *
  */
@@ -49,12 +50,12 @@ public class NGramTokenizerFactory extends AbstractTokenizerFactory {
     static final Map<String, CharMatcher> MATCHERS;
 
     static {
-        ImmutableMap.Builder<String, CharMatcher> builder = ImmutableMap.builder();
-        builder.put("letter", CharMatcher.Basic.LETTER);
-        builder.put("digit", CharMatcher.Basic.DIGIT);
-        builder.put("whitespace", CharMatcher.Basic.WHITESPACE);
-        builder.put("punctuation", CharMatcher.Basic.PUNCTUATION);
-        builder.put("symbol", CharMatcher.Basic.SYMBOL);
+        Map<String, CharMatcher> matchers = new HashMap<>();
+        matchers.put("letter", CharMatcher.Basic.LETTER);
+        matchers.put("digit", CharMatcher.Basic.DIGIT);
+        matchers.put("whitespace", CharMatcher.Basic.WHITESPACE);
+        matchers.put("punctuation", CharMatcher.Basic.PUNCTUATION);
+        matchers.put("symbol", CharMatcher.Basic.SYMBOL);
         // Populate with unicode categories from java.lang.Character
         for (Field field : Character.class.getFields()) {
             if (!field.getName().startsWith("DIRECTIONALITY")
@@ -62,14 +63,14 @@ public class NGramTokenizerFactory extends AbstractTokenizerFactory {
                     && Modifier.isStatic(field.getModifiers())
                     && field.getType() == byte.class) {
                 try {
-                    builder.put(field.getName().toLowerCase(Locale.ROOT), CharMatcher.ByUnicodeCategory.of(field.getByte(null)));
+                    matchers.put(field.getName().toLowerCase(Locale.ROOT), CharMatcher.ByUnicodeCategory.of(field.getByte(null)));
                 } catch (Exception e) {
                     // just ignore
                     continue;
                 }
             }
         }
-        MATCHERS = builder.build();
+        MATCHERS = unmodifiableMap(matchers);
     }
 
     static CharMatcher parseTokenChars(String[] characterClasses) {

+ 6 - 6
core/src/main/java/org/elasticsearch/index/mapper/Mapping.java

@@ -19,8 +19,6 @@
 
 package org.elasticsearch.index.mapper;
 
-import com.google.common.collect.ImmutableMap;
-
 import org.elasticsearch.Version;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -30,10 +28,12 @@ import org.elasticsearch.index.mapper.object.RootObjectMapper;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import static java.util.Collections.emptyMap;
+import static java.util.Collections.unmodifiableMap;
 
 /**
  * Wrapper around everything that defines a mapping, without references to
@@ -58,7 +58,7 @@ public final class Mapping implements ToXContent {
     final Version indexCreated;
     final RootObjectMapper root;
     final MetadataFieldMapper[] metadataMappers;
-    final ImmutableMap<Class<? extends MetadataFieldMapper>, MetadataFieldMapper> rootMappersMap;
+    final Map<Class<? extends MetadataFieldMapper>, MetadataFieldMapper> rootMappersMap;
     final SourceTransform[] sourceTransforms;
     volatile Map<String, Object> meta;
 
@@ -66,12 +66,12 @@ public final class Mapping implements ToXContent {
         this.indexCreated = indexCreated;
         this.root = rootObjectMapper;
         this.metadataMappers = metadataMappers;
-        ImmutableMap.Builder<Class<? extends MetadataFieldMapper>, MetadataFieldMapper> builder = ImmutableMap.builder();
+        Map<Class<? extends MetadataFieldMapper>, MetadataFieldMapper> rootMappersMap = new HashMap<>();
         for (MetadataFieldMapper metadataMapper : metadataMappers) {
             if (indexCreated.before(Version.V_2_0_0_beta1) && LEGACY_INCLUDE_IN_OBJECT.contains(metadataMapper.name())) {
                 root.putMapper(metadataMapper);
             }
-            builder.put(metadataMapper.getClass(), metadataMapper);
+            rootMappersMap.put(metadataMapper.getClass(), metadataMapper);
         }
         // keep root mappers sorted for consistent serialization
         Arrays.sort(metadataMappers, new Comparator<Mapper>() {
@@ -80,7 +80,7 @@ public final class Mapping implements ToXContent {
                 return o1.name().compareTo(o2.name());
             }
         });
-        this.rootMappersMap = builder.build();
+        this.rootMappersMap = unmodifiableMap(rootMappersMap);
         this.sourceTransforms = sourceTransforms;
         this.meta = meta;
     }

+ 15 - 15
core/src/main/java/org/elasticsearch/index/store/Store.java

@@ -19,8 +19,6 @@
 
 package org.elasticsearch.index.store;
 
-import com.google.common.collect.ImmutableMap;
-
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexCommit;
@@ -97,6 +95,7 @@ import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
 import static java.util.Collections.emptyMap;
+import static java.util.Collections.unmodifiableMap;
 
 /**
  * A Store provides plain access to files written by an elasticsearch index shard. Each shard
@@ -798,19 +797,19 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
 
         public MetadataSnapshot(StreamInput in) throws IOException {
             final int size = in.readVInt();
-            final ImmutableMap.Builder<String, StoreFileMetaData> metadataBuilder = ImmutableMap.builder();
+            Map<String, StoreFileMetaData> metadata = new HashMap<>();
             for (int i = 0; i < size; i++) {
                 StoreFileMetaData meta = StoreFileMetaData.readStoreFileMetaData(in);
-                metadataBuilder.put(meta.name(), meta);
+                metadata.put(meta.name(), meta);
             }
-            final ImmutableMap.Builder<String, String> commitUserDataBuilder = ImmutableMap.builder();
+            Map<String, String> commitUserData = new HashMap<>();
             int num = in.readVInt();
             for (int i = num; i > 0; i--) {
-                commitUserDataBuilder.put(in.readString(), in.readString());
+                commitUserData.put(in.readString(), in.readString());
             }
 
-            this.commitUserData = commitUserDataBuilder.build();
-            this.metadata = metadataBuilder.build();
+            this.metadata = unmodifiableMap(metadata);
+            this.commitUserData = unmodifiableMap(commitUserData);
             this.numDocs = in.readLong();
             assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles();
         }
@@ -823,11 +822,11 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
         }
 
         static class LoadedMetadata {
-            final ImmutableMap<String, StoreFileMetaData> fileMetadata;
-            final ImmutableMap<String, String> userData;
+            final Map<String, StoreFileMetaData> fileMetadata;
+            final Map<String, String> userData;
             final long numDocs;
 
-            LoadedMetadata(ImmutableMap<String, StoreFileMetaData> fileMetadata, ImmutableMap<String, String> userData, long numDocs) {
+            LoadedMetadata(Map<String, StoreFileMetaData> fileMetadata, Map<String, String> userData, long numDocs) {
                 this.fileMetadata = fileMetadata;
                 this.userData = userData;
                 this.numDocs = numDocs;
@@ -836,9 +835,9 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
 
         static LoadedMetadata loadMetadata(IndexCommit commit, Directory directory, ESLogger logger) throws IOException {
             long numDocs;
-            ImmutableMap.Builder<String, StoreFileMetaData> builder = ImmutableMap.builder();
+            Map<String, StoreFileMetaData> builder = new HashMap<>();
             Map<String, String> checksumMap = readLegacyChecksums(directory).v1();
-            ImmutableMap.Builder<String, String> commitUserDataBuilder = ImmutableMap.builder();
+            Map<String, String> commitUserDataBuilder = new HashMap<>();
             try {
                 final SegmentInfos segmentCommitInfos = Store.readSegmentsInfo(commit, directory);
                 numDocs = Lucene.getNumDocs(segmentCommitInfos);
@@ -895,7 +894,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
 
                 throw ex;
             }
-            return new LoadedMetadata(builder.build(), commitUserDataBuilder.build(), numDocs);
+            return new LoadedMetadata(unmodifiableMap(builder), unmodifiableMap(commitUserDataBuilder), numDocs);
         }
 
         /**
@@ -952,7 +951,8 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
             }
         }
 
-        private static void checksumFromLuceneFile(Directory directory, String file, ImmutableMap.Builder<String, StoreFileMetaData> builder, ESLogger logger, Version version, boolean readFileAsHash) throws IOException {
+        private static void checksumFromLuceneFile(Directory directory, String file, Map<String, StoreFileMetaData> builder,
+                ESLogger logger, Version version, boolean readFileAsHash) throws IOException {
             final String checksum;
             final BytesRefBuilder fileHash = new BytesRefBuilder();
             try (final IndexInput in = directory.openInput(file, IOContext.READONCE)) {

+ 2 - 4
core/src/main/java/org/elasticsearch/indices/flush/ShardsSyncedFlushResult.java

@@ -18,8 +18,6 @@
  */
 package org.elasticsearch.indices.flush;
 
-import com.google.common.collect.ImmutableMap;
-
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.index.shard.ShardId;
 
@@ -27,6 +25,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import static java.util.Collections.emptyMap;
+import static java.util.Collections.unmodifiableMap;
 
 /**
  * Result for all copies of a shard
@@ -62,8 +61,7 @@ public class ShardsSyncedFlushResult {
      */
     public ShardsSyncedFlushResult(ShardId shardId, String syncId, int totalShards, Map<ShardRouting, SyncedFlushService.SyncedFlushResponse> shardResponses) {
         this.failureReason = null;
-        ImmutableMap.Builder<ShardRouting, SyncedFlushService.SyncedFlushResponse> builder = ImmutableMap.builder();
-        this.shardResponses = builder.putAll(shardResponses).build();
+        this.shardResponses = unmodifiableMap(new HashMap<>(shardResponses));
         this.syncId = syncId;
         this.totalShards = totalShards;
         this.shardId = shardId;

+ 9 - 4
core/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java

@@ -48,7 +48,13 @@ import org.elasticsearch.indices.IndexClosedException;
 import org.elasticsearch.indices.IndicesLifecycle;
 import org.elasticsearch.indices.IndicesService;
 import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.transport.*;
+import org.elasticsearch.transport.BaseTransportResponseHandler;
+import org.elasticsearch.transport.TransportChannel;
+import org.elasticsearch.transport.TransportException;
+import org.elasticsearch.transport.TransportRequest;
+import org.elasticsearch.transport.TransportRequestHandler;
+import org.elasticsearch.transport.TransportResponse;
+import org.elasticsearch.transport.TransportService;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -174,7 +180,7 @@ public class SyncedFlushService extends AbstractComponent {
     * be written on a primary if no write operation was executed between step 1 and step 3 and sync id will only be written on
     * the replica if it contains the same changes that the primary contains.
     *
-    * Synced flush is a best effort operation. The sync id may be written on all, some or none of the copies. 
+    * Synced flush is a best effort operation. The sync id may be written on all, some or none of the copies.
     **/
     public void attemptSyncedFlush(final ShardId shardId, final ActionListener<ShardsSyncedFlushResult> actionListener) {
         try {
@@ -341,8 +347,7 @@ public class SyncedFlushService extends AbstractComponent {
     }
 
     private void contDownAndSendResponseIfDone(String syncId, List<ShardRouting> shards, ShardId shardId, int totalShards,
-                                               ActionListener<ShardsSyncedFlushResult> listener, CountDown countDown, Map<ShardRouting,
-            SyncedFlushResponse> results) {
+            ActionListener<ShardsSyncedFlushResult> listener, CountDown countDown, Map<ShardRouting, SyncedFlushResponse> results) {
         if (countDown.countDown()) {
             assert results.size() == shards.size();
             listener.onResponse(new ShardsSyncedFlushResult(shardId, syncId, totalShards, results));

+ 7 - 7
core/src/main/java/org/elasticsearch/monitor/jvm/DeadlockAnalyzer.java

@@ -19,17 +19,17 @@
 
 package org.elasticsearch.monitor.jvm;
 
-import com.google.common.collect.ImmutableMap;
-
 import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.Set;
 
+import static java.util.Collections.unmodifiableMap;
 import static java.util.Collections.unmodifiableSet;
 
 /**
@@ -55,7 +55,7 @@ public class DeadlockAnalyzer {
         if (deadlockedThreads == null || deadlockedThreads.length == 0) {
             return NULL_RESULT;
         }
-        ImmutableMap<Long, ThreadInfo> threadInfoMap = createThreadInfoMap(deadlockedThreads);
+        Map<Long, ThreadInfo> threadInfoMap = createThreadInfoMap(deadlockedThreads);
         Set<LinkedHashSet<ThreadInfo>> cycles = calculateCycles(threadInfoMap);
         Set<LinkedHashSet<ThreadInfo>> chains = calculateCycleDeadlockChains(threadInfoMap, cycles);
         cycles.addAll(chains);
@@ -89,7 +89,7 @@ public class DeadlockAnalyzer {
     }
 
 
-    private Set<LinkedHashSet<ThreadInfo>> calculateCycleDeadlockChains(ImmutableMap<Long, ThreadInfo> threadInfoMap, Set<LinkedHashSet<ThreadInfo>> cycles) {
+    private Set<LinkedHashSet<ThreadInfo>> calculateCycleDeadlockChains(Map<Long, ThreadInfo> threadInfoMap, Set<LinkedHashSet<ThreadInfo>> cycles) {
         ThreadInfo allThreads[] = threadBean.getThreadInfo(threadBean.getAllThreadIds());
         Set<LinkedHashSet<ThreadInfo>> deadlockChain = new HashSet<>();
         Set<Long> knownDeadlockedThreads = threadInfoMap.keySet();
@@ -113,13 +113,13 @@ public class DeadlockAnalyzer {
     }
 
 
-    private ImmutableMap<Long, ThreadInfo> createThreadInfoMap(long threadIds[]) {
+    private Map<Long, ThreadInfo> createThreadInfoMap(long threadIds[]) {
         ThreadInfo threadInfos[] = threadBean.getThreadInfo(threadIds);
-        ImmutableMap.Builder<Long, ThreadInfo> threadInfoMap = ImmutableMap.builder();
+        Map<Long, ThreadInfo> threadInfoMap = new HashMap<>();
         for (ThreadInfo threadInfo : threadInfos) {
             threadInfoMap.put(threadInfo.getThreadId(), threadInfo);
         }
-        return threadInfoMap.build();
+        return unmodifiableMap(threadInfoMap);
     }
 
     public static class Deadlock {

+ 5 - 6
core/src/main/java/org/elasticsearch/rest/action/cat/RestNodeAttrsAction.java

@@ -18,6 +18,8 @@
  */
 
 package org.elasticsearch.rest.action.cat;
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
 import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
 import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest;
 import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
@@ -41,8 +43,6 @@ import org.elasticsearch.rest.action.support.RestActionListener;
 import org.elasticsearch.rest.action.support.RestResponseListener;
 import org.elasticsearch.rest.action.support.RestTable;
 
-import java.util.Map;
-
 import static org.elasticsearch.rest.RestRequest.Method.GET;
 
 public class RestNodeAttrsAction extends AbstractCatAction {
@@ -111,8 +111,7 @@ public class RestNodeAttrsAction extends AbstractCatAction {
 
         for (DiscoveryNode node : nodes) {
             NodeInfo info = nodesInfo.getNodesMap().get(node.id());
-            Map<String, String> attrs = node.getAttributes();
-            for(String att : attrs.keySet()) {
+            for(ObjectObjectCursor<String, String> att : node.attributes()) {
                 table.startRow();
                 table.addCell(node.name());
                 table.addCell(fullId ? node.id() : Strings.substring(node.getId(), 0, 4));
@@ -124,8 +123,8 @@ public class RestNodeAttrsAction extends AbstractCatAction {
                 } else {
                     table.addCell("-");
                 }
-                table.addCell(att);
-                table.addCell(attrs.containsKey(att) ? attrs.get(att) : null);
+                table.addCell(att.key);
+                table.addCell(att.value);
                 table.endRow();
             }
         }

+ 10 - 8
core/src/main/java/org/elasticsearch/script/ScriptService.java

@@ -19,12 +19,10 @@
 
 package org.elasticsearch.script;
 
-import java.nio.charset.StandardCharsets;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.RemovalListener;
 import com.google.common.cache.RemovalNotification;
-import com.google.common.collect.ImmutableMap;
 
 import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.action.ActionListener;
@@ -67,14 +65,18 @@ import org.elasticsearch.watcher.ResourceWatcherService;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  *
  */
@@ -93,8 +95,8 @@ public class ScriptService extends AbstractComponent implements Closeable {
     private final String defaultLang;
 
     private final Set<ScriptEngineService> scriptEngines;
-    private final ImmutableMap<String, ScriptEngineService> scriptEnginesByLang;
-    private final ImmutableMap<String, ScriptEngineService> scriptEnginesByExt;
+    private final Map<String, ScriptEngineService> scriptEnginesByLang;
+    private final Map<String, ScriptEngineService> scriptEnginesByExt;
 
     private final ConcurrentMap<String, CompiledScript> staticCache = ConcurrentCollections.newConcurrentMap();
 
@@ -162,8 +164,8 @@ public class ScriptService extends AbstractComponent implements Closeable {
         }
         this.cache = cacheBuilder.removalListener(new ScriptCacheRemovalListener()).build();
 
-        ImmutableMap.Builder<String, ScriptEngineService> enginesByLangBuilder = ImmutableMap.builder();
-        ImmutableMap.Builder<String, ScriptEngineService> enginesByExtBuilder = ImmutableMap.builder();
+        Map<String, ScriptEngineService> enginesByLangBuilder = new HashMap<>();
+        Map<String, ScriptEngineService> enginesByExtBuilder = new HashMap<>();
         for (ScriptEngineService scriptEngine : scriptEngines) {
             for (String type : scriptEngine.types()) {
                 enginesByLangBuilder.put(type, scriptEngine);
@@ -172,8 +174,8 @@ public class ScriptService extends AbstractComponent implements Closeable {
                 enginesByExtBuilder.put(ext, scriptEngine);
             }
         }
-        this.scriptEnginesByLang = enginesByLangBuilder.build();
-        this.scriptEnginesByExt = enginesByExtBuilder.build();
+        this.scriptEnginesByLang = unmodifiableMap(enginesByLangBuilder);
+        this.scriptEnginesByExt = unmodifiableMap(enginesByExtBuilder);
 
         this.scriptModes = new ScriptModes(this.scriptEnginesByLang, scriptContextRegistry, settings);
 

+ 13 - 15
core/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java

@@ -18,7 +18,6 @@
  */
 package org.elasticsearch.search.aggregations;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Query;
@@ -35,34 +34,33 @@ import org.elasticsearch.search.query.QueryPhaseExecutionException;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  *
  */
 public class AggregationPhase implements SearchPhase {
-
-    private final AggregationParseElement parseElement;
-
-    private final AggregationBinaryParseElement binaryParseElement;
+    private final Map<String, SearchParseElement> parseElements;
 
     @Inject
     public AggregationPhase(AggregationParseElement parseElement, AggregationBinaryParseElement binaryParseElement) {
-        this.parseElement = parseElement;
-        this.binaryParseElement = binaryParseElement;
+        Map<String, SearchParseElement> parseElements = new HashMap<>();
+        parseElements.put("aggregations", parseElement);
+        parseElements.put("aggs", parseElement);
+        parseElements.put("aggregations_binary", binaryParseElement);
+        parseElements.put("aggregationsBinary", binaryParseElement);
+        parseElements.put("aggs_binary", binaryParseElement);
+        parseElements.put("aggsBinary", binaryParseElement);
+        this.parseElements = unmodifiableMap(parseElements);
     }
 
     @Override
     public Map<String, ? extends SearchParseElement> parseElements() {
-        return ImmutableMap.<String, SearchParseElement>builder()
-                .put("aggregations", parseElement)
-                .put("aggs", parseElement)
-                .put("aggregations_binary", binaryParseElement)
-                .put("aggregationsBinary", binaryParseElement)
-                .put("aggs_binary", binaryParseElement)
-                .put("aggsBinary", binaryParseElement)
-                .build();
+        return parseElements;
     }
 
     @Override

+ 3 - 3
core/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java

@@ -19,7 +19,6 @@
 
 package org.elasticsearch.search.fetch;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.search.DocIdSet;
@@ -64,6 +63,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static java.util.Collections.unmodifiableMap;
 import static org.elasticsearch.common.xcontent.XContentFactory.contentBuilder;
 
 /**
@@ -82,12 +82,12 @@ public class FetchPhase implements SearchPhase {
 
     @Override
     public Map<String, ? extends SearchParseElement> parseElements() {
-        ImmutableMap.Builder<String, SearchParseElement> parseElements = ImmutableMap.builder();
+        Map<String, SearchParseElement> parseElements = new HashMap<>();
         parseElements.put("fields", new FieldsParseElement());
         for (FetchSubPhase fetchSubPhase : fetchSubPhases) {
             parseElements.putAll(fetchSubPhase.parseElements());
         }
-        return parseElements.build();
+        return unmodifiableMap(parseElements);
     }
 
     @Override

+ 6 - 7
core/src/main/java/org/elasticsearch/search/fetch/fielddata/FieldDataFieldsFetchSubPhase.java

@@ -18,16 +18,13 @@
  */
 package org.elasticsearch.search.fetch.fielddata;
 
-import com.google.common.collect.ImmutableMap;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.index.fielddata.AtomicFieldData;
 import org.elasticsearch.index.fielddata.ScriptDocValues;
-import org.elasticsearch.index.mapper.FieldMapper;
 import org.elasticsearch.index.mapper.MappedFieldType;
 import org.elasticsearch.search.SearchHitField;
 import org.elasticsearch.search.SearchParseElement;
 import org.elasticsearch.search.fetch.FetchSubPhase;
-import org.elasticsearch.search.fetch.FetchSubPhaseContext;
 import org.elasticsearch.search.internal.InternalSearchHit;
 import org.elasticsearch.search.internal.InternalSearchHitField;
 import org.elasticsearch.search.internal.SearchContext;
@@ -36,6 +33,8 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  * Query sub phase which pulls data from field data (using the cache if
  * available, building it if not).
@@ -64,10 +63,10 @@ public class FieldDataFieldsFetchSubPhase implements FetchSubPhase {
 
     @Override
     public Map<String, ? extends SearchParseElement> parseElements() {
-        ImmutableMap.Builder<String, SearchParseElement> parseElements = ImmutableMap.builder();
-        parseElements.put("fielddata_fields", new FieldDataFieldsParseElement())
-                .put("fielddataFields", new FieldDataFieldsParseElement());
-        return parseElements.build();
+        Map<String, SearchParseElement> parseElements = new HashMap<>();
+        parseElements.put("fielddata_fields", new FieldDataFieldsParseElement());
+        parseElements.put("fielddataFields", new FieldDataFieldsParseElement());
+        return unmodifiableMap(parseElements);
     }
 
     @Override

+ 9 - 11
core/src/main/java/org/elasticsearch/search/fetch/script/ScriptFieldsFetchSubPhase.java

@@ -18,10 +18,6 @@
  */
 package org.elasticsearch.search.fetch.script;
 
-import com.google.common.collect.ImmutableMap;
-
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.script.LeafSearchScript;
 import org.elasticsearch.search.SearchHitField;
 import org.elasticsearch.search.SearchParseElement;
@@ -38,21 +34,23 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  *
  */
 public class ScriptFieldsFetchSubPhase implements FetchSubPhase {
-
-    @Inject
-    public ScriptFieldsFetchSubPhase() {
+    private static final Map<String, SearchParseElement> PARSE_ELEMENTS;
+    static {
+        Map<String, SearchParseElement> parseElements = new HashMap<>();
+        parseElements.put("script_fields", new ScriptFieldsParseElement());
+        parseElements.put("scriptFields", new ScriptFieldsParseElement());
+        PARSE_ELEMENTS = unmodifiableMap(parseElements);
     }
 
     @Override
     public Map<String, ? extends SearchParseElement> parseElements() {
-        ImmutableMap.Builder<String, SearchParseElement> parseElements = ImmutableMap.builder();
-        parseElements.put("script_fields", new ScriptFieldsParseElement())
-                .put("scriptFields", new ScriptFieldsParseElement());
-        return parseElements.build();
+        return PARSE_ELEMENTS;
     }
 
     @Override

+ 4 - 11
core/src/main/java/org/elasticsearch/search/fetch/source/FetchSourceSubPhase.java

@@ -19,12 +19,9 @@
 
 package org.elasticsearch.search.fetch.source;
 
-import com.google.common.collect.ImmutableMap;
 import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.search.SearchParseElement;
 import org.elasticsearch.search.fetch.FetchSubPhase;
 import org.elasticsearch.search.internal.InternalSearchHit;
@@ -34,20 +31,16 @@ import org.elasticsearch.search.lookup.SourceLookup;
 import java.io.IOException;
 import java.util.Map;
 
+import static java.util.Collections.singletonMap;
+
 /**
  */
 public class FetchSourceSubPhase implements FetchSubPhase {
-
-    @Inject
-    public FetchSourceSubPhase() {
-
-    }
+    private static final Map<String, SearchParseElement> PARSE_ELEMENTS = singletonMap("_source", new FetchSourceParseElement());
 
     @Override
     public Map<String, ? extends SearchParseElement> parseElements() {
-        ImmutableMap.Builder<String, SearchParseElement> parseElements = ImmutableMap.builder();
-        parseElements.put("_source", new FetchSourceParseElement());
-        return parseElements.build();
+        return PARSE_ELEMENTS;
     }
 
     @Override

+ 16 - 12
core/src/main/java/org/elasticsearch/search/lookup/IndexLookup.java

@@ -18,12 +18,24 @@
  */
 package org.elasticsearch.search.lookup;
 
-import com.google.common.collect.ImmutableMap.Builder;
-
 import org.apache.lucene.index.LeafReaderContext;
 
-public class IndexLookup {
+import java.util.HashMap;
+import java.util.Map;
 
+import static java.util.Collections.unmodifiableMap;
+
+public class IndexLookup {
+    public static final Map<String, Object> NAMES;
+    static {
+        Map<String, Object> names = new HashMap<>();
+        names.put("_FREQUENCIES", IndexLookup.FLAG_FREQUENCIES);
+        names.put("_POSITIONS", IndexLookup.FLAG_POSITIONS);
+        names.put("_OFFSETS", IndexLookup.FLAG_OFFSETS);
+        names.put("_PAYLOADS", IndexLookup.FLAG_PAYLOADS);
+        names.put("_CACHE", IndexLookup.FLAG_CACHE);
+        NAMES = unmodifiableMap(names);
+    }
     /**
      * Flag to pass to {@link IndexField#get(Object, int)} if you require
      * offsets in the returned {@link IndexFieldTerm}.
@@ -55,15 +67,7 @@ public class IndexLookup {
      */
     public static final int FLAG_CACHE = 32;
 
-    public IndexLookup(Builder<String, Object> builder) {
-        builder.put("_FREQUENCIES", IndexLookup.FLAG_FREQUENCIES);
-        builder.put("_POSITIONS", IndexLookup.FLAG_POSITIONS);
-        builder.put("_OFFSETS", IndexLookup.FLAG_OFFSETS);
-        builder.put("_PAYLOADS", IndexLookup.FLAG_PAYLOADS);
-        builder.put("_CACHE", IndexLookup.FLAG_CACHE);
-    }
-
-    public LeafIndexLookup getLeafIndexLookup(LeafReaderContext context) {
+    public static LeafIndexLookup getLeafIndexLookup(LeafReaderContext context) {
         return new LeafIndexLookup(context);
     }
 

+ 2 - 10
core/src/main/java/org/elasticsearch/search/lookup/SearchLookup.java

@@ -19,7 +19,6 @@
 
 package org.elasticsearch.search.lookup;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.index.LeafReaderContext;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
@@ -36,17 +35,10 @@ public class SearchLookup {
 
     final FieldsLookup fieldsLookup;
 
-    final IndexLookup indexLookup;
-
-    final ImmutableMap<String, Object> asMap;
-
     public SearchLookup(MapperService mapperService, IndexFieldDataService fieldDataService, @Nullable String[] types) {
-        ImmutableMap.Builder<String, Object> builder = ImmutableMap.builder();
         docMap = new DocLookup(mapperService, fieldDataService, types);
         sourceLookup = new SourceLookup();
         fieldsLookup = new FieldsLookup(mapperService, types);
-        indexLookup = new IndexLookup(builder);
-        asMap = builder.build();
     }
 
     public LeafSearchLookup getLeafSearchLookup(LeafReaderContext context) {
@@ -54,8 +46,8 @@ public class SearchLookup {
                 docMap.getLeafDocLookup(context),
                 sourceLookup,
                 fieldsLookup.getLeafFieldsLookup(context),
-                indexLookup.getLeafIndexLookup(context),
-                asMap);
+                IndexLookup.getLeafIndexLookup(context),
+                IndexLookup.NAMES);
     }
 
     public DocLookup doc() {

+ 46 - 26
core/src/main/java/org/elasticsearch/search/query/QueryPhase.java

@@ -19,12 +19,28 @@
 
 package org.elasticsearch.search.query;
 
-import com.google.common.collect.ImmutableMap;
-
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.MinDocQuery;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MultiCollector;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TimeLimitingCollector;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopDocsCollector;
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopScoreDocCollector;
+import org.apache.lucene.search.TotalHitCountCollector;
+import org.apache.lucene.search.Weight;
 import org.elasticsearch.action.search.SearchType;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.lucene.Lucene;
@@ -43,10 +59,13 @@ import org.elasticsearch.search.sort.TrackScoresParseElement;
 import org.elasticsearch.search.suggest.SuggestPhase;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  *
  */
@@ -65,29 +84,30 @@ public class QueryPhase implements SearchPhase {
 
     @Override
     public Map<String, ? extends SearchParseElement> parseElements() {
-        ImmutableMap.Builder<String, SearchParseElement> parseElements = ImmutableMap.builder();
-        parseElements.put("from", new FromParseElement()).put("size", new SizeParseElement())
-                .put("indices_boost", new IndicesBoostParseElement())
-                .put("indicesBoost", new IndicesBoostParseElement())
-                .put("query", new QueryParseElement())
-                .put("queryBinary", new QueryBinaryParseElement())
-                .put("query_binary", new QueryBinaryParseElement())
-                .put("filter", new PostFilterParseElement()) // For bw comp reason, should be removed in version 1.1
-                .put("post_filter", new PostFilterParseElement())
-                .put("postFilter", new PostFilterParseElement())
-                .put("filterBinary", new FilterBinaryParseElement())
-                .put("filter_binary", new FilterBinaryParseElement())
-                .put("sort", new SortParseElement())
-                .put("trackScores", new TrackScoresParseElement())
-                .put("track_scores", new TrackScoresParseElement())
-                .put("min_score", new MinScoreParseElement())
-                .put("minScore", new MinScoreParseElement())
-                .put("timeout", new TimeoutParseElement())
-                .put("terminate_after", new TerminateAfterParseElement())
-                .putAll(aggregationPhase.parseElements())
-                .putAll(suggestPhase.parseElements())
-                .putAll(rescorePhase.parseElements());
-        return parseElements.build();
+        Map<String, SearchParseElement> parseElements = new HashMap<>();
+        parseElements.put("from", new FromParseElement());
+        parseElements.put("size", new SizeParseElement());
+        parseElements.put("indices_boost", new IndicesBoostParseElement());
+        parseElements.put("indicesBoost", new IndicesBoostParseElement());
+        parseElements.put("query", new QueryParseElement());
+        parseElements.put("queryBinary", new QueryBinaryParseElement());
+        parseElements.put("query_binary", new QueryBinaryParseElement());
+        parseElements.put("filter", new PostFilterParseElement()); // For bw comp reason, should be removed in version 1.1
+        parseElements.put("post_filter", new PostFilterParseElement());
+        parseElements.put("postFilter", new PostFilterParseElement());
+        parseElements.put("filterBinary", new FilterBinaryParseElement());
+        parseElements.put("filter_binary", new FilterBinaryParseElement());
+        parseElements.put("sort", new SortParseElement());
+        parseElements.put("trackScores", new TrackScoresParseElement());
+        parseElements.put("track_scores", new TrackScoresParseElement());
+        parseElements.put("min_score", new MinScoreParseElement());
+        parseElements.put("minScore", new MinScoreParseElement());
+        parseElements.put("timeout", new TimeoutParseElement());
+        parseElements.put("terminate_after", new TerminateAfterParseElement());
+        parseElements.putAll(aggregationPhase.parseElements());
+        parseElements.putAll(suggestPhase.parseElements());
+        parseElements.putAll(rescorePhase.parseElements());
+        return unmodifiableMap(parseElements);
     }
 
     @Override

+ 5 - 6
core/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java

@@ -19,8 +19,6 @@
 
 package org.elasticsearch.search.rescore;
 
-import com.google.common.collect.ImmutableMap;
-import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.common.component.AbstractComponent;
@@ -33,10 +31,13 @@ import org.elasticsearch.search.internal.SearchContext;
 import java.io.IOException;
 import java.util.Map;
 
+import static java.util.Collections.singletonMap;
+
 /**
  */
 public class RescorePhase extends AbstractComponent implements SearchPhase {
-    
+    private static final Map<String, SearchParseElement> PARSE_ELEMENTS = singletonMap("rescore", new RescoreParseElement());
+
     @Inject
     public RescorePhase(Settings settings) {
         super(settings);
@@ -44,9 +45,7 @@ public class RescorePhase extends AbstractComponent implements SearchPhase {
 
     @Override
     public Map<String, ? extends SearchParseElement> parseElements() {
-        ImmutableMap.Builder<String, SearchParseElement> parseElements = ImmutableMap.builder();
-        parseElements.put("rescore", new RescoreParseElement());
-        return parseElements.build();
+        return PARSE_ELEMENTS;
     }
 
     @Override

+ 13 - 13
core/src/main/java/org/elasticsearch/search/sort/SortParseElement.java

@@ -19,8 +19,6 @@
 
 package org.elasticsearch.search.sort;
 
-import com.google.common.collect.ImmutableMap;
-
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.Sort;
@@ -34,17 +32,19 @@ import org.elasticsearch.index.fielddata.IndexFieldData;
 import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
 import org.elasticsearch.index.mapper.MappedFieldType;
 import org.elasticsearch.index.mapper.core.LongFieldMapper;
-import org.elasticsearch.index.mapper.object.ObjectMapper;
 import org.elasticsearch.index.query.support.NestedInnerQueryParseSupport;
 import org.elasticsearch.search.MultiValueMode;
 import org.elasticsearch.search.SearchParseElement;
 import org.elasticsearch.search.SearchParseException;
 import org.elasticsearch.search.internal.SearchContext;
-import org.elasticsearch.search.internal.SubSearchContext;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+
+import static java.util.Collections.unmodifiableMap;
 
 /**
  *
@@ -62,16 +62,16 @@ public class SortParseElement implements SearchParseElement {
     public static final String SCORE_FIELD_NAME = "_score";
     public static final String DOC_FIELD_NAME = "_doc";
 
-    private final ImmutableMap<String, SortParser> parsers;
+    private static final Map<String, SortParser> PARSERS;
 
-    public SortParseElement() {
-        ImmutableMap.Builder<String, SortParser> builder = ImmutableMap.builder();
-        addParser(builder, new ScriptSortParser());
-        addParser(builder, new GeoDistanceSortParser());
-        this.parsers = builder.build();
+    static {
+        Map<String, SortParser> parsers = new HashMap<>();
+        addParser(parsers, new ScriptSortParser());
+        addParser(parsers, new GeoDistanceSortParser());
+        PARSERS = unmodifiableMap(parsers);
     }
 
-    private void addParser(ImmutableMap.Builder<String, SortParser> parsers, SortParser parser) {
+    private static void addParser(Map<String, SortParser> parsers, SortParser parser) {
         for (String name : parser.names()) {
             parsers.put(name, parser);
         }
@@ -140,8 +140,8 @@ public class SortParseElement implements SearchParseElement {
                     }
                     addSortField(context, sortFields, fieldName, reverse, unmappedType, missing, sortMode, nestedFilterParseHelper);
                 } else {
-                    if (parsers.containsKey(fieldName)) {
-                        sortFields.add(parsers.get(fieldName).parse(parser, context));
+                    if (PARSERS.containsKey(fieldName)) {
+                        sortFields.add(PARSERS.get(fieldName).parse(parser, context));
                     } else {
                         while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
                             if (token == XContentParser.Token.FIELD_NAME) {

+ 5 - 5
core/src/main/java/org/elasticsearch/search/suggest/SuggestPhase.java

@@ -18,7 +18,6 @@
  */
 package org.elasticsearch.search.suggest;
 
-import com.google.common.collect.ImmutableMap;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.elasticsearch.ElasticsearchException;
@@ -38,23 +37,24 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import static java.util.Collections.singletonMap;
+
 /**
  */
 public class SuggestPhase extends AbstractComponent implements SearchPhase {
-
+    private final Map<String, SearchParseElement> parseElements;
     private final SuggestParseElement parseElement;
 
     @Inject
     public SuggestPhase(Settings settings, SuggestParseElement suggestParseElement) {
         super(settings);
         this.parseElement = suggestParseElement;
+        parseElements = singletonMap("suggest", parseElement);
     }
 
     @Override
     public Map<String, ? extends SearchParseElement> parseElements() {
-        ImmutableMap.Builder<String, SearchParseElement> parseElements = ImmutableMap.builder();
-        parseElements.put("suggest", parseElement);
-        return parseElements.build();
+        return parseElements;
     }
 
     public SuggestParseElement parseElement() {

+ 7 - 13
core/src/main/java/org/elasticsearch/search/suggest/completion/Completion090PostingsFormat.java

@@ -18,8 +18,6 @@
  */
 package org.elasticsearch.search.suggest.completion;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableMap.Builder;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
@@ -59,6 +57,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import static java.util.Collections.singletonMap;
+
 /**
  * This {@link PostingsFormat} is basically a T-Sink for a default postings
  * format that is used to store postings on disk fitting the lucene APIs and
@@ -75,18 +75,12 @@ public class Completion090PostingsFormat extends PostingsFormat {
     public static final int SUGGEST_VERSION_CURRENT = SUGGEST_CODEC_VERSION;
     public static final String EXTENSION = "cmp";
 
-    private final static ESLogger logger = Loggers.getLogger(Completion090PostingsFormat.class);
+    private static final ESLogger logger = Loggers.getLogger(Completion090PostingsFormat.class);
+    private static final CompletionLookupProvider LOOKUP_PROVIDER = new AnalyzingCompletionLookupProvider(true, false, true, false);
+    private static final Map<String, CompletionLookupProvider> PROVIDERS = singletonMap(LOOKUP_PROVIDER.getName(), LOOKUP_PROVIDER);
     private PostingsFormat delegatePostingsFormat;
-    private final static Map<String, CompletionLookupProvider> providers;
     private CompletionLookupProvider writeProvider;
 
-
-    static {
-        final CompletionLookupProvider provider = new AnalyzingCompletionLookupProvider(true, false, true, false);
-        final Builder<String, CompletionLookupProvider> builder = ImmutableMap.builder();
-        providers = builder.put(provider.getName(), provider).build();
-    }
-
     public Completion090PostingsFormat(PostingsFormat delegatePostingsFormat, CompletionLookupProvider provider) {
         super(CODEC_NAME);
         this.delegatePostingsFormat = delegatePostingsFormat;
@@ -173,11 +167,11 @@ public class Completion090PostingsFormat extends PostingsFormat {
             try {
                 PostingsFormat delegatePostingsFormat = PostingsFormat.forName(input.readString());
                 String providerName = input.readString();
-                CompletionLookupProvider completionLookupProvider = providers.get(providerName);
+                CompletionLookupProvider completionLookupProvider = PROVIDERS.get(providerName);
                 if (completionLookupProvider == null) {
                     throw new IllegalStateException("no provider with name [" + providerName + "] registered");
                 }
-                // TODO: we could clone the ReadState and make it always forward IOContext.MERGE to prevent unecessary heap usage? 
+                // TODO: we could clone the ReadState and make it always forward IOContext.MERGE to prevent unecessary heap usage?
                 delegateProducer = delegatePostingsFormat.fieldsProducer(state);
                 /*
                  * If we are merging we don't load the FSTs at all such that we

+ 8 - 6
core/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java

@@ -21,7 +21,6 @@ package org.elasticsearch.snapshots;
 
 import com.carrotsearch.hppc.cursors.ObjectCursor;
 import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
-import com.google.common.collect.ImmutableMap;
 
 import org.apache.lucene.util.CollectionUtil;
 import org.elasticsearch.ExceptionsHelper;
@@ -66,12 +65,15 @@ import java.io.IOException;
 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.Locale;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 
+import static java.util.Collections.unmodifiableMap;
 import static org.elasticsearch.cluster.SnapshotsInProgress.completed;
 
 /**
@@ -466,9 +468,9 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
      * @param snapshotId snapshot id
      * @return map of shard id to snapshot status
      */
-    public ImmutableMap<ShardId, IndexShardSnapshotStatus> snapshotShards(SnapshotId snapshotId) throws IOException {
+    public Map<ShardId, IndexShardSnapshotStatus> snapshotShards(SnapshotId snapshotId) throws IOException {
         validate(snapshotId);
-        ImmutableMap.Builder<ShardId, IndexShardSnapshotStatus> shardStatusBuilder = ImmutableMap.builder();
+        Map<ShardId, IndexShardSnapshotStatus> shardStatus = new HashMap<>();
         Repository repository = repositoriesService.repository(snapshotId.getRepository());
         IndexShardRepository indexShardRepository = repositoriesService.indexShardRepository(snapshotId.getRepository());
         Snapshot snapshot = repository.readSnapshot(snapshotId);
@@ -484,15 +486,15 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
                         IndexShardSnapshotStatus shardSnapshotStatus = new IndexShardSnapshotStatus();
                         shardSnapshotStatus.updateStage(IndexShardSnapshotStatus.Stage.FAILURE);
                         shardSnapshotStatus.failure(shardFailure.reason());
-                        shardStatusBuilder.put(shardId, shardSnapshotStatus);
+                        shardStatus.put(shardId, shardSnapshotStatus);
                     } else {
                         IndexShardSnapshotStatus shardSnapshotStatus = indexShardRepository.snapshotStatus(snapshotId, snapshot.version(), shardId);
-                        shardStatusBuilder.put(shardId, shardSnapshotStatus);
+                        shardStatus.put(shardId, shardSnapshotStatus);
                     }
                 }
             }
         }
-        return shardStatusBuilder.build();
+        return unmodifiableMap(shardStatus);
     }
 
 

+ 6 - 1
core/src/main/java/org/elasticsearch/tribe/TribeService.java

@@ -19,6 +19,8 @@
 
 package org.elasticsearch.tribe;
 
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
 import org.elasticsearch.cluster.ClusterChangedEvent;
@@ -242,7 +244,10 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
                     for (DiscoveryNode tribe : tribeState.nodes()) {
                         if (currentState.nodes().get(tribe.id()) == null) {
                             // a new node, add it, but also add the tribe name to the attributes
-                            Map<String, String> tribeAttr = new HashMap<>(tribe.attributes());
+                            Map<String, String> tribeAttr = new HashMap<>();
+                            for (ObjectObjectCursor<String, String> attr : tribe.attributes()) {
+                                tribeAttr.put(attr.key, attr.value);
+                            }
                             tribeAttr.put(TRIBE_NAME, tribeName);
                             DiscoveryNode discoNode = new DiscoveryNode(tribe.name(), tribe.id(), tribe.getHostName(), tribe.getHostAddress(), tribe.address(), unmodifiableMap(tribeAttr), tribe.version());
                             logger.info("[{}] adding node [{}]", tribeName, discoNode);

+ 1 - 0
dev-tools/src/main/resources/forbidden/all-signatures.txt

@@ -140,6 +140,7 @@ com.google.common.collect.ImmutableMap#of(java.lang.Object, java.lang.Object, ja
 com.google.common.collect.ImmutableMap#of(java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object)
 com.google.common.collect.ImmutableMap#of(java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object)
 com.google.common.collect.ImmutableMap#copyOf(java.util.Map)
+com.google.common.collect.ImmutableMap$Builder
 
 @defaultMessage Do not violate java's access system
 java.lang.reflect.AccessibleObject#setAccessible(boolean)

+ 11 - 3
plugins/discovery-multicast/src/main/java/org/elasticsearch/plugin/discovery/multicast/MulticastZenPing.java

@@ -19,6 +19,8 @@
 
 package org.elasticsearch.plugin.discovery.multicast;
 
+import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
+
 import org.apache.lucene.util.Constants;
 import org.elasticsearch.ExceptionsHelper;
 import org.elasticsearch.Version;
@@ -44,7 +46,13 @@ import org.elasticsearch.common.xcontent.XContentType;
 import org.elasticsearch.discovery.zen.ping.PingContextProvider;
 import org.elasticsearch.discovery.zen.ping.ZenPing;
 import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.transport.*;
+import org.elasticsearch.transport.EmptyTransportResponseHandler;
+import org.elasticsearch.transport.TransportChannel;
+import org.elasticsearch.transport.TransportException;
+import org.elasticsearch.transport.TransportRequest;
+import org.elasticsearch.transport.TransportRequestHandler;
+import org.elasticsearch.transport.TransportResponse;
+import org.elasticsearch.transport.TransportService;
 
 import java.io.IOException;
 import java.net.SocketAddress;
@@ -483,8 +491,8 @@ public class MulticastZenPing extends AbstractLifecycleComponent<ZenPing> implem
                 }
 
                 builder.startObject("attributes");
-                for (Map.Entry<String, String> attr : localNode.attributes().entrySet()) {
-                    builder.field(attr.getKey(), attr.getValue());
+                for (ObjectObjectCursor<String, String> attr : localNode.attributes()) {
+                    builder.field(attr.key, attr.value);
                 }
                 builder.endObject();