Browse Source

Merge branch 'master' into enhancement/remove_node_client_setting

javanna 9 years ago
parent
commit
a9f4982c40
49 changed files with 1849 additions and 704 deletions
  1. 6 29
      core/src/main/java/org/elasticsearch/action/ingest/SimulateExecutionService.java
  2. 14 0
      core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java
  3. 29 35
      core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java
  4. 28 7
      core/src/main/java/org/elasticsearch/bootstrap/BootstrapCheck.java
  5. 34 17
      core/src/main/java/org/elasticsearch/cluster/ClusterState.java
  6. 132 29
      core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java
  7. 0 4
      core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java
  8. 62 28
      core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java
  9. 1 1
      core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java
  10. 2 3
      core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java
  11. 17 12
      core/src/main/java/org/elasticsearch/index/IndexService.java
  12. 4 4
      core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java
  13. 84 23
      core/src/main/java/org/elasticsearch/index/shard/IndexShard.java
  14. 125 120
      core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java
  15. 21 11
      core/src/main/java/org/elasticsearch/ingest/core/CompoundProcessor.java
  16. 7 0
      core/src/main/java/org/elasticsearch/ingest/core/Pipeline.java
  17. 89 0
      core/src/main/java/org/elasticsearch/ingest/processor/TrackingResultProcessor.java
  18. 3 3
      core/src/main/java/org/elasticsearch/search/SearchModule.java
  19. 1 4
      core/src/main/java/org/elasticsearch/search/rescore/QueryRescoreMode.java
  20. 23 24
      core/src/main/java/org/elasticsearch/search/rescore/QueryRescorerBuilder.java
  21. 22 18
      core/src/main/java/org/elasticsearch/search/rescore/RescoreBuilder.java
  22. 44 77
      core/src/test/java/org/elasticsearch/action/ingest/SimulateExecutionServiceTests.java
  23. 10 0
      core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java
  24. 1 1
      core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java
  25. 10 7
      core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java
  26. 17 5
      core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java
  27. 15 5
      core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java
  28. 26 20
      core/src/test/java/org/elasticsearch/cluster/metadata/ToAndFromJsonMetaDataTests.java
  29. 241 0
      core/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java
  30. 0 1
      core/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java
  31. 22 16
      core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java
  32. 80 0
      core/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java
  33. 56 1
      core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java
  34. 153 40
      core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
  35. 1 0
      core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java
  36. 129 0
      core/src/test/java/org/elasticsearch/ingest/processor/TrackingResultProcessorTests.java
  37. 0 1
      core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java
  38. 2 1
      core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java
  39. 4 0
      distribution/src/main/packaging/scripts/postinst
  40. 8 0
      docs/reference/migration/migrate_5_0/settings.asciidoc
  41. 167 132
      docs/reference/modules/painless.asciidoc
  42. 3 3
      docs/reference/search/percolate.asciidoc
  43. 52 0
      qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/10_pipeline_with_mustache_templates.yaml
  44. 0 2
      qa/vagrant/src/test/resources/packaging/scripts/80_upgrade.bats
  45. 72 0
      qa/vagrant/src/test/resources/packaging/scripts/90_reinstall.bats
  46. 27 18
      qa/vagrant/src/test/resources/packaging/scripts/packaging_test_utils.bash
  47. 1 0
      rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json
  48. 1 1
      test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java
  49. 3 1
      test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java

+ 6 - 29
core/src/main/java/org/elasticsearch/action/ingest/SimulateExecutionService.java

@@ -23,13 +23,14 @@ import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.ActionRunnable;
 import org.elasticsearch.ingest.core.IngestDocument;
 import org.elasticsearch.ingest.core.Pipeline;
-import org.elasticsearch.ingest.core.Processor;
 import org.elasticsearch.ingest.core.CompoundProcessor;
 import org.elasticsearch.threadpool.ThreadPool;
 
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.elasticsearch.ingest.processor.TrackingResultProcessor.decorate;
+
 class SimulateExecutionService {
 
     private static final String THREAD_POOL_NAME = ThreadPool.Names.MANAGEMENT;
@@ -40,40 +41,16 @@ class SimulateExecutionService {
         this.threadPool = threadPool;
     }
 
-    void executeVerboseDocument(Processor processor, IngestDocument ingestDocument, List<SimulateProcessorResult> processorResultList) throws Exception {
-        if (processor instanceof CompoundProcessor) {
-            CompoundProcessor cp = (CompoundProcessor) processor;
-            try {
-                for (Processor p : cp.getProcessors()) {
-                    executeVerboseDocument(p, ingestDocument, processorResultList);
-                }
-            } catch (Exception e) {
-                for (Processor p : cp.getOnFailureProcessors()) {
-                    executeVerboseDocument(p, ingestDocument, processorResultList);
-                }
-            }
-        } else {
-            try {
-                processor.execute(ingestDocument);
-                processorResultList.add(new SimulateProcessorResult(processor.getTag(), new IngestDocument(ingestDocument)));
-            } catch (Exception e) {
-                processorResultList.add(new SimulateProcessorResult(processor.getTag(), e));
-                throw e;
-            }
-        }
-    }
-
     SimulateDocumentResult executeDocument(Pipeline pipeline, IngestDocument ingestDocument, boolean verbose) {
         if (verbose) {
             List<SimulateProcessorResult> processorResultList = new ArrayList<>();
-            IngestDocument currentIngestDocument = new IngestDocument(ingestDocument);
-            CompoundProcessor pipelineProcessor = new CompoundProcessor(pipeline.getProcessors(), pipeline.getOnFailureProcessors());
+            CompoundProcessor verbosePipelineProcessor = decorate(pipeline.getCompoundProcessor(), processorResultList);
             try {
-                executeVerboseDocument(pipelineProcessor, currentIngestDocument, processorResultList);
+                verbosePipelineProcessor.execute(ingestDocument);
+                return new SimulateDocumentVerboseResult(processorResultList);
             } catch (Exception e) {
-                return new SimulateDocumentBaseResult(e);
+                return new SimulateDocumentVerboseResult(processorResultList);
             }
-            return new SimulateDocumentVerboseResult(processorResultList);
         } else {
             try {
                 pipeline.execute(ingestDocument);

+ 14 - 0
core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java

@@ -51,6 +51,8 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
      */
     protected ShardId shardId;
 
+    long primaryTerm;
+
     protected TimeValue timeout = DEFAULT_TIMEOUT;
     protected String index;
 
@@ -148,6 +150,16 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
         return routedBasedOnClusterVersion;
     }
 
+    /** returns the primary term active at the time the operation was performed on the primary shard */
+    public long primaryTerm() {
+        return primaryTerm;
+    }
+
+    /** marks the primary term in which the operation was performed */
+    public void primaryTerm(long term) {
+        primaryTerm = term;
+    }
+
     @Override
     public ActionRequestValidationException validate() {
         ActionRequestValidationException validationException = null;
@@ -169,6 +181,7 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
         timeout = TimeValue.readTimeValue(in);
         index = in.readString();
         routedBasedOnClusterVersion = in.readVLong();
+        primaryTerm = in.readVLong();
     }
 
     @Override
@@ -184,6 +197,7 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
         timeout.writeTo(out);
         out.writeString(index);
         out.writeVLong(routedBasedOnClusterVersion);
+        out.writeVLong(primaryTerm);
     }
 
     @Override

+ 29 - 35
core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java

@@ -52,7 +52,6 @@ import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.util.concurrent.AbstractRunnable;
 import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
 import org.elasticsearch.common.util.concurrent.ThreadContext;
-import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexService;
 import org.elasticsearch.index.engine.VersionConflictEngineException;
 import org.elasticsearch.index.shard.IndexShard;
@@ -359,32 +358,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
                     }
                 });
             } else {
-                try {
-                    failReplicaIfNeeded(t);
-                } catch (Throwable unexpected) {
-                    logger.error("{} unexpected error while failing replica", unexpected, request.shardId().id());
-                } finally {
                     responseWithFailure(t);
-                }
-            }
-        }
-
-        private void failReplicaIfNeeded(Throwable t) {
-            Index index = request.shardId().getIndex();
-            int shardId = request.shardId().id();
-            logger.trace("failure on replica [{}][{}], action [{}], request [{}]", t, index, shardId, actionName, request);
-            if (ignoreReplicaException(t) == false) {
-                IndexService indexService = indicesService.indexService(index);
-                if (indexService == null) {
-                    logger.debug("ignoring failed replica {}[{}] because index was already removed.", index, shardId);
-                    return;
-                }
-                IndexShard indexShard = indexService.getShardOrNull(shardId);
-                if (indexShard == null) {
-                    logger.debug("ignoring failed replica {}[{}] because index was already removed.", index, shardId);
-                    return;
-                }
-                indexShard.failShard(actionName + " failed on replica", t);
             }
         }
 
