Bläddra i källkod

Merge branch 'master' into fix/bwc_creation

Ryan Ernst 10 år sedan
förälder
incheckning
448d3498b1
55 ändrade filer med 730 tillägg och 333 borttagningar
  1. 1 1
      core/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java
  2. 38 7
      core/src/main/java/org/elasticsearch/action/admin/indices/stats/ShardStats.java
  3. 1 1
      core/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java
  4. 13 3
      core/src/main/java/org/elasticsearch/action/admin/indices/warmer/delete/TransportDeleteWarmerAction.java
  5. 7 7
      core/src/main/java/org/elasticsearch/action/search/TransportClearScrollAction.java
  6. 3 6
      core/src/main/java/org/elasticsearch/action/search/type/ParsedScrollId.java
  7. 38 0
      core/src/main/java/org/elasticsearch/action/search/type/ScrollIdForNode.java
  8. 2 3
      core/src/main/java/org/elasticsearch/action/search/type/TransportSearchHelper.java
  9. 8 9
      core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollQueryAndFetchAction.java
  10. 5 6
      core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollQueryThenFetchAction.java
  11. 8 9
      core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java
  12. 14 12
      core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java
  13. 11 0
      core/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java
  14. 2 1
      core/src/main/java/org/elasticsearch/common/Strings.java
  15. 1 1
      core/src/main/java/org/elasticsearch/common/util/MultiDataPathUpgrader.java
  16. 34 19
      core/src/main/java/org/elasticsearch/index/IndexService.java
  17. 38 0
      core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperForType.java
  18. 5 16
      core/src/main/java/org/elasticsearch/index/mapper/MapperService.java
  19. 6 7
      core/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java
  20. 6 11
      core/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java
  21. 5 6
      core/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java
  22. 4 1
      core/src/main/java/org/elasticsearch/index/query/PrefixQueryParser.java
  23. 2 1
      core/src/main/java/org/elasticsearch/index/query/RangeQueryParser.java
  24. 4 1
      core/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java
  25. 6 3
      core/src/main/java/org/elasticsearch/index/query/TermQueryParser.java
  26. 3 3
      core/src/main/java/org/elasticsearch/index/query/support/InnerHitsQueryParserHelper.java
  27. 13 21
      core/src/main/java/org/elasticsearch/index/shard/IndexShard.java
  28. 39 6
      core/src/main/java/org/elasticsearch/index/shard/ShardPath.java
  29. 1 2
      core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java
  30. 7 26
      core/src/main/java/org/elasticsearch/index/termvectors/ShardTermVectorsService.java
  31. 30 11
      core/src/main/java/org/elasticsearch/index/translog/Translog.java
  32. 44 45
      core/src/main/java/org/elasticsearch/indices/IndicesService.java
  33. 6 7
      core/src/main/java/org/elasticsearch/percolator/PercolatorService.java
  34. 40 0
      core/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsSubSearchContext.java
  35. 2 0
      core/src/test/java/org/elasticsearch/common/StringsTests.java
  36. 2 2
      core/src/test/java/org/elasticsearch/common/util/MultiDataPathUpgraderTests.java
  37. 1 1
      core/src/test/java/org/elasticsearch/document/BulkIT.java
  38. 2 4
      core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java
  39. 6 33
      core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java
  40. 2 7
      core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java
  41. 2 2
      core/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java
  42. 40 0
      core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
  43. 69 0
      core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java
  44. 4 4
      core/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java
  45. 12 0
      core/src/test/java/org/elasticsearch/indices/warmer/SimpleIndicesWarmerIT.java
  46. 6 0
      core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java
  47. 73 5
      core/src/test/java/org/elasticsearch/ttl/SimpleTTLIT.java
  48. 19 8
      core/src/test/java/org/elasticsearch/update/UpdateNoopIT.java
  49. 8 10
      docs/reference/docs/update.asciidoc
  50. 4 0
      docs/reference/mapping/fields/ttl-field.asciidoc
  51. 1 1
      docs/reference/migration/migrate_2_0/query_dsl.asciidoc
  52. 18 0
      docs/reference/migration/migrate_2_0/removals.asciidoc
  53. 11 1
      docs/reference/migration/migrate_2_1.asciidoc
  54. 1 1
      rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_warmer.json
  55. 2 2
      rest-api-spec/src/main/resources/rest-api-spec/test/update/75_ttl.yaml

+ 1 - 1
core/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java

@@ -106,7 +106,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
             for (IndexShard indexShard : indexService) {
                 if (indexShard.routingEntry() != null && indexShard.routingEntry().active()) {
                     // only report on fully started shards
-                    shardsStats.add(new ShardStats(indexShard, indexShard.routingEntry(), SHARD_STATS_FLAGS));
+                    shardsStats.add(new ShardStats(indexShard, SHARD_STATS_FLAGS));
                 }
             }
         }

+ 38 - 7
core/src/main/java/org/elasticsearch/action/admin/indices/stats/ShardStats.java

@@ -29,6 +29,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentBuilderString;
 import org.elasticsearch.index.engine.CommitStats;
 import org.elasticsearch.index.shard.IndexShard;
+import org.elasticsearch.index.shard.ShardPath;
 
 import java.io.IOException;
 
@@ -37,20 +38,23 @@ import static org.elasticsearch.cluster.routing.ShardRouting.readShardRoutingEnt
 /**
  */
 public class ShardStats extends BroadcastShardResponse implements ToXContent {
-
     private ShardRouting shardRouting;
-
-    CommonStats commonStats;
-
+    private CommonStats commonStats;
     @Nullable
-    CommitStats commitStats;
+    private CommitStats commitStats;
+    private String dataPath;
+    private String statePath;
+    private boolean isCustomDataPath;
 
     ShardStats() {
     }
 
-    public ShardStats(IndexShard indexShard, ShardRouting shardRouting, CommonStatsFlags flags) {
+    public ShardStats(IndexShard indexShard, CommonStatsFlags flags) {
         super(indexShard.shardId());
-        this.shardRouting = shardRouting;
+        this.shardRouting = indexShard.routingEntry();
+        this.dataPath = indexShard.shardPath().getRootDataPath().toString();
+        this.statePath = indexShard.shardPath().getRootStatePath().toString();
+        this.isCustomDataPath = indexShard.shardPath().isCustomDataPath();
         this.commonStats = new CommonStats(indexShard, flags);
         this.commitStats = indexShard.commitStats();
     }
@@ -70,6 +74,18 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
         return this.commitStats;
     }
 
+    public String getDataPath() {
+        return dataPath;
+    }
+
+    public String getStatePath() {
+        return statePath;
+    }
+
+    public boolean isCustomDataPath() {
+        return isCustomDataPath;
+    }
+
     public static ShardStats readShardStats(StreamInput in) throws IOException {
         ShardStats stats = new ShardStats();
         stats.readFrom(in);
@@ -82,6 +98,9 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
         shardRouting = readShardRoutingEntry(in);
         commonStats = CommonStats.readCommonStats(in);
         commitStats = CommitStats.readOptionalCommitStatsFrom(in);
+        statePath = in.readString();
+        dataPath = in.readString();
+        isCustomDataPath = in.readBoolean();
     }
 
     @Override
@@ -90,6 +109,9 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
         shardRouting.writeTo(out);
         commonStats.writeTo(out);
         out.writeOptionalStreamable(commitStats);
+        out.writeString(statePath);
+        out.writeString(dataPath);
+        out.writeBoolean(isCustomDataPath);
     }
 
     @Override
@@ -105,12 +127,21 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
         if (commitStats != null) {
             commitStats.toXContent(builder, params);
         }
+        builder.startObject(Fields.SHARD_PATH);
+        builder.field(Fields.STATE_PATH, statePath);
+        builder.field(Fields.DATA_PATH, dataPath);
+        builder.field(Fields.IS_CUSTOM_DATA_PATH, isCustomDataPath);
+        builder.endObject();
         return builder;
     }
 
     static final class Fields {
         static final XContentBuilderString ROUTING = new XContentBuilderString("routing");
         static final XContentBuilderString STATE = new XContentBuilderString("state");
+        static final XContentBuilderString STATE_PATH = new XContentBuilderString("state_path");
+        static final XContentBuilderString DATA_PATH = new XContentBuilderString("data_path");
+        static final XContentBuilderString IS_CUSTOM_DATA_PATH = new XContentBuilderString("is_custom_data_path");
+        static final XContentBuilderString SHARD_PATH = new XContentBuilderString("shard_path");
         static final XContentBuilderString PRIMARY = new XContentBuilderString("primary");
         static final XContentBuilderString NODE = new XContentBuilderString("node");
         static final XContentBuilderString RELOCATING_NODE = new XContentBuilderString("relocating_node");

+ 1 - 1
core/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java

@@ -189,7 +189,7 @@ public class TransportIndicesStatsAction extends TransportBroadcastAction<Indice
             flags.set(CommonStatsFlags.Flag.Recovery);
         }
 
