瀏覽代碼

Introduce mapping version to index metadata (#33147)

This commit introduces mapping version to index metadata. This value is
monotonically increasing and is updated on mapping updates. This will be
useful in cross-cluster replication so that we can request mapping
updates from the leader only when there is a mapping update as opposed
to the strategy we employ today which is to request a mapping update any
time there is an index metadata update. As index metadata updates can
occur for many reasons other than mapping updates, this leads to some
unnecessary requests and work in cross-cluster replication.
Jason Tedor 7 年之前
父節點
當前提交
2aef7e0900

+ 1 - 1
server/src/main/java/org/elasticsearch/cluster/ClusterState.java

@@ -284,7 +284,7 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
         final String TAB = "   ";
         for (IndexMetaData indexMetaData : metaData) {
             sb.append(TAB).append(indexMetaData.getIndex());
-            sb.append(": v[").append(indexMetaData.getVersion()).append("]\n");
+            sb.append(": v[").append(indexMetaData.getVersion()).append("], mv[").append(indexMetaData.getMappingVersion()).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("], ");

+ 50 - 2
server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java

@@ -24,6 +24,7 @@ import com.carrotsearch.hppc.cursors.IntObjectCursor;
 import com.carrotsearch.hppc.cursors.ObjectCursor;
 import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
 
+import org.elasticsearch.Assertions;
 import org.elasticsearch.Version;
 import org.elasticsearch.action.admin.indices.rollover.RolloverInfo;
 import org.elasticsearch.action.support.ActiveShardCount;
@@ -291,6 +292,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
 
     public static final String KEY_IN_SYNC_ALLOCATIONS = "in_sync_allocations";
     static final String KEY_VERSION = "version";
+    static final String KEY_MAPPING_VERSION = "mapping_version";
     static final String KEY_ROUTING_NUM_SHARDS = "routing_num_shards";
     static final String KEY_SETTINGS = "settings";
     static final String KEY_STATE = "state";
@@ -309,6 +311,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
 
     private final Index index;
     private final long version;
+
+    private final long mappingVersion;
+
     private final long[] primaryTerms;
 
     private final State state;
@@ -336,7 +341,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
     private final ActiveShardCount waitForActiveShards;
     private final ImmutableOpenMap<String, RolloverInfo> rolloverInfos;
 
-    private IndexMetaData(Index index, long version, long[] primaryTerms, State state, int numberOfShards, int numberOfReplicas, Settings settings,
+    private IndexMetaData(Index index, long version, long mappingVersion, 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>> inSyncAllocationIds,
                           DiscoveryNodeFilters requireFilters, DiscoveryNodeFilters initialRecoveryFilters, DiscoveryNodeFilters includeFilters, DiscoveryNodeFilters excludeFilters,
@@ -345,6 +350,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
 
         this.index = index;
         this.version = version;
+        assert mappingVersion >= 0 : mappingVersion;
+        this.mappingVersion = mappingVersion;
         this.primaryTerms = primaryTerms;
         assert primaryTerms.length == numberOfShards;
         this.state = state;
@@ -394,6 +401,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
         return this.version;
     }
 
+    public long getMappingVersion() {
+        return mappingVersion;
+    }
 
     /**
      * The term of the current selected primary. This is a non-negative number incremented when
@@ -644,6 +654,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
         private final String index;
         private final int routingNumShards;
         private final long version;
+        private final long mappingVersion;
         private final long[] primaryTerms;
         private final State state;
         private final Settings settings;
@@ -656,6 +667,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
         IndexMetaDataDiff(IndexMetaData before, IndexMetaData after) {
             index = after.index.getName();
             version = after.version;
+            mappingVersion = after.mappingVersion;
             routingNumShards = after.routingNumShards;
             state = after.state;
             settings = after.settings;
@@ -672,6 +684,11 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
             index = in.readString();
             routingNumShards = in.readInt();
             version = in.readLong();
+            if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
+                mappingVersion = in.readVLong();
+            } else {
+                mappingVersion = 1;
+            }
             state = State.fromId(in.readByte());
             settings = Settings.readSettingsFromStream(in);
             primaryTerms = in.readVLongArray();
@@ -707,6 +724,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
             out.writeString(index);
             out.writeInt(routingNumShards);
             out.writeLong(version);
+            if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
+                out.writeVLong(mappingVersion);
+            }
             out.writeByte(state.id);
             Settings.writeSettingsToStream(settings, out);
             out.writeVLongArray(primaryTerms);
@@ -723,6 +743,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
         public IndexMetaData apply(IndexMetaData part) {
             Builder builder = builder(index);
             builder.version(version);
+            builder.mappingVersion(mappingVersion);
             builder.setRoutingNumShards(routingNumShards);
             builder.state(state);
             builder.settings(settings);
@@ -739,6 +760,11 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
     public static IndexMetaData readFrom(StreamInput in) throws IOException {
         Builder builder = new Builder(in.readString());
         builder.version(in.readLong());
+        if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
+            builder.mappingVersion(in.readVLong());
+        } else {
+            builder.mappingVersion(1);
+        }
         builder.setRoutingNumShards(in.readInt());
         builder.state(State.fromId(in.readByte()));
         builder.settings(readSettingsFromStream(in));
@@ -778,6 +804,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
     public void writeTo(StreamOutput out) throws IOException {
         out.writeString(index.getName()); // uuid will come as part of settings
         out.writeLong(version);
+        if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
+            out.writeVLong(mappingVersion);
+        }
         out.writeInt(routingNumShards);
         out.writeByte(state.id());
         writeSettingsToStream(settings, out);
@@ -821,6 +850,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
         private String index;
         private State state = State.OPEN;
         private long version = 1;
+        private long mappingVersion = 1;
         private long[] primaryTerms = null;
         private Settings settings = Settings.Builder.EMPTY_SETTINGS;
         private final ImmutableOpenMap.Builder<String, MappingMetaData> mappings;
@@ -843,6 +873,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
             this.index = indexMetaData.getIndex().getName();
             this.state = indexMetaData.state;
             this.version = indexMetaData.version;
+            this.mappingVersion = indexMetaData.mappingVersion;
             this.settings = indexMetaData.getSettings();
             this.primaryTerms = indexMetaData.primaryTerms.clone();
             this.mappings = ImmutableOpenMap.builder(indexMetaData.mappings);
@@ -1009,6 +1040,15 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
             return this;
         }
 
+        public long mappingVersion() {
+            return mappingVersion;
+        }
+
+        public Builder mappingVersion(final long mappingVersion) {
+            this.mappingVersion = mappingVersion;
+            return this;
+        }
+
         /**
          * returns the primary term for the given shard.
          * See {@link IndexMetaData#primaryTerm(int)} for more information.
@@ -1136,7 +1176,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
 
             final String uuid = settings.get(SETTING_INDEX_UUID, INDEX_UUID_NA_VALUE);
 
-            return new IndexMetaData(new Index(index, uuid), version, primaryTerms, state, numberOfShards, numberOfReplicas, tmpSettings, mappings.build(),
+            return new IndexMetaData(new Index(index, uuid), version, mappingVersion, primaryTerms, state, numberOfShards, numberOfReplicas, tmpSettings, mappings.build(),
                 tmpAliases.build(), customs.build(), filledInSyncAllocationIds.build(), requireFilters, initialRecoveryFilters, includeFilters, excludeFilters,
                 indexCreatedVersion, indexUpgradedVersion, getRoutingNumShards(), routingPartitionSize, waitForActiveShards, rolloverInfos.build());
         }
@@ -1145,6 +1185,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
             builder.startObject(indexMetaData.getIndex().getName());
 
             builder.field(KEY_VERSION, indexMetaData.getVersion());
+            builder.field(KEY_MAPPING_VERSION, indexMetaData.getMappingVersion());
             builder.field(KEY_ROUTING_NUM_SHARDS, indexMetaData.getRoutingNumShards());
             builder.field(KEY_STATE, indexMetaData.getState().toString().toLowerCase(Locale.ENGLISH));
 
@@ -1218,6 +1259,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
             if (token != XContentParser.Token.START_OBJECT) {
                 throw new IllegalArgumentException("expected object but got a " + token);
             }
+            boolean mappingVersion = false;
             while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
                 if (token == XContentParser.Token.FIELD_NAME) {
                     currentFieldName = parser.currentName();
@@ -1316,6 +1358,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
                         builder.state(State.fromString(parser.text()));
                     } else if (KEY_VERSION.equals(currentFieldName)) {
                         builder.version(parser.longValue());
+                    } else if (KEY_MAPPING_VERSION.equals(currentFieldName)) {
+                        mappingVersion = true;
+                        builder.mappingVersion(parser.longValue());
                     } else if (KEY_ROUTING_NUM_SHARDS.equals(currentFieldName)) {
                         builder.setRoutingNumShards(parser.intValue());
                     } else {
@@ -1325,6 +1370,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, ToXContentFragmen
                     throw new IllegalArgumentException("Unexpected token " + token);
                 }
             }
+            if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(Version.V_7_0_0_alpha1)) {
+                assert mappingVersion : "mapping version should be present for indices created on or after 7.0.0";
+            }
             return builder.build();
         }
     }

+ 12 - 2
server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java

@@ -287,6 +287,7 @@ public class MetaDataMappingService extends AbstractComponent {
             MetaData.Builder builder = MetaData.builder(metaData);
             boolean updated = false;
             for (IndexMetaData indexMetaData : updateList) {
+                boolean updatedMapping = false;
                 // do the actual merge here on the master, and update the mapping source
                 // we use the exact same indexService and metadata we used to validate above here to actually apply the update
                 final Index index = indexMetaData.getIndex();
@@ -303,7 +304,7 @@ public class MetaDataMappingService extends AbstractComponent {
                     if (existingSource.equals(updatedSource)) {
                         // same source, no changes, ignore it
                     } else {
-                        updated = true;
+                        updatedMapping = true;
                         // use the merged mapping source
                         if (logger.isDebugEnabled()) {
                             logger.debug("{} update_mapping [{}] with source [{}]", index, mergedMapper.type(), updatedSource);
@@ -313,7 +314,7 @@ public class MetaDataMappingService extends AbstractComponent {
 
                     }
                 } else {
-                    updated = true;
+                    updatedMapping = true;
                     if (logger.isDebugEnabled()) {
                         logger.debug("{} create_mapping [{}] with source [{}]", index, mappingType, updatedSource);
                     } else if (logger.isInfoEnabled()) {
@@ -329,7 +330,16 @@ public class MetaDataMappingService extends AbstractComponent {
                         indexMetaDataBuilder.putMapping(new MappingMetaData(mapper.mappingSource()));
                     }
                 }
+                if (updatedMapping) {
+                    indexMetaDataBuilder.mappingVersion(1 + indexMetaDataBuilder.mappingVersion());
+                }
+                /*
+                 * This implicitly increments the index metadata version and builds the index metadata. This means that we need to have
+                 * already incremented the mapping version if necessary. Therefore, the mapping version increment must remain before this
+                 * statement.
+                 */
                 builder.put(indexMetaDataBuilder);
+                updated |= updatedMapping;
             }
             if (updated) {
                 return ClusterState.builder(currentState).metaData(builder).build();

+ 2 - 2
server/src/main/java/org/elasticsearch/index/IndexService.java

@@ -522,8 +522,8 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
     }
 
     @Override
-    public boolean updateMapping(IndexMetaData indexMetaData) throws IOException {
-        return mapperService().updateMapping(indexMetaData);
+    public boolean updateMapping(final IndexMetaData currentIndexMetaData, final IndexMetaData newIndexMetaData) throws IOException {
+        return mapperService().updateMapping(currentIndexMetaData, newIndexMetaData);
     }
 
     private class StoreCloseListener implements Store.OnClose {

+ 46 - 4
server/src/main/java/org/elasticsearch/index/mapper/MapperService.java

@@ -25,6 +25,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.DelegatingAnalyzerWrapper;
 import org.apache.lucene.index.Term;
+import org.elasticsearch.Assertions;
 import org.elasticsearch.ElasticsearchGenerationException;
 import org.elasticsearch.Version;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
@@ -192,8 +193,8 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
     /**
      * Update mapping by only merging the metadata that is different between received and stored entries
      */
-    public boolean updateMapping(IndexMetaData indexMetaData) throws IOException {
-        assert indexMetaData.getIndex().equals(index()) : "index mismatch: expected " + index() + " but was " + indexMetaData.getIndex();
+    public boolean updateMapping(final IndexMetaData currentIndexMetaData, final IndexMetaData newIndexMetaData) throws IOException {
+        assert newIndexMetaData.getIndex().equals(index()) : "index mismatch: expected " + index() + " but was " + newIndexMetaData.getIndex();
         // go over and add the relevant mappings (or update them)
         Set<String> existingMappers = new HashSet<>();
         if (mapper != null) {
@@ -205,7 +206,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
         final Map<String, DocumentMapper> updatedEntries;
         try {
             // only update entries if needed
-            updatedEntries = internalMerge(indexMetaData, MergeReason.MAPPING_RECOVERY, true);
+            updatedEntries = internalMerge(newIndexMetaData, MergeReason.MAPPING_RECOVERY, true);
         } catch (Exception e) {
             logger.warn(() -> new ParameterizedMessage("[{}] failed to apply mappings", index()), e);
             throw e;
@@ -213,9 +214,11 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
 
         boolean requireRefresh = false;
 
+        assertMappingVersion(currentIndexMetaData, newIndexMetaData, updatedEntries);
+
         for (DocumentMapper documentMapper : updatedEntries.values()) {
             String mappingType = documentMapper.type();
-            CompressedXContent incomingMappingSource = indexMetaData.mapping(mappingType).source();
+            CompressedXContent incomingMappingSource = newIndexMetaData.mapping(mappingType).source();
 
             String op = existingMappers.contains(mappingType) ? "updated" : "added";
             if (logger.isDebugEnabled() && incomingMappingSource.compressed().length < 512) {
@@ -240,6 +243,45 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
         return requireRefresh;
     }
 
+    private void assertMappingVersion(
+            final IndexMetaData currentIndexMetaData,
+            final IndexMetaData newIndexMetaData,
+            final Map<String, DocumentMapper> updatedEntries) {
+        if (Assertions.ENABLED
+                && currentIndexMetaData != null
+                && currentIndexMetaData.getCreationVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
+            if (currentIndexMetaData.getMappingVersion() == newIndexMetaData.getMappingVersion()) {
+                // if the mapping version is unchanged, then there should not be any updates and all mappings should be the same
+                assert updatedEntries.isEmpty() : updatedEntries;
+                for (final ObjectCursor<MappingMetaData> mapping : newIndexMetaData.getMappings().values()) {
+                    final CompressedXContent currentSource = currentIndexMetaData.mapping(mapping.value.type()).source();
+                    final CompressedXContent newSource = mapping.value.source();
+                    assert currentSource.equals(newSource) :
+                            "expected current mapping [" + currentSource + "] for type [" + mapping.value.type() + "] "
+                                    + "to be the same as new mapping [" + newSource + "]";
+                }
+            } else {
+                // if the mapping version is changed, it should increase, there should be updates, and the mapping should be different
+                final long currentMappingVersion = currentIndexMetaData.getMappingVersion();
+                final long newMappingVersion = newIndexMetaData.getMappingVersion();
+                assert currentMappingVersion < newMappingVersion :
+                        "expected current mapping version [" + currentMappingVersion + "] "
+                                + "to be less than new mapping version [" + newMappingVersion + "]";
+                assert updatedEntries.isEmpty() == false;
+                for (final DocumentMapper documentMapper : updatedEntries.values()) {
+                    final MappingMetaData currentMapping = currentIndexMetaData.mapping(documentMapper.type());
+                    if (currentMapping != null) {
+                        final CompressedXContent currentSource = currentMapping.source();
+                        final CompressedXContent newSource = documentMapper.mappingSource();
+                        assert currentSource.equals(newSource) == false :
+                                "expected current mapping [" + currentSource + "] for type [" + documentMapper.type() + "] " +
+                                        "to be different than new mapping";
+                    }
+                }
+            }
+        }
+    }
+
     public void merge(Map<String, Map<String, Object>> mappings, MergeReason reason) {
         Map<String, CompressedXContent> mappingSourcesCompressed = new LinkedHashMap<>(mappings.size());
         for (Map.Entry<String, Map<String, Object>> entry : mappings.entrySet()) {

+ 3 - 3
server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java

@@ -456,7 +456,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
             AllocatedIndex<? extends Shard> indexService = null;
             try {
                 indexService = indicesService.createIndex(indexMetaData, buildInIndexListener);
-                if (indexService.updateMapping(indexMetaData) && sendRefreshMapping) {
+                if (indexService.updateMapping(null, indexMetaData) && sendRefreshMapping) {
                     nodeMappingRefreshAction.nodeMappingRefresh(state.nodes().getMasterNode(),
                         new NodeMappingRefreshAction.NodeMappingRefreshRequest(indexMetaData.getIndex().getName(),
                             indexMetaData.getIndexUUID(), state.nodes().getLocalNodeId())
@@ -490,7 +490,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
             if (ClusterChangedEvent.indexMetaDataChanged(currentIndexMetaData, newIndexMetaData)) {
                 indexService.updateMetaData(newIndexMetaData);
                 try {
-                    if (indexService.updateMapping(newIndexMetaData) && sendRefreshMapping) {
+                    if (indexService.updateMapping(currentIndexMetaData, newIndexMetaData) && sendRefreshMapping) {
                         nodeMappingRefreshAction.nodeMappingRefresh(state.nodes().getMasterNode(),
                             new NodeMappingRefreshAction.NodeMappingRefreshRequest(newIndexMetaData.getIndex().getName(),
                                 newIndexMetaData.getIndexUUID(), state.nodes().getLocalNodeId())
@@ -778,7 +778,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
         /**
          * Checks if index requires refresh from master.
          */
-        boolean updateMapping(IndexMetaData indexMetaData) throws IOException;
+        boolean updateMapping(IndexMetaData currentIndexMetaData, IndexMetaData newIndexMetaData) throws IOException;
 
         /**
          * Returns shard with given id.

+ 1 - 0
server/src/main/java/org/elasticsearch/snapshots/RestoreService.java

@@ -292,6 +292,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateApp
                                 // Index exists and it's closed - open it in metadata and start recovery
                                 IndexMetaData.Builder indexMdBuilder = IndexMetaData.builder(snapshotIndexMetaData).state(IndexMetaData.State.OPEN);
                                 indexMdBuilder.version(Math.max(snapshotIndexMetaData.getVersion(), currentIndexMetaData.getVersion() + 1));
+                                indexMdBuilder.mappingVersion(Math.max(snapshotIndexMetaData.getMappingVersion(), currentIndexMetaData.getMappingVersion() + 1));
                                 if (!request.includeAliases()) {
                                     // Remove all snapshot aliases
                                     if (!snapshotIndexMetaData.getAliases().isEmpty()) {

+ 30 - 0
server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataMappingServiceTests.java

@@ -84,4 +84,34 @@ public class MetaDataMappingServiceTests extends ESSingleNodeTestCase {
         assertSame(result, result2);
     }
 
+    public void testMappingVersion() throws Exception {
+        final IndexService indexService = createIndex("test", client().admin().indices().prepareCreate("test").addMapping("type"));
+        final long previousVersion = indexService.getMetaData().getMappingVersion();
+        final MetaDataMappingService mappingService = getInstanceFromNode(MetaDataMappingService.class);
+        final ClusterService clusterService = getInstanceFromNode(ClusterService.class);
+        final PutMappingClusterStateUpdateRequest request = new PutMappingClusterStateUpdateRequest().type("type");
+        request.indices(new Index[] {indexService.index()});
+        request.source("{ \"properties\": { \"field\": { \"type\": \"text\" }}}");
+        final ClusterStateTaskExecutor.ClusterTasksResult<PutMappingClusterStateUpdateRequest> result =
+                mappingService.putMappingExecutor.execute(clusterService.state(), Collections.singletonList(request));
+        assertThat(result.executionResults.size(), equalTo(1));
+        assertTrue(result.executionResults.values().iterator().next().isSuccess());
+        assertThat(result.resultingState.metaData().index("test").getMappingVersion(), equalTo(1 + previousVersion));
+    }
+
+    public void testMappingVersionUnchanged() throws Exception {
+        final IndexService indexService = createIndex("test", client().admin().indices().prepareCreate("test").addMapping("type"));
+        final long previousVersion = indexService.getMetaData().getMappingVersion();
+        final MetaDataMappingService mappingService = getInstanceFromNode(MetaDataMappingService.class);
+        final ClusterService clusterService = getInstanceFromNode(ClusterService.class);
+        final PutMappingClusterStateUpdateRequest request = new PutMappingClusterStateUpdateRequest().type("type");
+        request.indices(new Index[] {indexService.index()});
+        request.source("{ \"properties\": {}}");
+        final ClusterStateTaskExecutor.ClusterTasksResult<PutMappingClusterStateUpdateRequest> result =
+                mappingService.putMappingExecutor.execute(clusterService.state(), Collections.singletonList(request));
+        assertThat(result.executionResults.size(), equalTo(1));
+        assertTrue(result.executionResults.values().iterator().next().isSuccess());
+        assertThat(result.resultingState.metaData().index("test").getMappingVersion(), equalTo(previousVersion));
+    }
+
 }

+ 1 - 0
server/src/test/java/org/elasticsearch/gateway/MetaDataStateFormatTests.java

@@ -267,6 +267,7 @@ public class MetaDataStateFormatTests extends ESTestCase {
             IndexMetaData deserialized = indices.get(original.getIndex().getName());
             assertThat(deserialized, notNullValue());
             assertThat(deserialized.getVersion(), equalTo(original.getVersion()));
+            assertThat(deserialized.getMappingVersion(), equalTo(original.getMappingVersion()));
             assertThat(deserialized.getNumberOfReplicas(), equalTo(original.getNumberOfReplicas()));
             assertThat(deserialized.getNumberOfShards(), equalTo(original.getNumberOfShards()));
         }

+ 10 - 0
server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java

@@ -22,6 +22,7 @@ import org.apache.lucene.index.IndexOptions;
 import org.elasticsearch.Version;
 import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.compress.CompressedXContent;
@@ -741,4 +742,13 @@ public class DynamicMappingTests extends ESSingleNodeTestCase {
         client().prepareIndex("test", "type", "1").setSource("foo", "abc").get();
         assertThat(index.mapperService().fullName("foo"), instanceOf(KeywordFieldMapper.KeywordFieldType.class));
     }
+
+    public void testMappingVersionAfterDynamicMappingUpdate() {
+        createIndex("test", client().admin().indices().prepareCreate("test").addMapping("type"));
+        final ClusterService clusterService = getInstanceFromNode(ClusterService.class);
+        final long previousVersion = clusterService.state().metaData().index("test").getMappingVersion();
+        client().prepareIndex("test", "type", "1").setSource("field", "text").get();
+        assertThat(clusterService.state().metaData().index("test").getMappingVersion(), equalTo(1 + previousVersion));
+    }
+
 }

+ 29 - 0
server/src/test/java/org/elasticsearch/index/mapper/UpdateMappingTests.java

@@ -19,6 +19,8 @@
 
 package org.elasticsearch.index.mapper;
 
+import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
+import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.compress.CompressedXContent;
@@ -30,6 +32,7 @@ import org.elasticsearch.index.mapper.MapperService.MergeReason;
 import org.elasticsearch.plugins.Plugin;
 import org.elasticsearch.test.ESSingleNodeTestCase;
 import org.elasticsearch.test.InternalSettingsPlugin;
+import org.hamcrest.Matchers;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -188,4 +191,30 @@ public class UpdateMappingTests extends ESSingleNodeTestCase {
                 () -> mapperService2.merge("type", new CompressedXContent(mapping1), MergeReason.MAPPING_UPDATE));
         assertThat(e.getMessage(), equalTo("mapper [foo] of different type, current_type [long], merged_type [ObjectMapper]"));
     }
+
+    public void testMappingVersion() {
+        createIndex("test", client().admin().indices().prepareCreate("test").addMapping("type"));
+        final ClusterService clusterService = getInstanceFromNode(ClusterService.class);
+        {
+            final long previousVersion = clusterService.state().metaData().index("test").getMappingVersion();
+            final PutMappingRequest request = new PutMappingRequest();
+            request.indices("test");
+            request.type("type");
+            request.source("field", "type=text");
+            client().admin().indices().putMapping(request).actionGet();
+            assertThat(clusterService.state().metaData().index("test").getMappingVersion(), Matchers.equalTo(1 + previousVersion));
+        }
+
+        {
+            final long previousVersion = clusterService.state().metaData().index("test").getMappingVersion();
+            final PutMappingRequest request = new PutMappingRequest();
+            request.indices("test");
+            request.type("type");
+            request.source("field", "type=text");
+            client().admin().indices().putMapping(request).actionGet();
+            // the version should be unchanged after putting the same mapping again
+            assertThat(clusterService.state().metaData().index("test").getMappingVersion(), Matchers.equalTo(previousVersion));
+        }
+    }
+
 }

+ 1 - 1
server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java

@@ -273,7 +273,7 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
         }
 
         @Override
-        public boolean updateMapping(IndexMetaData indexMetaData) throws IOException {
+        public boolean updateMapping(final IndexMetaData currentIndexMetaData, final IndexMetaData newIndexMetaData) throws IOException {
             failRandomly();
             return false;
         }