@@ -401,7 +375,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
         protected void doRun() throws Exception {
             setPhase(task, "replica");
             assert request.shardId() != null : "request shardId must be set";
-            try (Releasable ignored = getIndexShardReferenceOnReplica(request.shardId())) {
+            try (Releasable ignored = getIndexShardReferenceOnReplica(request.shardId(), request.primaryTerm())) {
                 shardOperationOnReplica(request);
                 if (logger.isTraceEnabled()) {
                     logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction, request.shardId(), request);
@@ -707,7 +681,6 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
             indexShardReference = getIndexShardReferenceOnPrimary(shardId);
             if (indexShardReference.isRelocated() == false) {
                 executeLocally();
-
             } else {
                 executeRemotely();
             }
@@ -716,6 +689,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
         private void executeLocally() throws Exception {
             // execute locally
             Tuple<Response, ReplicaRequest> primaryResponse = shardOperationOnPrimary(state.metaData(), request);
+            primaryResponse.v2().primaryTerm(indexShardReference.opPrimaryTerm());
             if (logger.isTraceEnabled()) {
                 logger.trace("action [{}] completed on shard [{}] for request [{}] with cluster state version [{}]", transportPrimaryAction, shardId, request, state.version());
             }
@@ -825,17 +799,17 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
     protected IndexShardReference getIndexShardReferenceOnPrimary(ShardId shardId) {
         IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
         IndexShard indexShard = indexService.getShard(shardId.id());
-        return new IndexShardReferenceImpl(indexShard, true);
+        return IndexShardReferenceImpl.createOnPrimary(indexShard);
     }
 
     /**
      * returns a new reference to {@link IndexShard} on a node that the request is replicated to. The reference is closed as soon as
      * replication is completed on the node.
      */
-    protected IndexShardReference getIndexShardReferenceOnReplica(ShardId shardId) {
+    protected IndexShardReference getIndexShardReferenceOnReplica(ShardId shardId, long primaryTerm) {
         IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
         IndexShard indexShard = indexService.getShard(shardId.id());
-        return new IndexShardReferenceImpl(indexShard, false);
+        return IndexShardReferenceImpl.createOnReplica(indexShard, primaryTerm);
     }
 
     /**
@@ -1098,9 +1072,13 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
                                 totalShards,
                                 success.get(),
                                 failuresArray
-
                         )
                 );
+                if (logger.isTraceEnabled()) {
+                    logger.trace("finished replicating action [{}], request [{}], shardInfo [{}]", actionName, replicaRequest,
+                            finalResponse.getShardInfo());
+                }
+
                 try {
                     channel.sendResponse(finalResponse);
                 } catch (IOException responseException) {
@@ -1125,6 +1103,9 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
         boolean isRelocated();
         void failShard(String reason, @Nullable Throwable e);
         ShardRouting routingEntry();
+
+        /** returns the primary term of the current operation */
+        long opPrimaryTerm();
     }
 
     static final class IndexShardReferenceImpl implements IndexShardReference {
@@ -1132,15 +1113,23 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
         private final IndexShard indexShard;
         private final Releasable operationLock;
 
-        IndexShardReferenceImpl(IndexShard indexShard, boolean primaryAction) {
+        private IndexShardReferenceImpl(IndexShard indexShard, long primaryTerm) {
             this.indexShard = indexShard;
-            if (primaryAction) {
+            if (primaryTerm < 0) {
                 operationLock = indexShard.acquirePrimaryOperationLock();
             } else {
-                operationLock = indexShard.acquireReplicaOperationLock();
+                operationLock = indexShard.acquireReplicaOperationLock(primaryTerm);
             }
         }
 
+        static IndexShardReferenceImpl createOnPrimary(IndexShard indexShard) {
+            return new IndexShardReferenceImpl(indexShard, -1);
+        }
+
+        static IndexShardReferenceImpl createOnReplica(IndexShard indexShard, long primaryTerm) {
+            return new IndexShardReferenceImpl(indexShard, primaryTerm);
+        }
+
         @Override
         public void close() {
             operationLock.close();
@@ -1160,6 +1149,11 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
         public ShardRouting routingEntry() {
             return indexShard.routingEntry();
         }
+
+        @Override
+        public long opPrimaryTerm() {
+            return indexShard.getPrimaryTerm();
+        }
     }
 
     protected final void processAfterWrite(boolean refresh, IndexShard indexShard, Translog.Location location) {

+ 28 - 7
core/src/main/java/org/elasticsearch/bootstrap/BootstrapCheck.java

@@ -25,6 +25,7 @@ import org.elasticsearch.common.logging.Loggers;
 import org.elasticsearch.common.network.NetworkService;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.discovery.zen.elect.ElectMasterService;
 import org.elasticsearch.monitor.process.ProcessProbe;
 import org.elasticsearch.transport.TransportSettings;
 
@@ -39,7 +40,6 @@ import java.util.Set;
 /**
  * We enforce limits once any network host is configured. In this case we assume the node is running in production
  * and all production limit checks must pass. This should be extended as we go to settings like:
- * - discovery.zen.minimum_master_nodes
  * - discovery.zen.ping.unicast.hosts is set if we use zen disco
  * - ensure we can write in all data directories
  * - fail if vm.max_map_count is under a certain limit (not sure if this works cross platform)
@@ -114,10 +114,10 @@ final class BootstrapCheck {
     }
 
     // the list of checks to execute
-    private static List<Check> checks(final Settings settings) {
+    static List<Check> checks(final Settings settings) {
         final List<Check> checks = new ArrayList<>();
         final FileDescriptorCheck fileDescriptorCheck
-                = Constants.MAC_OS_X ? new OsXFileDescriptorCheck() : new FileDescriptorCheck();
+            = Constants.MAC_OS_X ? new OsXFileDescriptorCheck() : new FileDescriptorCheck();
         checks.add(fileDescriptorCheck);
         checks.add(new MlockallCheck(BootstrapSettings.MLOCKALL_SETTING.get(settings)));
         if (Constants.LINUX) {
@@ -126,6 +126,7 @@ final class BootstrapCheck {
         if (Constants.LINUX || Constants.MAC_OS_X) {
             checks.add(new MaxSizeVirtualMemoryCheck());
         }
+        checks.add(new MinMasterNodesCheck(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(settings)));
         return Collections.unmodifiableList(checks);
     }
 
@@ -186,10 +187,10 @@ final class BootstrapCheck {
         @Override
         public final String errorMessage() {
             return String.format(
-                    Locale.ROOT,
-                    "max file descriptors [%d] for elasticsearch process likely too low, increase to at least [%d]",
-                    getMaxFileDescriptorCount(),
-                    limit
+                Locale.ROOT,
+                "max file descriptors [%d] for elasticsearch process likely too low, increase to at least [%d]",
+                getMaxFileDescriptorCount(),
+                limit
             );
         }
 
@@ -226,6 +227,26 @@ final class BootstrapCheck {
 
     }
 
+    static class MinMasterNodesCheck implements Check {
+
+        final boolean minMasterNodesIsSet;
+
+        MinMasterNodesCheck(boolean minMasterNodesIsSet) {
+            this.minMasterNodesIsSet = minMasterNodesIsSet;
+        }
+
+        @Override
+        public boolean check() {
+            return minMasterNodesIsSet == false;
+        }
+
+        @Override
+        public String errorMessage() {
+            return "please set [" + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() +
+                "] to a majority of the number of master eligible nodes in your cluster.";
+        }
+    }
+
     static class MaxNumberOfThreadsCheck implements Check {
 
         private final long maxNumberOfThreadsThreshold = 1 << 11;

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

@@ -63,7 +63,7 @@ import java.util.Set;
 
 /**
  * Represents the current state of the cluster.
- *
+ * <p>
  * The cluster state object is immutable with an
  * exception of the {@link RoutingNodes} structure, which is built on demand from the {@link RoutingTable},
  * and cluster state {@link #status}, which is updated during cluster state publishing and applying
@@ -74,7 +74,7 @@ import java.util.Set;
  * the type of discovery. For example, for local discovery it is implemented by the {@link LocalDiscovery#publish}
  * method. In the Zen Discovery it is handled in the {@link PublishClusterStateAction#publish} method. The
  * publishing mechanism can be overridden by other discovery.
- *
+ * <p>
  * The cluster state implements the {@link Diffable} interface in order to support publishing of cluster state
  * differences instead of the entire state on each change. The publishing mechanism should only send differences
  * to a node if this node was present in the previous version of the cluster state. If a node is not present was
@@ -135,7 +135,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
 
     public static <T extends Custom> T lookupPrototypeSafe(String type) {
         @SuppressWarnings("unchecked")
-        T proto = (T)customPrototypes.get(type);
+        T proto = (T) customPrototypes.get(type);
         if (proto == null) {
             throw new IllegalArgumentException("No custom state prototype registered for type [" + type + "], node likely missing plugins");
         }
@@ -281,6 +281,16 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
         sb.append("state uuid: ").append(stateUUID).append("\n");
         sb.append("from_diff: ").append(wasReadFromDiff).append("\n");
         sb.append("meta data version: ").append(metaData.version()).append("\n");
+        for (IndexMetaData indexMetaData : metaData) {
+            final String TAB = "   ";
+            sb.append(TAB).append(indexMetaData.getIndex());
+            sb.append(": v[").append(indexMetaData.getVersion()).append("]\n");
+            for (int shard = 0; shard < indexMetaData.getNumberOfShards(); shard++) {
+                sb.append(TAB).append(TAB).append(shard).append(": ");
+                sb.append("p_term [").append(indexMetaData.primaryTerm(shard)).append("], ");
+                sb.append("a_ids ").append(indexMetaData.activeAllocationIds(shard)).append("\n");
+            }
+        }
         sb.append(blocks().prettyPrint());
         sb.append(nodes().prettyPrint());
         sb.append(routingTable().prettyPrint());
@@ -477,6 +487,12 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
                 }
                 builder.endArray();
 
+                builder.startObject(IndexMetaData.KEY_PRIMARY_TERMS);
+                for (int shard = 0; shard < indexMetaData.getNumberOfShards(); shard++) {
+                    builder.field(Integer.toString(shard), indexMetaData.primaryTerm(shard));
+                }
+                builder.endObject();
+
                 builder.startObject(IndexMetaData.KEY_ACTIVE_ALLOCATIONS);
                 for (IntObjectCursor<Set<String>> cursor : indexMetaData.getActiveAllocationIds()) {
                     builder.startArray(String.valueOf(cursor.key));
@@ -487,6 +503,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
                 }
                 builder.endObject();
 
+                // index metadata
                 builder.endObject();
             }
             builder.endObject();
@@ -683,16 +700,16 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
         }
 
         /**
-         * @param data               input bytes
-         * @param localNode          used to set the local node in the cluster state.
+         * @param data      input bytes
+         * @param localNode used to set the local node in the cluster state.
          */
         public static ClusterState fromBytes(byte[] data, DiscoveryNode localNode) throws IOException {
             return readFrom(StreamInput.wrap(data), localNode);
         }
 
         /**
-         * @param in                 input stream
-         * @param localNode          used to set the local node in the cluster state. can be null.
+         * @param in        input stream
+         * @param localNode used to set the local node in the cluster state. can be null.
          */
         public static ClusterState readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException {
             return PROTO.readFrom(in, localNode);
@@ -791,17 +808,17 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
             metaData = proto.metaData.readDiffFrom(in);
             blocks = proto.blocks.readDiffFrom(in);
             customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(),
-                    new DiffableUtils.DiffableValueSerializer<String, Custom>() {
-                @Override
-                public Custom read(StreamInput in, String key) throws IOException {
-                    return lookupPrototypeSafe(key).readFrom(in);
-                }
+                new DiffableUtils.DiffableValueSerializer<String, Custom>() {
+                    @Override
+                    public Custom read(StreamInput in, String key) throws IOException {
+                        return lookupPrototypeSafe(key).readFrom(in);
+                    }
 
-                @Override
-                public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
-                    return lookupPrototypeSafe(key).readDiffFrom(in);
-                }
-            });
+                    @Override
+                    public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
+                        return lookupPrototypeSafe(key).readDiffFrom(in);
+                    }
+                });
         }
 
         @Override

+ 132 - 29
core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java

@@ -19,6 +19,7 @@
 
 package org.elasticsearch.cluster.metadata;
 
+import com.carrotsearch.hppc.LongArrayList;
 import com.carrotsearch.hppc.cursors.IntObjectCursor;
 import com.carrotsearch.hppc.cursors.ObjectCursor;
 import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
@@ -29,6 +30,8 @@ import org.elasticsearch.cluster.DiffableUtils;
 import org.elasticsearch.cluster.block.ClusterBlock;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.node.DiscoveryNodeFilters;
+import org.elasticsearch.cluster.routing.RoutingTable;
+import org.elasticsearch.cluster.routing.allocation.AllocationService;
 import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.ParseFieldMatcher;
 import org.elasticsearch.common.collect.ImmutableOpenIntMap;
@@ -56,6 +59,7 @@ import org.joda.time.DateTimeZone;
 
 import java.io.IOException;
 import java.text.ParseException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -217,6 +221,13 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
             .numberOfShards(1).numberOfReplicas(0).build();
 
     public static final String KEY_ACTIVE_ALLOCATIONS = "active_allocations";
+    static final String KEY_VERSION = "version";
+    static final String KEY_SETTINGS = "settings";
+    static final String KEY_STATE = "state";
+    static final String KEY_MAPPINGS = "mappings";
+    static final String KEY_ALIASES = "aliases";
+    public static final String KEY_PRIMARY_TERMS = "primary_terms";
+
     public static final String INDEX_STATE_FILE_PREFIX = "state-";
 
     private final int numberOfShards;
@@ -224,6 +235,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
 
     private final Index index;
     private final long version;
+    private final long[] primaryTerms;
 
     private final State state;
 
@@ -247,7 +259,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
     private final Version indexUpgradedVersion;
     private final org.apache.lucene.util.Version minimumCompatibleLuceneVersion;
 
-    private IndexMetaData(Index index, long version, State state, int numberOfShards, int numberOfReplicas, Settings settings,
+    private IndexMetaData(Index index, long version, long[] primaryTerms, State state, int numberOfShards, int numberOfReplicas, Settings settings,
                           ImmutableOpenMap<String, MappingMetaData> mappings, ImmutableOpenMap<String, AliasMetaData> aliases,
                           ImmutableOpenMap<String, Custom> customs, ImmutableOpenIntMap<Set<String>> activeAllocationIds,
                           DiscoveryNodeFilters requireFilters, DiscoveryNodeFilters includeFilters, DiscoveryNodeFilters excludeFilters,
@@ -255,6 +267,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
 
         this.index = index;
         this.version = version;
+        this.primaryTerms = primaryTerms;
+        assert primaryTerms.length == numberOfShards;
         this.state = state;
         this.numberOfShards = numberOfShards;
         this.numberOfReplicas = numberOfReplicas;
@@ -296,6 +310,16 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
         return this.version;
     }
 
+
+    /**
+     * The term of the current selected primary. This is a non-negative number incremented when
+     * a primary shard is assigned after a full cluster restart or a replica shard is promoted to a primary
+     * See {@link AllocationService#updateMetaDataWithRoutingTable(MetaData, RoutingTable, RoutingTable)}.
+     **/
+    public long primaryTerm(int shardId) {
+        return this.primaryTerms[shardId];
+    }
+
     /**
      * Return the {@link Version} on which this index has been created. This
      * information is typically useful for backward compatibility.
@@ -416,6 +440,10 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
 
         IndexMetaData that = (IndexMetaData) o;
 
+        if (version != that.version) {
+            return false;
+        }
+
         if (!aliases.equals(that.aliases)) {
             return false;
         }
@@ -434,6 +462,10 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
         if (!customs.equals(that.customs)) {
             return false;
         }
+
+        if (Arrays.equals(primaryTerms, that.primaryTerms) == false) {
+            return false;
+        }
         if (!activeAllocationIds.equals(that.activeAllocationIds)) {
             return false;
         }
@@ -443,14 +475,18 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
     @Override
     public int hashCode() {
         int result = index.hashCode();
+        result = 31 * result + Long.hashCode(version);
         result = 31 * result + state.hashCode();
         result = 31 * result + aliases.hashCode();
         result = 31 * result + settings.hashCode();
         result = 31 * result + mappings.hashCode();
+        result = 31 * result + customs.hashCode();
+        result = 31 * result + Arrays.hashCode(primaryTerms);
         result = 31 * result + activeAllocationIds.hashCode();
         return result;
     }
 
+
     @Override
     public Diff<IndexMetaData> diff(IndexMetaData previousState) {
         return new IndexMetaDataDiff(previousState, this);
@@ -476,6 +512,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
 
         private final String index;
         private final long version;
+        private final long[] primaryTerms;
         private final State state;
         private final Settings settings;
         private final Diff<ImmutableOpenMap<String, MappingMetaData>> mappings;
@@ -488,11 +525,12 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
             version = after.version;
             state = after.state;
             settings = after.settings;
+            primaryTerms = after.primaryTerms;
             mappings = DiffableUtils.diff(before.mappings, after.mappings, DiffableUtils.getStringKeySerializer());
             aliases = DiffableUtils.diff(before.aliases, after.aliases, DiffableUtils.getStringKeySerializer());
             customs = DiffableUtils.diff(before.customs, after.customs, DiffableUtils.getStringKeySerializer());
             activeAllocationIds = DiffableUtils.diff(before.activeAllocationIds, after.activeAllocationIds,
-                    DiffableUtils.getVIntKeySerializer(), DiffableUtils.StringSetValueSerializer.getInstance());
+                DiffableUtils.getVIntKeySerializer(), DiffableUtils.StringSetValueSerializer.getInstance());
         }
 
         public IndexMetaDataDiff(StreamInput in) throws IOException {
@@ -500,22 +538,23 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
             version = in.readLong();
             state = State.fromId(in.readByte());
             settings = Settings.readSettingsFromStream(in);
+            primaryTerms = in.readVLongArray();
             mappings = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), MappingMetaData.PROTO);
             aliases = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), AliasMetaData.PROTO);
             customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(),
-                    new DiffableUtils.DiffableValueSerializer<String, Custom>() {
-                        @Override
-                        public Custom read(StreamInput in, String key) throws IOException {
-                            return lookupPrototypeSafe(key).readFrom(in);
-                        }
+                new DiffableUtils.DiffableValueSerializer<String, Custom>() {
+                    @Override
+                    public Custom read(StreamInput in, String key) throws IOException {
+                        return lookupPrototypeSafe(key).readFrom(in);
+                    }
 
-                        @Override
-                        public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
-                            return lookupPrototypeSafe(key).readDiffFrom(in);
-                        }
-                    });
+                    @Override
+                    public Diff<Custom> readDiff(StreamInput in, String key) throws IOException {
+                        return lookupPrototypeSafe(key).readDiffFrom(in);
+                    }
+                });
             activeAllocationIds = DiffableUtils.readImmutableOpenIntMapDiff(in, DiffableUtils.getVIntKeySerializer(),
-                    DiffableUtils.StringSetValueSerializer.getInstance());
+                DiffableUtils.StringSetValueSerializer.getInstance());
         }
 
         @Override
@@ -524,6 +563,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
             out.writeLong(version);
             out.writeByte(state.id);
             Settings.writeSettingsToStream(settings, out);
+            out.writeVLongArray(primaryTerms);
             mappings.writeTo(out);
             aliases.writeTo(out);
             customs.writeTo(out);
@@ -536,6 +576,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
             builder.version(version);
             builder.state(state);
             builder.settings(settings);
+            builder.primaryTerms(primaryTerms);
             builder.mappings.putAll(mappings.apply(part.mappings));
             builder.aliases.putAll(aliases.apply(part.aliases));
             builder.customs.putAll(customs.apply(part.customs));
@@ -550,6 +591,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
         builder.version(in.readLong());
         builder.state(State.fromId(in.readByte()));
         builder.settings(readSettingsFromStream(in));
+        builder.primaryTerms(in.readVLongArray());
         int mappingsSize = in.readVInt();
         for (int i = 0; i < mappingsSize; i++) {
             MappingMetaData mappingMd = MappingMetaData.PROTO.readFrom(in);
@@ -581,6 +623,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
         out.writeLong(version);
         out.writeByte(state.id());
         writeSettingsToStream(settings, out);
+        out.writeVLongArray(primaryTerms);
         out.writeVInt(mappings.size());
         for (ObjectCursor<MappingMetaData> cursor : mappings.values()) {
             cursor.value.writeTo(out);
@@ -614,6 +657,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
         private String index;
         private State state = State.OPEN;
         private long version = 1;
+        private long[] primaryTerms = null;
         private Settings settings = Settings.Builder.EMPTY_SETTINGS;
         private final ImmutableOpenMap.Builder<String, MappingMetaData> mappings;
         private final ImmutableOpenMap.Builder<String, AliasMetaData> aliases;
@@ -633,6 +677,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
             this.state = indexMetaData.state;
             this.version = indexMetaData.version;
             this.settings = indexMetaData.getSettings();
+            this.primaryTerms = indexMetaData.primaryTerms.clone();
             this.mappings = ImmutableOpenMap.builder(indexMetaData.mappings);
             this.aliases = ImmutableOpenMap.builder(indexMetaData.aliases);
             this.customs = ImmutableOpenMap.builder(indexMetaData.customs);
@@ -672,8 +717,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
         }
 
         public Builder settings(Settings.Builder settings) {
-            this.settings = settings.build();
-            return this;
+            return settings(settings.build());
         }
 
         public Builder settings(Settings settings) {
@@ -741,6 +785,42 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
             return this;
         }
 
+        /**
+         * returns the primary term for the given shard.
+         * See {@link IndexMetaData#primaryTerm(int)} for more information.
+         */
+        public long primaryTerm(int shardId) {
+            if (primaryTerms == null) {
+                initializePrimaryTerms();
+            }
+            return this.primaryTerms[shardId];
+        }
+
+        /**
+         * sets the primary term for the given shard.
+         * See {@link IndexMetaData#primaryTerm(int)} for more information.
+         */
+        public Builder primaryTerm(int shardId, long primaryTerm) {
+            if (primaryTerms == null) {
+                initializePrimaryTerms();
+            }
+            this.primaryTerms[shardId] = primaryTerm;
+            return this;
+        }
+
+        private void primaryTerms(long[] primaryTerms) {
+            this.primaryTerms = primaryTerms.clone();
+        }
+
+        private void initializePrimaryTerms() {
+            assert primaryTerms == null;
+            if (numberOfShards() < 0) {
+                throw new IllegalStateException("you must set the number of shards before setting/reading primary terms");
+            }
+            primaryTerms = new long[numberOfShards()];
+        }
+
+
         public IndexMetaData build() {
             ImmutableOpenMap.Builder<String, AliasMetaData> tmpAliases = aliases;
             Settings tmpSettings = settings;
@@ -815,27 +895,34 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
                 minimumCompatibleLuceneVersion = null;
             }
 
+            if (primaryTerms == null) {
+                initializePrimaryTerms();
+            } else if (primaryTerms.length != numberOfShards) {
+                throw new IllegalStateException("primaryTerms length is [" + primaryTerms.length
+                    + "] but should be equal to number of shards [" + numberOfShards() + "]");
+            }
+
             final String uuid = settings.get(SETTING_INDEX_UUID, INDEX_UUID_NA_VALUE);
-            return new IndexMetaData(new Index(index, uuid), version, state, numberOfShards, numberOfReplicas, tmpSettings, mappings.build(),
-                    tmpAliases.build(), customs.build(), filledActiveAllocationIds.build(), requireFilters, includeFilters, excludeFilters,
-                    indexCreatedVersion, indexUpgradedVersion, minimumCompatibleLuceneVersion);
+            return new IndexMetaData(new Index(index, uuid), version, primaryTerms, state, numberOfShards, numberOfReplicas, tmpSettings, mappings.build(),
+                tmpAliases.build(), customs.build(), filledActiveAllocationIds.build(), requireFilters, includeFilters, excludeFilters,
+                indexCreatedVersion, indexUpgradedVersion, minimumCompatibleLuceneVersion);
         }
 
         public static void toXContent(IndexMetaData indexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException {
             builder.startObject(indexMetaData.getIndex().getName(), XContentBuilder.FieldCaseConversion.NONE);
 
-            builder.field("version", indexMetaData.getVersion());
-            builder.field("state", indexMetaData.getState().toString().toLowerCase(Locale.ENGLISH));
+            builder.field(KEY_VERSION, indexMetaData.getVersion());
+            builder.field(KEY_STATE, indexMetaData.getState().toString().toLowerCase(Locale.ENGLISH));
 
             boolean binary = params.paramAsBoolean("binary", false);
 
-            builder.startObject("settings");
+            builder.startObject(KEY_SETTINGS);
             for (Map.Entry<String, String> entry : indexMetaData.getSettings().getAsMap().entrySet()) {
                 builder.field(entry.getKey(), entry.getValue());
             }
             builder.endObject();
 
-            builder.startArray("mappings");
+            builder.startArray(KEY_MAPPINGS);
             for (ObjectObjectCursor<String, MappingMetaData> cursor : indexMetaData.getMappings()) {
                 if (binary) {
                     builder.value(cursor.value.source().compressed());
@@ -855,12 +942,18 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
                 builder.endObject();
             }
 
-            builder.startObject("aliases");
+            builder.startObject(KEY_ALIASES);
             for (ObjectCursor<AliasMetaData> cursor : indexMetaData.getAliases().values()) {
                 AliasMetaData.Builder.toXContent(cursor.value, builder, params);
             }
             builder.endObject();
 
+            builder.startArray(KEY_PRIMARY_TERMS);
+            for (int i = 0; i < indexMetaData.getNumberOfShards(); i++) {
+                builder.value(indexMetaData.primaryTerm(i));
+            }
+            builder.endArray();
+
             builder.startObject(KEY_ACTIVE_ALLOCATIONS);
             for (IntObjectCursor<Set<String>> cursor : indexMetaData.activeAllocationIds) {
                 builder.startArray(String.valueOf(cursor.key));
@@ -895,9 +988,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
                 if (token == XContentParser.Token.FIELD_NAME) {
                     currentFieldName = parser.currentName();
                 } else if (token == XContentParser.Token.START_OBJECT) {
-                    if ("settings".equals(currentFieldName)) {
+                    if (KEY_SETTINGS.equals(currentFieldName)) {
                         builder.settings(Settings.settingsBuilder().put(SettingsLoader.Helper.loadNestedFromMap(parser.mapOrdered())));
-                    } else if ("mappings".equals(currentFieldName)) {
+                    } else if (KEY_MAPPINGS.equals(currentFieldName)) {
                         while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
                             if (token == XContentParser.Token.FIELD_NAME) {
                                 currentFieldName = parser.currentName();
@@ -909,7 +1002,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
                                 throw new IllegalArgumentException("Unexpected token: " + token);
                             }
                         }
-                    } else if ("aliases".equals(currentFieldName)) {
+                    } else if (KEY_ALIASES.equals(currentFieldName)) {
                         while (parser.nextToken() != XContentParser.Token.END_OBJECT) {
                             builder.putAlias(AliasMetaData.Builder.fromXContent(parser));
                         }
@@ -949,7 +1042,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
                         }
                     }
                 } else if (token == XContentParser.Token.START_ARRAY) {
-                    if ("mappings".equals(currentFieldName)) {
+                    if (KEY_MAPPINGS.equals(currentFieldName)) {
                         while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
                             if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) {
                                 builder.putMapping(new MappingMetaData(new CompressedXContent(parser.binaryValue())));
@@ -961,13 +1054,23 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
                                 }
                             }
                         }
+                    } else if (KEY_PRIMARY_TERMS.equals(currentFieldName)) {
+                        LongArrayList list = new LongArrayList();
+                        while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
+                            if (token == XContentParser.Token.VALUE_NUMBER) {
+                                list.add(parser.longValue());
+                            } else {
+                                throw new IllegalStateException("found a non-numeric value under [" + KEY_PRIMARY_TERMS + "]");
+                            }
+                        }
+                        builder.primaryTerms(list.toArray());
                     } else {
                         throw new IllegalArgumentException("Unexpected field for an array " + currentFieldName);
                     }
                 } else if (token.isValue()) {
-                    if ("state".equals(currentFieldName)) {
+                    if (KEY_STATE.equals(currentFieldName)) {
                         builder.state(State.fromString(parser.text()));
-                    } else if ("version".equals(currentFieldName)) {
+                    } else if (KEY_VERSION.equals(currentFieldName)) {
                         builder.version(parser.longValue());
                     } else {
                         throw new IllegalArgumentException("Unexpected field [" + currentFieldName + "]");

+ 0 - 4
core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java

@@ -586,10 +586,6 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
             if (indicesRouting == null) {
                 throw new IllegalStateException("once build is called the builder cannot be reused");
             }
-            // normalize the versions right before we build it...
-            for (ObjectCursor<IndexRoutingTable> indexRoutingTable : indicesRouting.values()) {
-                indicesRouting.put(indexRoutingTable.value.getIndex().getName(), indexRoutingTable.value);
-            }
             RoutingTable table = new RoutingTable(version, indicesRouting.build());
             indicesRouting = null;
             return table;

+ 62 - 28
core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java

@@ -42,6 +42,7 @@ import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.gateway.GatewayAllocator;
+import org.elasticsearch.index.shard.ShardId;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -98,7 +99,7 @@ public class AllocationService extends AbstractComponent {
         if (withReroute) {
             reroute(allocation);
         }
-        final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes);
+        final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), clusterState.routingTable(), routingNodes);
 
         String startedShardsAsString = firstListElementsToCommaDelimitedString(startedShards, s -> s.shardId().toString());
         logClusterHealthStateChange(
@@ -107,37 +108,44 @@ public class AllocationService extends AbstractComponent {
                 "shards started [" + startedShardsAsString + "] ..."
         );
         return result;
-    }
 
+    }
 
-    protected RoutingAllocation.Result buildChangedResult(MetaData metaData, RoutingNodes routingNodes) {
-        return buildChangedResult(metaData, routingNodes, new RoutingExplanations());
+    protected RoutingAllocation.Result buildChangedResult(MetaData oldMetaData, RoutingTable oldRoutingTable, RoutingNodes newRoutingNodes) {
+        return buildChangedResult(oldMetaData, oldRoutingTable, newRoutingNodes, new RoutingExplanations());
 
     }
-    protected RoutingAllocation.Result buildChangedResult(MetaData metaData, RoutingNodes routingNodes, RoutingExplanations explanations) {
-        final RoutingTable routingTable = new RoutingTable.Builder().updateNodes(routingNodes).build();
-        MetaData newMetaData = updateMetaDataWithRoutingTable(metaData,routingTable);
-        return new RoutingAllocation.Result(true, routingTable.validateRaiseException(newMetaData), newMetaData, explanations);
+
+    protected RoutingAllocation.Result buildChangedResult(MetaData oldMetaData, RoutingTable oldRoutingTable, RoutingNodes newRoutingNodes,
+                                                          RoutingExplanations explanations) {
+        final RoutingTable newRoutingTable = new RoutingTable.Builder().updateNodes(newRoutingNodes).build();
+        MetaData newMetaData = updateMetaDataWithRoutingTable(oldMetaData, oldRoutingTable, newRoutingTable);
+        return new RoutingAllocation.Result(true, newRoutingTable.validateRaiseException(newMetaData), newMetaData, explanations);
     }
 
     /**
-     * Updates the current {@link MetaData} based on the newly created {@link RoutingTable}.
+     * Updates the current {@link MetaData} based on the newly created {@link RoutingTable}. Specifically
+     * we update {@link IndexMetaData#getActiveAllocationIds()} and {@link IndexMetaData#primaryTerm(int)} based on
+     * the changes made during this allocation.
      *
-     * @param currentMetaData {@link MetaData} object from before the routing table was changed.
+     * @param oldMetaData     {@link MetaData} object from before the routing table was changed.
+     * @param oldRoutingTable {@link RoutingTable} from before the  change.
      * @param newRoutingTable new {@link RoutingTable} created by the allocation change
      * @return adapted {@link MetaData}, potentially the original one if no change was needed.
      */
-    static MetaData updateMetaDataWithRoutingTable(MetaData currentMetaData, RoutingTable newRoutingTable) {
-        // make sure index meta data and routing tables are in sync w.r.t active allocation ids
+    static MetaData updateMetaDataWithRoutingTable(MetaData oldMetaData, RoutingTable oldRoutingTable, RoutingTable newRoutingTable) {
         MetaData.Builder metaDataBuilder = null;
-        for (IndexRoutingTable indexRoutingTable : newRoutingTable) {
-            final IndexMetaData indexMetaData = currentMetaData.index(indexRoutingTable.getIndex());
-            if (indexMetaData == null) {
-                throw new IllegalStateException("no metadata found for index " + indexRoutingTable.getIndex().getName());
+        for (IndexRoutingTable newIndexTable : newRoutingTable) {
+            final IndexMetaData oldIndexMetaData = oldMetaData.index(newIndexTable.getIndex());
+            if (oldIndexMetaData == null) {
+                throw new IllegalStateException("no metadata found for index " + newIndexTable.getIndex().getName());
             }
             IndexMetaData.Builder indexMetaDataBuilder = null;
-            for (IndexShardRoutingTable shardRoutings : indexRoutingTable) {
-                Set<String> activeAllocationIds = shardRoutings.activeShards().stream()
+            for (IndexShardRoutingTable newShardTable : newIndexTable) {
+                final ShardId shardId = newShardTable.shardId();
+
+                // update activeAllocationIds
+                Set<String> activeAllocationIds = newShardTable.activeShards().stream()
                         .map(ShardRouting::allocationId)
                         .filter(Objects::nonNull)
                         .map(AllocationId::getId)
@@ -145,19 +153,44 @@ public class AllocationService extends AbstractComponent {
                 // only update active allocation ids if there is an active shard
                 if (activeAllocationIds.isEmpty() == false) {
                     // get currently stored allocation ids
-                    Set<String> storedAllocationIds = indexMetaData.activeAllocationIds(shardRoutings.shardId().id());
+                    Set<String> storedAllocationIds = oldIndexMetaData.activeAllocationIds(shardId.id());
                     if (activeAllocationIds.equals(storedAllocationIds) == false) {
                         if (indexMetaDataBuilder == null) {
-                            indexMetaDataBuilder = IndexMetaData.builder(indexMetaData);
+                            indexMetaDataBuilder = IndexMetaData.builder(oldIndexMetaData);
                         }
+                        indexMetaDataBuilder.putActiveAllocationIds(shardId.id(), activeAllocationIds);
+                    }
+                }
 
-                        indexMetaDataBuilder.putActiveAllocationIds(shardRoutings.shardId().id(), activeAllocationIds);
+                // update primary terms
+                final ShardRouting newPrimary = newShardTable.primaryShard();
+                if (newPrimary == null) {
+                    throw new IllegalStateException("missing primary shard for " + newShardTable.shardId());
+                }
+                final ShardRouting oldPrimary = oldRoutingTable.shardRoutingTable(shardId).primaryShard();
+                if (oldPrimary == null) {
+                    throw new IllegalStateException("missing primary shard for " + newShardTable.shardId());
+                }
+                // we update the primary term on initial assignment or when a replica is promoted. Most notably we do *not*
+                // update them when a primary relocates
+                if (newPrimary.unassigned() ||
+                        newPrimary.isSameAllocation(oldPrimary) ||
+                        // we do not use newPrimary.isTargetRelocationOf(oldPrimary) because that one enforces newPrimary to
+                        // be initializing. However, when the target shard is activated, we still want the primary term to staty
+                        // the same
+                        (oldPrimary.relocating() && newPrimary.isSameAllocation(oldPrimary.buildTargetRelocatingShard()))) {
+                    // do nothing
+                } else {
+                    // incrementing the primary term
+                    if (indexMetaDataBuilder == null) {
+                        indexMetaDataBuilder = IndexMetaData.builder(oldIndexMetaData);
                     }
+                    indexMetaDataBuilder.primaryTerm(shardId.id(), oldIndexMetaData.primaryTerm(shardId.id()) + 1);
                 }
             }
             if (indexMetaDataBuilder != null) {
                 if (metaDataBuilder == null) {
-                    metaDataBuilder = MetaData.builder(currentMetaData);
+                    metaDataBuilder = MetaData.builder(oldMetaData);
                 }
                 metaDataBuilder.put(indexMetaDataBuilder);
             }
@@ -165,7 +198,7 @@ public class AllocationService extends AbstractComponent {
         if (metaDataBuilder != null) {
             return metaDataBuilder.build();
         } else {
-            return currentMetaData;
+            return oldMetaData;
         }
     }
 
@@ -196,7 +229,7 @@ public class AllocationService extends AbstractComponent {
         }
         gatewayAllocator.applyFailedShards(allocation);
         reroute(allocation);
-        final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes);
+        final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), clusterState.routingTable(), routingNodes);
         String failedShardsAsString = firstListElementsToCommaDelimitedString(failedShards, s -> s.shard.shardId().toString());
         logClusterHealthStateChange(
                 new ClusterStateHealth(clusterState),
@@ -243,7 +276,7 @@ public class AllocationService extends AbstractComponent {
         // the assumption is that commands will move / act on shards (or fail through exceptions)
         // so, there will always be shard "movements", so no need to check on reroute
         reroute(allocation);
-        RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes, explanations);
+        RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), clusterState.routingTable(), routingNodes, explanations);
         logClusterHealthStateChange(
                 new ClusterStateHealth(clusterState),
                 new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()),
@@ -252,6 +285,7 @@ public class AllocationService extends AbstractComponent {
         return result;
     }
 
+
     /**
      * Reroutes the routing table based on the live nodes.
      * <p>
@@ -275,7 +309,7 @@ public class AllocationService extends AbstractComponent {
         if (!reroute(allocation)) {
             return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData());
         }
-        RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes);
+        RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), clusterState.routingTable(), routingNodes);
         logClusterHealthStateChange(
                 new ClusterStateHealth(clusterState),
                 new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()),
@@ -412,8 +446,8 @@ public class AllocationService extends AbstractComponent {
         boolean changed = false;
         for (ShardRouting routing : replicas) {
             changed |= applyFailedShard(allocation, routing, false,
-                new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, "primary failed while replica initializing",
-                    null, allocation.getCurrentNanoTime(), System.currentTimeMillis()));
+                    new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, "primary failed while replica initializing",
+                            null, allocation.getCurrentNanoTime(), System.currentTimeMillis()));
         }
         return changed;
     }

+ 1 - 1
core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java

@@ -44,7 +44,7 @@ import static java.util.Collections.unmodifiableSet;
 public class RoutingAllocation {
 
     /**
-     * this class is used to describe results of a {@link RoutingAllocation}  
+     * this class is used to describe results of a {@link RoutingAllocation}
      */
     public static class Result {
 

+ 2 - 3
core/src/main/java/org/elasticsearch/cluster/service/ClusterService.java

@@ -685,9 +685,8 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
             warnAboutSlowTaskIfNeeded(executionTime, source);
         } catch (Throwable t) {
             TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS)));
-            logger.warn("failed to apply updated cluster state in [{}]:\nversion [{}], uuid [{}], source [{}]\n{}{}{}", t, executionTime,
-                    newClusterState.version(), newClusterState.stateUUID(), source, newClusterState.nodes().prettyPrint(),
-                    newClusterState.routingTable().prettyPrint(), newClusterState.getRoutingNodes().prettyPrint());
+            logger.warn("failed to apply updated cluster state in [{}]:\nversion [{}], uuid [{}], source [{}]\n{}", t, executionTime,
+                    newClusterState.version(), newClusterState.stateUUID(), source, newClusterState.prettyPrint());
             // TODO: do we want to call updateTask.onFailure here?
         }
 

+ 17 - 12
core/src/main/java/org/elasticsearch/index/IndexService.java

@@ -19,18 +19,6 @@
 
 package org.elasticsearch.index;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.nio.file.Path;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Query;
@@ -82,6 +70,18 @@ import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
 import org.elasticsearch.indices.mapper.MapperRegistry;
 import org.elasticsearch.threadpool.ThreadPool;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import static java.util.Collections.emptyMap;
 import static java.util.Collections.unmodifiableMap;
 import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
@@ -621,6 +621,11 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
                 rescheduleFsyncTask(durability);
             }
         }
+
+        // update primary terms
+        for (final IndexShard shard : this.shards.values()) {
+            shard.updatePrimaryTerm(metadata.primaryTerm(shard.shardId().id()));
+        }
     }
 
     private void rescheduleFsyncTask(Translog.Durability durability) {

+ 4 - 4
core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java

@@ -33,12 +33,12 @@ public class IllegalIndexShardStateException extends ElasticsearchException {
 
     private final IndexShardState currentState;
 
-    public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg) {
-        this(shardId, currentState, msg, null);
+    public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg, Object... args) {
+        this(shardId, currentState, msg, null, args);
     }
 
-    public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg, Throwable ex) {
-        super("CurrentState[" + currentState + "] " + msg, ex);
+    public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg, Throwable ex, Object... args) {
+        super("CurrentState[" + currentState + "] " + msg, ex, args);
         setShard(shardId);
         this.currentState = currentState;
     }

+ 84 - 23
core/src/main/java/org/elasticsearch/index/shard/IndexShard.java

@@ -41,6 +41,7 @@ import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.common.logging.LoggerMessageFormat;
 import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.metrics.MeanMetric;
 import org.elasticsearch.common.settings.Settings;
@@ -144,13 +145,16 @@ public class IndexShard extends AbstractIndexShardComponent {
     private final TranslogConfig translogConfig;
     private final IndexEventListener indexEventListener;
 
-    /** How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh.  IndexingMemoryController polls this
-     *  across all shards to decide if throttling is necessary because moving bytes to disk is falling behind vs incoming documents
-     *  being indexed/deleted. */
+    /**
+     * How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh.  IndexingMemoryController polls this
+     * across all shards to decide if throttling is necessary because moving bytes to disk is falling behind vs incoming documents
+     * being indexed/deleted.
+     */
     private final AtomicLong writingBytes = new AtomicLong();
 
     protected volatile ShardRouting shardRouting;
     protected volatile IndexShardState state;
+    protected volatile long primaryTerm;
     protected final AtomicReference<Engine> currentEngineReference = new AtomicReference<>();
     protected final EngineFactory engineFactory;
 
@@ -236,13 +240,16 @@ public class IndexShard extends AbstractIndexShardComponent {
         this.engineConfig = newEngineConfig(translogConfig, cachingPolicy);
         this.suspendableRefContainer = new SuspendableRefContainer();
         this.searcherWrapper = indexSearcherWrapper;
+        this.primaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id());
     }
 
     public Store store() {
         return this.store;
     }
 
-    /** returns true if this shard supports indexing (i.e., write) operations. */
+    /**
+     * returns true if this shard supports indexing (i.e., write) operations.
+     */
     public boolean canIndex() {
         return true;
     }
@@ -279,6 +286,30 @@ public class IndexShard extends AbstractIndexShardComponent {
         return this.shardFieldData;
     }
 
+
+    /**
+     * Returns the primary term the index shard is on. See {@link org.elasticsearch.cluster.metadata.IndexMetaData#primaryTerm(int)}
+     */
+    public long getPrimaryTerm() {
+        return this.primaryTerm;
+    }
+
+    /**
+     * notifies the shard of an increase in the primary term
+     */
+    public void updatePrimaryTerm(final long newTerm) {
+        synchronized (mutex) {
+            if (newTerm != primaryTerm) {
+                assert shardRouting.primary() == false : "a primary shard should never update it's term. shard: " + shardRouting
+                    + " current term [" + primaryTerm + "] new term [" + newTerm + "]";
+                assert newTerm > primaryTerm : "primary terms can only go up. current [" + primaryTerm + "], new [" + newTerm + "]";
+                primaryTerm = newTerm;
+            }
+        }
+
+
+    }
+
     /**
      * Returns the latest cluster routing entry received with this shard. Might be null if the
      * shard was just created.
@@ -297,12 +328,12 @@ public class IndexShard extends AbstractIndexShardComponent {
      * unless explicitly disabled.
      *
      * @throws IndexShardRelocatedException if shard is marked as relocated and relocation aborted
-     * @throws IOException if shard state could not be persisted
+     * @throws IOException                  if shard state could not be persisted
      */
     public void updateRoutingEntry(final ShardRouting newRouting, final boolean persistState) throws IOException {
         final ShardRouting currentRouting = this.shardRouting;
         if (!newRouting.shardId().equals(shardId())) {
-            throw new IllegalArgumentException("Trying to set a routing entry with shardId [" + newRouting.shardId() + "] on a shard with shardId [" + shardId() + "]");
+            throw new IllegalArgumentException("Trying to set a routing entry with shardId " + newRouting.shardId() + " on a shard with shardId " + shardId() + "");
         }
         if ((currentRouting == null || newRouting.isSameAllocation(currentRouting)) == false) {
             throw new IllegalArgumentException("Trying to set a routing entry with a different allocation. Current " + currentRouting + ", new " + newRouting);
@@ -419,9 +450,7 @@ public class IndexShard extends AbstractIndexShardComponent {
 
     public Engine.Index prepareIndexOnPrimary(SourceToParse source, long version, VersionType versionType) {
         try {
-            if (shardRouting.primary() == false) {
-                throw new IllegalIndexShardStateException(shardId, state, "shard is not a primary");
-            }
+            verifyPrimary();
             return prepareIndex(docMapper(source.type()), source, version, versionType, Engine.Operation.Origin.PRIMARY);
         } catch (Throwable t) {
             verifyNotClosed(t);
@@ -431,6 +460,7 @@ public class IndexShard extends AbstractIndexShardComponent {
 
     public Engine.Index prepareIndexOnReplica(SourceToParse source, long version, VersionType versionType) {
         try {
+            verifyReplicationTarget();
             return prepareIndex(docMapper(source.type()), source, version, versionType, Engine.Operation.Origin.REPLICA);
         } catch (Throwable t) {
             verifyNotClosed(t);
@@ -474,9 +504,7 @@ public class IndexShard extends AbstractIndexShardComponent {
     }
 
     public Engine.Delete prepareDeleteOnPrimary(String type, String id, long version, VersionType versionType) {
-        if (shardRouting.primary() == false) {
-            throw new IllegalIndexShardStateException(shardId, state, "shard is not a primary");
-        }
+        verifyPrimary();
         final DocumentMapper documentMapper = docMapper(type).getDocumentMapper();
         return prepareDelete(type, id, documentMapper.uidMapper().term(Uid.createUid(type, id)), version, versionType, Engine.Operation.Origin.PRIMARY);
     }
@@ -515,7 +543,9 @@ public class IndexShard extends AbstractIndexShardComponent {
         return getEngine().get(get, this::acquireSearcher);
     }
 
-    /** Writes all indexing changes to disk and opens a new searcher reflecting all changes.  This can throw {@link EngineClosedException}. */
+    /**
+     * Writes all indexing changes to disk and opens a new searcher reflecting all changes.  This can throw {@link EngineClosedException}.
+     */
     public void refresh(String source) {
         verifyNotClosed();
         if (canIndex()) {
@@ -538,7 +568,9 @@ public class IndexShard extends AbstractIndexShardComponent {
         }
     }
 
-    /** Returns how many bytes we are currently moving from heap to disk */
+    /**
+     * Returns how many bytes we are currently moving from heap to disk
+     */
     public long getWritingBytes() {
         return writingBytes.get();
     }
@@ -940,6 +972,22 @@ public class IndexShard extends AbstractIndexShardComponent {
         }
     }
 
+    private void verifyPrimary() {
+        if (shardRouting.primary() == false) {
+            // must use exception that is not ignored by replication logic. See TransportActions.isShardNotAvailableException
+            throw new IllegalStateException("shard is not a primary " + shardRouting);
+        }
+    }
+
+    private void verifyReplicationTarget() {
+        final IndexShardState state = state();
+        if (shardRouting.primary() && shardRouting.active() && state != IndexShardState.RELOCATED) {
+            // must use exception that is not ignored by replication logic. See TransportActions.isShardNotAvailableException
+            throw new IllegalStateException("active primary shard cannot be a replication target before " +
+                " relocation hand off " + shardRouting + ", state is [" + state + "]");
+        }
+    }
+
     protected final void verifyStartedOrRecovering() throws IllegalIndexShardStateException {
         IndexShardState state = this.state; // one time volatile read
         if (state != IndexShardState.STARTED && state != IndexShardState.RECOVERING && state != IndexShardState.POST_RECOVERY) {
@@ -969,7 +1017,9 @@ public class IndexShard extends AbstractIndexShardComponent {
         }
     }
 
-    /** Returns number of heap bytes used by the indexing buffer for this shard, or 0 if the shard is closed */
+    /**
+     * Returns number of heap bytes used by the indexing buffer for this shard, or 0 if the shard is closed
+     */
     public long getIndexBufferRAMBytesUsed() {
         Engine engine = getEngineOrNull();
         if (engine == null) {
@@ -986,8 +1036,10 @@ public class IndexShard extends AbstractIndexShardComponent {
         this.shardEventListener.delegates.add(onShardFailure);
     }
 
-    /** Called by {@link IndexingMemoryController} to check whether more than {@code inactiveTimeNS} has passed since the last
-     *  indexing operation, and notify listeners that we are now inactive so e.g. sync'd flush can happen. */
+    /**
+     * Called by {@link IndexingMemoryController} to check whether more than {@code inactiveTimeNS} has passed since the last
+     * indexing operation, and notify listeners that we are now inactive so e.g. sync'd flush can happen.
+     */
     public void checkIdle(long inactiveTimeNS) {
         Engine engineOrNull = getEngineOrNull();
         if (engineOrNull != null && System.nanoTime() - engineOrNull.getLastWriteNanos() >= inactiveTimeNS) {
@@ -1132,11 +1184,12 @@ public class IndexShard extends AbstractIndexShardComponent {
             }
         } catch (Exception e) {
             handleRefreshException(e);
-        };
+        }
     }
 
     /**
      * Should be called for each no-op update operation to increment relevant statistics.
+     *
      * @param type the doc type of the update
      */
     public void noopUpdate(String type) {
@@ -1336,14 +1389,22 @@ public class IndexShard extends AbstractIndexShardComponent {
 
     public Releasable acquirePrimaryOperationLock() {
         verifyNotClosed();
-        if (shardRouting.primary() == false) {
-            throw new IllegalIndexShardStateException(shardId, state, "shard is not a primary");
-        }
+        verifyPrimary();
         return suspendableRefContainer.acquireUninterruptibly();
     }
 
-    public Releasable acquireReplicaOperationLock() {
+    /**
+     * acquires operation log. If the given primary term is lower then the one in {@link #shardRouting}
+     * an {@link IllegalArgumentException} is thrown.
+     */
+    public Releasable acquireReplicaOperationLock(long opPrimaryTerm) {
         verifyNotClosed();
+        verifyReplicationTarget();
+        if (primaryTerm > opPrimaryTerm) {
+            // must use exception that is not ignored by replication logic. See TransportActions.isShardNotAvailableException
+            throw new IllegalArgumentException(LoggerMessageFormat.format("{} operation term [{}] is too old (current [{}])",
+                shardId, opPrimaryTerm, primaryTerm));
+        }
         return suspendableRefContainer.acquireUninterruptibly();
     }
 
@@ -1447,7 +1508,7 @@ public class IndexShard extends AbstractIndexShardComponent {
      * Returns <code>true</code> iff one or more changes to the engine are not visible to via the current searcher.
      * Otherwise <code>false</code>.
      *
-     * @throws EngineClosedException if the engine is already closed
+     * @throws EngineClosedException  if the engine is already closed
      * @throws AlreadyClosedException if the internal indexwriter in the engine is already closed
      */
     public boolean isRefreshNeeded() {

+ 125 - 120
core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java

@@ -19,7 +19,6 @@
 
 package org.elasticsearch.indices.cluster;
 
-import com.carrotsearch.hppc.IntHashSet;
 import com.carrotsearch.hppc.cursors.ObjectCursor;
 import org.elasticsearch.cluster.ClusterChangedEvent;
 import org.elasticsearch.cluster.ClusterState;
@@ -71,9 +70,11 @@ import org.elasticsearch.snapshots.RestoreService;
 import org.elasticsearch.threadpool.ThreadPool;
 
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 
 /**
@@ -90,7 +91,8 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
     private final NodeMappingRefreshAction nodeMappingRefreshAction;
     private final NodeServicesProvider nodeServicesProvider;
 
-    private static final ShardStateAction.Listener SHARD_STATE_ACTION_LISTENER = new ShardStateAction.Listener() {};
+    private static final ShardStateAction.Listener SHARD_STATE_ACTION_LISTENER = new ShardStateAction.Listener() {
+    };
 
     // a list of shards that failed during recovery
     // we keep track of these shards in order to prevent repeated recovery of these shards on each cluster state update
@@ -174,41 +176,44 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
 
             cleanFailedShards(event);
 
+            // cleaning up indices that are completely deleted so we won't need to worry about them
+            // when checking for shards
             applyDeletedIndices(event);
+            applyDeletedShards(event);
+            // call after deleted shards so indices with no shards will be cleaned
+            applyCleanedIndices(event);
+            // make sure that newly created shards use the latest meta data
+            applyIndexMetaData(event);
             applyNewIndices(event);
+            // apply mappings also updates new indices. TODO: make new indices good to begin with
             applyMappings(event);
             applyNewOrUpdatedShards(event);
-            applyDeletedShards(event);
-            applyCleanedIndices(event);
-            applySettings(event);
         }
     }
 
-    private void applyCleanedIndices(final ClusterChangedEvent event) {
-        // handle closed indices, since they are not allocated on a node once they are closed
-        // so applyDeletedIndices might not take them into account
-        for (IndexService indexService : indicesService) {
-            Index index = indexService.index();
-            IndexMetaData indexMetaData = event.state().metaData().index(index);
-            if (indexMetaData != null && indexMetaData.getState() == IndexMetaData.State.CLOSE) {
-                for (Integer shardId : indexService.shardIds()) {
-                    logger.debug("{}[{}] removing shard (index is closed)", index, shardId);
-                    try {
-                        indexService.removeShard(shardId, "removing shard (index is closed)");
-                    } catch (Throwable e) {
-                        logger.warn("{} failed to remove shard (index is closed)", e, index);
-                    }
-                }
-            }
+    private void cleanFailedShards(final ClusterChangedEvent event) {
+        RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
+        if (routingNode == null) {
+            failedShards.clear();
+            return;
         }
-        for (IndexService indexService : indicesService) {
-            Index index = indexService.index();
-            if (indexService.shardIds().isEmpty()) {
-                if (logger.isDebugEnabled()) {
-                    logger.debug("{} cleaning index (no shards allocated)", index);
-                }
-                // clean the index
-                removeIndex(index, "removing index (no shards allocated)");
+        RoutingTable routingTable = event.state().routingTable();
+        for (Iterator<Map.Entry<ShardId, ShardRouting>> iterator = failedShards.entrySet().iterator(); iterator.hasNext(); ) {
+            Map.Entry<ShardId, ShardRouting> entry = iterator.next();
+            ShardId failedShardId = entry.getKey();
+            ShardRouting failedShardRouting = entry.getValue();
+            IndexRoutingTable indexRoutingTable = routingTable.index(failedShardId.getIndex());
+            if (indexRoutingTable == null) {
+                iterator.remove();
+                continue;
+            }
+            IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(failedShardId.id());
+            if (shardRoutingTable == null) {
+                iterator.remove();
+                continue;
+            }
+            if (shardRoutingTable.assignedShards().stream().noneMatch(shr -> shr.isSameAllocation(failedShardRouting))) {
+                iterator.remove();
             }
         }
     }
@@ -218,16 +223,6 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
         final String localNodeId = event.state().nodes().localNodeId();
         assert localNodeId != null;
 
-        for (IndexService indexService : indicesService) {
-            IndexMetaData indexMetaData = event.state().metaData().index(indexService.index().getName());
-            if (indexMetaData != null) {
-                if (!indexMetaData.isSameUUID(indexService.indexUUID())) {
-                    logger.debug("[{}] mismatch on index UUIDs between cluster state and local state, cleaning the index so it will be recreated", indexMetaData.getIndex());
-                    deleteIndex(indexMetaData.getIndex(), "mismatch on index UUIDs between cluster state and local state, cleaning the index so it will be recreated");
-                }
-            }
-        }
-
         for (Index index : event.indicesDeleted()) {
             if (logger.isDebugEnabled()) {
                 logger.debug("[{}] cleaning index, no longer part of the metadata", index);
@@ -249,7 +244,17 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
             }
         }
 
-
+        for (IndexService indexService : indicesService) {
+            IndexMetaData indexMetaData = event.state().metaData().index(indexService.index());
+            if (indexMetaData == null) {
+                assert false : "index" + indexService.index() + " exists locally, doesn't have a metadata but is not part "
+                    + " of the delete index list. \nprevious state: " + event.previousState().prettyPrint()
+                    + "\n current state:\n" + event.state().prettyPrint();
+                logger.warn("[{}] isn't part of metadata but is part of in memory structures. removing",
+                    indexService.index());
+                deleteIndex(indexService.index(), "isn't part of metadata (explicit check)");
+            }
+        }
     }
 
     private void applyDeletedShards(final ClusterChangedEvent event) {
@@ -257,62 +262,77 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
         if (routingNode == null) {
             return;
         }
-        IntHashSet newShardIds = new IntHashSet();
+        Set<String> newShardAllocationIds = new HashSet<>();
         for (IndexService indexService : indicesService) {
             Index index = indexService.index();
-            IndexMetaData indexMetaData = event.state().metaData().getIndexSafe(index);
-            if (indexMetaData == null) {
-                continue;
-            }
+            IndexMetaData indexMetaData = event.state().metaData().index(index);
+            assert indexMetaData != null : "local index doesn't have metadata, should have been cleaned up by applyDeletedIndices: " + index;
             // now, go over and delete shards that needs to get deleted
-            newShardIds.clear();
+            newShardAllocationIds.clear();
             for (ShardRouting shard : routingNode) {
                 if (shard.index().equals(index)) {
-                    newShardIds.add(shard.id());
+                    // use the allocation id and not object so we won't be influence by relocation targets
+                    newShardAllocationIds.add(shard.allocationId().getId());
                 }
             }
-            for (Integer existingShardId : indexService.shardIds()) {
-                if (!newShardIds.contains(existingShardId)) {
+            for (IndexShard existingShard : indexService) {
+                if (newShardAllocationIds.contains(existingShard.routingEntry().allocationId().getId()) == false) {
                     if (indexMetaData.getState() == IndexMetaData.State.CLOSE) {
                         if (logger.isDebugEnabled()) {
-                            logger.debug("{}[{}] removing shard (index is closed)", index, existingShardId);
+                            logger.debug("{} removing shard (index is closed)", existingShard.shardId());
                         }
-                        indexService.removeShard(existingShardId, "removing shard (index is closed)");
+                        indexService.removeShard(existingShard.shardId().id(), "removing shard (index is closed)");
                     } else {
                         // we can just remove the shard, without cleaning it locally, since we will clean it
                         // when all shards are allocated in the IndicesStore
                         if (logger.isDebugEnabled()) {
-                            logger.debug("{}[{}] removing shard (not allocated)", index, existingShardId);
+                            logger.debug("{} removing shard (not allocated)", existingShard.shardId());
                         }
-                        indexService.removeShard(existingShardId, "removing shard (not allocated)");
+                        indexService.removeShard(existingShard.shardId().id(), "removing shard (not allocated)");
                     }
                 }
             }
         }
     }
 
-    private void applyNewIndices(final ClusterChangedEvent event) {
-        // we only create indices for shards that are allocated
-        RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
-        if (routingNode == null) {
-            return;
+    private void applyCleanedIndices(final ClusterChangedEvent event) {
+        // handle closed indices, since they are not allocated on a node once they are closed
+        // so applyDeletedIndices might not take them into account
+        for (IndexService indexService : indicesService) {
+            Index index = indexService.index();
+            IndexMetaData indexMetaData = event.state().metaData().index(index);
+            if (indexMetaData != null && indexMetaData.getState() == IndexMetaData.State.CLOSE) {
+                for (Integer shardId : indexService.shardIds()) {
+                    logger.debug("{}[{}] removing shard (index is closed)", index, shardId);
+                    try {
+                        indexService.removeShard(shardId, "removing shard (index is closed)");
+                    } catch (Throwable e) {
+                        logger.warn("{} failed to remove shard (index is closed)", e, index);
+                    }
+                }
+            }
         }
-        for (ShardRouting shard : routingNode) {
-            if (!indicesService.hasIndex(shard.index())) {
-                final IndexMetaData indexMetaData = event.state().metaData().getIndexSafe(shard.index());
+
+        Set<Index> hasAllocations = new HashSet<>();
+        for (ShardRouting routing : event.state().getRoutingNodes().node(event.state().nodes().localNodeId())) {
+            hasAllocations.add(routing.index());
+        }
+        for (IndexService indexService : indicesService) {
+            Index index = indexService.index();
+            if (hasAllocations.contains(index) == false) {
+                assert indexService.shardIds().isEmpty() :
+                    "no locally assigned shards, but index wasn't emptied by applyDeletedShards."
+                        + " index " + index + ", shards: " + indexService.shardIds();
                 if (logger.isDebugEnabled()) {
-                    logger.debug("[{}] creating index", indexMetaData.getIndex());
-                }
-                try {
-                    indicesService.createIndex(nodeServicesProvider, indexMetaData, buildInIndexListener);
-                } catch (Throwable e) {
-                    sendFailShard(shard, "failed to create index", e);
+                    logger.debug("{} cleaning index (no shards allocated)", index);
                 }
+                // clean the index
+                removeIndex(index, "removing index (no shards allocated)");
             }
         }
     }
 
-    private void applySettings(ClusterChangedEvent event) {
+    private void applyIndexMetaData(ClusterChangedEvent event) {
         if (!event.metaDataChanged()) {
             return;
         }
@@ -335,6 +355,26 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
         }
     }
 
+    private void applyNewIndices(final ClusterChangedEvent event) {
+        // we only create indices for shards that are allocated
+        RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
+        if (routingNode == null) {
+            return;
+        }
+        for (ShardRouting shard : routingNode) {
+            if (!indicesService.hasIndex(shard.index())) {
+                final IndexMetaData indexMetaData = event.state().metaData().getIndexSafe(shard.index());
+                if (logger.isDebugEnabled()) {
+                    logger.debug("[{}] creating index", indexMetaData.getIndex());
+                }
+                try {
+                    indicesService.createIndex(nodeServicesProvider, indexMetaData, buildInIndexListener);
+                } catch (Throwable e) {
+                    sendFailShard(shard, "failed to create index", e);
+                }
+            }
+        }
+    }
 
     private void applyMappings(ClusterChangedEvent event) {
         // go over and update mappings
@@ -361,8 +401,8 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
                 }
                 if (requireRefresh && sendRefreshMapping) {
                     nodeMappingRefreshAction.nodeMappingRefresh(event.state(),
-                            new NodeMappingRefreshAction.NodeMappingRefreshRequest(index.getName(), indexMetaData.getIndexUUID(),
-                                    event.state().nodes().localNodeId())
+                        new NodeMappingRefreshAction.NodeMappingRefreshRequest(index.getName(), indexMetaData.getIndexUUID(),
+                            event.state().nodes().localNodeId())
                     );
                 }
             } catch (Throwable t) {
@@ -426,14 +466,13 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
         for (final ShardRouting shardRouting : routingNode) {
             final IndexService indexService = indicesService.indexService(shardRouting.index());
             if (indexService == null) {
-                // got deleted on us, ignore
+                // creation failed for some reasons
+                assert failedShards.containsKey(shardRouting.shardId()) :
+                    "index has local allocation but is not created by applyNewIndices and is not failed " + shardRouting;
                 continue;
             }
             final IndexMetaData indexMetaData = event.state().metaData().index(shardRouting.index());
-            if (indexMetaData == null) {
-                // the index got deleted on the metadata, we will clean it later in the apply deleted method call
-                continue;
-            }
+            assert indexMetaData != null : "index has local allocation but no meta data. " + shardRouting.index();
 
             final int shardId = shardRouting.id();
 
@@ -458,12 +497,10 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
                 // for example: a shard that recovers from one node and now needs to recover to another node,
                 //              or a replica allocated and then allocating a primary because the primary failed on another node
                 boolean shardHasBeenRemoved = false;
-                if (currentRoutingEntry.isSameAllocation(shardRouting) == false) {
-                    logger.debug("[{}][{}] removing shard (different instance of it allocated on this node, current [{}], global [{}])", shardRouting.index(), shardRouting.id(), currentRoutingEntry, shardRouting);
-                    // closing the shard will also cancel any ongoing recovery.
-                    indexService.removeShard(shardRouting.id(), "removing shard (different instance of it allocated on this node)");
-                    shardHasBeenRemoved = true;
-                } else if (isPeerRecovery(shardRouting)) {
+                assert currentRoutingEntry.isSameAllocation(shardRouting) :
+                    "local shard has a different allocation id but wasn't cleaning by applyDeletedShards. "
+                        + "cluster state: " + shardRouting + " local: " + currentRoutingEntry;
+                if (isPeerRecovery(shardRouting)) {
                     final DiscoveryNode sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting);
                     // check if there is an existing recovery going, and if so, and the source node is not the same, cancel the recovery to restart it
                     if (recoveryTargetService.cancelRecoveriesForShard(indexShard.shardId(), "recovery source node changed", status -> !status.sourceNode().equals(sourceNode))) {
@@ -477,7 +514,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
                 if (shardHasBeenRemoved == false) {
                     // shadow replicas do not support primary promotion. The master would reinitialize the shard, giving it a new allocation, meaning we should be there.
                     assert (shardRouting.primary() && currentRoutingEntry.primary() == false) == false || indexShard.allowsPrimaryPromotion() :
-                            "shard for doesn't support primary promotion but master promoted it with changing allocation. New routing " + shardRouting + ", current routing " + currentRoutingEntry;
+                        "shard for doesn't support primary promotion but master promoted it with changing allocation. New routing " + shardRouting + ", current routing " + currentRoutingEntry;
                     try {
                         indexShard.updateRoutingEntry(shardRouting, event.state().blocks().disableStatePersistence() == false);
                     } catch (Throwable e) {
@@ -487,44 +524,12 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
             }
 
             if (shardRouting.initializing()) {
-                applyInitializingShard(event.state(), indexMetaData, shardRouting);
-            }
-        }
-    }
-
-    private void cleanFailedShards(final ClusterChangedEvent event) {
-        RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
-        if (routingNode == null) {
-            failedShards.clear();
-            return;
-        }
-        RoutingTable routingTable = event.state().routingTable();
-        for (Iterator<Map.Entry<ShardId, ShardRouting>> iterator = failedShards.entrySet().iterator(); iterator.hasNext(); ) {
-            Map.Entry<ShardId, ShardRouting> entry = iterator.next();
-            ShardId failedShardId = entry.getKey();
-            ShardRouting failedShardRouting = entry.getValue();
-            IndexRoutingTable indexRoutingTable = routingTable.index(failedShardId.getIndex());
-            if (indexRoutingTable == null) {
-                iterator.remove();
-                continue;
-            }
-            IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(failedShardId.id());
-            if (shardRoutingTable == null) {
-                iterator.remove();
-                continue;
-            }
-            if (shardRoutingTable.assignedShards().stream().noneMatch(shr -> shr.isSameAllocation(failedShardRouting))) {
-                iterator.remove();
+                applyInitializingShard(event.state(), indexMetaData, indexService, shardRouting);
             }
         }
     }
 
-    private void applyInitializingShard(final ClusterState state, final IndexMetaData indexMetaData, final ShardRouting shardRouting) {
-        final IndexService indexService = indicesService.indexService(shardRouting.index());
-        if (indexService == null) {
-            // got deleted on us, ignore
-            return;
-        }
+    private void applyInitializingShard(final ClusterState state, final IndexMetaData indexMetaData, IndexService indexService, final ShardRouting shardRouting) {
         final RoutingTable routingTable = state.routingTable();
         final DiscoveryNodes nodes = state.getNodes();
         final int shardId = shardRouting.id();
@@ -537,7 +542,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
                 // we managed to tell the master we started), mark us as started
                 if (logger.isTraceEnabled()) {
                     logger.trace("{} master marked shard as initializing, but shard has state [{}], resending shard started to {}",
-                            indexShard.shardId(), indexShard.state(), nodes.masterNode());
+                        indexShard.shardId(), indexShard.state(), nodes.masterNode());
                 }
                 if (nodes.masterNode() != null) {
                     shardStateAction.shardStarted(shardRouting,
@@ -618,8 +623,8 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
             assert indexShard.routingEntry().equals(shardRouting); // should have already be done before
             // recover from filesystem store
             final RecoveryState recoveryState = new RecoveryState(indexShard.shardId(), shardRouting.primary(),
-                    RecoveryState.Type.STORE,
-                    nodes.localNode(), nodes.localNode());
+                RecoveryState.Type.STORE,
+                nodes.localNode(), nodes.localNode());
             indexShard.markAsRecovering("from store", recoveryState); // mark the shard as recovering on the cluster state thread
             threadPool.generic().execute(() -> {
                 try {
@@ -634,7 +639,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
         } else {
             // recover from a restore
             final RecoveryState recoveryState = new RecoveryState(indexShard.shardId(), shardRouting.primary(),
-                    RecoveryState.Type.SNAPSHOT, shardRouting.restoreSource(), nodes.localNode());
+                RecoveryState.Type.SNAPSHOT, shardRouting.restoreSource(), nodes.localNode());
             indexShard.markAsRecovering("from snapshot", recoveryState); // mark the shard as recovering on the cluster state thread
             threadPool.generic().execute(() -> {
                 final ShardId sId = indexShard.shardId();

+ 21 - 11
core/src/main/java/org/elasticsearch/ingest/core/CompoundProcessor.java

@@ -28,15 +28,16 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.stream.Collectors;
 
 /**
  * A Processor that executes a list of other "processors". It executes a separate list of
  * "onFailureProcessors" when any of the processors throw an {@link Exception}.
  */
 public class CompoundProcessor implements Processor {
-    static final String ON_FAILURE_MESSAGE_FIELD = "on_failure_message";
-    static final String ON_FAILURE_PROCESSOR_TYPE_FIELD = "on_failure_processor_type";
-    static final String ON_FAILURE_PROCESSOR_TAG_FIELD = "on_failure_processor_tag";
+    public static final String ON_FAILURE_MESSAGE_FIELD = "on_failure_message";
+    public static final String ON_FAILURE_PROCESSOR_TYPE_FIELD = "on_failure_processor_type";
+    public static final String ON_FAILURE_PROCESSOR_TAG_FIELD = "on_failure_processor_tag";
 
     private final List<Processor> processors;
     private final List<Processor> onFailureProcessors;
@@ -84,7 +85,7 @@ public class CompoundProcessor implements Processor {
 
     @Override
     public String getTag() {
-        return "compound-processor-" + Objects.hash(processors, onFailureProcessors);
+        return "CompoundProcessor-" + flattenProcessors().stream().map(Processor::getTag).collect(Collectors.joining("-"));
     }
 
     @Override
@@ -104,18 +105,27 @@ public class CompoundProcessor implements Processor {
     }
 
     void executeOnFailure(IngestDocument ingestDocument, Exception cause, String failedProcessorType, String failedProcessorTag) throws Exception {
-        Map<String, String> ingestMetadata = ingestDocument.getIngestMetadata();
         try {
-            ingestMetadata.put(ON_FAILURE_MESSAGE_FIELD, cause.getMessage());
-            ingestMetadata.put(ON_FAILURE_PROCESSOR_TYPE_FIELD, failedProcessorType);
-            ingestMetadata.put(ON_FAILURE_PROCESSOR_TAG_FIELD, failedProcessorTag);
+            putFailureMetadata(ingestDocument, cause, failedProcessorType, failedProcessorTag);
             for (Processor processor : onFailureProcessors) {
                 processor.execute(ingestDocument);
             }
         } finally {
-            ingestMetadata.remove(ON_FAILURE_MESSAGE_FIELD);
-            ingestMetadata.remove(ON_FAILURE_PROCESSOR_TYPE_FIELD);
-            ingestMetadata.remove(ON_FAILURE_PROCESSOR_TAG_FIELD);
+            removeFailureMetadata(ingestDocument);
         }
     }
+
+    private void putFailureMetadata(IngestDocument ingestDocument, Exception cause, String failedProcessorType, String failedProcessorTag) {
+        Map<String, String> ingestMetadata = ingestDocument.getIngestMetadata();
+        ingestMetadata.put(ON_FAILURE_MESSAGE_FIELD, cause.getMessage());
+        ingestMetadata.put(ON_FAILURE_PROCESSOR_TYPE_FIELD, failedProcessorType);
+        ingestMetadata.put(ON_FAILURE_PROCESSOR_TAG_FIELD, failedProcessorTag);
+    }
+
+    private void removeFailureMetadata(IngestDocument ingestDocument) {
+        Map<String, String> ingestMetadata = ingestDocument.getIngestMetadata();
+        ingestMetadata.remove(ON_FAILURE_MESSAGE_FIELD);
+        ingestMetadata.remove(ON_FAILURE_PROCESSOR_TYPE_FIELD);
+        ingestMetadata.remove(ON_FAILURE_PROCESSOR_TAG_FIELD);
+    }
 }

+ 7 - 0
core/src/main/java/org/elasticsearch/ingest/core/Pipeline.java

@@ -68,6 +68,13 @@ public final class Pipeline {
         return description;
     }
 
+    /**
+     * Get the underlying {@link CompoundProcessor} containing the Pipeline's processors
+     */
+    public CompoundProcessor getCompoundProcessor() {
+        return compoundProcessor;
+    }
+
     /**
      * Unmodifiable list containing each processor that operates on the data.
      */

+ 89 - 0
core/src/main/java/org/elasticsearch/ingest/processor/TrackingResultProcessor.java

@@ -0,0 +1,89 @@
+/*
+ * 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.ingest.processor;
+
+import org.elasticsearch.action.ingest.SimulateProcessorResult;
+import org.elasticsearch.ingest.core.CompoundProcessor;
+import org.elasticsearch.ingest.core.IngestDocument;
+import org.elasticsearch.ingest.core.Processor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Processor to be used within Simulate API to keep track of processors executed in pipeline.
+ */
+public final class TrackingResultProcessor implements Processor {
+
+    private final Processor actualProcessor;
+    private final List<SimulateProcessorResult> processorResultList;
+
+    public TrackingResultProcessor(Processor actualProcessor, List<SimulateProcessorResult> processorResultList) {
+        this.processorResultList = processorResultList;
+        if (actualProcessor instanceof CompoundProcessor) {
+            CompoundProcessor trackedCompoundProcessor = decorate((CompoundProcessor) actualProcessor, processorResultList);
+            this.actualProcessor = trackedCompoundProcessor;
+        } else {
+            this.actualProcessor = actualProcessor;
+        }
+    }
+
+    @Override
+    public void execute(IngestDocument ingestDocument) throws Exception {
+        try {
+            actualProcessor.execute(ingestDocument);
+            processorResultList.add(new SimulateProcessorResult(actualProcessor.getTag(), new IngestDocument(ingestDocument)));
+        } catch (Exception e) {
+            processorResultList.add(new SimulateProcessorResult(actualProcessor.getTag(), e));
+            throw e;
+        }
+    }
+
+    @Override
+    public String getType() {
+        return actualProcessor.getType();
+    }
+
+    @Override
+    public String getTag() {
+        return actualProcessor.getTag();
+    }
+
+    public static CompoundProcessor decorate(CompoundProcessor compoundProcessor, List<SimulateProcessorResult> processorResultList) {
+        List<Processor> processors = new ArrayList<>(compoundProcessor.getProcessors().size());
+        for (Processor processor : compoundProcessor.getProcessors()) {
+            if (processor instanceof CompoundProcessor) {
+                processors.add(decorate((CompoundProcessor) processor, processorResultList));
+            } else {
+                processors.add(new TrackingResultProcessor(processor, processorResultList));
+            }
+        }
+        List<Processor> onFailureProcessors = new ArrayList<>(compoundProcessor.getProcessors().size());
+        for (Processor processor : compoundProcessor.getOnFailureProcessors()) {
+            if (processor instanceof CompoundProcessor) {
+                onFailureProcessors.add(decorate((CompoundProcessor) processor, processorResultList));
+            } else {
+                onFailureProcessors.add(new TrackingResultProcessor(processor, processorResultList));
+            }
+        }
+        return new CompoundProcessor(processors, onFailureProcessors);
+    }
+}
+

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

@@ -286,6 +286,7 @@ public class SearchModule extends AbstractModule {
 
         registerBuiltinFunctionScoreParsers();
         registerBuiltinQueryParsers();
+        registerBuiltinRescorers();
     }
 
     public void registerHighlighter(String key, Class<? extends Highlighter> clazz) {
@@ -350,7 +351,6 @@ public class SearchModule extends AbstractModule {
         configureSuggesters();
         configureFetchSubPhase();
         configureShapes();
-        configureRescorers();
         configureSorts();
     }
 
@@ -491,8 +491,8 @@ public class SearchModule extends AbstractModule {
         }
     }
 
-    private void configureRescorers() {
-        namedWriteableRegistry.registerPrototype(RescoreBuilder.class, QueryRescorerBuilder.PROTOTYPE);
+    private void registerBuiltinRescorers() {
+        namedWriteableRegistry.register(RescoreBuilder.class, QueryRescorerBuilder.NAME, QueryRescorerBuilder::new);
     }
 
     private void configureSorts() {

+ 1 - 4
core/src/main/java/org/elasticsearch/search/rescore/QueryRescoreMode.java

@@ -85,10 +85,7 @@ public enum QueryRescoreMode implements Writeable<QueryRescoreMode> {
 
     public abstract float combine(float primary, float secondary);
 
-    static QueryRescoreMode PROTOTYPE = Total;
-
-    @Override
-    public QueryRescoreMode readFrom(StreamInput in) throws IOException {
+    public static QueryRescoreMode readFromStream(StreamInput in) throws IOException {
         int ordinal = in.readVInt();
         if (ordinal < 0 || ordinal >= values().length) {
             throw new IOException("Unknown ScoreMode ordinal [" + ordinal + "]");

+ 23 - 24
core/src/main/java/org/elasticsearch/search/rescore/QueryRescorerBuilder.java

@@ -25,7 +25,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.xcontent.ObjectParser;
 import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.MatchAllQueryBuilder;
 import org.elasticsearch.index.query.QueryBuilder;
 import org.elasticsearch.index.query.QueryParseContext;
 import org.elasticsearch.index.query.QueryShardContext;
@@ -39,8 +38,6 @@ public class QueryRescorerBuilder extends RescoreBuilder<QueryRescorerBuilder> {
 
     public static final String NAME = "query";
 
-    public static final QueryRescorerBuilder PROTOTYPE = new QueryRescorerBuilder(new MatchAllQueryBuilder());
-
     public static final float DEFAULT_RESCORE_QUERYWEIGHT = 1.0f;
     public static final float DEFAULT_QUERYWEIGHT = 1.0f;
     public static final QueryRescoreMode DEFAULT_SCORE_MODE = QueryRescoreMode.Total;
@@ -77,6 +74,25 @@ public class QueryRescorerBuilder extends RescoreBuilder<QueryRescorerBuilder> {
         this.queryBuilder = builder;
     }
 
+    /**
+     * Read from a stream.
+     */
+    public QueryRescorerBuilder(StreamInput in) throws IOException {
+        super(in);
+        queryBuilder = in.readQuery();
+        scoreMode = QueryRescoreMode.readFromStream(in);
+        rescoreQueryWeight = in.readFloat();
+        queryWeight = in.readFloat();
+    }
+
+    @Override
+    public void doWriteTo(StreamOutput out) throws IOException {
+        out.writeQuery(queryBuilder);
+        scoreMode.writeTo(out);
+        out.writeFloat(rescoreQueryWeight);
+        out.writeFloat(queryWeight);
+    }
+
     /**
      * @return the query used for this rescore query
      */
@@ -140,9 +156,9 @@ public class QueryRescorerBuilder extends RescoreBuilder<QueryRescorerBuilder> {
         builder.endObject();
     }
 
-    public QueryRescorerBuilder fromXContent(QueryParseContext parseContext) throws IOException {
-            InnerBuilder innerBuilder = QUERY_RESCORE_PARSER.parse(parseContext.parser(), new InnerBuilder(), parseContext);
-            return innerBuilder.build();
+    public static QueryRescorerBuilder fromXContent(QueryParseContext parseContext) throws IOException {
+        InnerBuilder innerBuilder = QUERY_RESCORE_PARSER.parse(parseContext.parser(), new InnerBuilder(), parseContext);
+        return innerBuilder.build();
     }
 
     @Override
@@ -181,23 +197,6 @@ public class QueryRescorerBuilder extends RescoreBuilder<QueryRescorerBuilder> {
                Objects.equals(queryBuilder, other.queryBuilder);
     }
 
-    @Override
-    public QueryRescorerBuilder doReadFrom(StreamInput in) throws IOException {
-        QueryRescorerBuilder rescorer = new QueryRescorerBuilder(in.readQuery());
-        rescorer.setScoreMode(QueryRescoreMode.PROTOTYPE.readFrom(in));
-        rescorer.setRescoreQueryWeight(in.readFloat());
-        rescorer.setQueryWeight(in.readFloat());
-        return rescorer;
-    }
-
-    @Override
-    public void doWriteTo(StreamOutput out) throws IOException {
-        out.writeQuery(queryBuilder);
-        scoreMode.writeTo(out);
-        out.writeFloat(rescoreQueryWeight);
-        out.writeFloat(queryWeight);
-    }
-
     @Override
     public String getWriteableName() {
         return NAME;
@@ -208,7 +207,7 @@ public class QueryRescorerBuilder extends RescoreBuilder<QueryRescorerBuilder> {
      * for the constructor of {@link QueryRescorerBuilder}, but {@link ObjectParser} only
      * allows filling properties of an already constructed value.
      */
-    private class InnerBuilder {
+    private static class InnerBuilder {
 
         private QueryBuilder<?> queryBuilder;
         private float rescoreQueryWeight = DEFAULT_RESCORE_QUERYWEIGHT;

+ 22 - 18
core/src/main/java/org/elasticsearch/search/rescore/RescoreBuilder.java

@@ -46,6 +46,27 @@ public abstract class RescoreBuilder<RB extends RescoreBuilder<RB>> implements T
 
     private static ParseField WINDOW_SIZE_FIELD = new ParseField("window_size");
 
+    /**
+     * Construct an empty RescoreBuilder.
+     */
+    public RescoreBuilder() {
+    }
+
+    /**
+     * Read from a stream.
+     */
+    protected RescoreBuilder(StreamInput in) throws IOException {
+        windowSize = in.readOptionalVInt();
+    }
+
+    @Override
+    public final void writeTo(StreamOutput out) throws IOException {
+        out.writeOptionalVInt(this.windowSize);
+        doWriteTo(out);
+    }
+
+    protected abstract void doWriteTo(StreamOutput out) throws IOException;
+
     @SuppressWarnings("unchecked")
     public RB windowSize(int windowSize) {
         this.windowSize = windowSize;
@@ -74,7 +95,7 @@ public abstract class RescoreBuilder<RB extends RescoreBuilder<RB>> implements T
             } else if (token == XContentParser.Token.START_OBJECT) {
                 // we only have QueryRescorer at this point
                 if (QueryRescorerBuilder.NAME.equals(fieldName)) {
-                    rescorer = QueryRescorerBuilder.PROTOTYPE.fromXContent(parseContext);
+                    rescorer = QueryRescorerBuilder.fromXContent(parseContext);
                 } else {
                     throw new ParsingException(parser.getTokenLocation(), "rescore doesn't support rescorer with name [" + fieldName + "]");
                 }
@@ -128,23 +149,6 @@ public abstract class RescoreBuilder<RB extends RescoreBuilder<RB>> implements T
         return Objects.equals(windowSize, other.windowSize);
     }
 
-    @Override
-    public RB readFrom(StreamInput in) throws IOException {
-        RB builder = doReadFrom(in);
-        builder.windowSize = in.readOptionalVInt();
-        return builder;
-    }
-
-    protected abstract RB doReadFrom(StreamInput in) throws IOException;
-
-    @Override
-    public void writeTo(StreamOutput out) throws IOException {
-        doWriteTo(out);
-        out.writeOptionalVInt(this.windowSize);
-    }
-
-    protected abstract void doWriteTo(StreamOutput out) throws IOException;
-
     @Override
     public final String toString() {
         try {

+ 44 - 77
core/src/test/java/org/elasticsearch/action/ingest/SimulateExecutionServiceTests.java

@@ -31,10 +31,8 @@ import org.elasticsearch.threadpool.ThreadPool;
 import org.junit.After;
 import org.junit.Before;
 
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
-import java.util.List;
+import java.util.Map;
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.instanceOf;
@@ -46,7 +44,6 @@ public class SimulateExecutionServiceTests extends ESTestCase {
 
     private ThreadPool threadPool;
     private SimulateExecutionService executionService;
-    private Pipeline pipeline;
     private Processor processor;
     private IngestDocument ingestDocument;
 
@@ -59,7 +56,6 @@ public class SimulateExecutionServiceTests extends ESTestCase {
         );
         executionService = new SimulateExecutionService(threadPool);
         processor = new TestProcessor("id", "mock", ingestDocument -> {});
-        pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor));
         ingestDocument = RandomDocumentPicks.randomIngestDocument(random());
     }
 
@@ -68,74 +64,6 @@ public class SimulateExecutionServiceTests extends ESTestCase {
         threadPool.shutdown();
     }
 
-    public void testExecuteVerboseDocumentSimple() throws Exception {
-        List<SimulateProcessorResult> processorResultList = new ArrayList<>();
-        executionService.executeVerboseDocument(processor, ingestDocument, processorResultList);
-        SimulateProcessorResult result = new SimulateProcessorResult("id", ingestDocument);
-        assertThat(processorResultList.size(), equalTo(1));
-        assertThat(processorResultList.get(0).getProcessorTag(), equalTo(result.getProcessorTag()));
-        assertThat(processorResultList.get(0).getIngestDocument(), equalTo(result.getIngestDocument()));
-        assertThat(processorResultList.get(0).getFailure(), nullValue());
-    }
-
-    public void testExecuteVerboseDocumentSimpleException() throws Exception {
-        RuntimeException exception = new RuntimeException("mock_exception");
-        TestProcessor processor = new TestProcessor("id", "mock", ingestDocument -> { throw exception; });
-        List<SimulateProcessorResult> processorResultList = new ArrayList<>();
-        try {
-            executionService.executeVerboseDocument(processor, ingestDocument, processorResultList);
-            fail("should throw exception");
-        } catch (RuntimeException e) {
-            assertThat(e.getMessage(), equalTo("mock_exception"));
-        }
-        SimulateProcessorResult result = new SimulateProcessorResult("id", exception);
-        assertThat(processorResultList.size(), equalTo(1));
-        assertThat(processorResultList.get(0).getProcessorTag(), equalTo(result.getProcessorTag()));
-        assertThat(processorResultList.get(0).getFailure(), equalTo(result.getFailure()));
-    }
-
-    public void testExecuteVerboseDocumentCompoundSuccess() throws Exception {
-        TestProcessor processor1 = new TestProcessor("p1", "mock", ingestDocument -> { });
-        TestProcessor processor2 = new TestProcessor("p2", "mock", ingestDocument -> { });
-
-        Processor compoundProcessor = new CompoundProcessor(processor1, processor2);
-        List<SimulateProcessorResult> processorResultList = new ArrayList<>();
-        executionService.executeVerboseDocument(compoundProcessor, ingestDocument, processorResultList);
-        assertThat(processor1.getInvokedCounter(), equalTo(1));
-        assertThat(processor2.getInvokedCounter(), equalTo(1));
-        assertThat(processorResultList.size(), equalTo(2));
-        assertThat(processorResultList.get(0).getProcessorTag(), equalTo("p1"));
-        assertThat(processorResultList.get(0).getIngestDocument(), equalTo(ingestDocument));
-        assertThat(processorResultList.get(0).getFailure(), nullValue());
-        assertThat(processorResultList.get(1).getProcessorTag(), equalTo("p2"));
-        assertThat(processorResultList.get(1).getIngestDocument(), equalTo(ingestDocument));
-        assertThat(processorResultList.get(1).getFailure(), nullValue());
-    }
-
-    public void testExecuteVerboseDocumentCompoundOnFailure() throws Exception {
-        TestProcessor processor1 = new TestProcessor("p1", "mock", ingestDocument -> { });
-        TestProcessor processor2 = new TestProcessor("p2", "mock", ingestDocument -> { throw new RuntimeException("p2_exception"); });
-        TestProcessor onFailureProcessor1 = new TestProcessor("fail_p1", "mock", ingestDocument -> { });
-        TestProcessor onFailureProcessor2 = new TestProcessor("fail_p2", "mock", ingestDocument -> { throw new RuntimeException("fail_p2_exception"); });
-        TestProcessor onFailureProcessor3 = new TestProcessor("fail_p3", "mock", ingestDocument -> { });
-        CompoundProcessor onFailureCompoundProcessor = new CompoundProcessor(Collections.singletonList(onFailureProcessor2), Collections.singletonList(onFailureProcessor3));
-
-        Processor compoundProcessor = new CompoundProcessor(Arrays.asList(processor1, processor2), Arrays.asList(onFailureProcessor1, onFailureCompoundProcessor));
-        List<SimulateProcessorResult> processorResultList = new ArrayList<>();
-        executionService.executeVerboseDocument(compoundProcessor, ingestDocument, processorResultList);
-        assertThat(processor1.getInvokedCounter(), equalTo(1));
-        assertThat(processor2.getInvokedCounter(), equalTo(1));
-        assertThat(onFailureProcessor1.getInvokedCounter(), equalTo(1));
-        assertThat(onFailureProcessor2.getInvokedCounter(), equalTo(1));
-        assertThat(onFailureProcessor3.getInvokedCounter(), equalTo(1));
-        assertThat(processorResultList.size(), equalTo(5));
-        assertThat(processorResultList.get(0).getProcessorTag(), equalTo("p1"));
-        assertThat(processorResultList.get(1).getProcessorTag(), equalTo("p2"));
-        assertThat(processorResultList.get(2).getProcessorTag(), equalTo("fail_p1"));
-        assertThat(processorResultList.get(3).getProcessorTag(), equalTo("fail_p2"));
-        assertThat(processorResultList.get(4).getProcessorTag(), equalTo("fail_p3"));
-    }
-
     public void testExecuteVerboseItem() throws Exception {
         TestProcessor processor = new TestProcessor("test-id", "mock", ingestDocument -> {});
         Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor, processor));
@@ -170,16 +98,43 @@ public class SimulateExecutionServiceTests extends ESTestCase {
         assertThat(simulateDocumentBaseResult.getFailure(), nullValue());
     }
 
-    public void testExecuteVerboseItemWithFailure() throws Exception {
+    public void testExecuteVerboseItemExceptionWithoutOnFailure() throws Exception {
+        TestProcessor processor1 = new TestProcessor("processor_0", "mock", ingestDocument -> {});
+        TestProcessor processor2 = new TestProcessor("processor_1", "mock", ingestDocument -> { throw new RuntimeException("processor failed"); });
+        TestProcessor processor3 = new TestProcessor("processor_2", "mock", ingestDocument -> {});
+        Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor1, processor2, processor3));
+        SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, true);
+        assertThat(processor1.getInvokedCounter(), equalTo(1));
+        assertThat(processor2.getInvokedCounter(), equalTo(1));
+        assertThat(processor3.getInvokedCounter(), equalTo(0));
+        assertThat(actualItemResponse, instanceOf(SimulateDocumentVerboseResult.class));
+        SimulateDocumentVerboseResult simulateDocumentVerboseResult = (SimulateDocumentVerboseResult) actualItemResponse;
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(2));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getProcessorTag(), equalTo("processor_0"));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure(), nullValue());
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument(), not(sameInstance(ingestDocument)));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument(), equalTo(ingestDocument));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument().getSourceAndMetadata(), not(sameInstance(ingestDocument.getSourceAndMetadata())));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getProcessorTag(), equalTo("processor_1"));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), nullValue());
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getFailure(), instanceOf(RuntimeException.class));
+        RuntimeException runtimeException = (RuntimeException) simulateDocumentVerboseResult.getProcessorResults().get(1).getFailure();
+        assertThat(runtimeException.getMessage(), equalTo("processor failed"));
+    }
+
+    public void testExecuteVerboseItemWithOnFailure() throws Exception {
         TestProcessor processor1 = new TestProcessor("processor_0", "mock", ingestDocument -> { throw new RuntimeException("processor failed"); });
         TestProcessor processor2 = new TestProcessor("processor_1", "mock", ingestDocument -> {});
-        Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(Collections.singletonList(processor1), Collections.singletonList(processor2)));
+        TestProcessor processor3 = new TestProcessor("processor_2", "mock", ingestDocument -> {});
+        Pipeline pipeline = new Pipeline("_id", "_description",
+                new CompoundProcessor(new CompoundProcessor(Collections.singletonList(processor1),
+                                Collections.singletonList(processor2)), processor3));
         SimulateDocumentResult actualItemResponse = executionService.executeDocument(pipeline, ingestDocument, true);
         assertThat(processor1.getInvokedCounter(), equalTo(1));
         assertThat(processor2.getInvokedCounter(), equalTo(1));
         assertThat(actualItemResponse, instanceOf(SimulateDocumentVerboseResult.class));
         SimulateDocumentVerboseResult simulateDocumentVerboseResult = (SimulateDocumentVerboseResult) actualItemResponse;
-        assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(2));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().size(), equalTo(3));
         assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getProcessorTag(), equalTo("processor_0"));
         assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getIngestDocument(), nullValue());
         assertThat(simulateDocumentVerboseResult.getProcessorResults().get(0).getFailure(), instanceOf(RuntimeException.class));
@@ -187,8 +142,20 @@ public class SimulateExecutionServiceTests extends ESTestCase {
         assertThat(runtimeException.getMessage(), equalTo("processor failed"));
         assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getProcessorTag(), equalTo("processor_1"));
         assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), not(sameInstance(ingestDocument)));
-        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), equalTo(ingestDocument));
+
+        IngestDocument ingestDocumentWithOnFailureMetadata = new IngestDocument(ingestDocument);
+        Map<String, String> metadata = ingestDocumentWithOnFailureMetadata.getIngestMetadata();
+        metadata.put(CompoundProcessor.ON_FAILURE_PROCESSOR_TYPE_FIELD, "mock");
+        metadata.put(CompoundProcessor.ON_FAILURE_PROCESSOR_TAG_FIELD, "processor_0");
+        metadata.put(CompoundProcessor.ON_FAILURE_MESSAGE_FIELD, "processor failed");
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getIngestDocument(), equalTo(ingestDocumentWithOnFailureMetadata));
+
         assertThat(simulateDocumentVerboseResult.getProcessorResults().get(1).getFailure(), nullValue());
+
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(2).getProcessorTag(), equalTo("processor_2"));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(2).getIngestDocument(), not(sameInstance(ingestDocument)));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(2).getIngestDocument(), equalTo(ingestDocument));
+        assertThat(simulateDocumentVerboseResult.getProcessorResults().get(2).getFailure(), nullValue());
     }
 
     public void testExecuteItemWithFailure() throws Exception {

+ 10 - 0
core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java

@@ -34,7 +34,9 @@ import org.elasticsearch.cluster.block.ClusterBlock;
 import org.elasticsearch.cluster.block.ClusterBlockException;
 import org.elasticsearch.cluster.block.ClusterBlockLevel;
 import org.elasticsearch.cluster.block.ClusterBlocks;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
+import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.node.DiscoveryNodes;
 import org.elasticsearch.cluster.routing.IndexRoutingTable;
@@ -214,10 +216,12 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
         IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(new Index(index, "_na_"));
 
         int shardIndex = -1;
+        int totalIndexShards = 0;
         for (int i = 0; i < numberOfNodes; i++) {
             final DiscoveryNode node = newNode(i);
             discoBuilder = discoBuilder.put(node);
             int numberOfShards = randomIntBetween(1, 10);
+            totalIndexShards += numberOfShards;
             for (int j = 0; j < numberOfShards; j++) {
                 final ShardId shardId = new ShardId(index, "_na_", ++shardIndex);
                 ShardRouting shard = TestShardRouting.newShardRouting(index, shardId.getId(), node.id(), true, ShardRoutingState.STARTED);
@@ -230,6 +234,12 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
         discoBuilder.masterNodeId(newNode(numberOfNodes - 1).id());
         ClusterState.Builder stateBuilder = ClusterState.builder(new ClusterName(TEST_CLUSTER));
         stateBuilder.nodes(discoBuilder);
+        final IndexMetaData.Builder indexMetaData = IndexMetaData.builder(index)
+                .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT))
+                .numberOfReplicas(0)
+                .numberOfShards(totalIndexShards);
+
+        stateBuilder.metaData(MetaData.builder().put(indexMetaData));
         stateBuilder.routingTable(RoutingTable.builder().add(indexRoutingTable.build()).build());
         ClusterState clusterState = stateBuilder.build();
         setState(clusterService, clusterState);

+ 1 - 1
core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java

@@ -142,7 +142,7 @@ public class BroadcastReplicationTests extends ESTestCase {
 
     public void testResultCombine() throws InterruptedException, ExecutionException, IOException {
         final String index = "test";
-        int numShards = randomInt(3);
+        int numShards = 1 + randomInt(3);
         setState(clusterService, stateWithAssignedPrimariesAndOneReplica(index, numShards));
         logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
         Future<BroadcastResponse> response = (broadcastReplicationAction.execute(new DummyBroadcastRequest().indices(index)));

+ 10 - 7
core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java

@@ -47,6 +47,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF
 import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
 import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED;
 import static org.elasticsearch.test.ESTestCase.randomFrom;
+import static org.elasticsearch.test.ESTestCase.randomInt;
 import static org.elasticsearch.test.ESTestCase.randomIntBetween;
 
 /**
@@ -85,10 +86,11 @@ public class ClusterStateCreationUtils {
         }
         discoBuilder.localNodeId(newNode(0).id());
         discoBuilder.masterNodeId(newNode(1).id()); // we need a non-local master to test shard failures
+        final int primaryTerm = randomInt(200);
         IndexMetaData indexMetaData = IndexMetaData.builder(index).settings(Settings.builder()
                 .put(SETTING_VERSION_CREATED, Version.CURRENT)
                 .put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, numberOfReplicas)
-                .put(SETTING_CREATION_DATE, System.currentTimeMillis())).build();
+                .put(SETTING_CREATION_DATE, System.currentTimeMillis())).primaryTerm(0, primaryTerm).build();
 
         RoutingTable.Builder routing = new RoutingTable.Builder();
         routing.addAsNew(indexMetaData);
@@ -156,7 +158,7 @@ public class ClusterStateCreationUtils {
         discoBuilder.masterNodeId(newNode(1).id()); // we need a non-local master to test shard failures
         IndexMetaData indexMetaData = IndexMetaData.builder(index).settings(Settings.builder()
                 .put(SETTING_VERSION_CREATED, Version.CURRENT)
-                .put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 1)
+                .put(SETTING_NUMBER_OF_SHARDS, numberOfShards).put(SETTING_NUMBER_OF_REPLICAS, 1)
                 .put(SETTING_CREATION_DATE, System.currentTimeMillis())).build();
         ClusterState.Builder state = ClusterState.builder(new ClusterName("test"));
         state.nodes(discoBuilder);
@@ -167,10 +169,10 @@ public class ClusterStateCreationUtils {
             routing.addAsNew(indexMetaData);
             final ShardId shardId = new ShardId(index, "_na_", i);
             IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId);
-            indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(0).id(), null, null,
-                    true, ShardRoutingState.STARTED, null));
-            indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(1).id(), null, null,
-                    false, ShardRoutingState.STARTED, null));
+            indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(0).id(), null, null, true,
+                    ShardRoutingState.STARTED, null));
+            indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(1).id(), null, null, false,
+                    ShardRoutingState.STARTED, null));
             indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build());
         }
         state.routingTable(RoutingTable.builder().add(indexRoutingTableBuilder.build()).build());
@@ -230,12 +232,13 @@ public class ClusterStateCreationUtils {
 
     /**
      * Creates a cluster state where local node and master node can be specified
+     *
      * @param localNode  node in allNodes that is the local node
      * @param masterNode node in allNodes that is the master node. Can be null if no master exists
      * @param allNodes   all nodes in the cluster
      * @return cluster state
      */
-    public static  ClusterState state(DiscoveryNode localNode, DiscoveryNode masterNode, DiscoveryNode... allNodes) {
+    public static ClusterState state(DiscoveryNode localNode, DiscoveryNode masterNode, DiscoveryNode... allNodes) {
         DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder();
         for (DiscoveryNode node : allNodes) {
             discoBuilder.put(node);

+ 17 - 5
core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java

@@ -630,11 +630,13 @@ public class TransportReplicationActionTests extends ESTestCase {
         final ShardIterator shardIt = shardRoutingTable.shardsIt();
         final ShardId shardId = shardIt.shardId();
         final Request request = new Request(shardId);
+        final long primaryTerm = randomInt(200);
+        request.primaryTerm(primaryTerm);
         final PlainActionFuture<Response> listener = new PlainActionFuture<>();
         ReplicationTask task = maybeTask();
         logger.debug("expecting [{}] assigned replicas, [{}] total shards. using state: \n{}", assignedReplicas, totalShards, clusterService.state().prettyPrint());
 
-        TransportReplicationAction.IndexShardReference reference = getOrCreateIndexShardOperationsCounter();
+        TransportReplicationAction.IndexShardReference reference = getOrCreateIndexShardOperationsCounter(0);
 
         ShardRouting primaryShard = state.getRoutingTable().shardRoutingTable(shardId).primaryShard();
         indexShardRouting.set(primaryShard);
@@ -767,6 +769,9 @@ public class TransportReplicationActionTests extends ESTestCase {
         }
         // all replicas have responded so the counter should be decreased again
         assertIndexShardCounter(1);
+
+        // assert that nothing in the replica logic changes the primary term of the operation
+        assertThat(request.primaryTerm(), equalTo(primaryTerm));
     }
 
     public void testCounterOnPrimary() throws Exception {
@@ -989,7 +994,7 @@ public class TransportReplicationActionTests extends ESTestCase {
     /**
      * Returns testIndexShardOperationsCounter or initializes it if it was already created in this test run.
      */
-    private synchronized TransportReplicationAction.IndexShardReference getOrCreateIndexShardOperationsCounter() {
+    private synchronized TransportReplicationAction.IndexShardReference getOrCreateIndexShardOperationsCounter(long primaryTerm) {
         count.incrementAndGet();
         return new TransportReplicationAction.IndexShardReference() {
             @Override
@@ -1009,6 +1014,11 @@ public class TransportReplicationActionTests extends ESTestCase {
                 return shardRouting;
             }
 
+            @Override
+            public long opPrimaryTerm() {
+                return primaryTerm;
+            }
+
             @Override
             public void close() {
                 count.decrementAndGet();
@@ -1104,13 +1114,15 @@ public class TransportReplicationActionTests extends ESTestCase {
             return false;
         }
 
+
         @Override
         protected IndexShardReference getIndexShardReferenceOnPrimary(ShardId shardId) {
-            return getOrCreateIndexShardOperationsCounter();
+            final IndexMetaData indexMetaData = clusterService.state().metaData().index(shardId.getIndex());
+            return getOrCreateIndexShardOperationsCounter(indexMetaData.primaryTerm(shardId.id()));
         }
 
-        protected IndexShardReference getIndexShardReferenceOnReplica(ShardId shardId) {
-            return getOrCreateIndexShardOperationsCounter();
+        protected IndexShardReference getIndexShardReferenceOnReplica(ShardId shardId, long opPrimaryTerm) {
+            return getOrCreateIndexShardOperationsCounter(opPrimaryTerm);
         }
     }
 

+ 15 - 5
core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java

@@ -32,6 +32,8 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
 import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.Matchers.not;
 
 public class BootstrapCheckTests extends ESTestCase {
 
@@ -80,9 +82,9 @@ public class BootstrapCheckTests extends ESTestCase {
 
     public void testFileDescriptorLimitsThrowsOnInvalidLimit() {
         final IllegalArgumentException e =
-                expectThrows(
-                        IllegalArgumentException.class,
-                        () -> new BootstrapCheck.FileDescriptorCheck(-randomIntBetween(0, Integer.MAX_VALUE)));
+            expectThrows(
+                IllegalArgumentException.class,
+                () -> new BootstrapCheck.FileDescriptorCheck(-randomIntBetween(0, Integer.MAX_VALUE)));
         assertThat(e.getMessage(), containsString("limit must be positive but was"));
     }
 
@@ -121,8 +123,8 @@ public class BootstrapCheckTests extends ESTestCase {
                     fail("should have failed due to memory not being locked");
                 } catch (final RuntimeException e) {
                     assertThat(
-                            e.getMessage(),
-                            containsString("memory locking requested for elasticsearch process but memory is not locked"));
+                        e.getMessage(),
+                        containsString("memory locking requested for elasticsearch process but memory is not locked"));
                 }
             } else {
                 // nothing should happen
@@ -197,4 +199,12 @@ public class BootstrapCheckTests extends ESTestCase {
         assertTrue(BootstrapCheck.enforceLimits(settings));
     }
 
+    public void testMinMasterNodes() {
+        boolean isSet = randomBoolean();
+        BootstrapCheck.Check check = new BootstrapCheck.MinMasterNodesCheck(isSet);
+        assertThat(check.check(), not(equalTo(isSet)));
+        List<BootstrapCheck.Check> defaultChecks = BootstrapCheck.checks(Settings.EMPTY);
+
+        expectThrows(RuntimeException.class, () -> BootstrapCheck.check(true, defaultChecks));
+    }
 }

+ 26 - 20
core/src/test/java/org/elasticsearch/cluster/metadata/ToAndFromJsonMetaDataTests.java

@@ -41,11 +41,14 @@ public class ToAndFromJsonMetaDataTests extends ESTestCase {
                 .put(IndexMetaData.builder("test1")
                         .settings(settings(Version.CURRENT))
                         .numberOfShards(1)
-                        .numberOfReplicas(2))
+                        .numberOfReplicas(2)
+                        .primaryTerm(0, 1))
                 .put(IndexMetaData.builder("test2")
                         .settings(settings(Version.CURRENT).put("setting1", "value1").put("setting2", "value2"))
                         .numberOfShards(2)
-                        .numberOfReplicas(3))
+                        .numberOfReplicas(3)
+                        .primaryTerm(0, 2)
+                        .primaryTerm(1, 2))
                 .put(IndexMetaData.builder("test3")
                         .settings(settings(Version.CURRENT))
                         .numberOfShards(1)
@@ -112,15 +115,15 @@ public class ToAndFromJsonMetaDataTests extends ESTestCase {
                         .putAlias(newAliasMetaDataBuilder("alias1").filter(ALIAS_FILTER1))
                         .putAlias(newAliasMetaDataBuilder("alias2"))
                         .putAlias(newAliasMetaDataBuilder("alias4").filter(ALIAS_FILTER2)))
-                        .put(IndexTemplateMetaData.builder("foo")
-                                .template("bar")
-                                .order(1)
-                                .settings(settingsBuilder()
-                                        .put("setting1", "value1")
-                                        .put("setting2", "value2"))
-                                .putAlias(newAliasMetaDataBuilder("alias-bar1"))
-                                .putAlias(newAliasMetaDataBuilder("alias-bar2").filter("{\"term\":{\"user\":\"kimchy\"}}"))
-                                .putAlias(newAliasMetaDataBuilder("alias-bar3").routing("routing-bar")))
+                .put(IndexTemplateMetaData.builder("foo")
+                        .template("bar")
+                        .order(1)
+                        .settings(settingsBuilder()
+                                .put("setting1", "value1")
+                                .put("setting2", "value2"))
+                        .putAlias(newAliasMetaDataBuilder("alias-bar1"))
+                        .putAlias(newAliasMetaDataBuilder("alias-bar2").filter("{\"term\":{\"user\":\"kimchy\"}}"))
+                        .putAlias(newAliasMetaDataBuilder("alias-bar3").routing("routing-bar")))
                 .put(IndexMetaData.builder("test12")
                         .settings(settings(Version.CURRENT)
                                 .put("setting1", "value1")
@@ -133,15 +136,15 @@ public class ToAndFromJsonMetaDataTests extends ESTestCase {
                         .putAlias(newAliasMetaDataBuilder("alias1").filter(ALIAS_FILTER1))
                         .putAlias(newAliasMetaDataBuilder("alias2"))
                         .putAlias(newAliasMetaDataBuilder("alias4").filter(ALIAS_FILTER2)))
-                        .put(IndexTemplateMetaData.builder("foo")
-                                .template("bar")
-                                .order(1)
-                                .settings(settingsBuilder()
-                                        .put("setting1", "value1")
-                                        .put("setting2", "value2"))
-                                .putAlias(newAliasMetaDataBuilder("alias-bar1"))
-                                .putAlias(newAliasMetaDataBuilder("alias-bar2").filter("{\"term\":{\"user\":\"kimchy\"}}"))
-                                .putAlias(newAliasMetaDataBuilder("alias-bar3").routing("routing-bar")))
+                .put(IndexTemplateMetaData.builder("foo")
+                        .template("bar")
+                        .order(1)
+                        .settings(settingsBuilder()
+                                .put("setting1", "value1")
+                                .put("setting2", "value2"))
+                        .putAlias(newAliasMetaDataBuilder("alias-bar1"))
+                        .putAlias(newAliasMetaDataBuilder("alias-bar2").filter("{\"term\":{\"user\":\"kimchy\"}}"))
+                        .putAlias(newAliasMetaDataBuilder("alias-bar3").routing("routing-bar")))
                 .build();
 
         String metaDataSource = MetaData.Builder.toXContent(metaData);
@@ -150,6 +153,7 @@ public class ToAndFromJsonMetaDataTests extends ESTestCase {
         MetaData parsedMetaData = MetaData.Builder.fromXContent(XContentFactory.xContent(XContentType.JSON).createParser(metaDataSource));
 
         IndexMetaData indexMetaData = parsedMetaData.index("test1");
+        assertThat(indexMetaData.primaryTerm(0), equalTo(1L));
         assertThat(indexMetaData.getNumberOfShards(), equalTo(1));
         assertThat(indexMetaData.getNumberOfReplicas(), equalTo(2));
         assertThat(indexMetaData.getCreationDate(), equalTo(-1L));
@@ -159,6 +163,8 @@ public class ToAndFromJsonMetaDataTests extends ESTestCase {
         indexMetaData = parsedMetaData.index("test2");
         assertThat(indexMetaData.getNumberOfShards(), equalTo(2));
         assertThat(indexMetaData.getNumberOfReplicas(), equalTo(3));
+        assertThat(indexMetaData.primaryTerm(0), equalTo(2L));
+        assertThat(indexMetaData.primaryTerm(1), equalTo(2L));
         assertThat(indexMetaData.getCreationDate(), equalTo(-1L));
         assertThat(indexMetaData.getSettings().getAsMap().size(), equalTo(5));
         assertThat(indexMetaData.getSettings().get("setting1"), equalTo("value1"));

+ 241 - 0
core/src/test/java/org/elasticsearch/cluster/routing/PrimaryTermsTests.java

@@ -0,0 +1,241 @@
+/*
+ * 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.cluster.routing;
+
+import org.elasticsearch.Version;
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.health.ClusterStateHealth;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.cluster.metadata.MetaData;
+import org.elasticsearch.cluster.node.DiscoveryNodes;
+import org.elasticsearch.cluster.node.DiscoveryNodes.Builder;
+import org.elasticsearch.cluster.routing.allocation.AllocationService;
+import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
+import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.test.ESAllocationTestCase;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
+import static org.elasticsearch.common.settings.Settings.settingsBuilder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+
+public class PrimaryTermsTests extends ESAllocationTestCase {
+
+    private static final String TEST_INDEX_1 = "test1";
+    private static final String TEST_INDEX_2 = "test2";
+    private RoutingTable testRoutingTable;
+    private int numberOfShards;
+    private int numberOfReplicas;
+    private final static Settings DEFAULT_SETTINGS = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
+    private AllocationService allocationService;
+    private ClusterState clusterState;
+
+    private final Map<String, long[]> primaryTermsPerIndex = new HashMap<>();
+
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+        this.allocationService = createAllocationService(settingsBuilder()
+                .put("cluster.routing.allocation.node_concurrent_recoveries", Integer.MAX_VALUE) // don't limit recoveries
+                .put("cluster.routing.allocation.node_initial_primaries_recoveries", Integer.MAX_VALUE)
+                .build());
+        this.numberOfShards = randomIntBetween(1, 5);
+        this.numberOfReplicas = randomIntBetween(1, 5);
+        logger.info("Setup test with " + this.numberOfShards + " shards and " + this.numberOfReplicas + " replicas.");
+        this.primaryTermsPerIndex.clear();
+        MetaData metaData = MetaData.builder()
+                .put(createIndexMetaData(TEST_INDEX_1))
+                .put(createIndexMetaData(TEST_INDEX_2))
+                .build();
+
+        this.testRoutingTable = new RoutingTable.Builder()
+                .add(new IndexRoutingTable.Builder(metaData.index(TEST_INDEX_1).getIndex()).initializeAsNew(metaData.index(TEST_INDEX_1))
+                        .build())
+                .add(new IndexRoutingTable.Builder(metaData.index(TEST_INDEX_2).getIndex()).initializeAsNew(metaData.index(TEST_INDEX_2))
+                        .build())
+                .build();
+
+        this.clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData)
+                .routingTable(testRoutingTable).build();
+    }
+
+    /**
+     * puts primary shard routings into initializing state
+     */
+    private void initPrimaries() {
+        logger.info("adding " + (this.numberOfReplicas + 1) + " nodes and performing rerouting");
+        Builder discoBuilder = DiscoveryNodes.builder();
+        for (int i = 0; i < this.numberOfReplicas + 1; i++) {
+            discoBuilder = discoBuilder.put(newNode("node" + i));
+        }
+        this.clusterState = ClusterState.builder(clusterState).nodes(discoBuilder).build();
+        RoutingAllocation.Result rerouteResult = allocationService.reroute(clusterState, "reroute");
+        this.testRoutingTable = rerouteResult.routingTable();
+        assertThat(rerouteResult.changed(), is(true));
+        applyRerouteResult(rerouteResult);
+        primaryTermsPerIndex.keySet().forEach(this::incrementPrimaryTerm);
+    }
+
+    private void incrementPrimaryTerm(String index) {
+        final long[] primaryTerms = primaryTermsPerIndex.get(index);
+        for (int i = 0; i < primaryTerms.length; i++) {
+            primaryTerms[i]++;
+        }
+    }
+
+    private void incrementPrimaryTerm(String index, int shard) {
+        primaryTermsPerIndex.get(index)[shard]++;
+    }
+
+    private boolean startInitializingShards(String index) {
+        this.clusterState = ClusterState.builder(clusterState).routingTable(this.testRoutingTable).build();
+        final List<ShardRouting> startedShards = this.clusterState.getRoutingNodes().shardsWithState(index, INITIALIZING);
+        logger.info("start primary shards for index [{}]: {} ", index, startedShards);
+        RoutingAllocation.Result rerouteResult = allocationService.applyStartedShards(this.clusterState, startedShards);
+        applyRerouteResult(rerouteResult);
+        return rerouteResult.changed();
+    }
+
+    private void applyRerouteResult(RoutingAllocation.Result rerouteResult) {
+        ClusterState previousClusterState = this.clusterState;
+        ClusterState newClusterState = ClusterState.builder(previousClusterState).routingResult(rerouteResult).build();
+        ClusterState.Builder builder = ClusterState.builder(newClusterState).incrementVersion();
+        if (previousClusterState.routingTable() != newClusterState.routingTable()) {
+            builder.routingTable(RoutingTable.builder(newClusterState.routingTable()).version(newClusterState.routingTable().version() + 1)
+                    .build());
+        }
+        if (previousClusterState.metaData() != newClusterState.metaData()) {
+            builder.metaData(MetaData.builder(newClusterState.metaData()).version(newClusterState.metaData().version() + 1));
+        }
+        this.clusterState = builder.build();
+        this.testRoutingTable = rerouteResult.routingTable();
+        final ClusterStateHealth clusterHealth = new ClusterStateHealth(clusterState);
+        logger.info("applied reroute. active shards: p [{}], t [{}], init shards: [{}], relocating: [{}]",
+                clusterHealth.getActivePrimaryShards(), clusterHealth.getActiveShards(),
+                clusterHealth.getInitializingShards(), clusterHealth.getRelocatingShards());
+    }
+
+    private void failSomePrimaries(String index) {
+        this.clusterState = ClusterState.builder(clusterState).routingTable(this.testRoutingTable).build();
+        final IndexRoutingTable indexShardRoutingTable = testRoutingTable.index(index);
+        Set<Integer> shardIdsToFail = new HashSet<>();
+        for (int i = 1 + randomInt(numberOfShards - 1); i > 0; i--) {
+            shardIdsToFail.add(randomInt(numberOfShards - 1));
+        }
+        logger.info("failing primary shards {} for index [{}]", shardIdsToFail, index);
+        List<FailedRerouteAllocation.FailedShard> failedShards = new ArrayList<>();
+        for (int shard : shardIdsToFail) {
+            failedShards.add(new FailedRerouteAllocation.FailedShard(indexShardRoutingTable.shard(shard).primaryShard(), "test", null));
+            incrementPrimaryTerm(index, shard); // the primary failure should increment the primary term;
+        }
+        RoutingAllocation.Result rerouteResult = allocationService.applyFailedShards(this.clusterState, failedShards);
+        applyRerouteResult(rerouteResult);
+    }
+
+    private void addNodes() {
+        DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(clusterState.nodes());
+        final int newNodes = randomInt(10);
+        logger.info("adding [{}] nodes", newNodes);
+        for (int i = 0; i < newNodes; i++) {
+            nodesBuilder.put(newNode("extra_" + i));
+        }
+        this.clusterState = ClusterState.builder(clusterState).nodes(nodesBuilder).build();
+        RoutingAllocation.Result rerouteResult = allocationService.reroute(this.clusterState, "nodes added");
+        applyRerouteResult(rerouteResult);
+
+    }
+
+    private IndexMetaData.Builder createIndexMetaData(String indexName) {
+        primaryTermsPerIndex.put(indexName, new long[numberOfShards]);
+        final IndexMetaData.Builder builder = new IndexMetaData.Builder(indexName)
+                .settings(DEFAULT_SETTINGS)
+                .numberOfReplicas(this.numberOfReplicas)
+                .numberOfShards(this.numberOfShards);
+        for (int i = 0; i < numberOfShards; i++) {
+            builder.primaryTerm(i, randomInt(200));
+            primaryTermsPerIndex.get(indexName)[i] = builder.primaryTerm(i);
+        }
+        return builder;
+    }
+
+    private void assertAllPrimaryTerm() {
+        primaryTermsPerIndex.keySet().forEach(this::assertPrimaryTerm);
+    }
+
+    private void assertPrimaryTerm(String index) {
+        final long[] terms = primaryTermsPerIndex.get(index);
+        final IndexMetaData indexMetaData = clusterState.metaData().index(index);
+        for (IndexShardRoutingTable shardRoutingTable : this.testRoutingTable.index(index)) {
+            final int shard = shardRoutingTable.shardId().id();
+            assertThat("primary term mismatch between indexMetaData of [" + index + "] and shard [" + shard + "]'s routing",
+                    indexMetaData.primaryTerm(shard), equalTo(terms[shard]));
+        }
+    }
+
+    public void testPrimaryTermMetaDataSync() {
+        assertAllPrimaryTerm();
+
+        initPrimaries();
+        assertAllPrimaryTerm();
+
+        startInitializingShards(TEST_INDEX_1);
+        assertAllPrimaryTerm();
+
+        startInitializingShards(TEST_INDEX_2);
+        assertAllPrimaryTerm();
+
+        // now start all replicas too
+        startInitializingShards(TEST_INDEX_1);
+        startInitializingShards(TEST_INDEX_2);
+        assertAllPrimaryTerm();
+
+        // relocations shouldn't change much
+        addNodes();
+        assertAllPrimaryTerm();
+        boolean changed = true;
+        while (changed) {
+            changed = startInitializingShards(TEST_INDEX_1);
+            assertAllPrimaryTerm();
+            changed |= startInitializingShards(TEST_INDEX_2);
+            assertAllPrimaryTerm();
+        }
+
+        // primary promotion
+        failSomePrimaries(TEST_INDEX_1);
+        assertAllPrimaryTerm();
+
+        // stablize cluster
+        changed = true;
+        while (changed) {
+            changed = startInitializingShards(TEST_INDEX_1);
+            assertAllPrimaryTerm();
+            changed |= startInitializingShards(TEST_INDEX_2);
+            assertAllPrimaryTerm();
+        }
+    }
+}

+ 0 - 1
core/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java

@@ -30,7 +30,6 @@ import org.elasticsearch.cluster.routing.RoutingTable;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.ShardRoutingState;
 import org.elasticsearch.cluster.routing.TestShardRouting;
-import org.elasticsearch.index.shard.ShardId;
 import org.elasticsearch.test.ESAllocationTestCase;
 
 import java.io.BufferedReader;

+ 22 - 16
core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java

@@ -58,29 +58,31 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
         ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
 
         logger.info("Adding two nodes and performing rerouting");
-        clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2"))).build();
-        RoutingTable prevRoutingTable = routingTable;
-        routingTable = strategy.reroute(clusterState, "reroute").routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
+        clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1"))).build();
+        RoutingAllocation.Result result = strategy.reroute(clusterState, "reroute");
+        clusterState = ClusterState.builder(clusterState).routingResult(result).build();
+
+        clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).put(newNode("node2"))).build();
+        result = strategy.reroute(clusterState, "reroute");
+        clusterState = ClusterState.builder(clusterState).routingResult(result).build();
 
         logger.info("Start the primary shard (on node1)");
         RoutingNodes routingNodes = clusterState.getRoutingNodes();
-        prevRoutingTable = routingTable;
-        routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
+        result = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING));
+        clusterState = ClusterState.builder(clusterState).routingResult(result).build();
 
         logger.info("Start the backup shard (on node2)");
         routingNodes = clusterState.getRoutingNodes();
-        prevRoutingTable = routingTable;
-        routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)).routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
+        result = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING));
+        clusterState = ClusterState.builder(clusterState).routingResult(result).build();
 
         logger.info("Adding third node and reroute and kill first node");
         clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).put(newNode("node3")).remove("node1")).build();
-        prevRoutingTable = routingTable;
-        routingTable = strategy.reroute(clusterState, "reroute").routingTable();
-        clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
+        RoutingTable prevRoutingTable = clusterState.routingTable();
+        result = strategy.reroute(clusterState, "reroute");
+        clusterState = ClusterState.builder(clusterState).routingResult(result).build();
         routingNodes = clusterState.getRoutingNodes();
+        routingTable = clusterState.routingTable();
 
         assertThat(prevRoutingTable != routingTable, equalTo(true));
         assertThat(routingTable.index("test").shards().size(), equalTo(1));
@@ -89,6 +91,7 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
         assertThat(routingNodes.node("node3").numberOfShardsWithState(INITIALIZING), equalTo(1));
         // verify where the primary is
         assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo("node2"));
+        assertThat(clusterState.metaData().index("test").primaryTerm(0), equalTo(2L));
         assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo("node3"));
     }
 
@@ -110,16 +113,18 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
         logger.info("Adding two nodes and performing rerouting");
         clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2"))).build();
         RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
-        clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
+        clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
 
         logger.info("Start the primary shards");
         RoutingNodes routingNodes = clusterState.getRoutingNodes();
         rerouteResult = allocation.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
-        clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
+        clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
         routingNodes = clusterState.getRoutingNodes();
 
         assertThat(routingNodes.shardsWithState(STARTED).size(), equalTo(2));
         assertThat(routingNodes.shardsWithState(INITIALIZING).size(), equalTo(2));
+        assertThat(clusterState.metaData().index("test").primaryTerm(0), equalTo(1L));
+        assertThat(clusterState.metaData().index("test").primaryTerm(1), equalTo(1L));
 
         // now, fail one node, while the replica is initializing, and it also holds a primary
         logger.info("--> fail node with primary");
@@ -129,12 +134,13 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
                 .put(newNode(nodeIdRemaining))
         ).build();
         rerouteResult = allocation.reroute(clusterState, "reroute");
-        clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
+        clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
         routingNodes = clusterState.getRoutingNodes();
 
         assertThat(routingNodes.shardsWithState(STARTED).size(), equalTo(1));
         assertThat(routingNodes.shardsWithState(INITIALIZING).size(), equalTo(1));
         assertThat(routingNodes.node(nodeIdRemaining).shardsWithState(INITIALIZING).get(0).primary(), equalTo(true));
+        assertThat(clusterState.metaData().index("test").primaryTerm(0), equalTo(2L));
 
     }
 }

+ 80 - 0
core/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java

@@ -0,0 +1,80 @@
+/*
+ * 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.cluster.routing.allocation;
+
+import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.health.ClusterHealthStatus;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.index.IndexService;
+import org.elasticsearch.index.shard.IndexShard;
+import org.elasticsearch.indices.IndicesService;
+import org.elasticsearch.test.ESIntegTestCase;
+
+import static org.hamcrest.Matchers.equalTo;
+
+public class ShardStateIT extends ESIntegTestCase {
+
+    public void testPrimaryFailureIncreasesTerm() throws Exception {
+        internalCluster().ensureAtLeastNumDataNodes(2);
+        prepareCreate("test").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).get();
+        ensureGreen();
+        assertPrimaryTerms(1, 1);
+
+        logger.info("--> disabling allocation to capture shard failure");
+        disableAllocation("test");
+
+        ClusterState state = client().admin().cluster().prepareState().get().getState();
+        final int shard = randomBoolean() ? 0 : 1;
+        final String nodeId = state.routingTable().index("test").shard(shard).primaryShard().currentNodeId();
+        final String node = state.nodes().get(nodeId).name();
+        logger.info("--> failing primary of [{}] on node [{}]", shard, node);
+        IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node);
+        indicesService.indexService(resolveIndex("test")).getShard(shard).failShard("simulated test failure", null);
+
+        logger.info("--> waiting for a yellow index");
+        assertBusy(() -> assertThat(client().admin().cluster().prepareHealth().get().getStatus(), equalTo(ClusterHealthStatus.YELLOW)));
+
+        final long term0 = shard == 0 ? 2 : 1;
+        final long term1 = shard == 1 ? 2 : 1;
+        assertPrimaryTerms(term0, term1);
+
+        logger.info("--> enabling allocation");
+        enableAllocation("test");
+        ensureGreen();
+        assertPrimaryTerms(term0, term1);
+    }
+
+    protected void assertPrimaryTerms(long term0, long term1) {
+        for (String node : internalCluster().getNodeNames()) {
+            logger.debug("--> asserting primary terms terms on [{}]", node);
+            ClusterState state = client(node).admin().cluster().prepareState().setLocal(true).get().getState();
+            IndexMetaData metaData = state.metaData().index("test");
+            assertThat(metaData.primaryTerm(0), equalTo(term0));
+            assertThat(metaData.primaryTerm(1), equalTo(term1));
+            IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node);
+            IndexService indexService = indicesService.indexService(metaData.getIndex());
+            if (indexService != null) {
+                for (IndexShard shard : indexService) {
+                    assertThat("term mismatch for shard " + shard.shardId(),
+                        shard.getPrimaryTerm(), equalTo(metaData.primaryTerm(shard.shardId().id())));
+                }
+            }
+        }
+    }
+}

+ 56 - 1
core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java

@@ -19,11 +19,13 @@
 
 package org.elasticsearch.gateway;
 
+import com.carrotsearch.hppc.cursors.ObjectCursor;
 import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
 import org.elasticsearch.action.admin.indices.stats.IndexStats;
 import org.elasticsearch.action.admin.indices.stats.ShardStats;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
 import org.elasticsearch.common.settings.Settings;
@@ -37,11 +39,13 @@ import org.elasticsearch.test.ESIntegTestCase;
 import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
 import org.elasticsearch.test.ESIntegTestCase.Scope;
 import org.elasticsearch.test.InternalTestCluster.RestartCallback;
-import org.elasticsearch.test.junit.annotations.TestLogging;
 import org.elasticsearch.test.store.MockFSDirectoryService;
 import org.elasticsearch.test.store.MockFSIndexStore;
 
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.IntStream;
 
 import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
 import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
@@ -88,10 +92,13 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
         assertHitCount(client().prepareSearch().setSize(0).setQuery(termQuery("appAccountIds", 179)).execute().actionGet(), 2);
         ensureYellow("test"); // wait for primary allocations here otherwise if we have a lot of shards we might have a
         // shard that is still in post recovery when we restart and the ensureYellow() below will timeout
+
+        Map<String, long[]> primaryTerms = assertAndCapturePrimaryTerms(null);
         internalCluster().fullRestart();
 
         logger.info("Running Cluster Health (wait for the shards to startup)");
         ensureYellow();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         client().admin().indices().prepareRefresh().execute().actionGet();
         assertHitCount(client().prepareSearch().setSize(0).setQuery(termQuery("appAccountIds", 179)).execute().actionGet(), 2);
@@ -100,11 +107,37 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
 
         logger.info("Running Cluster Health (wait for the shards to startup)");
         ensureYellow();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         client().admin().indices().prepareRefresh().execute().actionGet();
         assertHitCount(client().prepareSearch().setSize(0).setQuery(termQuery("appAccountIds", 179)).execute().actionGet(), 2);
     }
 
+    private Map<String, long[]> assertAndCapturePrimaryTerms(Map<String, long[]> previousTerms) {
+        if (previousTerms == null) {
+            previousTerms = new HashMap<>();
+        }
+        final Map<String, long[]> result = new HashMap<>();
+        final ClusterState state = client().admin().cluster().prepareState().get().getState();
+        for (ObjectCursor<IndexMetaData> cursor : state.metaData().indices().values()) {
+            final IndexMetaData indexMetaData = cursor.value;
+            final String index = indexMetaData.getIndex().getName();
+            final long[] previous = previousTerms.get(index);
+            final long[] current = IntStream.range(0, indexMetaData.getNumberOfShards()).mapToLong(indexMetaData::primaryTerm).toArray();
+            if (previous == null) {
+                result.put(index, current);
+            } else {
+                assertThat("number of terms changed for index [" + index + "]", current.length, equalTo(previous.length));
+                for (int shard = 0; shard < current.length; shard++) {
+                    assertThat("primary term didn't increase for [" + index + "][" + shard + "]", current[shard], greaterThan(previous[shard]));
+                }
+                result.put(index, current);
+            }
+        }
+
+        return result;
+    }
+
     public void testSingleNodeNoFlush() throws Exception {
         internalCluster().startNode();
 
@@ -163,10 +196,14 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
             logger.info("Ensure all primaries have been started");
             ensureYellow();
         }
+
+        Map<String, long[]> primaryTerms = assertAndCapturePrimaryTerms(null);
+
         internalCluster().fullRestart();
 
         logger.info("Running Cluster Health (wait for the shards to startup)");
         ensureYellow();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         for (int i = 0; i <= randomInt(10); i++) {
             assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(), value1Docs + value2Docs);
@@ -180,6 +217,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
 
         logger.info("Running Cluster Health (wait for the shards to startup)");
         ensureYellow();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         for (int i = 0; i <= randomInt(10); i++) {
             assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(), value1Docs + value2Docs);
@@ -201,10 +239,13 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
         ensureYellow("test"); // wait for primary allocations here otherwise if we have a lot of shards we might have a
         // shard that is still in post recovery when we restart and the ensureYellow() below will timeout
 
+        Map<String, long[]> primaryTerms = assertAndCapturePrimaryTerms(null);
+
         internalCluster().fullRestart();
 
         logger.info("Running Cluster Health (wait for the shards to startup)");
         ensureYellow();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         for (int i = 0; i < 10; i++) {
             assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet(), 2);
@@ -214,6 +255,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
 
         logger.info("Running Cluster Health (wait for the shards to startup)");
         ensureYellow();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         for (int i = 0; i < 10; i++) {
             assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet(), 2);
@@ -236,6 +278,8 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
             assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet(), 2);
         }
 
+        Map<String, long[]> primaryTerms = assertAndCapturePrimaryTerms(null);
+
         internalCluster().fullRestart(new RestartCallback() {
             @Override
             public Settings onNodeStopped(String nodeName) throws Exception {
@@ -251,6 +295,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
 
         logger.info("Running Cluster Health (wait for the shards to startup)");
         ensureGreen();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         for (int i = 0; i < 10; i++) {
             assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet(), 2);
@@ -276,6 +321,8 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
         String metaDataUuid = client().admin().cluster().prepareState().execute().get().getState().getMetaData().clusterUUID();
         assertThat(metaDataUuid, not(equalTo("_na_")));
 
+        Map<String, long[]> primaryTerms = assertAndCapturePrimaryTerms(null);
+
         logger.info("--> closing first node, and indexing more data to the second node");
         internalCluster().fullRestart(new RestartCallback() {
 
@@ -315,6 +362,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
 
         logger.info("--> running cluster_health (wait for the shards to startup)");
         ensureGreen();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         assertThat(client().admin().cluster().prepareState().execute().get().getState().getMetaData().clusterUUID(), equalTo(metaDataUuid));
 
@@ -386,11 +434,15 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
                 .setTransientSettings(settingsBuilder()
                         .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), EnableAllocationDecider.Allocation.NONE))
                 .get();
+
+        Map<String, long[]> primaryTerms = assertAndCapturePrimaryTerms(null);
+
         logger.info("--> full cluster restart");
         internalCluster().fullRestart();
 
         logger.info("--> waiting for cluster to return to green after {}shutdown", useSyncIds ? "" : "second ");
         ensureGreen();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         if (useSyncIds) {
             assertSyncIdsNotNull();
@@ -445,6 +497,8 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
         internalCluster().startNode(settingsBuilder().put(Environment.PATH_DATA_SETTING.getKey(), createTempDir()).build());
 
         ensureGreen();
+        Map<String, long[]> primaryTerms = assertAndCapturePrimaryTerms(null);
+
 
         internalCluster().fullRestart(new RestartCallback() {
 
@@ -455,6 +509,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
         });
 
         ensureYellow();
+        primaryTerms = assertAndCapturePrimaryTerms(primaryTerms);
 
         assertThat(client().admin().indices().prepareExists("test").execute().actionGet().isExists(), equalTo(true));
         assertHitCount(client().prepareSearch("test").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet(), 1);

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

@@ -37,18 +37,22 @@ import org.elasticsearch.action.admin.indices.stats.CommonStats;
 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.index.IndexRequest;
+import org.elasticsearch.action.index.TransportIndexAction;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.support.IndicesOptions;
 import org.elasticsearch.cluster.ClusterInfoService;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.InternalClusterInfoService;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.cluster.metadata.MetaData;
 import org.elasticsearch.cluster.metadata.SnapshotId;
 import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.routing.AllocationId;
 import org.elasticsearch.cluster.routing.RestoreSource;
 import org.elasticsearch.cluster.routing.ShardRouting;
 import org.elasticsearch.cluster.routing.ShardRoutingHelper;
+import org.elasticsearch.cluster.routing.ShardRoutingState;
 import org.elasticsearch.cluster.routing.TestShardRouting;
 import org.elasticsearch.cluster.routing.UnassignedInfo;
 import org.elasticsearch.cluster.service.ClusterService;
@@ -59,7 +63,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.logging.ESLogger;
-import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.DummyTransportAddress;
 import org.elasticsearch.common.unit.ByteSizeUnit;
@@ -119,6 +122,7 @@ import static java.util.Collections.emptySet;
 import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
 import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
 import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED;
+import static org.elasticsearch.common.lucene.Lucene.cleanLuceneIndex;
 import static org.elasticsearch.common.settings.Settings.settingsBuilder;
 import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS;
 import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
@@ -127,6 +131,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcke
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits;
+import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.greaterThan;
 
@@ -168,6 +173,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         createIndex("test");
         ensureGreen();
         NodeEnvironment env = getInstanceFromNode(NodeEnvironment.class);
+
         ClusterService cs = getInstanceFromNode(ClusterService.class);
         final Index index = cs.state().metaData().index("test").getIndex();
         Path[] shardPaths = env.availableShardPaths(new ShardId(index, 0));
@@ -295,31 +301,133 @@ public class IndexShardTests extends ESSingleNodeTestCase {
             // expected
         }
         try {
-            indexShard.acquireReplicaOperationLock();
+            indexShard.acquireReplicaOperationLock(indexShard.getPrimaryTerm());
             fail("we should not be able to increment anymore");
         } catch (IndexShardClosedException e) {
             // expected
         }
     }
 
-    public void testIndexOperationsCounter() throws InterruptedException, ExecutionException, IOException {
+    public void testOperationLocksOnPrimaryShards() throws InterruptedException, ExecutionException, IOException {
         assertAcked(client().admin().indices().prepareCreate("test").setSettings(Settings.builder().put("index.number_of_shards", 1).put("index.number_of_replicas", 0)).get());
         ensureGreen("test");
         IndicesService indicesService = getInstanceFromNode(IndicesService.class);
         IndexService indexService = indicesService.indexServiceSafe(resolveIndex("test"));
         IndexShard indexShard = indexService.getShardOrNull(0);
+        long primaryTerm = indexShard.getPrimaryTerm();
+
+        ShardRouting temp = indexShard.routingEntry();
+        final ShardRouting newPrimaryShardRouting;
+        if (randomBoolean()) {
+            // relocation target
+            newPrimaryShardRouting = TestShardRouting.newShardRouting(temp.index(), temp.id(), temp.currentNodeId(), "other node",
+                true, ShardRoutingState.INITIALIZING, AllocationId.newRelocation(temp.allocationId()));
+        } else if (randomBoolean()) {
+            // simulate promotion
+            ShardRouting newReplicaShardRouting = TestShardRouting.newShardRouting(temp.index(), temp.id(), temp.currentNodeId(), null,
+                false, ShardRoutingState.STARTED, temp.allocationId());
+            indexShard.updateRoutingEntry(newReplicaShardRouting, false);
+            primaryTerm = primaryTerm + 1;
+            indexShard.updatePrimaryTerm(primaryTerm);
+            newPrimaryShardRouting = TestShardRouting.newShardRouting(temp.index(), temp.id(), temp.currentNodeId(), null,
+                true, ShardRoutingState.STARTED, temp.allocationId());
+        } else {
+            newPrimaryShardRouting = temp;
+        }
+        indexShard.updateRoutingEntry(newPrimaryShardRouting, false);
+
         assertEquals(0, indexShard.getActiveOperationsCount());
+        if (newPrimaryShardRouting.isRelocationTarget() == false) {
+            try {
+                indexShard.acquireReplicaOperationLock(primaryTerm);
+                fail("shard shouldn't accept operations as replica");
+            } catch (IllegalStateException ignored) {
+
+            }
+        }
         Releasable operation1 = indexShard.acquirePrimaryOperationLock();
         assertEquals(1, indexShard.getActiveOperationsCount());
         Releasable operation2 = indexShard.acquirePrimaryOperationLock();
         assertEquals(2, indexShard.getActiveOperationsCount());
+
+        Releasables.close(operation1, operation2);
+        assertEquals(0, indexShard.getActiveOperationsCount());
+    }
+
+    public void testOperationLocksOnReplicaShards() throws InterruptedException, ExecutionException, IOException {
+        assertAcked(client().admin().indices().prepareCreate("test").setSettings(Settings.builder().put("index.number_of_shards", 1).put("index.number_of_replicas", 0)).get());
+        ensureGreen("test");
+        IndicesService indicesService = getInstanceFromNode(IndicesService.class);
+        IndexService indexService = indicesService.indexServiceSafe(resolveIndex("test"));
+        IndexShard indexShard = indexService.getShardOrNull(0);
+        long primaryTerm = indexShard.getPrimaryTerm();
+
+        // ugly hack to allow the shard to operated as a replica
+        final ShardRouting temp = indexShard.routingEntry();
+        final ShardRouting newShardRouting;
+        switch (randomInt(2)) {
+            case 0:
+                // started replica
+                newShardRouting = TestShardRouting.newShardRouting(temp.index(), temp.id(), temp.currentNodeId(), null,
+                    false, ShardRoutingState.STARTED, AllocationId.newRelocation(temp.allocationId()));
+
+                indexShard.updateRoutingEntry(newShardRouting, false);
+                break;
+            case 1:
+                // initializing replica / primary
+                final boolean relocating = randomBoolean();
+                newShardRouting = TestShardRouting.newShardRouting(temp.index(), temp.id(), temp.currentNodeId(),
+                    relocating ? "sourceNode" : null,
+                    relocating ? randomBoolean() : false,
+                    ShardRoutingState.INITIALIZING,
+                    relocating ? AllocationId.newRelocation(temp.allocationId()) : temp.allocationId());
+                indexShard.updateRoutingEntry(newShardRouting, false);
+                break;
+            case 2:
+                // relocation source
+                newShardRouting = TestShardRouting.newShardRouting(temp.index(), temp.id(), temp.currentNodeId(), "otherNode",
+                    false, ShardRoutingState.RELOCATING, AllocationId.newRelocation(temp.allocationId()));
+                indexShard.updateRoutingEntry(newShardRouting, false);
+                indexShard.relocated("test");
+                break;
+            default:
+                throw new UnsupportedOperationException("get your numbers straight");
+
+        }
+        logger.info("updated shard routing to {}", newShardRouting);
+
+        assertEquals(0, indexShard.getActiveOperationsCount());
+        if (newShardRouting.primary() == false) {
+            try {
+                indexShard.acquirePrimaryOperationLock();
+                fail("shard shouldn't accept primary ops");
+            } catch (IllegalStateException ignored) {
+
+            }
+        }
+
+        Releasable operation1 = indexShard.acquireReplicaOperationLock(primaryTerm);
+        assertEquals(1, indexShard.getActiveOperationsCount());
+        Releasable operation2 = indexShard.acquireReplicaOperationLock(primaryTerm);
+        assertEquals(2, indexShard.getActiveOperationsCount());
+
+        try {
+            indexShard.acquireReplicaOperationLock(primaryTerm - 1);
+            fail("you can not increment the operation counter with an older primary term");
+        } catch (IllegalArgumentException e) {
+            assertThat(e.getMessage(), containsString("operation term"));
+            assertThat(e.getMessage(), containsString("too old"));
+        }
+
+        // but you can increment with a newer one..
+        indexShard.acquireReplicaOperationLock(primaryTerm + 1 + randomInt(20)).close();
         Releasables.close(operation1, operation2);
         assertEquals(0, indexShard.getActiveOperationsCount());
     }
 
     public void testMarkAsInactiveTriggersSyncedFlush() throws Exception {
         assertAcked(client().admin().indices().prepareCreate("test")
-                .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0));
+            .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0));
         client().prepareIndex("test", "test").setSource("{}").get();
         ensureGreen("test");
         IndicesService indicesService = getInstanceFromNode(IndicesService.class);
@@ -364,14 +472,14 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         assertTrue(shard.getEngine().getTranslog().syncNeeded());
         setDurability(shard, Translog.Durability.REQUEST);
         assertNoFailures(client().prepareBulk()
-                .add(client().prepareIndex("test", "bar", "3").setSource("{}"))
-                .add(client().prepareDelete("test", "bar", "1")).get());
+            .add(client().prepareIndex("test", "bar", "3").setSource("{}"))
+            .add(client().prepareDelete("test", "bar", "1")).get());
         assertFalse(shard.getEngine().getTranslog().syncNeeded());
 
         setDurability(shard, Translog.Durability.ASYNC);
         assertNoFailures(client().prepareBulk()
-                .add(client().prepareIndex("test", "bar", "4").setSource("{}"))
-                .add(client().prepareDelete("test", "bar", "3")).get());
+            .add(client().prepareIndex("test", "bar", "4").setSource("{}"))
+            .add(client().prepareDelete("test", "bar", "3")).get());
         setDurability(shard, Translog.Durability.REQUEST);
         assertTrue(shard.getEngine().getTranslog().syncNeeded());
     }
@@ -384,7 +492,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
     public void testMinimumCompatVersion() {
         Version versionCreated = VersionUtils.randomVersion(random());
         assertAcked(client().admin().indices().prepareCreate("test")
-                .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0, SETTING_VERSION_CREATED, versionCreated.id));
+            .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0, SETTING_VERSION_CREATED, versionCreated.id));
         client().prepareIndex("test", "test").setSource("{}").get();
         ensureGreen("test");
         IndicesService indicesService = getInstanceFromNode(IndicesService.class);
@@ -398,7 +506,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
 
     public void testUpdatePriority() {
         assertAcked(client().admin().indices().prepareCreate("test")
-                .setSettings(IndexMetaData.SETTING_PRIORITY, 200));
+            .setSettings(IndexMetaData.SETTING_PRIORITY, 200));
         IndexService indexService = getInstanceFromNode(IndicesService.class).indexService(resolveIndex("test"));
         assertEquals(200, indexService.getIndexSettings().getSettings().getAsInt(IndexMetaData.SETTING_PRIORITY, 0).intValue());
         client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_PRIORITY, 400).build()).get();
@@ -434,8 +542,8 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         Path idxPath = env.sharedDataFile().resolve(randomAsciiOfLength(10));
         logger.info("--> idxPath: [{}]", idxPath);
         Settings idxSettings = Settings.builder()
-                .put(IndexMetaData.SETTING_DATA_PATH, idxPath)
-                .build();
+            .put(IndexMetaData.SETTING_DATA_PATH, idxPath)
+            .build();
         createIndex("test", idxSettings);
         ensureGreen("test");
         client().prepareIndex("test", "bar", "1").setSource("{}").setRefresh(true).get();
@@ -447,7 +555,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
 
     public void testExpectedShardSizeIsPresent() throws InterruptedException {
         assertAcked(client().admin().indices().prepareCreate("test")
-                .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0));
+            .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0));
         for (int i = 0; i < 50; i++) {
             client().prepareIndex("test", "test").setSource("{}").get();
         }
@@ -475,11 +583,11 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         IOUtils.rm(endDir);
 
         Settings sb = Settings.builder()
-                .put(IndexMetaData.SETTING_DATA_PATH, startDir.toAbsolutePath().toString())
-                .build();
+            .put(IndexMetaData.SETTING_DATA_PATH, startDir.toAbsolutePath().toString())
+            .build();
         Settings sb2 = Settings.builder()
-                .put(IndexMetaData.SETTING_DATA_PATH, endDir.toAbsolutePath().toString())
-                .build();
+            .put(IndexMetaData.SETTING_DATA_PATH, endDir.toAbsolutePath().toString())
+            .build();
 
         logger.info("--> creating an index with data_path [{}]", startDir.toAbsolutePath().toString());
         createIndex(INDEX, sb);
@@ -510,9 +618,9 @@ public class IndexShardTests extends ESSingleNodeTestCase {
 
         logger.info("--> updating settings...");
         client().admin().indices().prepareUpdateSettings(INDEX)
-                .setSettings(sb2)
-                .setIndicesOptions(IndicesOptions.fromOptions(true, false, true, true))
-                .get();
+            .setSettings(sb2)
+            .setIndicesOptions(IndicesOptions.fromOptions(true, false, true, true))
+            .get();
 
         assert Files.exists(startDir) == false : "start dir shouldn't exist";
 
@@ -642,7 +750,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         try {
             shard.index(index);
             fail();
-        }catch (IllegalIndexShardStateException e){
+        } catch (IllegalIndexShardStateException e) {
 
         }
 
@@ -655,7 +763,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         try {
             shard.delete(delete);
             fail();
-        }catch (IllegalIndexShardStateException e){
+        } catch (IllegalIndexShardStateException e) {
 
         }
 
@@ -692,7 +800,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         long size = shard.getEngine().getTranslog().sizeInBytes();
         logger.info("--> current translog size: [{}] num_ops [{}] generation [{}]", shard.getEngine().getTranslog().sizeInBytes(), shard.getEngine().getTranslog().totalOperations(), shard.getEngine().getTranslog().getGeneration());
         client().admin().indices().prepareUpdateSettings("test").setSettings(settingsBuilder().put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(size, ByteSizeUnit.BYTES))
-                .build()).get();
+            .build()).get();
         client().prepareDelete("test", "test", "2").get();
         logger.info("--> translog size after delete: [{}] num_ops [{}] generation [{}]", shard.getEngine().getTranslog().sizeInBytes(), shard.getEngine().getTranslog().totalOperations(), shard.getEngine().getTranslog().getGeneration());
         assertBusy(() -> { // this is async
@@ -877,7 +985,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         newShard.updateRoutingEntry(routing, false);
         DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT);
         newShard.markAsRecovering("store", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.STORE, localNode,
-                localNode));
+            localNode));
         assertTrue(newShard.recoverFromStore(localNode));
         assertEquals(0, newShard.recoveryState().getTranslog().recoveredOperations());
         assertEquals(0, newShard.recoveryState().getTranslog().totalOperations());
@@ -890,7 +998,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         assertHitCount(response, 0);
     }
 
-    public void testFailIfIndexNotPresentInRecoverFromStore() throws IOException {
+    public void testFailIfIndexNotPresentInRecoverFromStore() throws Exception {
         createIndex("test");
         ensureGreen();
         IndicesService indicesService = getInstanceFromNode(IndicesService.class);
@@ -907,7 +1015,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         Store store = shard.store();
         store.incRef();
         test.removeShard(0, "b/c simon says so");
-        Lucene.cleanLuceneIndex(store.directory());
+        cleanLuceneIndex(store.directory());
         store.decRef();
         ShardRoutingHelper.reinit(routing);
         IndexShard newShard = test.createShard(routing);
@@ -940,7 +1048,12 @@ public class IndexShardTests extends ESSingleNodeTestCase {
         newShard.updateRoutingEntry(routing, true);
         SearchResponse response = client().prepareSearch().get();
         assertHitCount(response, 0);
-        client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(true).get();
+        // we can't issue this request through a client because of the inconsistencies we created with the cluster state
+        // doing it directly instead
+        IndexRequest request = client().prepareIndex("test", "test", "0").setSource("{}").request();
+        request.process(MetaData.builder().put(test.getMetaData(), false).build(), null, false, "test");
+        TransportIndexAction.executeIndexRequestOnPrimary(request, newShard, null);
+        newShard.refresh("test");
         assertHitCount(client().prepareSearch().get(), 1);
     }
 
@@ -999,7 +1112,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
             @Override
             public void restore(SnapshotId snapshotId, Version version, ShardId shardId, ShardId snapshotShardId, RecoveryState recoveryState) {
                 try {
-                    Lucene.cleanLuceneIndex(targetStore.directory());
+                    cleanLuceneIndex(targetStore.directory());
                     for (String file : sourceStore.directory().listAll()) {
                         if (file.equals("write.lock") || file.startsWith("extra")) {
                             continue;
@@ -1205,12 +1318,12 @@ public class IndexShardTests extends ESSingleNodeTestCase {
     public void testIndexingBufferDuringInternalRecovery() throws IOException {
         createIndex("index");
         client().admin().indices().preparePutMapping("index").setType("testtype").setSource(jsonBuilder().startObject()
-                .startObject("testtype")
-                .startObject("properties")
-                .startObject("foo")
-                .field("type", "text")
-                .endObject()
-                .endObject().endObject().endObject()).get();
+            .startObject("testtype")
+            .startObject("properties")
+            .startObject("foo")
+            .field("type", "text")
+            .endObject()
+            .endObject().endObject().endObject()).get();
         ensureGreen();
         IndicesService indicesService = getInstanceFromNode(IndicesService.class);
         IndexService test = indicesService.indexService(resolveIndex("index"));
@@ -1234,12 +1347,12 @@ public class IndexShardTests extends ESSingleNodeTestCase {
     public void testIndexingBufferDuringPeerRecovery() throws IOException {
         createIndex("index");
         client().admin().indices().preparePutMapping("index").setType("testtype").setSource(jsonBuilder().startObject()
-                .startObject("testtype")
-                .startObject("properties")
-                .startObject("foo")
-                .field("type", "text")
-                .endObject()
-                .endObject().endObject().endObject()).get();
+            .startObject("testtype")
+            .startObject("properties")
+            .startObject("foo")
+            .field("type", "text")
+            .endObject()
+            .endObject().endObject().endObject()).get();
         ensureGreen();
         IndicesService indicesService = getInstanceFromNode(IndicesService.class);
         IndexService test = indicesService.indexService(resolveIndex("index"));

+ 1 - 0
core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java

@@ -40,6 +40,7 @@ public class IndexPrimaryRelocationIT extends ESIntegTestCase {
 
     private static final int RELOCATION_COUNT = 25;
 
+    @TestLogging("_root:DEBUG,action.delete:TRACE,action.index:TRACE,index.shard:TRACE,cluster.service:TRACE")
     public void testPrimaryRelocationWhileIndexing() throws Exception {
         internalCluster().ensureAtLeastNumDataNodes(randomIntBetween(2, 3));
         client().admin().indices().prepareCreate("test")

+ 129 - 0
core/src/test/java/org/elasticsearch/ingest/processor/TrackingResultProcessorTests.java

@@ -0,0 +1,129 @@
+/*
+ * 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.ingest.processor;
+
+import org.elasticsearch.action.ingest.SimulateProcessorResult;
+import org.elasticsearch.ingest.TestProcessor;
+import org.elasticsearch.ingest.core.CompoundProcessor;
+import org.elasticsearch.ingest.core.IngestDocument;
+import org.elasticsearch.test.ESTestCase;
+import org.junit.Before;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.elasticsearch.ingest.core.CompoundProcessor.ON_FAILURE_MESSAGE_FIELD;
+import static org.elasticsearch.ingest.core.CompoundProcessor.ON_FAILURE_PROCESSOR_TAG_FIELD;
+import static org.elasticsearch.ingest.core.CompoundProcessor.ON_FAILURE_PROCESSOR_TYPE_FIELD;
+import static org.elasticsearch.ingest.processor.TrackingResultProcessor.decorate;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.Matchers.nullValue;
+
+public class TrackingResultProcessorTests extends ESTestCase {
+
+    private IngestDocument ingestDocument;
+    private List<SimulateProcessorResult> resultList;
+
+    @Before
+    public void init() {
+        ingestDocument = new IngestDocument(new HashMap<>(), new HashMap<>());
+        resultList = new ArrayList<>();
+    }
+
+    public void testActualProcessor() throws Exception {
+        TestProcessor actualProcessor = new TestProcessor(ingestDocument -> {});
+        TrackingResultProcessor trackingProcessor = new TrackingResultProcessor(actualProcessor, resultList);
+        trackingProcessor.execute(ingestDocument);
+
+        SimulateProcessorResult expectedResult = new SimulateProcessorResult(actualProcessor.getTag(), ingestDocument);
+
+        assertThat(actualProcessor.getInvokedCounter(), equalTo(1));
+        assertThat(resultList.size(), equalTo(1));
+
+        assertThat(resultList.get(0).getIngestDocument(), equalTo(expectedResult.getIngestDocument()));
+        assertThat(resultList.get(0).getFailure(), nullValue());
+        assertThat(resultList.get(0).getProcessorTag(), equalTo(expectedResult.getProcessorTag()));
+    }
+
+    public void testActualCompoundProcessorWithoutOnFailure() throws Exception {
+        RuntimeException exception = new RuntimeException("processor failed");
+        TestProcessor testProcessor = new TestProcessor(ingestDocument -> {  throw exception; });
+        CompoundProcessor actualProcessor = new CompoundProcessor(testProcessor);
+        CompoundProcessor trackingProcessor = decorate(actualProcessor, resultList);
+
+        try {
+            trackingProcessor.execute(ingestDocument);
+        } catch (Exception e) {
+            assertThat(e.getMessage(), equalTo(exception.getMessage()));
+        }
+
+        SimulateProcessorResult expectedFirstResult = new SimulateProcessorResult(testProcessor.getTag(), ingestDocument);
+        assertThat(testProcessor.getInvokedCounter(), equalTo(1));
+        assertThat(resultList.size(), equalTo(1));
+        assertThat(resultList.get(0).getIngestDocument(), nullValue());
+        assertThat(resultList.get(0).getFailure(), equalTo(exception));
+        assertThat(resultList.get(0).getProcessorTag(), equalTo(expectedFirstResult.getProcessorTag()));
+    }
+
+    public void testActualCompoundProcessorWithOnFailure() throws Exception {
+        RuntimeException exception = new RuntimeException("fail");
+        TestProcessor failProcessor = new TestProcessor("fail", "test", ingestDocument -> {  throw exception; });
+        TestProcessor onFailureProcessor = new TestProcessor("success", "test", ingestDocument -> {});
+        CompoundProcessor actualProcessor = new CompoundProcessor(
+            Arrays.asList(new CompoundProcessor(
+                Arrays.asList(failProcessor, onFailureProcessor),
+                Arrays.asList(onFailureProcessor, failProcessor))),
+                Arrays.asList(onFailureProcessor));
+        CompoundProcessor trackingProcessor = decorate(actualProcessor, resultList);
+        trackingProcessor.execute(ingestDocument);
+
+        SimulateProcessorResult expectedFailResult = new SimulateProcessorResult(failProcessor.getTag(), ingestDocument);
+        SimulateProcessorResult expectedSuccessResult = new SimulateProcessorResult(onFailureProcessor.getTag(), ingestDocument);
+
+        assertThat(failProcessor.getInvokedCounter(), equalTo(2));
+        assertThat(onFailureProcessor.getInvokedCounter(), equalTo(2));
+        assertThat(resultList.size(), equalTo(4));
+
+        assertThat(resultList.get(0).getIngestDocument(), nullValue());
+        assertThat(resultList.get(0).getFailure(), equalTo(exception));
+        assertThat(resultList.get(0).getProcessorTag(), equalTo(expectedFailResult.getProcessorTag()));
+
+        Map<String, String> metadata = resultList.get(1).getIngestDocument().getIngestMetadata();
+        assertThat(metadata.get(ON_FAILURE_MESSAGE_FIELD), equalTo("fail"));
+        assertThat(metadata.get(ON_FAILURE_PROCESSOR_TYPE_FIELD), equalTo("test"));
+        assertThat(metadata.get(ON_FAILURE_PROCESSOR_TAG_FIELD), equalTo("fail"));
+        assertThat(resultList.get(1).getFailure(), nullValue());
+        assertThat(resultList.get(1).getProcessorTag(), equalTo(expectedSuccessResult.getProcessorTag()));
+
+        assertThat(resultList.get(2).getIngestDocument(), nullValue());
+        assertThat(resultList.get(2).getFailure(), equalTo(exception));
+        assertThat(resultList.get(2).getProcessorTag(), equalTo(expectedFailResult.getProcessorTag()));
+
+        metadata = resultList.get(3).getIngestDocument().getIngestMetadata();
+        assertThat(metadata.get(ON_FAILURE_MESSAGE_FIELD), equalTo("fail"));
+        assertThat(metadata.get(ON_FAILURE_PROCESSOR_TYPE_FIELD), equalTo("compound"));
+        assertThat(metadata.get(ON_FAILURE_PROCESSOR_TAG_FIELD), equalTo("CompoundProcessor-fail-success-success-fail"));
+        assertThat(resultList.get(3).getFailure(), nullValue());
+        assertThat(resultList.get(3).getProcessorTag(), equalTo(expectedSuccessResult.getProcessorTag()));
+    }
+}

+ 0 - 1
core/src/test/java/org/elasticsearch/search/rescore/QueryRescoreBuilderTests.java

@@ -69,7 +69,6 @@ public class QueryRescoreBuilderTests extends ESTestCase {
     @BeforeClass
     public static void init() {
         namedWriteableRegistry = new NamedWriteableRegistry();
-        namedWriteableRegistry.registerPrototype(RescoreBuilder.class, QueryRescorerBuilder.PROTOTYPE);
         indicesQueriesRegistry = new SearchModule(Settings.EMPTY, namedWriteableRegistry).buildQueryParserRegistry();
     }
 

+ 2 - 1
core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java

@@ -293,8 +293,9 @@ public class RandomShapeGenerator extends RandomGeoGenerator {
                 xDivisible(yRange.getMax()*10e3)/10e3);
     }
 
+    /** creates a small random rectangle by default to keep shape test performance at bay */
     public static Rectangle xRandomRectangle(Random r, Point nearP) {
-        return xRandomRectangle(r, nearP, ctx.getWorldBounds(), false);
+        return xRandomRectangle(r, nearP, ctx.getWorldBounds(), true);
     }
 
     public static Rectangle xRandomRectangle(Random r, Point nearP, boolean small) {

+ 4 - 0
distribution/src/main/packaging/scripts/postinst

@@ -96,4 +96,8 @@ elif [ "$RESTART_ON_UPGRADE" = "true" ]; then
     echo " OK"
 fi
 
+chown -R $ES_USER:$ES_GROUP /var/lib/elasticsearch
+chown -R $ES_USER:$ES_GROUP /var/log/elasticsearch
+chown -R root:$ES_GROUP /etc/elasticsearch
+
 ${scripts.footer}

+ 8 - 0
docs/reference/migration/migrate_5_0/settings.asciidoc

@@ -210,3 +210,11 @@ setting settings via `--name.of.setting value.of.setting`. This feature
 has been removed. Instead, use
 `-Ees.name.of.setting=value.of.setting`. Note that in all cases the
 name of the setting must be prefixed with `es.`.
+
+==== Discovery Settings
+
+The `discovery.zen.minimum_master_node` must bet set for nodes that are bound
+to a non-loopback network interface. We see those nodes as in "production" mode and
+thus require the setting.
+
+

+ 167 - 132
docs/reference/modules/painless.asciidoc

@@ -1,19 +1,21 @@
 [[modules-scripting-painless]]
 == Painless Scripting Language
 
-_Painless_ is a simple, secure scripting language built in to Elasticsearch as a module. 
+experimental[The Painless scripting language is new and is still marked as experimental. The syntax or API may be changed in the future in non-backwards compatible ways if required.]
+
+_Painless_ is a simple, secure scripting language built in to Elasticsearch as a module.
 It is designed specifically for use with Elasticsearch and can safely be used dynamically.
 
-A Painless script is essentially a single function. Painless does not provide support 
-for defining multiple functions within a script. The Painless syntax is similar to 
-http://groovy-lang.org/index.html[Groovy]. 
+A Painless script is essentially a single function. Painless does not provide support
+for defining multiple functions within a script. The Painless syntax is similar to
+http://groovy-lang.org/index.html[Groovy].
 
-You can use Painless anywhere a script can be used in Elasticsearch--simply set the `lang` parameter 
+You can use Painless anywhere a script can be used in Elasticsearch--simply set the `lang` parameter
 to `painless`.
 
 [[painless-features]]
 [float]
-=== Painless Features
+== Painless Features
 
 * Control flow: `for` loops, `while` loops, `do/while` loops, `if/else`
 
@@ -32,196 +34,229 @@ to `painless`.
 
 [[painless-examples]]
 [float]
-=== Painless Examples
+== Painless Examples
 
 To illustrate how Painless works, let's load some hockey stats into an Elasticsearch index:
 
 [source,sh]
 ----------------------------------------------------------------
-curl -XDELETE http://localhost:9200/hockey-stats
-curl -XPUT http://localhost:9200/hockey-stats
-curl -XPUT http://localhost:9200/hockey-stats/player/1 -d '{"first":"johnny", "last":"gaudreau", "goals":[9, 27, 1], "assists":[17, 46, 0], "gp":[26, 82, 1]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/2 -d '{"first":"sean", "last":"monohan", "goals":[7, 54, 26], "assists":[11, 26, 13], "gp":[26, 82, 82]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/3 -d '{"first":"jiri", "last":"hudler", "goals":[5, 34, 36], "assists":[11, 62, 42], "gp":[24, 80, 79]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/4 -d '{"first":"micheal", "last":"frolik", "goals":[4, 6, 15], "assists":[8, 23, 15], "gp":[26, 82, 82]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/5 -d '{"first":"sam", "last":"bennett", "goals":[5, 0, 0], "assists":[8, 1, 0], "gp":[26, 1, 0]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/6 -d '{"first":"dennis", "last":"wideman", "goals":[0, 26, 15], "assists":[11, 30, 24], "gp":[26, 81, 82]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/7 -d '{"first":"david", "last":"jones", "goals":[7, 19, 5], "assists":[3, 17, 4], "gp":[26, 45, 34]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/8 -d '{"first":"tj", "last":"brodie", "goals":[2, 14, 7], "assists":[8, 42, 30], "gp":[26, 82, 82]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/9 -d '{"first":"mark", "last":"giordano", "goals":[6, 30, 15], "assists":[3, 30, 24], "gp":[26, 60, 63]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/10 -d '{"first":"mikael", "last":"backlund", "goals":[3, 15, 13], "assists":[6, 24, 18], "gp":[26, 82, 82]}'
-curl -XPUT http://localhost:9200/hockey-stats/player/11 -d '{"first":"joe", "last":"colborne", "goals":[3, 18, 13], "assists":[6, 20, 24], "gp":[26, 67, 82]}'
+DELETE /hockey-stats
+
+PUT /hockey-stats/player/_bulk
+{"index":{"_id":1}}
+{"first":"johnny","last":"gaudreau","goals":[9,27,1],"assists":[17,46,0],"gp":[26,82,1]}
+{"index":{"_id":2}}
+{"first":"sean","last":"monohan","goals":[7,54,26],"assists":[11,26,13],"gp":[26,82,82]}
+{"index":{"_id":3}}
+{"first":"jiri","last":"hudler","goals":[5,34,36],"assists":[11,62,42],"gp":[24,80,79]}
+{"index":{"_id":4}}
+{"first":"micheal","last":"frolik","goals":[4,6,15],"assists":[8,23,15],"gp":[26,82,82]}
+{"index":{"_id":5}}
+{"first":"sam","last":"bennett","goals":[5,0,0],"assists":[8,1,0],"gp":[26,1,0]}
+{"index":{"_id":6}}
+{"first":"dennis","last":"wideman","goals":[0,26,15],"assists":[11,30,24],"gp":[26,81,82]}
+{"index":{"_id":7}}
+{"first":"david","last":"jones","goals":[7,19,5],"assists":[3,17,4],"gp":[26,45,34]}
+{"index":{"_id":8}}
+{"first":"tj","last":"brodie","goals":[2,14,7],"assists":[8,42,30],"gp":[26,82,82]}
+{"index":{"_id":39}}
+{"first":"mark","last":"giordano","goals":[6,30,15],"assists":[3,30,24],"gp":[26,60,63]}
+{"index":{"_id":10}}
+{"first":"mikael","last":"backlund","goals":[3,15,13],"assists":[6,24,18],"gp":[26,82,82]}
+{"index":{"_id":11}}
+{"first":"joe","last":"colborne","goals":[3,18,13],"assists":[6,20,24],"gp":[26,67,82]}
 ----------------------------------------------------------------
+// AUTOSENSE
 
 [float]
-==== Accessing Doc Values from Painless
+=== Accessing Doc Values from Painless
 
-All Painless scripts take in a `Map<String,def>` of values called `input`.  Document values can be accessed through another `Map<String,def>` within the `input` variable.  
+All Painless scripts take in a `Map<String,def>` of values called `input`.  Document values can be accessed through another `Map<String,def>` within the `input` variable.
 
-For example, the following script calculates a player's total goals. This example uses a strongly typed `int` and a `for` loop.  
+For example, the following script calculates a player's total goals. This example uses a strongly typed `int` and a `for` loop.
 
 [source,sh]
 ----------------------------------------------------------------
-curl -XGET http://localhost:9200/hockey-stats/_search -d '{
-   "query": {
-      "function_score": {
-         "script_score" : {
-            "script" : {
-               "inline": 
-                  "int total = 0; for (int i = 0; i < input.doc.goals.size(); ++i) { total += input.doc.goals[i]; } return total;", 
-               "lang": "painless"
-            }
+GET /hockey-stats/_search
+{
+  "query": {
+    "function_score": {
+      "script_score": {
+        "script": {
+          "lang": "painless",
+          "inline": "int total = 0; for (int i = 0; i < input.doc.goals.size(); ++i) { total += input.doc.goals[i]; } return total;"
         }
+      }
     }
-   }
-}'
+  }
+}
 ----------------------------------------------------------------
+// AUTOSENSE
 
 Alternatively, you could do the same thing using a script field instead of a function score:
 
 [source,sh]
 ----------------------------------------------------------------
-curl -XGET http://localhost:9200/hockey-stats/_search -d '{
-   "query": {
-      "match_all": {}}, 
-      "script_fields": {
-         "total_goals": {
-            "script": {
-               "inline": "int total = 0; for (int i = 0; i < input.doc.goals.size(); ++i) { total += input.doc.goals[i]; } return total;", 
-               "lang": "painless"
-            }
-        }
+GET /hockey-stats/_search
+{
+  "query": {
+    "match_all": {}
+  },
+  "script_fields": {
+    "total_goals": {
+      "script": {
+        "lang": "painless",
+        "inline": "int total = 0; for (int i = 0; i < input.doc.goals.size(); ++i) { total += input.doc.goals[i]; } return total;"
+      }
     }
-}'
+  }
+}
 ----------------------------------------------------------------
+// AUTOSENSE
 
-You must always specify the index of the field value you want, even if there's only a single item in the field. 
+You must always specify the index of the field value you want, even if there's only a single item in the field.
 All fields in Elasticsearch are multi-valued and Painless does not provide a `.value` shortcut. The following example uses a Painless script to sort the players by their combined first and last names. The names are accessed using
-`input.doc.first.0` and `input.doc.last.0`.  
+`input.doc.first.0` and `input.doc.last.0`.
 
 [source,sh]
 ----------------------------------------------------------------
-curl -XGET http://localhost:9200/hockey-stats/_search -d '{
-   "query" : {
-      "match_all": {}}, 
-      "sort" : {
-         "_script" : {
-            "type" : "string", 
-            "script" : {"inline": "input.doc.first.0 + \" \" + input.doc.last.0", 
-            "lang": "painless"}, 
-            "order" : "asc"
-        }
+GET /hockey-stats/_search
+{
+  "query": {
+    "match_all": {}
+  },
+  "sort": {
+    "_script": {
+      "type": "string",
+      "order": "asc",
+      "script": {
+        "lang": "painless",
+        "inline": "input.doc.first.0 + \" \" + input.doc.last.0"
+      }
     }
-}'
+  }
+}
 ----------------------------------------------------------------
+// AUTOSENSE
 
 [float]
-==== Updating Fields with Painless 
+=== Updating Fields with Painless
 
-You can also easily update fields. You access the original source for a field as `input.ctx._source.<field-name>`. 
+You can also easily update fields. You access the original source for a field as `input.ctx._source.<field-name>`.
 
 First, let's look at the source data for a player by submitting the following request:
 
 [source,sh]
 ----------------------------------------------------------------
-curl -XGET http://localhost:9200/hockey-stats/_search -d '{
-   "fields" : ["_id", "_source"], "query" : {
-      "term" : { "_id" : 1 }
+GET /hockey-stats/_search
+{
+  "fields": [
+    "_id",
+    "_source"
+  ],
+  "query": {
+    "term": {
+      "_id": 1
     }
-}'
+  }
+}
 ----------------------------------------------------------------
+// AUTOSENSE
 
-To change player 1's last name to _hockey_, simply set `input.ctx._source.last` to the new value:
+To change player 1's last name to `hockey`, simply set `input.ctx._source.last` to the new value:
 
 [source,sh]
 ----------------------------------------------------------------
-curl -XPOST http://localhost:9200/hockey-stats/player/1/_update -d '{
-   "script": {
-      "inline": "input.ctx._source.last = input.last", 
-      "params": {"last": "hockey"}, 
-      "lang": "painless"
-   }
-}'
+POST /hockey-stats/player/1/_update
+{
+  "script": {
+    "lang": "painless",
+    "inline": "input.ctx._source.last = input.last",
+    "params": {
+      "last": "hockey"
+    }
+  }
+}
 ----------------------------------------------------------------
+// AUTOSENSE
 
-You can also add fields to a document. For example, this script adds a new field that contains 
+You can also add fields to a document. For example, this script adds a new field that contains
 the player's nickname,  _hockey_.
 
 [source,sh]
 ----------------------------------------------------------------
-curl -XPOST http://localhost:9200/hockey-stats/player/1/_update -d '{
-   "script": {
-      "inline": "input.ctx._source.last = input.last input.ctx._source.nick = input.nick", 
-      "params": {"last": "gaudreau", "nick": "hockey"}, 
-      "lang": "painless"
-   }
-}'
+POST /hockey-stats/player/1/_update
+{
+  "script": {
+    "lang": "painless",
+    "inline": "input.ctx._source.last = input.last input.ctx._source.nick = input.nick",
+    "params": {
+      "last": "gaudreau",
+      "nick": "hockey"
+    }
+  }
+}
 ----------------------------------------------------------------
+// AUTOSENSE
 
 [float]
-==== Writing Type-Safe Scripts to Improve Performance
+=== Writing Type-Safe Scripts to Improve Performance
 
-If you explicitly specify types, the compiler doesn't have to perform type lookups at runtime, which can significantly 
-improve performance. For example, the following script performs the same first name, last name sort we showed before, 
+If you explicitly specify types, the compiler doesn't have to perform type lookups at runtime, which can significantly
+improve performance. For example, the following script performs the same first name, last name sort we showed before,
 but it's fully type-safe.
 
 [source,sh]
 ----------------------------------------------------------------
-curl -XGET http://localhost:9200/hockey-stats/_search -d '{
-   "query": {
-      "match_all": {}
-   }, 
-   "script_fields": {
-      "full_name_dynamic": {
-         "script": {
-            "inline": "def first = input.doc.first.0; def last = input.doc.last.0; return first + \" \" + last;", 
-            "lang": "painless"
-         }
-      }, 
-      "full_name_static": {
-         "script": {
-            "inline": 
-               "String first = (String)((List)((Map)input.get(\"doc\")).get(\"first\")).get(0); String last = (String)((List)((Map)input.get(\"doc\")).get(\"last\")).get(0); return first + \" \" + last;", 
-           "lang": "painless"
-         }
-       }
+GET /hockey-stats/_search
+{
+  "query": {
+    "match_all": {}
+  },
+  "script_fields": {
+    "full_name_dynamic": {
+      "script": {
+        "lang": "painless",
+        "inline": "def first = input.doc.first.0; def last = input.doc.last.0; return first + \" \" + last;"
+      }
+    },
+    "full_name_static": {
+      "script": {
+        "lang": "painless",
+        "inline": "String first = (String)((List)((Map)input.get(\"doc\")).get(\"first\")).get(0); String last = (String)((List)((Map)input.get(\"doc\")).get(\"last\")).get(0); return first + \" \" + last;"
+      }
     }
-}'
+  }
+}
 ----------------------------------------------------------------
+// AUTOSENSE
 
 [[painless-api]]
 [float]
-=== Painless API
+== Painless API
 
 The following types are available for use in the Painless language. Most types and methods map directly to their Java equivalents--for more information, see the corresponding https://docs.oracle.com/javase/8/docs/api/java/lang/package-summary.html[Javadoc].
 
 
 [float]
-==== Dynamic Types
+=== Dynamic Types
 
-`def` (This type can be used to represent any other type.)
+* `def` (This type can be used to represent any other type.)
 
 [float]
-==== Basic Types
-
-`void`
-
-`boolean`
-
-`short`
-
-`char`
+=== Basic Types
 
-`int`
-
-`long`
-
-`float`
-
-`double`
+* `void`
+* `boolean`
+* `short`
+* `char`
+* `int`
+* `long`
+* `float`
+* `double`
 
 [float]
-==== Complex Types
+=== Complex Types
 
 Non-static methods/members in superclasses are available to subclasses.
 Generic types with unspecified generic parameters are parameters of type `def`.
@@ -242,7 +277,7 @@ ArrayList<Object> extends List<Object>
 -----
 
 -----
-ArrayList<String> extends List<String>    
+ArrayList<String> extends List<String>
     <init>()
 -----
 
@@ -254,13 +289,13 @@ Boolean extends Object
 -----
 
 -----
-Character extends Object    
+Character extends Object
     <init>(char)
     static Character valueOf(char)
     char charValue()
     static char MIN_VALUE
     static char MAX_VALUE
------    
+-----
 
 -----
 CharSequence extends Object
@@ -311,7 +346,7 @@ Double extends Number
 
 -----
 Exception extends Object
-    String getMessage()    
+    String getMessage()
 -----
 
 -----
@@ -324,7 +359,7 @@ Float extends Number
 
 -----
 HashMap extends Map
-    <init>()    
+    <init>()
 -----
 
 -----
@@ -334,14 +369,14 @@ HashMap<Object,Object> extends Map<Object,Object>
 
 -----
 HashMap<String,def> extends Map<String,def>
-    <init>()    
+    <init>()
 -----
 
 -----
 HashMap<String,Object> extends Map<String,Object>
     <init>()
 -----
-    
+
 -----
 IllegalArgument extends Exception
     <init>()
@@ -349,7 +384,7 @@ IllegalArgument extends Exception
 
 -----
 IllegalState extends Exception
-    <init>()    
+    <init>()
 -----
 
 -----
@@ -413,7 +448,7 @@ Map extends Object
     boolean containsKey(def)
     boolean containsValue(def)
     Set keySet()
-    Collection values()    
+    Collection values()
 -----
 
 -----
@@ -549,7 +584,7 @@ Math
    static double min(double, double)
    static float fmin(float, float)
    static long lmin(long, long)
-   static int imin(int, int)   
+   static int imin(int, int)
    static double pow(double, double)
    static double random()
    static double rint(double)

+ 3 - 3
docs/reference/search/percolate.asciidoc

@@ -1,8 +1,8 @@
 [[search-percolate]]
 == Percolator
 
-added[5.0.0,Percolator queries modifications aren't visible immediately and a refresh is required]
+deprecated[5.0.0,Percolate and multi percolate APIs are deprecated and have been replaced by the new <<query-dsl-percolator-query,`percolator` query>>]
 
-added[5.0.0,Percolate and multi percolate APIs have been deprecated and has been replaced by <<query-dsl-percolator-query, the new `percolator` query>>]
+added[5.0.0,Percolator query modifications only become visible after a refresh has occurred. Previously, they became visible immediately]
 
-added[5.0.0,For indices created on or after version 5.0.0 the percolator automatically indexes the query terms with the percolator queries this allows the percolator to percolate documents quicker. It is advisable to reindex any pre 5.0.0 indices to take advantage of this new optimization]
+added[5.0.0,For indices created on or after version 5.0.0-alpha1 the percolator automatically indexes the query terms with the percolator queries. This allows the percolator to percolate documents more quickly. It is advisable to reindex any pre 5.0.0 indices to take advantage of this new optimization]

+ 52 - 0
qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/10_pipeline_with_mustache_templates.yaml

@@ -274,3 +274,55 @@
         id: 1
   - length: { _source: 2 }
   - match: { _source.values_flat: ["foo_bar", "foo_baz"] }
+
+---
+"Test verbose simulate with error context":
+  - do:
+      cluster.health:
+          wait_for_status: green
+  - do:
+      ingest.simulate:
+        verbose: true
+        body: >
+          {
+            "pipeline" : {
+              "description": "_description",
+              "processors": [
+                {
+                  "rename" : {
+                    "tag" : "rename-status",
+                    "field" : "status",
+                    "to" : "bar",
+                    "on_failure" : [
+                      {
+                        "set" : {
+                          "tag" : "set_on_rename_failure",
+                          "field" : "error",
+                          "value" : "processor {{ _ingest.on_failure_processor_tag }} [{{ _ingest.on_failure_processor_type }}]: {{ _ingest.on_failure_message }}"
+                        }
+                      }
+                    ]
+                  }
+                }
+              ]
+            },
+            "docs": [
+              {
+                "_index": "index",
+                "_type": "type",
+                "_id": "id",
+                "_source": {
+                  "foo": "bar"
+                }
+              }
+            ]
+          }
+  - length: { docs: 1 }
+  - length: { docs.0.processor_results: 2 }
+  - match: { docs.0.processor_results.0.tag: "rename-status" }
+  - match: { docs.0.processor_results.0.error.type: "illegal_argument_exception" }
+  - match: { docs.0.processor_results.0.error.reason: "field [status] doesn't exist" }
+  - match: { docs.0.processor_results.1.tag: "set_on_rename_failure" }
+  - length: { docs.0.processor_results.1.doc._source: 2 }
+  - match: { docs.0.processor_results.1.doc._source.foo: "bar"  }
+  - match: { docs.0.processor_results.1.doc._source.error: "processor rename-status [rename]: field [status] doesn't exist" }

+ 0 - 2
qa/vagrant/src/test/resources/packaging/scripts/80_upgrade.bats

@@ -42,7 +42,6 @@ setup() {
 @test "[UPGRADE] install old version" {
     clean_before_test
     install_package -v $(cat upgrade_from_version)
-    perl -p -i -e 's/es.logger.level: INFO/es.logger.level: DEBUG/' /etc/elasticsearch/logging.yml
 }
 
 @test "[UPGRADE] start old version" {
@@ -81,7 +80,6 @@ setup() {
 
 @test "[UPGRADE] install version under test" {
     install_package -u
-    perl -p -i -e 's/es.logger.level: INFO/es.logger.level: DEBUG/' /etc/elasticsearch/logging.yml
 }
 
 @test "[UPGRADE] start version under test" {

+ 72 - 0
qa/vagrant/src/test/resources/packaging/scripts/90_reinstall.bats

@@ -0,0 +1,72 @@
+#!/usr/bin/env bats
+
+# Tests upgrading elasticsearch from a previous version with the deb or rpm
+# packages. Just uses a single node cluster on the current machine rather than
+# fancy rolling restarts.
+
+# WARNING: This testing file must be executed as root and can
+# dramatically change your system. It removes the 'elasticsearch'
+# user/group and also many directories. Do not execute this file
+# unless you know exactly what you are doing.
+
+# The test case can be executed with the Bash Automated
+# Testing System tool available at https://github.com/sstephenson/bats
+# Thanks to Sam Stephenson!
+
+# 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.
+
+# Load test utilities
+load packaging_test_utils
+load os_package
+
+# Cleans everything for the 1st execution
+setup() {
+    skip_not_dpkg_or_rpm
+}
+
+@test "[REINSTALL] install" {
+    clean_before_test
+    install_package
+}
+
+@test "[REINSTALL] purge elasticsearch" {
+    purge_elasticsearch
+}
+
+@test "[REINSTALL] chown directories" {
+    # to simulate the loss of ownership
+    if [ -d /var/lib/elasticsearch ]; then
+      sudo chown -R root:root /var/lib/elasticsearch
+    fi
+    if [ -d "/var/log/elasticsearch" ]; then
+      sudo chown -R root:root /var/log/elasticsearch
+    fi
+    if [ -d /etc/elasticsearch ]; then
+      sudo chown -R root:root /etc/elasticsearch
+    fi
+}
+
+@test "[REINSTALL] reinstall elasticsearch" {
+    install_package
+}
+
+@test "[REINSTALL] check ownership" {
+    assert_recursive_ownership /var/lib/elasticsearch elasticsearch elasticsearch
+    assert_recursive_ownership /var/log/elasticsearch elasticsearch elasticsearch
+    assert_recursive_ownership /etc/elasticsearch root elasticsearch
+}

+ 27 - 18
qa/vagrant/src/test/resources/packaging/scripts/packaging_test_utils.bash

@@ -209,6 +209,17 @@ assert_output() {
     echo "$output" | grep -E "$1"
 }
 
+assert_recursive_ownership() {
+    local directory=$1
+    local user=$2
+    local group=$3
+
+    realuser=$(find $directory -printf "%u\n" | sort | uniq)
+    [ "$realuser" = "$user" ]
+    realgroup=$(find $directory -printf "%g\n" | sort | uniq)
+    [ "$realgroup" = "$group" ]
+}
+
 # Deletes everything before running a test file
 clean_before_test() {
 
@@ -235,6 +246,22 @@ clean_before_test() {
     # Kills all running Elasticsearch processes
     ps aux | grep -i "org.elasticsearch.bootstrap.Elasticsearch" | awk {'print $2'} | xargs kill -9 > /dev/null 2>&1 || true
 
+    purge_elasticsearch
+
+    # Removes user & group
+    userdel elasticsearch > /dev/null 2>&1 || true
+    groupdel elasticsearch > /dev/null 2>&1 || true
+
+
+    # Removes all files
+    for d in "${ELASTICSEARCH_TEST_FILES[@]}"; do
+        if [ -e "$d" ]; then
+            rm -rf "$d"
+        fi
+    done
+}
+
+purge_elasticsearch() {
     # Removes RPM package
     if is_rpm; then
         rpm --quiet -e elasticsearch > /dev/null 2>&1 || true
@@ -252,18 +279,6 @@ clean_before_test() {
     if [ -x "`which apt-get 2>/dev/null`" ]; then
         apt-get --quiet --yes purge elasticsearch > /dev/null 2>&1 || true
     fi
-
-    # Removes user & group
-    userdel elasticsearch > /dev/null 2>&1 || true
-    groupdel elasticsearch > /dev/null 2>&1 || true
-
-
-    # Removes all files
-    for d in "${ELASTICSEARCH_TEST_FILES[@]}"; do
-        if [ -e "$d" ]; then
-            rm -rf "$d"
-        fi
-    done
 }
 
 # Start elasticsearch and wait for it to come up with a status.
@@ -385,10 +400,6 @@ wait_for_elasticsearch_status() {
     local desiredStatus=${1:-green}
     local index=$2
 
-    if [ -f /var/log/elasitcsearch/elasticsearch.log ]; then
-      cat /var/log/elasticsearch/elasticsearch.log >&3
-    fi
-
     echo "Making sure elasticsearch is up..."
     wget -O - --retry-connrefused --waitretry=1 --timeout=60 --tries 60 http://localhost:9200/_cluster/health || {
           echo "Looks like elasticsearch never started. Here is its log:"
@@ -398,7 +409,6 @@ wait_for_elasticsearch_status() {
               echo "The elasticsearch log doesn't exist. Maybe /var/log/messages has something:"
               tail -n20 /var/log/messages
           fi
-          cat /var/log/elasticsearch/elasticsearch.log >&3
           false
     }
 
@@ -413,7 +423,6 @@ wait_for_elasticsearch_status() {
         echo "Connected"
     else
         echo "Unable to connect to Elastisearch"
-        cat /var/log/elasticsearch/elasticsearch.log >&3
         false
     fi
 

+ 1 - 0
rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json

@@ -8,6 +8,7 @@
       "parts": {
         "repository": {
           "type" : "list",
+          "required": true,
           "description": "Name of repository from which to fetch the snapshot information"
         }
       },

+ 1 - 1
test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java

@@ -30,7 +30,7 @@ import org.elasticsearch.test.ESTestCase;
 public class TestShardRouting {
 
     public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, boolean primary, ShardRoutingState state) {
-        return newShardRouting(new Index(index, IndexMetaData.INDEX_UUID_NA_VALUE), shardId, currentNodeId,primary, state);
+        return newShardRouting(new Index(index, IndexMetaData.INDEX_UUID_NA_VALUE), shardId, currentNodeId, primary, state);
     }
 
     public static ShardRouting newShardRouting(Index index, int shardId, String currentNodeId, boolean primary, ShardRoutingState state) {

+ 3 - 1
test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java

@@ -30,6 +30,7 @@ import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.Version;
 import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
 import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
+import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag;
 import org.elasticsearch.cache.recycler.PageCacheRecycler;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.client.transport.TransportClient;
@@ -1840,7 +1841,8 @@ public final class InternalTestCluster extends TestCluster {
                 }
 
                 NodeService nodeService = getInstanceFromNode(NodeService.class, nodeAndClient.node);
-                NodeStats stats = nodeService.stats(CommonStatsFlags.ALL, false, false, false, false, false, false, false, false, false, false, false);
+                CommonStatsFlags flags = new CommonStatsFlags(Flag.FieldData, Flag.QueryCache, Flag.Segments);
+                NodeStats stats = nodeService.stats(flags, false, false, false, false, false, false, false, false, false, false, false);
                 assertThat("Fielddata size must be 0 on node: " + stats.getNode(), stats.getIndices().getFieldData().getMemorySizeInBytes(), equalTo(0L));
                 assertThat("Query cache size must be 0 on node: " + stats.getNode(), stats.getIndices().getQueryCache().getMemorySizeInBytes(), equalTo(0L));
                 assertThat("FixedBitSet cache size must be 0 on node: " + stats.getNode(), stats.getIndices().getSegments().getBitsetMemoryInBytes(), equalTo(0L));