-        return new ShardStats(indexShard, indexShard.routingEntry(), flags);
+        return new ShardStats(indexShard, flags);
     }
 
     static class IndexShardStatsRequest extends BroadcastShardRequest {

+ 13 - 3
core/src/main/java/org/elasticsearch/action/admin/indices/warmer/delete/TransportDeleteWarmerAction.java

@@ -92,6 +92,14 @@ public class TransportDeleteWarmerAction extends TransportMasterNodeAction<Delet
                 MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
 
                 boolean globalFoundAtLeastOne = false;
+                boolean deleteAll = false;
+                for (int i=0; i<request.names().length; i++){
+                    if (request.names()[i].equals(MetaData.ALL)) {
+                        deleteAll = true;
+                        break;
+                    }
+                }
+
                 for (String index : concreteIndices) {
                     IndexMetaData indexMetaData = currentState.metaData().index(index);
                     if (indexMetaData == null) {
@@ -103,7 +111,7 @@ public class TransportDeleteWarmerAction extends TransportMasterNodeAction<Delet
                         for (IndexWarmersMetaData.Entry entry : warmers.entries()) {
                             boolean keepWarmer = true;
                             for (String warmer : request.names()) {
-                                if (Regex.simpleMatch(warmer, entry.name()) || warmer.equals("_all")) {
+                                if (Regex.simpleMatch(warmer, entry.name()) || warmer.equals(MetaData.ALL)) {
                                     globalFoundAtLeastOne = true;
                                     keepWarmer =  false;
                                     // don't add it...
@@ -123,7 +131,7 @@ public class TransportDeleteWarmerAction extends TransportMasterNodeAction<Delet
                     }
                 }
 
-                if (!globalFoundAtLeastOne) {
+                if (globalFoundAtLeastOne == false && deleteAll == false) {
                     throw new IndexWarmerMissingException(request.names());
                 }
 
@@ -137,11 +145,13 @@ public class TransportDeleteWarmerAction extends TransportMasterNodeAction<Delet
                         if (warmers != null) {
                             for (IndexWarmersMetaData.Entry entry : warmers.entries()) {
                                 for (String warmer : request.names()) {
-                                    if (Regex.simpleMatch(warmer, entry.name()) || warmer.equals("_all")) {
+                                    if (Regex.simpleMatch(warmer, entry.name()) || warmer.equals(MetaData.ALL)) {
                                         logger.info("[{}] delete warmer [{}]", index, entry.name());
                                     }
                                 }
                             }
+                        } else if(deleteAll){
+                            logger.debug("no warmers to delete on index [{}]", index);
                         }
                     }
                 }

+ 7 - 7
core/src/main/java/org/elasticsearch/action/search/TransportClearScrollAction.java

@@ -20,6 +20,7 @@
 package org.elasticsearch.action.search;
 
 import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.search.type.ScrollIdForNode;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.HandledTransportAction;
 import org.elasticsearch.cluster.ClusterService;
@@ -27,7 +28,6 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.CountDown;
@@ -69,7 +69,7 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
         final DiscoveryNodes nodes;
         final CountDown expectedOps;
         final ClearScrollRequest request;
-        final List<Tuple<String, Long>[]> contexts = new ArrayList<>();
+        final List<ScrollIdForNode[]> contexts = new ArrayList<>();
         final ActionListener<ClearScrollResponse> listener;
         final AtomicReference<Throwable> expHolder;
         final AtomicInteger numberOfFreedSearchContexts = new AtomicInteger(0);
@@ -81,7 +81,7 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
                 expectedOps = nodes.size();
             } else {
                 for (String parsedScrollId : request.getScrollIds()) {
-                    Tuple<String, Long>[] context = parseScrollId(parsedScrollId).getContext();
+                    ScrollIdForNode[] context = parseScrollId(parsedScrollId).getContext();
                     expectedOps += context.length;
                     this.contexts.add(context);
                 }
@@ -114,15 +114,15 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
                     });
                 }
             } else {
-                for (Tuple<String, Long>[] context : contexts) {
-                    for (Tuple<String, Long> target : context) {
-                        final DiscoveryNode node = nodes.get(target.v1());
+                for (ScrollIdForNode[] context : contexts) {
+                    for (ScrollIdForNode target : context) {
+                        final DiscoveryNode node = nodes.get(target.getNode());
                         if (node == null) {
                             onFreedContext(false);
                             continue;
                         }
 
-                        searchServiceTransportAction.sendFreeContext(node, target.v2(), request, new ActionListener<SearchServiceTransportAction.SearchFreeContextResponse>() {
+                        searchServiceTransportAction.sendFreeContext(node, target.getScrollId(), request, new ActionListener<SearchServiceTransportAction.SearchFreeContextResponse>() {
                             @Override
                             public void onResponse(SearchServiceTransportAction.SearchFreeContextResponse freed) {
                                 onFreedContext(freed.isFreed());

+ 3 - 6
core/src/main/java/org/elasticsearch/action/search/type/ParsedScrollId.java

@@ -19,9 +19,6 @@
 
 package org.elasticsearch.action.search.type;
 
-import org.elasticsearch.Version;
-import org.elasticsearch.common.collect.Tuple;
-
 import java.util.Map;
 
 /**
@@ -39,11 +36,11 @@ public class ParsedScrollId {
 
     private final String type;
 
-    private final Tuple<String, Long>[] context;
+    private final ScrollIdForNode[] context;
 
     private final Map<String, String> attributes;
 
-    public ParsedScrollId(String source, String type, Tuple<String, Long>[] context, Map<String, String> attributes) {
+    public ParsedScrollId(String source, String type, ScrollIdForNode[] context, Map<String, String> attributes) {
         this.source = source;
         this.type = type;
         this.context = context;
@@ -58,7 +55,7 @@ public class ParsedScrollId {
         return type;
     }
 
-    public Tuple<String, Long>[] getContext() {
+    public ScrollIdForNode[] getContext() {
         return context;
     }
 

+ 38 - 0
core/src/main/java/org/elasticsearch/action/search/type/ScrollIdForNode.java

@@ -0,0 +1,38 @@
+/*
+ * 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.action.search.type;
+
+public class ScrollIdForNode {
+    private final String node;
+    private final long scrollId;
+
+    public ScrollIdForNode(String node, long scrollId) {
+        this.node = node;
+        this.scrollId = scrollId;
+    }
+
+    public String getNode() {
+        return node;
+    }
+
+    public long getScrollId() {
+        return scrollId;
+    }
+}

+ 2 - 3
core/src/main/java/org/elasticsearch/action/search/type/TransportSearchHelper.java

@@ -30,7 +30,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.common.Base64;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.util.concurrent.AtomicArray;
 import org.elasticsearch.search.SearchPhaseResult;
 import org.elasticsearch.search.internal.InternalScrollSearchRequest;
@@ -103,14 +102,14 @@ public abstract class TransportSearchHelper {
             throw new IllegalArgumentException("Malformed scrollId [" + scrollId + "]");
         }
 
-        @SuppressWarnings({"unchecked"}) Tuple<String, Long>[] context = new Tuple[contextSize];
+        ScrollIdForNode[] context = new ScrollIdForNode[contextSize];
         for (int i = 0; i < contextSize; i++) {
             String element = elements[index++];
             int sep = element.indexOf(':');
             if (sep == -1) {
                 throw new IllegalArgumentException("Malformed scrollId [" + scrollId + "]");
             }
-            context[i] = new Tuple<>(element.substring(sep + 1), Long.parseLong(element.substring(0, sep)));
+            context[i] = new ScrollIdForNode(element.substring(sep + 1), Long.parseLong(element.substring(0, sep)));
         }
         Map<String, String> attributes;
         int attributesSize = Integer.parseInt(elements[index++]);

+ 8 - 9
core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollQueryAndFetchAction.java

@@ -25,7 +25,6 @@ import org.elasticsearch.action.search.*;
 import org.elasticsearch.cluster.ClusterService;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.Settings;
@@ -115,15 +114,15 @@ public class TransportSearchScrollQueryAndFetchAction extends AbstractComponent
                 return;
             }
 
-            Tuple<String, Long>[] context = scrollId.getContext();
+            ScrollIdForNode[] context = scrollId.getContext();
             for (int i = 0; i < context.length; i++) {
-                Tuple<String, Long> target = context[i];
-                DiscoveryNode node = nodes.get(target.v1());
+                ScrollIdForNode target = context[i];
+                DiscoveryNode node = nodes.get(target.getNode());
                 if (node != null) {
-                    executePhase(i, node, target.v2());
+                    executePhase(i, node, target.getScrollId());
                 } else {
                     if (logger.isDebugEnabled()) {
-                        logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
+                        logger.debug("Node [" + target.getNode() + "] not available for scroll request [" + scrollId.getSource() + "]");
                     }
                     successfulOps.decrementAndGet();
                     if (counter.decrementAndGet() == 0) {
@@ -132,11 +131,11 @@ public class TransportSearchScrollQueryAndFetchAction extends AbstractComponent
                 }
             }
 
-            for (Tuple<String, Long> target : scrollId.getContext()) {
-                DiscoveryNode node = nodes.get(target.v1());
+            for (ScrollIdForNode target : scrollId.getContext()) {
+                DiscoveryNode node = nodes.get(target.getNode());
                 if (node == null) {
                     if (logger.isDebugEnabled()) {
-                        logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
+                        logger.debug("Node [" + target.getNode() + "] not available for scroll request [" + scrollId.getSource() + "]");
                     }
                     successfulOps.decrementAndGet();
                     if (counter.decrementAndGet() == 0) {

+ 5 - 6
core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollQueryThenFetchAction.java

@@ -26,7 +26,6 @@ import org.elasticsearch.action.search.*;
 import org.elasticsearch.cluster.ClusterService;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.Settings;
@@ -124,15 +123,15 @@ public class TransportSearchScrollQueryThenFetchAction extends AbstractComponent
             }
             final AtomicInteger counter = new AtomicInteger(scrollId.getContext().length);
 
-            Tuple<String, Long>[] context = scrollId.getContext();
+            ScrollIdForNode[] context = scrollId.getContext();
             for (int i = 0; i < context.length; i++) {
-                Tuple<String, Long> target = context[i];
-                DiscoveryNode node = nodes.get(target.v1());
+                ScrollIdForNode target = context[i];
+                DiscoveryNode node = nodes.get(target.getNode());
                 if (node != null) {
-                    executeQueryPhase(i, counter, node, target.v2());
+                    executeQueryPhase(i, counter, node, target.getScrollId());
                 } else {
                     if (logger.isDebugEnabled()) {
-                        logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
+                        logger.debug("Node [" + target.getNode() + "] not available for scroll request [" + scrollId.getSource() + "]");
                     }
                     successfulOps.decrementAndGet();
                     if (counter.decrementAndGet() == 0) {

+ 8 - 9
core/src/main/java/org/elasticsearch/action/search/type/TransportSearchScrollScanAction.java

@@ -28,7 +28,6 @@ import org.elasticsearch.action.search.ShardSearchFailure;
 import org.elasticsearch.cluster.ClusterService;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.Settings;
@@ -125,15 +124,15 @@ public class TransportSearchScrollScanAction extends AbstractComponent {
                 return;
             }
 
-            Tuple<String, Long>[] context = scrollId.getContext();
+            ScrollIdForNode[] context = scrollId.getContext();
             for (int i = 0; i < context.length; i++) {
-                Tuple<String, Long> target = context[i];
-                DiscoveryNode node = nodes.get(target.v1());
+                ScrollIdForNode target = context[i];
+                DiscoveryNode node = nodes.get(target.getNode());
                 if (node != null) {
-                    executePhase(i, node, target.v2());
+                    executePhase(i, node, target.getScrollId());
                 } else {
                     if (logger.isDebugEnabled()) {
-                        logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
+                        logger.debug("Node [" + target.getScrollId() + "] not available for scroll request [" + scrollId.getSource() + "]");
                     }
                     successfulOps.decrementAndGet();
                     if (counter.decrementAndGet() == 0) {
@@ -142,11 +141,11 @@ public class TransportSearchScrollScanAction extends AbstractComponent {
                 }
             }
 
-            for (Tuple<String, Long> target : scrollId.getContext()) {
-                DiscoveryNode node = nodes.get(target.v1());
+            for (ScrollIdForNode target : scrollId.getContext()) {
+                DiscoveryNode node = nodes.get(target.getNode());
                 if (node == null) {
                     if (logger.isDebugEnabled()) {
-                        logger.debug("Node [" + target.v1() + "] not available for scroll request [" + scrollId.getSource() + "]");
+                        logger.debug("Node [" + target.getNode() + "] not available for scroll request [" + scrollId.getSource() + "]");
                     }
                     successfulOps.decrementAndGet();
                     if (counter.decrementAndGet() == 0) {

+ 14 - 12
core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java

@@ -78,7 +78,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
 
     private boolean scriptedUpsert = false;
     private boolean docAsUpsert = false;
-    private boolean detectNoop = false;
+    private boolean detectNoop = true;
 
     @Nullable
     private IndexRequest doc;
@@ -243,7 +243,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
      * The script to execute. Note, make sure not to send different script each
      * times and instead use script params if possible with the same
      * (automatically compiled) script.
-     * 
+     *
      * @deprecated Use {@link #script(Script)} instead
      */
     @Deprecated
@@ -256,7 +256,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
      * The script to execute. Note, make sure not to send different script each
      * times and instead use script params if possible with the same
      * (automatically compiled) script.
-     * 
+     *
      * @deprecated Use {@link #script(Script)} instead
      */
     @Deprecated
@@ -267,7 +267,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
 
     /**
      * The language of the script to execute.
-     * 
+     *
      * @deprecated Use {@link #script(Script)} instead
      */
     @Deprecated
@@ -286,7 +286,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
 
     /**
      * Add a script parameter.
-     * 
+     *
      * @deprecated Use {@link #script(Script)} instead
      */
     @Deprecated
@@ -311,7 +311,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
 
     /**
      * Sets the script parameters to use with the script.
-     * 
+     *
      * @deprecated Use {@link #script(Script)} instead
      */
     @Deprecated
@@ -338,7 +338,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
      * The script to execute. Note, make sure not to send different script each
      * times and instead use script params if possible with the same
      * (automatically compiled) script.
-     * 
+     *
      * @deprecated Use {@link #script(Script)} instead
      */
     @Deprecated
@@ -360,7 +360,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
      *            The script type
      * @param scriptParams
      *            The script parameters
-     * 
+     *
      * @deprecated Use {@link #script(Script)} instead
      */
     @Deprecated
@@ -623,7 +623,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
     }
 
     /**
-     * Should this update attempt to detect if it is a noop?
+     * Should this update attempt to detect if it is a noop? Defaults to true.
      * @return this for chaining
      */
     public UpdateRequest detectNoop(boolean detectNoop) {
@@ -631,6 +631,9 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
         return this;
     }
 
+    /**
+     * Should this update attempt to detect if it is a noop? Defaults to true.
+     */
     public boolean detectNoop() {
         return detectNoop;
     }
@@ -699,16 +702,15 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
         this.docAsUpsert = shouldUpsertDoc;
         return this;
     }
-    
+
     public boolean scriptedUpsert(){
         return this.scriptedUpsert;
     }
-    
+
     public UpdateRequest scriptedUpsert(boolean scriptedUpsert) {
         this.scriptedUpsert = scriptedUpsert;
         return this;
     }
-    
 
     @Override
     public void readFrom(StreamInput in) throws IOException {

+ 11 - 0
core/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java

@@ -308,6 +308,7 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<U
 
     /**
      * Sets whether to perform extra effort to detect noop updates via docAsUpsert.
+     * Defautls to true.
      */
     public UpdateRequestBuilder setDetectNoop(boolean detectNoop) {
         request.detectNoop(detectNoop);
@@ -322,4 +323,14 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<U
         request.scriptedUpsert(scriptedUpsert);
         return this;
     }
+
+    /**
+     * Set the new ttl of the document. Note that if detectNoop is true (the default)
+     * and the source of the document isn't changed then the ttl update won't take
+     * effect.
+     */
+    public UpdateRequestBuilder setTtl(Long ttl) {
+        request.doc().ttl(ttl);
+        return this;
+    }
 }

+ 2 - 1
core/src/main/java/org/elasticsearch/common/Strings.java

@@ -943,7 +943,8 @@ public class Strings {
         boolean changed = false;
         for (int i = 0; i < value.length(); i++) {
             char c = value.charAt(i);
-            if (c == '_') {
+            //e.g. _name stays as-is, _first_name becomes _firstName
+            if (c == '_' && i > 0) {
                 if (!changed) {
                     if (sb != null) {
                         sb.setLength(0);

+ 1 - 1
core/src/main/java/org/elasticsearch/common/util/MultiDataPathUpgrader.java

@@ -198,7 +198,7 @@ public class MultiDataPathUpgrader {
                 }
             }
         }
-        return new ShardPath(target.path, target.path, IndexMetaData.INDEX_UUID_NA_VALUE /* we don't know */, shard);
+        return new ShardPath(false, target.path, target.path, IndexMetaData.INDEX_UUID_NA_VALUE /* we don't know */, shard);
     }
 
     private ShardFileInfo[] getShardFileInfo(ShardId shard, NodeEnvironment.NodePath[] paths) throws IOException {

+ 34 - 19
core/src/main/java/org/elasticsearch/index/IndexService.java

@@ -22,14 +22,12 @@ package org.elasticsearch.index;
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterators;
-
 import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.inject.*;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.env.NodeEnvironment;
@@ -40,7 +38,6 @@ import org.elasticsearch.index.cache.IndexCache;
 import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
 import org.elasticsearch.index.deletionpolicy.DeletionPolicyModule;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
-import org.elasticsearch.index.shard.StoreRecoveryService;
 import org.elasticsearch.index.mapper.MapperService;
 import org.elasticsearch.index.query.IndexQueryParserService;
 import org.elasticsearch.index.settings.IndexSettings;
@@ -102,7 +99,25 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
     private final NodeEnvironment nodeEnv;
     private final IndicesService indicesServices;
 
-    private volatile ImmutableMap<Integer, Tuple<IndexShard, Injector>> shards = ImmutableMap.of();
+    private volatile ImmutableMap<Integer, IndexShardInjectorPair> shards = ImmutableMap.of();
+
+    private static class IndexShardInjectorPair {
+        private final IndexShard indexShard;
+        private final Injector injector;
+
+        public IndexShardInjectorPair(IndexShard indexShard, Injector injector) {
+            this.indexShard = indexShard;
+            this.injector = injector;
+        }
+
+        public IndexShard getIndexShard() {
+            return indexShard;
+        }
+
+        public Injector getInjector() {
+            return injector;
+        }
+    }
 
     private final AtomicBoolean closed = new AtomicBoolean(false);
     private final AtomicBoolean deleted = new AtomicBoolean(false);
@@ -147,10 +162,10 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
 
     @Override
     public Iterator<IndexShard> iterator() {
-        return Iterators.transform(shards.values().iterator(), new Function<Tuple<IndexShard, Injector>, IndexShard>() {
+        return Iterators.transform(shards.values().iterator(), new Function<IndexShardInjectorPair, IndexShard>() {
             @Override
-            public IndexShard apply(Tuple<IndexShard, Injector> input) {
-                return input.v1();
+            public IndexShard apply(IndexShardInjectorPair input) {
+                return input.getIndexShard();
             }
         });
     }
@@ -164,9 +179,9 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
      */
     @Nullable
     public IndexShard shard(int shardId) {
-        Tuple<IndexShard, Injector> indexShardInjectorTuple = shards.get(shardId);
-        if (indexShardInjectorTuple != null) {
-            return indexShardInjectorTuple.v1();
+        IndexShardInjectorPair indexShardInjectorPair = shards.get(shardId);
+        if (indexShardInjectorPair != null) {
+            return indexShardInjectorPair.getIndexShard();
         }
         return null;
     }
@@ -244,11 +259,11 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
      * Return the shard injector for the provided id, or throw an exception if there is no such shard.
      */
     public Injector shardInjectorSafe(int shardId)  {
-        Tuple<IndexShard, Injector> tuple = shards.get(shardId);
-        if (tuple == null) {
+        IndexShardInjectorPair indexShardInjectorPair = shards.get(shardId);
+        if (indexShardInjectorPair == null) {
             throw new ShardNotFoundException(new ShardId(index, shardId));
         }
-        return tuple.v2();
+        return indexShardInjectorPair.getInjector();
     }
 
     public String indexUUID() {
@@ -286,6 +301,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
         Injector shardInjector = null;
         try {
             lock = nodeEnv.shardLock(shardId, TimeUnit.SECONDS.toMillis(5));
+            indicesLifecycle.beforeIndexShardCreated(shardId, indexSettings);
             ShardPath path;
             try {
                 path = ShardPath.loadShardPath(logger, nodeEnv, shardId, indexSettings);
@@ -310,7 +326,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
                 throw new IndexShardAlreadyExistsException(shardId + " already exists");
             }
 
-            indicesLifecycle.beforeIndexShardCreated(shardId, indexSettings);
             logger.debug("creating shard_id {}", shardId);
             // if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary.
             final boolean canDeleteShardContent = IndexMetaData.isOnSharedFilesystem(indexSettings) == false ||
@@ -348,7 +363,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
             indicesLifecycle.indexShardStateChanged(indexShard, null, "shard created");
             indicesLifecycle.afterIndexShardCreated(indexShard);
 
-            shards = newMapBuilder(shards).put(shardId.id(), new Tuple<>(indexShard, shardInjector)).immutableMap();
+            shards = newMapBuilder(shards).put(shardId.id(), new IndexShardInjectorPair(indexShard, shardInjector)).immutableMap();
             success = true;
             return indexShard;
         } catch (IOException e) {
@@ -374,10 +389,10 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
             return;
         }
         logger.debug("[{}] closing... (reason: [{}])", shardId, reason);
-        HashMap<Integer, Tuple<IndexShard, Injector>> tmpShardsMap = newHashMap(shards);
-        Tuple<IndexShard, Injector> tuple = tmpShardsMap.remove(shardId);
-        indexShard = tuple.v1();
-        shardInjector = tuple.v2();
+        HashMap<Integer, IndexShardInjectorPair> tmpShardsMap = newHashMap(shards);
+        IndexShardInjectorPair indexShardInjectorPair = tmpShardsMap.remove(shardId);
+        indexShard = indexShardInjectorPair.getIndexShard();
+        shardInjector = indexShardInjectorPair.getInjector();
         shards = ImmutableMap.copyOf(tmpShardsMap);
         closeShardInjector(reason, sId, shardInjector, indexShard);
         logger.debug("[{}] closed (reason: [{}])", shardId, reason);

+ 38 - 0
core/src/main/java/org/elasticsearch/index/mapper/DocumentMapperForType.java

@@ -0,0 +1,38 @@
+/*
+ * 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.index.mapper;
+
+public class DocumentMapperForType {
+    private final DocumentMapper documentMapper;
+    private final Mapping mapping;
+
+    public DocumentMapperForType(DocumentMapper documentMapper, Mapping mapping) {
+        this.mapping = mapping;
+        this.documentMapper = documentMapper;
+    }
+
+    public DocumentMapper getDocumentMapper() {
+        return documentMapper;
+    }
+
+    public Mapping getMapping() {
+        return mapping;
+    }
+}

+ 5 - 16
core/src/main/java/org/elasticsearch/index/mapper/MapperService.java

@@ -26,24 +26,18 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterators;
-
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.TermsQuery;
-import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.*;
 import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.ElasticsearchGenerationException;
 import org.elasticsearch.Version;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.collect.ImmutableOpenMap;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.compress.CompressedXContent;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.lucene.search.Queries;
@@ -65,12 +59,7 @@ import org.elasticsearch.script.ScriptService;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -395,16 +384,16 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
      * Returns the document mapper created, including a mapping update if the
      * type has been dynamically created.
      */
-    public Tuple<DocumentMapper, Mapping> documentMapperWithAutoCreate(String type) {
+    public DocumentMapperForType documentMapperWithAutoCreate(String type) {
         DocumentMapper mapper = mappers.get(type);
         if (mapper != null) {
-            return Tuple.tuple(mapper, null);
+            return new DocumentMapperForType(mapper, null);
         }
         if (!dynamic) {
             throw new TypeMissingException(index, type, "trying to auto create mapping, but dynamic mapping is disabled");
         }
         mapper = parse(type, null, true);
-        return Tuple.tuple(mapper, mapper.mapping());
+        return new DocumentMapperForType(mapper, mapper.mapping());
     }
 
     /**

+ 6 - 7
core/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java

@@ -21,17 +21,16 @@ package org.elasticsearch.index.query;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiDocValues;
-import org.apache.lucene.search.*;
 import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.join.BitDocIdSetFilter;
-import org.elasticsearch.common.ParseField;
 import org.apache.lucene.search.join.JoinUtil;
 import org.apache.lucene.search.join.ScoreMode;
 import org.elasticsearch.Version;
+import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.lucene.search.Queries;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -45,8 +44,8 @@ import org.elasticsearch.index.search.child.ChildrenConstantScoreQuery;
 import org.elasticsearch.index.search.child.ChildrenQuery;
 import org.elasticsearch.index.search.child.ScoreType;
 import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
+import org.elasticsearch.search.fetch.innerhits.InnerHitsSubSearchContext;
 import org.elasticsearch.search.internal.SearchContext;
-import org.elasticsearch.search.internal.SubSearchContext;
 
 import java.io.IOException;
 
@@ -82,7 +81,7 @@ public class HasChildQueryParser implements QueryParser {
         int maxChildren = 0;
         int shortCircuitParentDocSet = 8192;
         String queryName = null;
-        Tuple<String, SubSearchContext> innerHits = null;
+        InnerHitsSubSearchContext innerHits = null;
 
         String currentFieldName = null;
         XContentParser.Token token;
@@ -152,8 +151,8 @@ public class HasChildQueryParser implements QueryParser {
 
         if (innerHits != null) {
             ParsedQuery parsedQuery = new ParsedQuery(innerQuery, parseContext.copyNamedQueries());
-            InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.v2(), parsedQuery, null, parseContext.mapperService(), childDocMapper);
-            String name = innerHits.v1() != null ? innerHits.v1() : childType;
+            InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.getSubSearchContext(), parsedQuery, null, parseContext.mapperService(), childDocMapper);
+            String name = innerHits.getName() != null ? innerHits.getName() : childType;
             parseContext.addInnerHits(name, parentChildInnerHits);
         }
 

+ 6 - 11
core/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java

@@ -18,15 +18,10 @@
  */
 package org.elasticsearch.index.query;
 
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.Query;
+import org.apache.lucene.search.*;
 import org.elasticsearch.Version;
-import org.apache.lucene.search.QueryWrapperFilter;
 import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.lucene.search.Queries;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -39,7 +34,7 @@ import org.elasticsearch.index.search.child.ParentConstantScoreQuery;
 import org.elasticsearch.index.search.child.ParentQuery;
 import org.elasticsearch.index.search.child.ScoreType;
 import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
-import org.elasticsearch.search.internal.SubSearchContext;
+import org.elasticsearch.search.fetch.innerhits.InnerHitsSubSearchContext;
 
 import java.io.IOException;
 import java.util.HashSet;
@@ -73,7 +68,7 @@ public class HasParentQueryParser implements QueryParser {
         String parentType = null;
         boolean score = false;
         String queryName = null;
-        Tuple<String, SubSearchContext> innerHits = null;
+        InnerHitsSubSearchContext innerHits = null;
 
         String currentFieldName = null;
         XContentParser.Token token;
@@ -146,7 +141,7 @@ public class HasParentQueryParser implements QueryParser {
         return query;
     }
 
-    static Query createParentQuery(Query innerQuery, String parentType, boolean score, QueryParseContext parseContext, Tuple<String, SubSearchContext> innerHits) throws IOException {
+    static Query createParentQuery(Query innerQuery, String parentType, boolean score, QueryParseContext parseContext, InnerHitsSubSearchContext innerHits) throws IOException {
         DocumentMapper parentDocMapper = parseContext.mapperService().documentMapper(parentType);
         if (parentDocMapper == null) {
             throw new QueryParsingException(parseContext, "[has_parent] query configured 'parent_type' [" + parentType
@@ -155,8 +150,8 @@ public class HasParentQueryParser implements QueryParser {
 
         if (innerHits != null) {
             ParsedQuery parsedQuery = new ParsedQuery(innerQuery, parseContext.copyNamedQueries());
-            InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.v2(), parsedQuery, null, parseContext.mapperService(), parentDocMapper);
-            String name = innerHits.v1() != null ? innerHits.v1() : parentType;
+            InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.getSubSearchContext(), parsedQuery, null, parseContext.mapperService(), parentDocMapper);
+            String name = innerHits.getName() != null ? innerHits.getName() : parentType;
             parseContext.addInnerHits(name, parentChildInnerHits);
         }
 

+ 5 - 6
core/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java

@@ -26,14 +26,13 @@ import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.lucene.search.Queries;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.support.InnerHitsQueryParserHelper;
 import org.elasticsearch.index.query.support.NestedInnerQueryParseSupport;
 import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
-import org.elasticsearch.search.internal.SubSearchContext;
+import org.elasticsearch.search.fetch.innerhits.InnerHitsSubSearchContext;
 
 import java.io.IOException;
 
@@ -120,7 +119,7 @@ public class NestedQueryParser implements QueryParser {
     public static class ToBlockJoinQueryBuilder extends NestedInnerQueryParseSupport {
 
         private ScoreMode scoreMode;
-        private Tuple<String, SubSearchContext> innerHits;
+        private InnerHitsSubSearchContext innerHits;
 
         public ToBlockJoinQueryBuilder(QueryParseContext parseContext) throws IOException {
             super(parseContext);
@@ -130,7 +129,7 @@ public class NestedQueryParser implements QueryParser {
             this.scoreMode = scoreMode;
         }
 
-        public void setInnerHits(Tuple<String, SubSearchContext> innerHits) {
+        public void setInnerHits(InnerHitsSubSearchContext innerHits) {
             this.innerHits = innerHits;
         }
 
@@ -152,8 +151,8 @@ public class NestedQueryParser implements QueryParser {
 
             if (innerHits != null) {
                 ParsedQuery parsedQuery = new ParsedQuery(innerQuery, parseContext.copyNamedQueries());
-                InnerHitsContext.NestedInnerHits nestedInnerHits = new InnerHitsContext.NestedInnerHits(innerHits.v2(), parsedQuery, null, getParentObjectMapper(), nestedObjectMapper);
-                String name = innerHits.v1() != null ? innerHits.v1() : path;
+                InnerHitsContext.NestedInnerHits nestedInnerHits = new InnerHitsContext.NestedInnerHits(innerHits.getSubSearchContext(), parsedQuery, null, getParentObjectMapper(), nestedObjectMapper);
+                String name = innerHits.getName() != null ? innerHits.getName() : path;
                 parseContext.addInnerHits(name, nestedInnerHits);
             }
 

+ 4 - 1
core/src/main/java/org/elasticsearch/index/query/PrefixQueryParser.java

@@ -23,6 +23,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.PrefixQuery;
 import org.apache.lucene.search.Query;
+import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.lucene.BytesRefs;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -38,6 +39,8 @@ public class PrefixQueryParser implements QueryParser {
 
     public static final String NAME = "prefix";
 
+    private static final ParseField NAME_FIELD = new ParseField("_name").withAllDeprecated("query name is not supported in short version of prefix query");
+
     @Inject
     public PrefixQueryParser() {
     }
@@ -84,7 +87,7 @@ public class PrefixQueryParser implements QueryParser {
                     }
                 }
             } else {
-                if ("_name".equals(currentFieldName)) {
+                if (parseContext.parseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
                     queryName = parser.text();
                 } else {
                     fieldName = currentFieldName;

+ 2 - 1
core/src/main/java/org/elasticsearch/index/query/RangeQueryParser.java

@@ -40,6 +40,7 @@ public class RangeQueryParser implements QueryParser {
 
     public static final String NAME = "range";
     private static final ParseField FIELDDATA_FIELD = new ParseField("fielddata").withAllDeprecated("[no replacement]");
+    private static final ParseField NAME_FIELD = new ParseField("_name").withAllDeprecated("query name is not supported in short version of range query");
 
     @Inject
     public RangeQueryParser() {
@@ -109,7 +110,7 @@ public class RangeQueryParser implements QueryParser {
                     }
                 }
             } else if (token.isValue()) {
-                if ("_name".equals(currentFieldName)) {
+                if (parseContext.parseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
                     queryName = parser.text();
                 } else if (parseContext.parseFieldMatcher().match(currentFieldName, FIELDDATA_FIELD)) {
                     // ignore

+ 4 - 1
core/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java

@@ -24,6 +24,7 @@ import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.RegexpQuery;
 import org.apache.lucene.util.automaton.Operations;
+import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.lucene.BytesRefs;
 import org.elasticsearch.common.xcontent.XContentParser;
@@ -41,6 +42,8 @@ public class RegexpQueryParser implements QueryParser {
 
     public static final int DEFAULT_FLAGS_VALUE = RegexpFlag.ALL.value();
 
+    private static final ParseField NAME_FIELD = new ParseField("_name").withAllDeprecated("query name is not supported in short version of regexp query");
+
     @Inject
     public RegexpQueryParser() {
     }
@@ -96,7 +99,7 @@ public class RegexpQueryParser implements QueryParser {
                     }
                 }
             } else {
-                if ("_name".equals(currentFieldName)) {
+                if (parseContext.parseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
                     queryName = parser.text();
                 } else {
                     fieldName = currentFieldName;

+ 6 - 3
core/src/main/java/org/elasticsearch/index/query/TermQueryParser.java

@@ -22,10 +22,10 @@ package org.elasticsearch.index.query;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
+import org.elasticsearch.common.ParseField;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.lucene.BytesRefs;
 import org.elasticsearch.common.xcontent.XContentParser;
-import org.elasticsearch.index.mapper.FieldMapper;
 import org.elasticsearch.index.mapper.MappedFieldType;
 
 import java.io.IOException;
@@ -37,6 +37,9 @@ public class TermQueryParser implements QueryParser {
 
     public static final String NAME = "term";
 
+    private static final ParseField NAME_FIELD = new ParseField("_name").withAllDeprecated("query name is not supported in short version of term query");
+    private static final ParseField BOOST_FIELD = new ParseField("boost").withAllDeprecated("boost is not supported in short version of term query");
+
     @Inject
     public TermQueryParser() {
     }
@@ -85,9 +88,9 @@ public class TermQueryParser implements QueryParser {
                     }
                 }
             } else if (token.isValue()) {
-                if ("_name".equals(currentFieldName)) {
+                if (parseContext.parseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
                     queryName = parser.text();
-                } else if ("boost".equals(currentFieldName)) {
+                } else if (parseContext.parseFieldMatcher().match(currentFieldName, BOOST_FIELD)) {
                     boost = parser.floatValue();
                 } else {
                     if (fieldName != null) {

+ 3 - 3
core/src/main/java/org/elasticsearch/index/query/support/InnerHitsQueryParserHelper.java

@@ -19,12 +19,12 @@
 
 package org.elasticsearch.index.query.support;
 
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryParsingException;
 import org.elasticsearch.search.fetch.fielddata.FieldDataFieldsParseElement;
+import org.elasticsearch.search.fetch.innerhits.InnerHitsSubSearchContext;
 import org.elasticsearch.search.fetch.script.ScriptFieldsParseElement;
 import org.elasticsearch.search.fetch.source.FetchSourceParseElement;
 import org.elasticsearch.search.highlight.HighlighterParseElement;
@@ -51,7 +51,7 @@ public class InnerHitsQueryParserHelper {
         this.fieldDataFieldsParseElement = fieldDataFieldsParseElement;
     }
 
-    public Tuple<String, SubSearchContext> parse(QueryParseContext parserContext) throws IOException, QueryParsingException {
+    public InnerHitsSubSearchContext parse(QueryParseContext parserContext) throws IOException, QueryParsingException {
         String fieldName = null;
         XContentParser.Token token;
         String innerHitName = null;
@@ -74,7 +74,7 @@ public class InnerHitsQueryParserHelper {
         } catch (Exception e) {
             throw new QueryParsingException(parserContext, "Failed to parse [_inner_hits]", e);
         }
-        return new Tuple<>(innerHitName, subSearchContext);
+        return new InnerHitsSubSearchContext(innerHitName, subSearchContext);
     }
 
     public static void parseCommonInnerHitOptions(XContentParser parser, XContentParser.Token token, String fieldName, SubSearchContext subSearchContext,

+ 13 - 21
core/src/main/java/org/elasticsearch/index/shard/IndexShard.java

@@ -21,17 +21,10 @@ package org.elasticsearch.index.shard;
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
-
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.index.CheckIndex;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.DisjunctionMaxQuery;
-import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.search.MatchNoDocsQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.QueryCachingPolicy;
-import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
+import org.apache.lucene.search.*;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.ThreadInterruptedException;
@@ -48,7 +41,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.ShardRoutingState;
 import org.elasticsearch.common.Booleans;
 import org.elasticsearch.common.Nullable;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.logging.ESLogger;
@@ -506,13 +498,13 @@ public class IndexShard extends AbstractIndexShardComponent {
         }
     }
 
-    static Engine.Create prepareCreate(Tuple<DocumentMapper, Mapping> docMapper, SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin, boolean canHaveDuplicates, boolean autoGeneratedId) {
+    static Engine.Create prepareCreate(DocumentMapperForType docMapper, SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin, boolean canHaveDuplicates, boolean autoGeneratedId) {
         long startTime = System.nanoTime();
-        ParsedDocument doc = docMapper.v1().parse(source);
-        if (docMapper.v2() != null) {
-            doc.addDynamicMappingsUpdate(docMapper.v2());
+        ParsedDocument doc = docMapper.getDocumentMapper().parse(source);
+        if (docMapper.getMapping() != null) {
+            doc.addDynamicMappingsUpdate(docMapper.getMapping());
         }
-        return new Engine.Create(docMapper.v1().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime, canHaveDuplicates, autoGeneratedId);
+        return new Engine.Create(docMapper.getDocumentMapper().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime, canHaveDuplicates, autoGeneratedId);
     }
 
     public void create(Engine.Create create) {
@@ -540,13 +532,13 @@ public class IndexShard extends AbstractIndexShardComponent {
         }
     }
 
-    static Engine.Index prepareIndex(Tuple<DocumentMapper, Mapping> docMapper, SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin, boolean canHaveDuplicates) {
+    static Engine.Index prepareIndex(DocumentMapperForType docMapper, SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin, boolean canHaveDuplicates) {
         long startTime = System.nanoTime();
-        ParsedDocument doc = docMapper.v1().parse(source);
-        if (docMapper.v2() != null) {
-            doc.addDynamicMappingsUpdate(docMapper.v2());
+        ParsedDocument doc = docMapper.getDocumentMapper().parse(source);
+        if (docMapper.getMapping() != null) {
+            doc.addDynamicMappingsUpdate(docMapper.getMapping());
         }
-        return new Engine.Index(docMapper.v1().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime, canHaveDuplicates);
+        return new Engine.Index(docMapper.getDocumentMapper().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime, canHaveDuplicates);
     }
 
     /**
@@ -573,7 +565,7 @@ public class IndexShard extends AbstractIndexShardComponent {
 
     public Engine.Delete prepareDelete(String type, String id, long version, VersionType versionType, Engine.Operation.Origin origin) {
         long startTime = System.nanoTime();
-        final DocumentMapper documentMapper = docMapper(type).v1();
+        final DocumentMapper documentMapper = docMapper(type).getDocumentMapper();
         return new Engine.Delete(type, id, documentMapper.uidMapper().term(Uid.createUid(type, id)), version, versionType, origin, startTime, false);
     }
 
@@ -1389,7 +1381,7 @@ public class IndexShard extends AbstractIndexShardComponent {
         return indexSettings.get(IndexMetaData.SETTING_INDEX_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
     }
 
-    private Tuple<DocumentMapper, Mapping> docMapper(String type) {
+    private DocumentMapperForType docMapper(String type) {
         return mapperService.documentMapperWithAutoCreate(type);
     }
 

+ 39 - 6
core/src/main/java/org/elasticsearch/index/shard/ShardPath.java

@@ -20,6 +20,7 @@ package org.elasticsearch.index.shard;
 
 import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.logging.ESLogger;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.env.NodeEnvironment;
@@ -41,10 +42,18 @@ public final class ShardPath {
     private final String indexUUID;
     private final ShardId shardId;
     private final Path shardStatePath;
-
-
-    public ShardPath(Path path, Path shardStatePath, String indexUUID, ShardId shardId) {
-        this.path = path;
+    private final boolean isCustomDataPath;
+
+    public ShardPath(boolean isCustomDataPath, Path dataPath, Path shardStatePath, String indexUUID, ShardId shardId) {
+        assert dataPath.getFileName().toString().equals(Integer.toString(shardId.id())) : "dataPath must end with the shard ID but didn't: " + dataPath.toString();
+        assert shardStatePath.getFileName().toString().equals(Integer.toString(shardId.id())) : "shardStatePath must end with the shard ID but didn't: " + dataPath.toString();
+        assert dataPath.getParent().getFileName().toString().equals(shardId.getIndex()) : "dataPath must end with index/shardID but didn't: " + dataPath.toString();
+        assert shardStatePath.getParent().getFileName().toString().equals(shardId.getIndex()) : "shardStatePath must end with index/shardID but didn't: " + dataPath.toString();
+        if (isCustomDataPath && dataPath.equals(shardStatePath)) {
+            throw new IllegalArgumentException("shard state path must be different to the data path when using custom data paths");
+        }
+        this.isCustomDataPath = isCustomDataPath;
+        this.path = dataPath;
         this.indexUUID = indexUUID;
         this.shardId = shardId;
         this.shardStatePath = shardStatePath;
@@ -78,6 +87,30 @@ public final class ShardPath {
         return shardStatePath;
     }
 
+    /**
+     * Returns the data-path root for this shard. The root is a parent of {@link #getDataPath()} without the index name
+     * and the shard ID.
+     */
+    public Path getRootDataPath() {
+        Path noIndexShardId = getDataPath().getParent().getParent();
+        return isCustomDataPath ? noIndexShardId : noIndexShardId.getParent(); // also strip the indices folder
+    }
+
+    /**
+     * Returns the state-path root for this shard. The root is a parent of {@link #getRootStatePath()} ()} without the index name
+     * and the shard ID.
+     */
+    public Path getRootStatePath() {
+        return getShardStatePath().getParent().getParent().getParent(); // also strip the indices folder
+    }
+
+    /**
+     * Returns <code>true</code> iff the data location is a custom data location and therefore outside of the nodes configured data paths.
+     */
+    public boolean isCustomDataPath() {
+        return isCustomDataPath;
+    }
+
     /**
      * This method walks through the nodes shard paths to find the data and state path for the given shard. If multiple
      * directories with a valid shard state exist the one with the highest version will be used.
@@ -113,7 +146,7 @@ public final class ShardPath {
                 dataPath = statePath;
             }
             logger.debug("{} loaded data path [{}], state path [{}]", shardId, dataPath, statePath);
-            return new ShardPath(dataPath, statePath, indexUUID, shardId);
+            return new ShardPath(NodeEnvironment.hasCustomDataPath(indexSettings), dataPath, statePath, indexUUID, shardId);
         }
     }
 
@@ -202,7 +235,7 @@ public final class ShardPath {
             dataPath = statePath;
         }
 
-        return new ShardPath(dataPath, statePath, indexUUID, shardId);
+        return new ShardPath(NodeEnvironment.hasCustomDataPath(indexSettings), dataPath, statePath, indexUUID, shardId);
     }
 
     @Override

+ 1 - 2
core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java

@@ -25,7 +25,6 @@ import org.elasticsearch.Version;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.lucene.search.Queries;
@@ -67,7 +66,7 @@ public class TranslogRecoveryPerformer {
         this.indexCache = indexCache;
     }
 
-    protected Tuple<DocumentMapper, Mapping> docMapper(String type) {
+    protected DocumentMapperForType docMapper(String type) {
         return mapperService.documentMapperWithAutoCreate(type); // protected for testing
     }
 

+ 7 - 26
core/src/main/java/org/elasticsearch/index/termvectors/ShardTermVectorsService.java

@@ -20,12 +20,7 @@
 package org.elasticsearch.index.termvectors;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.*;
 import org.apache.lucene.index.memory.MemoryIndex;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.termvectors.TermVectorsFilter;
@@ -38,20 +33,13 @@ import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.lucene.uid.Versions;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.engine.Engine;
 import org.elasticsearch.index.get.GetField;
 import org.elasticsearch.index.get.GetResult;
-import org.elasticsearch.index.mapper.DocumentMapper;
-import org.elasticsearch.index.mapper.MappedFieldType;
-import org.elasticsearch.index.mapper.MapperService;
-import org.elasticsearch.index.mapper.Mapping;
-import org.elasticsearch.index.mapper.ParseContext;
-import org.elasticsearch.index.mapper.ParsedDocument;
-import org.elasticsearch.index.mapper.Uid;
+import org.elasticsearch.index.mapper.*;
 import org.elasticsearch.index.mapper.core.StringFieldMapper;
 import org.elasticsearch.index.mapper.internal.UidFieldMapper;
 import org.elasticsearch.index.settings.IndexSettings;
@@ -61,14 +49,7 @@ import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.search.dfs.AggregatedDfs;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
+import java.util.*;
 
 import static org.elasticsearch.index.mapper.SourceToParse.source;
 
@@ -308,10 +289,10 @@ public class ShardTermVectorsService extends AbstractIndexShardComponent {
         MapperService mapperService = indexShard.mapperService();
 
         // TODO: make parsing not dynamically create fields not in the original mapping
-        Tuple<DocumentMapper, Mapping> docMapper = mapperService.documentMapperWithAutoCreate(type);
-        ParsedDocument parsedDocument = docMapper.v1().parse(source(doc).index(index).type(type).flyweight(true));
-        if (docMapper.v2() != null) {
-            parsedDocument.addDynamicMappingsUpdate(docMapper.v2());
+        DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(type);
+        ParsedDocument parsedDocument = docMapper.getDocumentMapper().parse(source(doc).index(index).type(type).flyweight(true));
+        if (docMapper.getMapping() != null) {
+            parsedDocument.addDynamicMappingsUpdate(docMapper.getMapping());
         }
         if (parsedDocument.dynamicMappingsUpdate() != null) {
             mappingUpdatedAction.updateMappingOnMasterSynchronously(index, type, parsedDocument.dynamicMappingsUpdate());

+ 30 - 11
core/src/main/java/org/elasticsearch/index/translog/Translog.java

@@ -32,8 +32,10 @@ import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.bytes.ReleasablePagedBytesReference;
-import org.elasticsearch.common.collect.Tuple;
-import org.elasticsearch.common.io.stream.*;
+import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
+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.lease.Releasable;
 import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.logging.ESLogger;
@@ -231,7 +233,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
             }
         })) {
             long latestGeneration = -1;
-            List<Tuple<Path, Long>> filesToUpgrade = new ArrayList<>();
+            List<PathWithGeneration> filesToUpgrade = new ArrayList<>();
             for (Path path : stream) {
                 Matcher matcher = parseLegacyIdPattern.matcher(path.getFileName().toString());
                 if (matcher.matches()) {
@@ -239,7 +241,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
                     if (generation >= translogGeneration.translogFileGeneration) {
                         latestGeneration = Math.max(translogGeneration.translogFileGeneration, generation);
                     }
-                    filesToUpgrade.add(new Tuple<>(path, generation));
+                    filesToUpgrade.add(new PathWithGeneration(path, generation));
                 } else {
                     Matcher strict_matcher = PARSE_STRICT_ID_PATTERN.matcher(path.getFileName().toString());
                     if (strict_matcher.matches()) {
@@ -250,17 +252,17 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
             if (latestGeneration < translogGeneration.translogFileGeneration) {
                 throw new IllegalStateException("latest found translog has a lower generation that the excepcted uncommitted " + translogGeneration.translogFileGeneration + " > " + latestGeneration);
             }
-            CollectionUtil.timSort(filesToUpgrade, new Comparator<Tuple<Path, Long>>() {
+            CollectionUtil.timSort(filesToUpgrade, new Comparator<PathWithGeneration>() {
                 @Override
-                public int compare(Tuple<Path, Long> o1, Tuple<Path, Long> o2) {
-                    long gen1 = o1.v2();
-                    long gen2 = o2.v2();
+                public int compare(PathWithGeneration o1, PathWithGeneration o2) {
+                    long gen1 = o1.getGeneration();
+                    long gen2 = o2.getGeneration();
                     return Long.compare(gen1, gen2);
                 }
             });
-            for (Tuple<Path, Long> pathAndGeneration : filesToUpgrade) {
-                final Path path = pathAndGeneration.v1();
-                final long generation = pathAndGeneration.v2();
+            for (PathWithGeneration pathAndGeneration : filesToUpgrade) {
+                final Path path = pathAndGeneration.getPath();
+                final long generation = pathAndGeneration.getGeneration();
                 final Path target = path.resolveSibling(getFilename(generation));
                 logger.debug("upgrading translog copy file from {} to {}", path, target);
                 Files.move(path, target, StandardCopyOption.ATOMIC_MOVE);
@@ -1798,4 +1800,21 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
         return outstandingViews.size();
     }
 
+    private static class PathWithGeneration {
+        private final Path path;
+        private final long generation;
+
+        public PathWithGeneration(Path path, long generation) {
+            this.path = path;
+            this.generation = generation;
+        }
+
+        public Path getPath() {
+            return path;
+        }
+
+        public long getGeneration() {
+            return generation;
+        }
+    }
 }

+ 44 - 45
core/src/main/java/org/elasticsearch/indices/IndicesService.java

@@ -20,12 +20,7 @@
 package org.elasticsearch.indices;
 
 import com.google.common.base.Function;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import com.google.common.collect.*;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.IOUtils;
@@ -38,14 +33,8 @@ import org.elasticsearch.action.admin.indices.stats.ShardStats;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.common.Nullable;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.component.AbstractLifecycleComponent;
-import org.elasticsearch.common.inject.CreationException;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.inject.Injector;
-import org.elasticsearch.common.inject.Injectors;
-import org.elasticsearch.common.inject.Module;
-import org.elasticsearch.common.inject.ModulesBuilder;
+import org.elasticsearch.common.inject.*;
 import org.elasticsearch.common.io.FileSystemUtils;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
@@ -53,12 +42,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.env.ShardLock;
 import org.elasticsearch.gateway.MetaDataStateFormat;
-import org.elasticsearch.index.Index;
-import org.elasticsearch.index.IndexModule;
-import org.elasticsearch.index.IndexNameModule;
-import org.elasticsearch.index.IndexNotFoundException;
-import org.elasticsearch.index.IndexService;
-import org.elasticsearch.index.LocalNodeIdModule;
+import org.elasticsearch.index.*;
 import org.elasticsearch.index.aliases.IndexAliasesServiceModule;
 import org.elasticsearch.index.analysis.AnalysisModule;
 import org.elasticsearch.index.analysis.AnalysisService;
@@ -91,11 +75,7 @@ import org.elasticsearch.plugins.PluginsService;
 import java.io.Closeable;
 import java.io.IOException;
 import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -124,7 +104,26 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
     private final NodeEnvironment nodeEnv;
     private final TimeValue shardsClosedTimeout;
 
-    private volatile Map<String, Tuple<IndexService, Injector>> indices = ImmutableMap.of();
+    private volatile Map<String, IndexServiceInjectorPair> indices = ImmutableMap.of();
+    
+    static class IndexServiceInjectorPair {
+        private final IndexService indexService;
+        private final Injector injector;
+
+        public IndexServiceInjectorPair(IndexService indexService, Injector injector) {
+            this.indexService = indexService;
+            this.injector = injector;
+        }
+
+        public IndexService getIndexService() {
+            return indexService;
+        }
+
+        public Injector getInjector() {
+            return injector;
+        }
+    }
+    
     private final Map<Index, List<PendingDelete>> pendingDeletes = new HashMap<>();
 
     private final OldShardsStats oldShardsStats = new OldShardsStats();
@@ -229,14 +228,14 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
         }
 
         Map<Index, List<IndexShardStats>> statsByShard = Maps.newHashMap();
-        for (Tuple<IndexService, Injector> value : indices.values()) {
-            IndexService indexService = value.v1();
+        for (IndexServiceInjectorPair value : indices.values()) {
+            IndexService indexService = value.getIndexService();
             for (IndexShard indexShard : indexService) {
                 try {
                     if (indexShard.routingEntry() == null) {
                         continue;
                     }
-                    IndexShardStats indexShardStats = new IndexShardStats(indexShard.shardId(), new ShardStats[] { new ShardStats(indexShard, indexShard.routingEntry(), flags) });
+                    IndexShardStats indexShardStats = new IndexShardStats(indexShard.shardId(), new ShardStats[] { new ShardStats(indexShard, flags) });
                     if (!statsByShard.containsKey(indexService.index())) {
                         statsByShard.put(indexService.index(), Lists.<IndexShardStats>newArrayList(indexShardStats));
                     } else {
@@ -261,10 +260,10 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
 
     @Override
     public Iterator<IndexService> iterator() {
-        return Iterators.transform(indices.values().iterator(), new Function<Tuple<IndexService, Injector>, IndexService>() {
+        return Iterators.transform(indices.values().iterator(), new Function<IndexServiceInjectorPair, IndexService>() {
             @Override
-            public IndexService apply(Tuple<IndexService, Injector> input) {
-                return input.v1();
+            public IndexService apply(IndexServiceInjectorPair input) {
+                return input.getIndexService();
             }
         });
     }
@@ -279,11 +278,11 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
      */
     @Nullable
     public IndexService indexService(String index) {
-        Tuple<IndexService, Injector> indexServiceInjectorTuple = indices.get(index);
-        if (indexServiceInjectorTuple == null) {
+        IndexServiceInjectorPair indexServiceInjectorPair = indices.get(index);
+        if (indexServiceInjectorPair == null) {
             return null;
         } else {
-            return indexServiceInjectorTuple.v1();
+            return indexServiceInjectorPair.getIndexService();
         }
     }
 
@@ -352,7 +351,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
 
         indicesLifecycle.afterIndexCreated(indexService);
 
-        indices = newMapBuilder(indices).put(index.name(), new Tuple<>(indexService, indexInjector)).immutableMap();
+        indices = newMapBuilder(indices).put(index.name(), new IndexServiceInjectorPair(indexService, indexInjector)).immutableMap();
 
         return indexService;
     }
@@ -377,10 +376,10 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
                 }
 
                 logger.debug("[{}] closing ... (reason [{}])", index, reason);
-                Map<String, Tuple<IndexService, Injector>> tmpMap = newHashMap(indices);
-                Tuple<IndexService, Injector> remove = tmpMap.remove(index);
-                indexService = remove.v1();
-                indexInjector = remove.v2();
+                Map<String, IndexServiceInjectorPair> tmpMap = newHashMap(indices);
+                IndexServiceInjectorPair remove = tmpMap.remove(index);
+                indexService = remove.getIndexService();
+                indexInjector = remove.getInjector();
                 indices = ImmutableMap.copyOf(tmpMap);
             }
 
@@ -488,7 +487,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
             synchronized (this) {
                 String indexName = metaData.index();
                 if (indices.containsKey(indexName)) {
-                    String localUUid = indices.get(indexName).v1().indexUUID();
+                    String localUUid = indices.get(indexName).getIndexService().indexUUID();
                     throw new IllegalStateException("Can't delete index store for [" + indexName + "] - it's still part of the indices service [" + localUUid + "] [" + metaData.getIndexUUID() + "]");
                 }
                 if (clusterState.metaData().hasIndex(indexName) && (clusterState.nodes().localNode().masterNode() == true)) {
@@ -589,9 +588,9 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
      * @return true if the index can be deleted on this node
      */
     public boolean canDeleteIndexContents(Index index, Settings indexSettings) {
-        final Tuple<IndexService, Injector> indexServiceInjectorTuple = this.indices.get(index.name());
+        final IndexServiceInjectorPair indexServiceInjectorPair = this.indices.get(index.name());
         if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) {
-            if (indexServiceInjectorTuple == null && nodeEnv.hasNodeFile()) {
+            if (indexServiceInjectorPair == null && nodeEnv.hasNodeFile()) {
                 return true;
             }
         } else {
@@ -622,10 +621,10 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
     }
 
     private boolean canDeleteShardContent(ShardId shardId, @IndexSettings Settings indexSettings) {
-        final Tuple<IndexService, Injector> indexServiceInjectorTuple = this.indices.get(shardId.getIndex());
+        final IndexServiceInjectorPair indexServiceInjectorPair = this.indices.get(shardId.getIndex());
         if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) {
-            if (indexServiceInjectorTuple != null && nodeEnv.hasNodeFile()) {
-                final IndexService indexService = indexServiceInjectorTuple.v1();
+            if (indexServiceInjectorPair != null && nodeEnv.hasNodeFile()) {
+                final IndexService indexService = indexServiceInjectorPair.getIndexService();
                 return indexService.hasShard(shardId.id()) == false;
             } else if (nodeEnv.hasNodeFile()) {
                 if (NodeEnvironment.hasCustomDataPath(indexSettings)) {

+ 6 - 7
core/src/main/java/org/elasticsearch/percolator/PercolatorService.java

@@ -38,7 +38,6 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.common.ParseFieldMatcher;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
@@ -275,10 +274,10 @@ public class PercolatorService extends AbstractComponent {
                         }
 
                         MapperService mapperService = documentIndexService.mapperService();
-                        Tuple<DocumentMapper, Mapping> docMapper = mapperService.documentMapperWithAutoCreate(request.documentType());
-                        doc = docMapper.v1().parse(source(parser).index(index).type(request.documentType()).flyweight(true));
-                        if (docMapper.v2() != null) {
-                            doc.addDynamicMappingsUpdate(docMapper.v2());
+                        DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(request.documentType());
+                        doc = docMapper.getDocumentMapper().parse(source(parser).index(index).type(request.documentType()).flyweight(true));
+                        if (docMapper.getMapping() != null) {
+                            doc.addDynamicMappingsUpdate(docMapper.getMapping());
                         }
                         if (doc.dynamicMappingsUpdate() != null) {
                             mappingUpdatedAction.updateMappingOnMasterSynchronously(request.shardId().getIndex(), request.documentType(), doc.dynamicMappingsUpdate());
@@ -384,8 +383,8 @@ public class PercolatorService extends AbstractComponent {
         try {
             parser = XContentFactory.xContent(fetchedDoc).createParser(fetchedDoc);
             MapperService mapperService = documentIndexService.mapperService();
-            Tuple<DocumentMapper, Mapping> docMapper = mapperService.documentMapperWithAutoCreate(type);
-            doc = docMapper.v1().parse(source(parser).index(index).type(type).flyweight(true));
+            DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(type);
+            doc = docMapper.getDocumentMapper().parse(source(parser).index(index).type(type).flyweight(true));
 
             if (context.highlight() != null) {
                 doc.setSource(fetchedDoc);

+ 40 - 0
core/src/main/java/org/elasticsearch/search/fetch/innerhits/InnerHitsSubSearchContext.java

@@ -0,0 +1,40 @@
+/*
+ * 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.search.fetch.innerhits;
+
+import org.elasticsearch.search.internal.SubSearchContext;
+
+public class InnerHitsSubSearchContext {
+    private final String name;
+    private final SubSearchContext subSearchContext;
+
+    public InnerHitsSubSearchContext(String name, SubSearchContext subSearchContext) {
+        this.name = name;
+        this.subSearchContext = subSearchContext;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public SubSearchContext getSubSearchContext() {
+        return subSearchContext;
+    }
+}

+ 2 - 0
core/src/test/java/org/elasticsearch/common/StringsTests.java

@@ -29,6 +29,8 @@ public class StringsTests extends ESTestCase {
         assertEquals("fooBar", Strings.toCamelCase("foo_bar"));
         assertEquals("fooBarFooBar", Strings.toCamelCase("foo_bar_foo_bar"));
         assertEquals("fooBar", Strings.toCamelCase("foo_bar_"));
+        assertEquals("_foo", Strings.toCamelCase("_foo"));
+        assertEquals("_fooBar", Strings.toCamelCase("_foo_bar_"));
     }
 
     public void testSubstring() {

+ 2 - 2
core/src/test/java/org/elasticsearch/common/util/MultiDataPathUpgraderTests.java

@@ -83,7 +83,7 @@ public class MultiDataPathUpgraderTests extends ESTestCase {
                 ShardStateMetaData.FORMAT.write(new ShardStateMetaData(metaStateVersion, true, uuid), metaStateVersion, shardDataPaths);
             }
             final Path path = randomFrom(shardDataPaths);
-            ShardPath targetPath = new ShardPath(path, path, uuid, new ShardId("foo", 0));
+            ShardPath targetPath = new ShardPath(false, path, path, uuid, new ShardId("foo", 0));
             MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment);
             helper.upgrade(shardId, targetPath);
             assertFalse(helper.needsUpgrading(shardId));
@@ -177,7 +177,7 @@ public class MultiDataPathUpgraderTests extends ESTestCase {
             }
             logger.info("--> injecting index [{}] into multiple data paths", indexName);
             OldIndexBackwardsCompatibilityIT.copyIndex(logger, src, indexName, multiDataPath);
-            final ShardPath shardPath = new ShardPath(nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], IndexMetaData.INDEX_UUID_NA_VALUE, new ShardId(indexName, 0));
+            final ShardPath shardPath = new ShardPath(false, nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], IndexMetaData.INDEX_UUID_NA_VALUE, new ShardId(indexName, 0));
 
             logger.info("{}", FileSystemUtils.files(shardPath.resolveIndex()));
 

+ 1 - 1
core/src/test/java/org/elasticsearch/document/BulkIT.java

@@ -195,7 +195,7 @@ public class BulkIT extends ESIntegTestCase {
         bulkResponse = client().prepareBulk()
                 .add(client().prepareUpdate("test", "type", "e1").setDoc("field", "2").setVersion(10)) // INTERNAL
                 .add(client().prepareUpdate("test", "type", "e1").setDoc("field", "3").setVersion(20).setVersionType(VersionType.FORCE))
-                .add(client().prepareUpdate("test", "type", "e1").setDoc("field", "3").setVersion(20).setVersionType(VersionType.INTERNAL)).get();
+                .add(client().prepareUpdate("test", "type", "e1").setDoc("field", "4").setVersion(20).setVersionType(VersionType.INTERNAL)).get();
 
         assertThat(bulkResponse.getItems()[0].getFailureMessage(), containsString("version conflict"));
         assertThat(((UpdateResponse) bulkResponse.getItems()[1].getResponse()).getVersion(), equalTo(20l));

+ 2 - 4
core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java

@@ -54,9 +54,7 @@ import java.nio.file.Path;
 import java.util.Set;
 
 import static org.elasticsearch.common.settings.Settings.settingsBuilder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.instanceOf;
-import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.*;
 
 /**
  *
@@ -249,7 +247,7 @@ public class AnalysisModuleTests extends ESTestCase {
             fail("This should fail with IllegalArgumentException because the analyzers name starts with _");
         } catch (ProvisionException e) {
             assertTrue(e.getCause() instanceof IllegalArgumentException);
-            assertThat(e.getCause().getMessage(), equalTo("analyzer name must not start with '_'. got \"_invalid_name\""));
+            assertThat(e.getCause().getMessage(), either(equalTo("analyzer name must not start with '_'. got \"_invalid_name\"")).or(equalTo("analyzer name must not start with '_'. got \"_invalidName\"")));
         }
     }
 

+ 6 - 33
core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java

@@ -29,15 +29,7 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexDeletionPolicy;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LiveIndexWriterConfig;
-import org.apache.lucene.index.LogByteSizeMergePolicy;
-import org.apache.lucene.index.MergePolicy;
-import org.apache.lucene.index.NoMergePolicy;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.lucene.index.*;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.TermQuery;
@@ -55,7 +47,6 @@ import org.elasticsearch.common.Base64;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.collect.Tuple;
 import org.elasticsearch.common.io.FileSystemUtils;
 import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.lucene.uid.Versions;
@@ -69,16 +60,9 @@ import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy;
 import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
 import org.elasticsearch.index.engine.Engine.Searcher;
 import org.elasticsearch.index.indexing.ShardIndexingService;
-import org.elasticsearch.index.mapper.ContentPath;
-import org.elasticsearch.index.mapper.DocumentMapper;
-import org.elasticsearch.index.mapper.DocumentMapperParser;
+import org.elasticsearch.index.mapper.*;
 import org.elasticsearch.index.mapper.Mapper.BuilderContext;
-import org.elasticsearch.index.mapper.MapperBuilders;
-import org.elasticsearch.index.mapper.MapperService;
-import org.elasticsearch.index.mapper.Mapping;
-import org.elasticsearch.index.mapper.MetadataFieldMapper;
 import org.elasticsearch.index.mapper.ParseContext.Document;
-import org.elasticsearch.index.mapper.ParsedDocument;
 import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
 import org.elasticsearch.index.mapper.internal.UidFieldMapper;
 import org.elasticsearch.index.mapper.object.RootObjectMapper;
@@ -107,13 +91,7 @@ import java.nio.charset.Charset;
 import java.nio.file.DirectoryStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
-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.*;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
@@ -121,12 +99,7 @@ import java.util.regex.Pattern;
 import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
 import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY;
 import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.hasKey;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.notNullValue;
-import static org.hamcrest.Matchers.nullValue;
+import static org.hamcrest.Matchers.*;
 
 public class InternalEngineTests extends ESTestCase {
 
@@ -1984,8 +1957,8 @@ public class InternalEngineTests extends ESTestCase {
         }
 
         @Override
-        protected Tuple<DocumentMapper, Mapping> docMapper(String type) {
-            return new Tuple<>(docMapper, mappingUpdate);
+        protected DocumentMapperForType docMapper(String type) {
+            return new DocumentMapperForType(docMapper, mappingUpdate);
         }
 
         @Override

+ 2 - 7
core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java

@@ -19,16 +19,11 @@
 package org.elasticsearch.index.mapper;
 
 import com.google.common.collect.ImmutableMap;
-
 import org.elasticsearch.Version;
 import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.xcontent.ToXContent;
-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.common.xcontent.*;
 import org.elasticsearch.index.IndexService;
 import org.elasticsearch.index.mapper.core.IntegerFieldMapper;
 import org.elasticsearch.index.mapper.core.StringFieldMapper;
@@ -368,7 +363,7 @@ public class DynamicMappingTests extends ESSingleNodeTestCase {
 
         // Even if the dynamic type of our new field is long, we already have a mapping for the same field
         // of type string so it should be mapped as a string
-        DocumentMapper newMapper = indexService.mapperService().documentMapperWithAutoCreate("type2").v1();
+        DocumentMapper newMapper = indexService.mapperService().documentMapperWithAutoCreate("type2").getDocumentMapper();
         Mapper update = parse(newMapper, indexService.mapperService().documentMapperParser(),
                 XContentFactory.jsonBuilder().startObject().field("my_field1", 42).endObject());
         Mapper myField1Mapper = null;

+ 2 - 2
core/src/test/java/org/elasticsearch/index/mapper/source/DefaultSourceMappingTests.java

@@ -38,7 +38,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
-import static org.hamcrest.Matchers.*;
+import static org.hamcrest.Matchers.equalTo;
 
 public class DefaultSourceMappingTests extends ESSingleNodeTestCase {
 
@@ -202,7 +202,7 @@ public class DefaultSourceMappingTests extends ESSingleNodeTestCase {
         MapperService mapperService = createIndex("test").mapperService();
         mapperService.merge(MapperService.DEFAULT_MAPPING, new CompressedXContent(defaultMapping), true, false);
 
-        DocumentMapper mapper = mapperService.documentMapperWithAutoCreate("my_type").v1();
+        DocumentMapper mapper = mapperService.documentMapperWithAutoCreate("my_type").getDocumentMapper();
         assertThat(mapper.type(), equalTo("my_type"));
         assertThat(mapper.sourceMapper().enabled(), equalTo(false));
     }

+ 40 - 0
core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java

@@ -20,9 +20,12 @@ package org.elasticsearch.index.shard;
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.Version;
+import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
 import org.elasticsearch.action.admin.indices.stats.IndexStats;
+import org.elasticsearch.action.admin.indices.stats.ShardStats;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.support.IndicesOptions;
 import org.elasticsearch.cluster.*;
@@ -31,11 +34,16 @@ import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.ShardRoutingState;
 import org.elasticsearch.cluster.routing.TestShardRouting;
 import org.elasticsearch.common.bytes.BytesArray;
+import org.elasticsearch.common.io.stream.BytesStreamOutput;
+import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.logging.ESLogger;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentFactory;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
 import org.elasticsearch.env.ShardLock;
+import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexService;
 import org.elasticsearch.index.engine.Engine;
 import org.elasticsearch.index.query.QueryParsingException;
@@ -539,4 +547,36 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         assertPathHasBeenCleared(startDir.toAbsolutePath().toString());
         assertPathHasBeenCleared(endDir.toAbsolutePath().toString());
     }
+
+    public void testShardStats() throws IOException {
+        createIndex("test");
+        ensureGreen();
+        IndicesService indicesService = getInstanceFromNode(IndicesService.class);
+        IndexService test = indicesService.indexService("test");
+        IndexShard shard = test.shard(0);
+        ShardStats stats = new ShardStats(shard, new CommonStatsFlags());
+        assertEquals(shard.shardPath().getRootDataPath().toString(), stats.getDataPath());
+        assertEquals(shard.shardPath().getRootStatePath().toString(), stats.getStatePath());
+        assertEquals(shard.shardPath().isCustomDataPath(), stats.isCustomDataPath());
+
+        if (randomBoolean() || true) { // try to serialize it to ensure values survive the serialization
+            BytesStreamOutput out = new BytesStreamOutput();
+            stats.writeTo(out);
+            StreamInput in = StreamInput.wrap(out.bytes());
+            stats = ShardStats.readShardStats(in);
+        }
+        XContentBuilder builder = XContentFactory.jsonBuilder();
+        builder.startObject();
+        stats.toXContent(builder, EMPTY_PARAMS);
+        builder.endObject();
+        String xContent = builder.string();
+        StringBuilder expectedSubSequence = new StringBuilder("\"shard_path\":{\"state_path\":\"");
+        expectedSubSequence.append(shard.shardPath().getRootStatePath().toString());
+        expectedSubSequence.append("\",\"data_path\":\"");
+        expectedSubSequence.append(shard.shardPath().getRootDataPath().toString());
+        expectedSubSequence.append("\",\"is_custom_data_path\":").append(shard.shardPath().isCustomDataPath()).append("}");
+        assumeFalse("Some path weirdness on windows", Constants.WINDOWS);
+        assertTrue(xContent.contains(expectedSubSequence));
+    }
+
 }

+ 69 - 0
core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java

@@ -18,6 +18,7 @@
  */
 package org.elasticsearch.index.shard;
 
+import com.carrotsearch.randomizedtesting.annotations.Repeat;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.env.NodeEnvironment;
@@ -26,6 +27,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.nio.file.Path;
+import java.util.Set;
 
 import static org.elasticsearch.common.settings.Settings.settingsBuilder;
 
@@ -78,4 +80,71 @@ public class ShardPathTests extends ESTestCase {
         }
     }
 
+    @Test(expected = IllegalArgumentException.class)
+    public void testIllegalCustomDataPath() {
+        final Path path = createTempDir().resolve("foo").resolve("0");
+        new ShardPath(true, path, path, "foo", new ShardId("foo", 0));
+    }
+
+    public void testValidCtor() {
+        final Path path = createTempDir().resolve("foo").resolve("0");
+        ShardPath shardPath = new ShardPath(false, path, path, "foo", new ShardId("foo", 0));
+        assertFalse(shardPath.isCustomDataPath());
+        assertEquals(shardPath.getDataPath(), path);
+        assertEquals(shardPath.getShardStatePath(), path);
+    }
+
+    public void testGetRootPaths() throws IOException {
+        boolean useCustomDataPath = randomBoolean();
+        final Settings indexSetttings;
+        final Settings nodeSettings;
+        Settings.Builder indexSettingsBuilder = settingsBuilder().put(IndexMetaData.SETTING_INDEX_UUID, "0xDEADBEEF");
+        final Path customPath;
+        if (useCustomDataPath) {
+            final Path path = createTempDir();
+            final boolean includeNodeId = randomBoolean();
+            indexSetttings = indexSettingsBuilder.put(IndexMetaData.SETTING_DATA_PATH, "custom").build();
+            nodeSettings = settingsBuilder().put("path.shared_data", path.toAbsolutePath().toAbsolutePath())
+                    .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH, includeNodeId).build();
+            if (includeNodeId) {
+                customPath = path.resolve("custom").resolve("0");
+            } else {
+                customPath = path.resolve("custom");
+            }
+        } else {
+            customPath = null;
+            indexSetttings = indexSettingsBuilder.build();
+            nodeSettings = Settings.EMPTY;
+        }
+        try (final NodeEnvironment env = newNodeEnvironment(nodeSettings)) {
+            ShardId shardId = new ShardId("foo", 0);
+            Path[] paths = env.availableShardPaths(shardId);
+            Path path = randomFrom(paths);
+            ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, "0xDEADBEEF"), 2, path);
+            ShardPath shardPath = ShardPath.loadShardPath(logger, env, shardId, indexSetttings);
+            boolean found = false;
+            for (Path p : env.nodeDataPaths()) {
+                if (p.equals(shardPath.getRootStatePath())) {
+                    found = true;
+                    break;
+                }
+            }
+            assertTrue("root state paths must be a node path but wasn't: " + shardPath.getRootStatePath(), found);
+            found = false;
+            if (useCustomDataPath) {
+                assertNotEquals(shardPath.getRootDataPath(), shardPath.getRootStatePath());
+                assertEquals(customPath, shardPath.getRootDataPath());
+            } else {
+                assertNull(customPath);
+                for (Path p : env.nodeDataPaths()) {
+                    if (p.equals(shardPath.getRootDataPath())) {
+                        found = true;
+                        break;
+                    }
+                }
+                assertTrue("root state paths must be a node path but wasn't: " + shardPath.getRootDataPath(), found);
+            }
+        }
+    }
+
 }

+ 4 - 4
core/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java

@@ -35,11 +35,11 @@ import java.util.Locale;
 public class IndexStoreTests extends ESTestCase {
 
     public void testStoreDirectory() throws IOException {
-        final Path tempDir = createTempDir();
+        final Path tempDir = createTempDir().resolve("foo").resolve("0");
         final IndexStoreModule.Type[] values = IndexStoreModule.Type.values();
         final IndexStoreModule.Type type = RandomPicks.randomFrom(random(), values);
         Settings settings = Settings.settingsBuilder().put(IndexStoreModule.STORE_TYPE, type.name().toLowerCase(Locale.ROOT)).build();
-        FsDirectoryService service = new FsDirectoryService(settings, null, new ShardPath(tempDir, tempDir, "foo", new ShardId("foo", 0)));
+        FsDirectoryService service = new FsDirectoryService(settings, null, new ShardPath(false, tempDir, tempDir, "foo", new ShardId("foo", 0)));
         try (final Directory directory = service.newFSDirectory(tempDir, NoLockFactory.INSTANCE)) {
             switch (type) {
                 case NIOFS:
@@ -70,9 +70,9 @@ public class IndexStoreTests extends ESTestCase {
     }
 
     public void testStoreDirectoryDefault() throws IOException {
-        final Path tempDir = createTempDir();
+        final Path tempDir = createTempDir().resolve("foo").resolve("0");
         Settings settings = Settings.EMPTY;
-        FsDirectoryService service = new FsDirectoryService(settings, null, new ShardPath(tempDir, tempDir, "foo", new ShardId("foo", 0)));
+        FsDirectoryService service = new FsDirectoryService(settings, null, new ShardPath(false, tempDir, tempDir, "foo", new ShardId("foo", 0)));
         try (final Directory directory = service.newFSDirectory(tempDir, NoLockFactory.INSTANCE)) {
             if (Constants.WINDOWS) {
                 assertTrue(directory.toString(), directory instanceof MMapDirectory || directory instanceof SimpleFSDirectory);

+ 12 - 0
core/src/test/java/org/elasticsearch/indices/warmer/SimpleIndicesWarmerIT.java

@@ -173,6 +173,18 @@ public class SimpleIndicesWarmerIT extends ESIntegTestCase {
         }
     }
 
+    @Test // issue 8991
+    public void deleteAllIndexWarmerDoesNotThrowWhenNoWarmers() {
+        createIndex("test");
+        DeleteWarmerResponse deleteWarmerResponse = client().admin().indices().prepareDeleteWarmer()
+                .setIndices("test").setNames("_all").execute().actionGet();
+        assertThat(deleteWarmerResponse.isAcknowledged(), equalTo(true));
+
+        deleteWarmerResponse = client().admin().indices().prepareDeleteWarmer()
+                .setIndices("test").setNames("foo", "_all", "bar").execute().actionGet();
+        assertThat(deleteWarmerResponse.isAcknowledged(), equalTo(true));
+    }
+
     @Test
     public void deleteIndexWarmerTest() {
         createIndex("test");

+ 6 - 0
core/src/test/java/org/elasticsearch/search/aggregations/bucket/ChildrenIT.java

@@ -255,9 +255,15 @@ public class ChildrenIT extends ESIntegTestCase {
             assertThat(count.getValue(), equalTo(4.));
 
             String idToUpdate = Integer.toString(randomInt(3));
+            /*
+             * The whole point of this test is to test these things with deleted
+             * docs in the index so we turn off detect_noop to make sure that
+             * the updates cause that.
+             */
             UpdateResponse updateResponse = client().prepareUpdate(indexName, "child", idToUpdate)
                     .setParent("1")
                     .setDoc("count", 1)
+                    .setDetectNoop(false)
                     .get();
             assertThat(updateResponse.getVersion(), greaterThan(1l));
             refresh();

+ 73 - 5
core/src/test/java/org/elasticsearch/ttl/SimpleTTLIT.java

@@ -20,17 +20,21 @@
 package org.elasticsearch.ttl;
 
 import com.google.common.base.Predicate;
+
 import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
 import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
 import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
 import org.elasticsearch.action.get.GetResponse;
 import org.elasticsearch.action.index.IndexResponse;
+import org.elasticsearch.action.update.UpdateRequestBuilder;
+import org.elasticsearch.action.update.UpdateResponse;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.index.get.GetField;
 import org.elasticsearch.test.ESIntegTestCase;
 import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
-import org.junit.Test;
+import org.elasticsearch.test.ESIntegTestCase.Scope;
 
 import java.io.IOException;
 import java.util.Locale;
@@ -39,9 +43,17 @@ import java.util.concurrent.TimeUnit;
 
 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.assertAcked;
-import static org.hamcrest.Matchers.*;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
+import static org.hamcrest.Matchers.both;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.hasKey;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.hamcrest.Matchers.nullValue;
 
 @ClusterScope(scope= Scope.SUITE, numDataNodes = 1)
 public class SimpleTTLIT extends ESIntegTestCase {
@@ -63,7 +75,6 @@ public class SimpleTTLIT extends ESIntegTestCase {
                 .build();
     }
 
-    @Test
     public void testSimpleTTL() throws Exception {
         assertAcked(prepareCreate("test")
                 .addMapping("type1", XContentFactory.jsonBuilder()
@@ -200,7 +211,7 @@ public class SimpleTTLIT extends ESIntegTestCase {
         assertThat(getResponse.isExists(), equalTo(false));
     }
 
-    @Test // issue 5053
+    // issue 5053
     public void testThatUpdatingMappingShouldNotRemoveTTLConfiguration() throws Exception {
         String index = "foo";
         String type = "mytype";
@@ -220,6 +231,63 @@ public class SimpleTTLIT extends ESIntegTestCase {
         assertTTLMappingEnabled(index, type);
     }
 
+    /**
+     * Test that updates with detect_noop set to true (the default) that don't
+     * change the source don't change the ttl. This is unexpected behavior and
+     * documented in ttl-field.asciidoc. If this behavior changes it is safe to
+     * rewrite this test to reflect the new behavior and to change the
+     * documentation.
+     */
+    public void testNoopUpdate() throws IOException {
+        assertAcked(prepareCreate("test")
+                .addMapping("type1", XContentFactory.jsonBuilder()
+                        .startObject()
+                        .startObject("type1")
+                        .startObject("_timestamp").field("enabled", true).endObject()
+                        .startObject("_ttl").field("enabled", true).endObject()
+                        .endObject()
+                        .endObject()));
+        ensureYellow("test");
+
+        long aLongTime = 10000000;
+        long firstTtl = aLongTime * 3;
+        long secondTtl = aLongTime * 2;
+        long thirdTtl = aLongTime * 1;
+        IndexResponse indexResponse = client().prepareIndex("test", "type1", "1").setSource("field1", "value1")
+                .setTTL(firstTtl).setRefresh(true).get();
+        assertTrue(indexResponse.isCreated());
+        assertThat(getTtl("type1", 1), both(lessThan(firstTtl)).and(greaterThan(secondTtl)));
+
+        // Updating with the default detect_noop without a change to the document doesn't change the ttl.
+        UpdateRequestBuilder update = client().prepareUpdate("test", "type1", "1").setDoc("field1", "value1").setTtl(secondTtl);
+        assertThat(updateAndGetTtl(update), both(lessThan(firstTtl)).and(greaterThan(secondTtl)));
+
+        // Updating with the default detect_noop with a change to the document does change the ttl.
+        update = client().prepareUpdate("test", "type1", "1").setDoc("field1", "value2").setTtl(secondTtl);
+        assertThat(updateAndGetTtl(update), both(lessThan(secondTtl)).and(greaterThan(thirdTtl)));
+
+        // Updating with detect_noop=true without a change to the document doesn't change the ttl.
+        update = client().prepareUpdate("test", "type1", "1").setDoc("field1", "value2").setTtl(secondTtl).setDetectNoop(true);
+        assertThat(updateAndGetTtl(update), both(lessThan(secondTtl)).and(greaterThan(thirdTtl)));
+
+        // Updating with detect_noop=false without a change to the document does change the ttl.
+        update = client().prepareUpdate("test", "type1", "1").setDoc("field1", "value2").setTtl(thirdTtl).setDetectNoop(false);
+        assertThat(updateAndGetTtl(update), lessThan(thirdTtl));
+    }
+
+    private long updateAndGetTtl(UpdateRequestBuilder update) {
+        UpdateResponse updateResponse = update.setFields("_ttl").get();
+        assertThat(updateResponse.getShardInfo().getFailed(), equalTo(0));
+        // You can't actually fetch _ttl from an update so we use a get.
+        return getTtl(updateResponse.getType(), updateResponse.getId());
+    }
+
+    private long getTtl(String type, Object id) {
+        GetResponse getResponse = client().prepareGet("test", type, id.toString()).setFields("_ttl").setRealtime(true).execute()
+                .actionGet();
+        return ((Number) getResponse.getField("_ttl").getValue()).longValue();
+    }
+
     private void assertTTLMappingEnabled(String index, String type) throws IOException {
         String errMsg = String.format(Locale.ROOT, "Expected ttl field mapping to be enabled for %s/%s", index, type);
 

+ 19 - 8
core/src/test/java/org/elasticsearch/update/UpdateNoopIT.java

@@ -19,6 +19,7 @@
 
 package org.elasticsearch.update;
 
+import org.elasticsearch.action.update.UpdateRequestBuilder;
 import org.elasticsearch.action.update.UpdateResponse;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
@@ -44,8 +45,10 @@ public class UpdateNoopIT extends ESIntegTestCase {
         updateAndCheckSource(4, fields("bar", null));
         updateAndCheckSource(4, fields("bar", null));
         updateAndCheckSource(5, fields("bar", "foo"));
+        // detect_noop defaults to true
+        updateAndCheckSource(5, null, fields("bar", "foo"));
 
-        assertEquals(3, totalNoopUpdates());
+        assertEquals(4, totalNoopUpdates());
     }
 
     @Test
@@ -210,7 +213,8 @@ public class UpdateNoopIT extends ESIntegTestCase {
     }
 
     /**
-     * Totally empty requests are noop if and only if detect noops is true.
+     * Totally empty requests are noop if and only if detect noops is true and
+     * its true by default.
      */
     @Test
     public void totallyEmpty() throws Exception {
@@ -223,6 +227,7 @@ public class UpdateNoopIT extends ESIntegTestCase {
                 .endObject());
         update(true, 1, XContentFactory.jsonBuilder().startObject().endObject());
         update(false, 2, XContentFactory.jsonBuilder().startObject().endObject());
+        update(null, 2, XContentFactory.jsonBuilder().startObject().endObject());
     }
 
     private XContentBuilder fields(Object... fields) throws IOException {
@@ -237,17 +242,23 @@ public class UpdateNoopIT extends ESIntegTestCase {
     }
 
     private void updateAndCheckSource(long expectedVersion, XContentBuilder xContentBuilder) {
-        UpdateResponse updateResponse = update(true, expectedVersion, xContentBuilder);
+        updateAndCheckSource(expectedVersion, true, xContentBuilder);
+    }
+
+    private void updateAndCheckSource(long expectedVersion, Boolean detectNoop, XContentBuilder xContentBuilder) {
+        UpdateResponse updateResponse = update(detectNoop, expectedVersion, xContentBuilder);
         assertEquals(updateResponse.getGetResult().sourceRef().toUtf8(), xContentBuilder.bytes().toUtf8());
     }
 
-    private UpdateResponse update(boolean detectNoop, long expectedVersion, XContentBuilder xContentBuilder) {
-        UpdateResponse updateResponse = client().prepareUpdate("test", "type1", "1")
+    private UpdateResponse update(Boolean detectNoop, long expectedVersion, XContentBuilder xContentBuilder) {
+        UpdateRequestBuilder updateRequest = client().prepareUpdate("test", "type1", "1")
                 .setDoc(xContentBuilder.bytes().toUtf8())
                 .setDocAsUpsert(true)
-                .setDetectNoop(detectNoop)
-                .setFields("_source")
-                .execute().actionGet();
+                .setFields("_source");
+        if (detectNoop != null) {
+            updateRequest.setDetectNoop(detectNoop);
+        }
+        UpdateResponse updateResponse = updateRequest.get();
         assertThat(updateResponse.getGetResult(), notNullValue());
         assertEquals(expectedVersion, updateResponse.getVersion());
         return updateResponse;

+ 8 - 10
docs/reference/docs/update.asciidoc

@@ -114,25 +114,23 @@ If both `doc` and `script` is specified, then `doc` is ignored. Best is
 to put your field pairs of the partial document in the script itself.
 
 [float]
-=== Detecting noop
-
-By default if `doc` is specified then the document is always updated even
-if the merging process doesn't cause any changes.  Specifying `detect_noop`
-as `true` will cause Elasticsearch to check if there are changes and, if
-there aren't, turn the update request into a noop. For example:
-
+=== Detecting noop updates
+If `doc` is specified its value is merged with the existing `_source`. By
+default the document is only reindexed if the new `_source` field differs from
+the old. Setting `detect_noop` to `false` will cause Elasticsearch to always
+update the document even if it hasn't changed. For example:
 [source,js]
 --------------------------------------------------
 curl -XPOST 'localhost:9200/test/type1/1/_update' -d '{
     "doc" : {
         "name" : "new_name"
     },
-    "detect_noop": true
+    "detect_noop": false
 }'
 --------------------------------------------------
 
-If `name` was `new_name` before the request was sent then the entire update
-request is ignored.
+If `name` was `new_name` before the request was sent then document is still
+reindexed.
 
 [[upserts]]
 [float]

+ 4 - 0
docs/reference/mapping/fields/ttl-field.asciidoc

@@ -104,3 +104,7 @@ may still be retrieved before they are purged.
 How many deletions are handled by a single <<docs-bulk,`bulk`>> request. The
 default value is `10000`.
 
+==== Note on `detect_noop`
+If an update tries to update just the `_ttl` without changing the `_source` of
+the document it's expiration time won't be updated if `detect_noop` is `true`.
+In 2.1 `detect_noop` defaults to `true`.

+ 1 - 1
docs/reference/migration/migrate_2_0/query_dsl.asciidoc

@@ -178,7 +178,7 @@ The parameter `percent_terms_to_match` has been removed in favor of
 The `limit` filter is deprecated and becomes a no-op. You can achieve similar
 behaviour using the <<search-request-body,terminate_after>> parameter.
 
-==== Jave plugins registering custom queries
+==== Java plugins registering custom queries
 
 Java plugins that register custom queries can do so by using the
 `IndicesQueriesModule#addQuery(Class<? extends QueryParser>)` method. Other

+ 18 - 0
docs/reference/migration/migrate_2_0/removals.asciidoc

@@ -40,6 +40,24 @@ can install the plugin with:
 ./bin/plugin install delete-by-query
 ------------------
 
+See {plugins}/plugins-delete-by-query.html for more information.
+
+==== Multicast Discovery is now a plugin
+
+Support for multicast is very patchy. Linux doesn’t allow multicast listening on localhost,
+while OS/X sends multicast broadcasts across all interfaces regardless of the configured
+bind address. On top of that, some networks have multicast disabled by default.
+
+This feature has been moved to a plugin. The default discovery mechanism now uses
+unicast, with a default setup which looks for the first 5 ports on localhost. If you
+still need to use multicast discovery, you can install the plugin with:
+
+[source,sh]
+------------------
+./bin/plugin install discovery-multicast
+------------------
+
+See {plugins}/discovery-multicast.html for more information.
 
 ==== `_shutdown` API
 

+ 11 - 1
docs/reference/migration/migrate_2_1.asciidoc

@@ -24,4 +24,14 @@ GET /my_index/_search?scroll=2m
 
 Scroll requests sorted by `_doc` have been optimized to more efficiently resume
 from where the previous request stopped, so this will have the same performance
-characteristics as the former `scan` search type.
+characteristics as the former `scan` search type.
+
+=== Update changes
+
+==== Updates now `detect_noop` by default
+
+We've switched the default value of the `detect_noop` option from `false` to
+`true`. This means that Elasticsearch will ignore updates that don't change
+source unless you explicitly set `"detect_noop": false`. `detect_noop` was
+always computationally cheap compared to the expense of the update which can be
+thought of as a delete operation followed by an index operation.

+ 1 - 1
rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_warmer.json

@@ -41,7 +41,7 @@
         },
         "request_cache": {
           "type" : "boolean",
-          "description" : "Specify whether the request to be wamred shoyd use the request cache, defaults to index level setting"
+          "description" : "Specify whether the request to be warmed should use the request cache, defaults to index level setting"
         }
       }
     },

+ 2 - 2
rest-api-spec/src/main/resources/rest-api-spec/test/update/75_ttl.yaml

@@ -56,7 +56,7 @@
  - lte:   { _ttl: 100000}
  - gt:    { _ttl: 10000}
 
-# duration
+# seconds
 
  - do:
       update:
@@ -66,6 +66,7 @@
           body:
             doc:        { foo: baz }
             upsert:     { foo: bar }
+            detect_noop:  false
           ttl:       20s
 
  - do:
@@ -89,4 +90,3 @@
           body:      { foo: bar }
           ttl:       20s
           timestamp: 2013-06-23T18:14:40
-