diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_index_template/15_composition.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_index_template/15_composition.yml index 9bad6437dcf6a..571f95e42ab59 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_index_template/15_composition.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_index_template/15_composition.yml @@ -200,3 +200,89 @@ index: eggplant - match: {eggplant.settings.index.number_of_shards: "3"} + +--- +"Index template mapping merging": + - skip: + version: " - 7.99.99" + reason: "index template v2 mapping merging not yet backported to 7.9" + features: allowed_warnings + + - do: + cluster.put_component_template: + name: red + body: + template: + mappings: + properties: + object1.red: + type: keyword + object2.red: + type: keyword + + - do: + cluster.put_component_template: + name: blue + body: + template: + mappings: + properties: + object2.red: + type: text + object1.blue: + type: text + object2.blue: + type: text + + - do: + allowed_warnings: + - "index template [my-template] has index patterns [baz*] matching patterns from existing older templates [global] with patterns (global => [*]); this template [my-template] will take precedence during new index creation" + indices.put_index_template: + name: blue + body: + index_patterns: ["purple-index"] + composed_of: ["red", "blue"] + template: + mappings: + properties: + object2.blue: + type: integer + object1.purple: + type: integer + object2.purple: + type: integer + nested: + type: nested + include_in_root: true + + - do: + indices.create: + index: purple-index + body: + mappings: + properties: + object2.purple: + type: double + object3.purple: + type: double + nested: + type: nested + include_in_root: false + include_in_parent: true + + - do: + indices.get: + index: purple-index + + - match: {purple-index.mappings.properties.object1.properties.red: {type: keyword}} + - match: {purple-index.mappings.properties.object1.properties.blue: {type: text}} + - match: {purple-index.mappings.properties.object1.properties.purple: {type: integer}} + + - match: {purple-index.mappings.properties.object2.properties.red: {type: text}} + - match: {purple-index.mappings.properties.object2.properties.blue: {type: integer}} + - match: {purple-index.mappings.properties.object2.properties.purple: {type: double}} + + - match: {purple-index.mappings.properties.object3.properties.purple: {type: double}} + + - is_false: purple-index.mappings.properties.nested.include_in_root + - is_true: purple-index.mappings.properties.nested.include_in_parent diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java index 917a90373dc7e..e57b2a3c56886 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java @@ -28,22 +28,22 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.AliasValidator; +import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.MetadataCreateIndexService; import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService; import org.elasticsearch.cluster.metadata.Template; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.NamedXContentRegistry; -import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; @@ -58,7 +58,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.elasticsearch.cluster.metadata.MetadataCreateIndexService.resolveV2Mappings; import static org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.findConflictingV1Templates; import static org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.findConflictingV2Templates; import static org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.findV2Template; @@ -172,10 +171,6 @@ public static Template resolveTemplate(final String matchingTemplate, final Stri final AliasValidator aliasValidator) throws Exception { Settings settings = resolveSettings(simulatedState.metadata(), matchingTemplate); - // empty request mapping as the user can't specify any explicit mappings via the simulate api - Map mappings = resolveV2Mappings("{}", simulatedState, matchingTemplate, xContentRegistry); - String mappingsJson = Strings.toString(XContentFactory.jsonBuilder().map(mappings)); - List> resolvedAliases = MetadataIndexTemplateService.resolveAliases(simulatedState.metadata(), matchingTemplate); @@ -200,8 +195,27 @@ public static Template resolveTemplate(final String matchingTemplate, final Stri // the context is only used for validation so it's fine to pass fake values for the // shard id and the current timestamp tempIndexService.newQueryShardContext(0, null, () -> 0L, null))); + Map aliasesByName = aliases.stream().collect( + Collectors.toMap(AliasMetadata::getAlias, Function.identity())); - return new Template(settings, mappingsJson == null ? null : new CompressedXContent(mappingsJson), - aliases.stream().collect(Collectors.toMap(AliasMetadata::getAlias, Function.identity()))); + // empty request mapping as the user can't specify any explicit mappings via the simulate api + List> mappings = MetadataCreateIndexService.collectV2Mappings( + "{}", simulatedState, matchingTemplate, xContentRegistry); + + CompressedXContent mergedMapping = indicesService.withTempIndexService(indexMetadata, + tempIndexService -> { + MapperService mapperService = tempIndexService.mapperService(); + for (Map mapping : mappings) { + if (!mapping.isEmpty()) { + assert mapping.size() == 1 : mapping; + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, mapping, MapperService.MergeReason.INDEX_TEMPLATE); + } + } + + DocumentMapper documentMapper = mapperService.documentMapper(); + return documentMapper != null ? documentMapper.mappingSource() : null; + }); + + return new Template(settings, mergedMapping, aliasesByName); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStream.java b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStream.java index e73f190666639..81191d4a1db6e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStream.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStream.java @@ -27,7 +27,6 @@ import org.elasticsearch.common.xcontent.ConstructingObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.Index; @@ -244,28 +243,24 @@ public TimestampField(StreamInput in) throws IOException { } /** - * Force fully inserts the timestamp field mapping into the provided mapping. - * Existing mapping definitions for the timestamp field will be completely overwritten. - * Takes into account if the name of the timestamp field is nested. - * - * @param mappings The mapping to update + * Creates a map representing the full timestamp field mapping, taking into + * account if the timestamp field is nested under object mappers (its path + * contains dots). */ - public void insertTimestampFieldMapping(Map mappings) { - assert mappings.containsKey("_doc"); - + public Map getTimestampFieldMapping() { String mappingPath = convertFieldPathToMappingPath(name); String parentObjectFieldPath = "_doc." + mappingPath.substring(0, mappingPath.lastIndexOf('.')); String leafFieldName = mappingPath.substring(mappingPath.lastIndexOf('.') + 1); - Map changes = new HashMap<>(); - Map current = changes; + Map result = new HashMap<>(); + Map current = result; for (String key : parentObjectFieldPath.split("\\.")) { Map map = new HashMap<>(); current.put(key, map); current = map; } current.put(leafFieldName, fieldMapping); - XContentHelper.update(mappings, changes, false); + return result; } @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java index 492583efa4dda..007a2b2fd2f9e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java @@ -83,11 +83,9 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; @@ -380,7 +378,7 @@ public ClusterState applyCreateIndexRequest(ClusterState currentState, CreateInd * @param silent a boolean for whether logging should be at a lower or higher level * @param sourceMetadata when recovering from an existing index, metadata that should be copied to the new index * @param temporaryIndexMeta metadata for the new index built from templates, source metadata, and request settings - * @param mappings a map of mappings for the new index + * @param mappings a list of all mapping definitions to apply, in order * @param aliasSupplier a function that takes the real {@link IndexService} and returns a list of {@link AliasMetadata} aliases * @param templatesApplied a list of the names of the templates applied, for logging * @param metadataTransformer if provided, a function that may alter cluster metadata in the same cluster state update that @@ -392,7 +390,7 @@ private ClusterState applyCreateIndexWithTemporaryService(final ClusterState cur final boolean silent, final IndexMetadata sourceMetadata, final IndexMetadata temporaryIndexMeta, - final Map mappings, + final List> mappings, final Function> aliasSupplier, final List templatesApplied, final BiConsumer metadataTransformer) @@ -417,9 +415,8 @@ private ClusterState applyCreateIndexWithTemporaryService(final ClusterState cur throw e; } - logger.log(silent ? Level.DEBUG : Level.INFO, "[{}] creating index, cause [{}], templates {}, shards [{}]/[{}], mappings {}", - request.index(), request.cause(), templatesApplied, indexMetadata.getNumberOfShards(), - indexMetadata.getNumberOfReplicas(), mappings.keySet()); + logger.log(silent ? Level.DEBUG : Level.INFO, "[{}] creating index, cause [{}], templates {}, shards [{}]/[{}]", + request.index(), request.cause(), templatesApplied, indexMetadata.getNumberOfShards(), indexMetadata.getNumberOfReplicas()); indexService.getIndexEventListener().beforeIndexAddedToCluster(indexMetadata.getIndex(), indexMetadata.getSettings()); @@ -474,7 +471,7 @@ private ClusterState applyCreateIndexRequestWithV1Templates(final ClusterState c int routingNumShards = getIndexNumberOfRoutingShards(aggregatedIndexSettings, null); IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(currentState, aggregatedIndexSettings, request, routingNumShards); - return applyCreateIndexWithTemporaryService(currentState, request, silent, null, tmpImd, mappings, + return applyCreateIndexWithTemporaryService(currentState, request, silent, null, tmpImd, List.of(mappings), indexService -> resolveAndValidateAliases(request.index(), request.aliases(), MetadataIndexTemplateService.resolveAliases(templates), currentState.metadata(), aliasValidator, // the context is only used for validation so it's fine to pass fake values for the @@ -491,12 +488,12 @@ private ClusterState applyCreateIndexRequestWithV2Template(final ClusterState cu throws Exception { logger.debug("applying create index request using composable template [{}]", templateName); - final Map mappings = resolveV2Mappings(request.mappings(), currentState, templateName, xContentRegistry); + final List> mappings = collectV2Mappings(request.mappings(), currentState, templateName, xContentRegistry); if (request.dataStreamName() != null) { DataStream dataStream = currentState.metadata().dataStreams().get(request.dataStreamName()); if (dataStream != null) { - dataStream.getTimeStampField().insertTimestampFieldMapping(mappings); + mappings.add(dataStream.getTimeStampField().getTimestampFieldMapping()); } } @@ -516,15 +513,22 @@ private ClusterState applyCreateIndexRequestWithV2Template(final ClusterState cu Collections.singletonList(templateName), metadataTransformer); } - public static Map resolveV2Mappings(final String requestMappings, - final ClusterState currentState, - final String templateName, - final NamedXContentRegistry xContentRegistry) throws Exception { - final Map mappings = Collections.unmodifiableMap(parseV2Mappings(requestMappings, - MetadataIndexTemplateService.resolveMappings(currentState, templateName), xContentRegistry)); - return mappings; - } + public static List> collectV2Mappings(final String requestMappings, + final ClusterState currentState, + final String templateName, + final NamedXContentRegistry xContentRegistry) throws Exception { + List> result = new ArrayList<>(); + List templateMappings = MetadataIndexTemplateService.collectMappings(currentState, templateName); + for (CompressedXContent templateMapping : templateMappings) { + Map parsedTemplateMapping = MapperService.parseMapping(xContentRegistry, templateMapping.string()); + result.add(parsedTemplateMapping); + } + + Map parsedRequestMappings = MapperService.parseMapping(xContentRegistry, requestMappings); + result.add(parsedRequestMappings); + return result; + } private ClusterState applyCreateIndexRequestWithExistingMetadata(final ClusterState currentState, final CreateIndexClusterStateUpdateRequest request, final boolean silent, @@ -544,7 +548,7 @@ private ClusterState applyCreateIndexRequestWithExistingMetadata(final ClusterSt final int routingNumShards = getIndexNumberOfRoutingShards(aggregatedIndexSettings, sourceMetadata); IndexMetadata tmpImd = buildAndValidateTemporaryIndexMetadata(currentState, aggregatedIndexSettings, request, routingNumShards); - return applyCreateIndexWithTemporaryService(currentState, request, silent, sourceMetadata, tmpImd, mappings, + return applyCreateIndexWithTemporaryService(currentState, request, silent, sourceMetadata, tmpImd, List.of(mappings), indexService -> resolveAndValidateAliases(request.index(), request.aliases(), Collections.emptyList(), currentState.metadata(), aliasValidator, xContentRegistry, // the context is only used for validation so it's fine to pass fake values for the @@ -553,110 +557,6 @@ private ClusterState applyCreateIndexRequestWithExistingMetadata(final ClusterSt List.of(), metadataTransformer); } - /** - * Parses the provided mappings json and the inheritable mappings from the templates (if any) - * into a map. - * - * The template mappings are applied in the order they are encountered in the list, with the - * caveat that mapping fields are only merged at the top-level, meaning that field settings are - * not merged, instead they replace any previous field definition. - */ - @SuppressWarnings("unchecked") - static Map parseV2Mappings(String mappingsJson, List templateMappings, - NamedXContentRegistry xContentRegistry) throws Exception { - Map requestMappings = MapperService.parseMapping(xContentRegistry, mappingsJson); - // apply templates, merging the mappings into the request mapping if exists - Map properties = new HashMap<>(); - Map nonProperties = new HashMap<>(); - for (CompressedXContent mapping : templateMappings) { - if (mapping != null) { - Map templateMapping = MapperService.parseMapping(xContentRegistry, mapping.string()); - if (templateMapping.isEmpty()) { - // Someone provided an empty '{}' for mappings, which is okay, but to avoid - // tripping the below assertion, we can safely ignore it - continue; - } - assert templateMapping.size() == 1 : "expected exactly one mapping value, got: " + templateMapping; - if (templateMapping.get(MapperService.SINGLE_MAPPING_NAME) instanceof Map == false) { - throw new IllegalStateException("invalid mapping definition, expected a single map underneath [" + - MapperService.SINGLE_MAPPING_NAME + "] but it was: [" + templateMapping + "]"); - } - - Map innerTemplateMapping = (Map) templateMapping.get(MapperService.SINGLE_MAPPING_NAME); - Map innerTemplateNonProperties = new HashMap<>(innerTemplateMapping); - Map maybeProperties = (Map) innerTemplateNonProperties.remove("properties"); - - nonProperties = mergeFailingOnReplacement(nonProperties, innerTemplateNonProperties); - - if (maybeProperties != null) { - properties = mergeFailingOnReplacement(properties, maybeProperties); - } - } - } - - if (requestMappings.get(MapperService.SINGLE_MAPPING_NAME) != null) { - Map innerRequestMappings = (Map) requestMappings.get(MapperService.SINGLE_MAPPING_NAME); - Map innerRequestNonProperties = new HashMap<>(innerRequestMappings); - Map maybeRequestProperties = (Map) innerRequestNonProperties.remove("properties"); - - nonProperties = mergeFailingOnReplacement(nonProperties, innerRequestNonProperties); - - if (maybeRequestProperties != null) { - properties = mergeFailingOnReplacement(properties, maybeRequestProperties); - } - } - - Map finalMappings = dedupDynamicTemplates(nonProperties); - finalMappings.put("properties", properties); - return Collections.singletonMap(MapperService.SINGLE_MAPPING_NAME, finalMappings); - } - - /** - * Parses the `dynamic_templates` from the provided mappings, if any are configured, and returns a mappings map containing dynamic - * templates with unique names. - * - * The later templates in the provided mapping's `dynamic_templates` array will override the templates with the same name defined - * earlier in the `dynamic_templates` array. - */ - @SuppressWarnings("unchecked") - private static Map dedupDynamicTemplates(Map mappings) { - Objects.requireNonNull(mappings, "deduping the dynamic templates a non-null mapping"); - Map results = new HashMap<>(mappings); - List> dynamicTemplates = (List>) mappings.get("dynamic_templates"); - if (dynamicTemplates == null) { - return results; - } - - LinkedHashMap> dedupedDynamicTemplates = new LinkedHashMap<>(dynamicTemplates.size(), 1f); - for (Map dynamicTemplate : dynamicTemplates) { - dedupedDynamicTemplates.put(dynamicTemplate.keySet().iterator().next(), dynamicTemplate); - } - - results.put("dynamic_templates", new ArrayList<>(dedupedDynamicTemplates.values())); - return results; - } - - /** - * Add the objects in the second map to the first, A duplicated field is treated as illegal and - * an exception is thrown. - */ - static Map mergeFailingOnReplacement(Map first, Map second) { - Objects.requireNonNull(first, "merging requires two non-null maps but the first map was null"); - Objects.requireNonNull(second, "merging requires two non-null maps but the second map was null"); - Map results = new HashMap<>(first); - Set prefixes = second.keySet().stream().map(MetadataCreateIndexService::prefix).collect(Collectors.toSet()); - List matchedPrefixes = results.keySet().stream().filter(k -> prefixes.contains(prefix(k))).collect(Collectors.toList()); - if (matchedPrefixes.size() > 0) { - throw new IllegalArgumentException("mapping fields " + matchedPrefixes + " cannot be replaced during template composition"); - } - results.putAll(second); - return results; - } - - private static String prefix(String s) { - return s.split("\\.", 2)[0]; - } - /** * Parses the provided mappings json and the inheritable mappings from the templates (if any) * into a map. @@ -939,12 +839,14 @@ private static ClusterBlocks.Builder createClusterBlocksBuilder(ClusterState cur return blocksBuilder; } - private static void updateIndexMappingsAndBuildSortOrder(IndexService indexService, Map mappings, + private static void updateIndexMappingsAndBuildSortOrder(IndexService indexService, List> mappings, @Nullable IndexMetadata sourceMetadata) throws IOException { MapperService mapperService = indexService.mapperService(); - if (!mappings.isEmpty()) { - assert mappings.size() == 1 : mappings; - mapperService.merge(MapperService.SINGLE_MAPPING_NAME, mappings, MergeReason.MAPPING_UPDATE); + for (Map mapping : mappings) { + if (!mapping.isEmpty()) { + assert mapping.size() == 1 : mapping; + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, mapping, MergeReason.INDEX_TEMPLATE); + } } if (sourceMetadata == null) { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java index d8b577bbdc914..0b0ec82e98f8c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java @@ -871,15 +871,16 @@ public static String findV2Template(Metadata metadata, String indexName, boolean } /** - * Resolve the given v2 template into an ordered list of mappings + * Collect the given v2 template into an ordered list of mappings. */ - public static List resolveMappings(final ClusterState state, final String templateName) { + public static List collectMappings(final ClusterState state, final String templateName) { final ComposableIndexTemplate template = state.metadata().templatesV2().get(templateName); assert template != null : "attempted to resolve mappings for a template [" + templateName + "] that did not exist in the cluster state"; if (template == null) { return List.of(); } + final Map componentTemplates = state.metadata().componentTemplates(); List mappings = template.composedOf().stream() .map(componentTemplates::get) @@ -1038,20 +1039,15 @@ private static void validateCompositeTemplate(final ClusterState state, xContentRegistry, tempIndexService.newQueryShardContext(0, null, () -> 0L, null)); // Parse mappings to ensure they are valid after being composed - List mappings = resolveMappings(stateWithIndex, templateName); + List mappings = collectMappings(stateWithIndex, templateName); try { - Map finalMappings = MetadataCreateIndexService.parseV2Mappings("{}", mappings, xContentRegistry); - - MapperService dummyMapperService = tempIndexService.mapperService(); - if (finalMappings.isEmpty() == false) { - assert finalMappings.size() == 1 : finalMappings; - // TODO: Eventually change this to: - // dummyMapperService.merge(MapperService.SINGLE_MAPPING_NAME, mapping, MergeReason.INDEX_TEMPLATE); - dummyMapperService.merge(MapperService.SINGLE_MAPPING_NAME, finalMappings, MergeReason.MAPPING_UPDATE); - } - if (template.getDataStreamTemplate() != null) { - String tsFieldName = template.getDataStreamTemplate().getTimestampField(); - validateTimestampFieldMapping(tsFieldName, dummyMapperService); + MapperService mapperService = tempIndexService.mapperService(); + for (CompressedXContent mapping : mappings) { + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, mapping, MergeReason.INDEX_TEMPLATE); + if (template.getDataStreamTemplate() != null) { + String tsFieldName = template.getDataStreamTemplate().getTimestampField(); + validateTimestampFieldMapping(tsFieldName, mapperService); + } } } catch (Exception e) { throw new IllegalArgumentException("invalid composite mappings for [" + templateName + "]", e); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataMappingService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataMappingService.java index 067c6c85cb954..e36dbd297cd78 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataMappingService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataMappingService.java @@ -256,7 +256,7 @@ private ClusterState applyRequest(ClusterState currentState, PutMappingClusterSt DocumentMapper newMapper = mapperService.parse(MapperService.SINGLE_MAPPING_NAME, mappingUpdateSource); if (existingMapper != null) { // first, simulate: just call merge and ignore the result - existingMapper.merge(newMapper.mapping()); + existingMapper.merge(newMapper.mapping(), MergeReason.MAPPING_UPDATE); } } Metadata.Builder builder = Metadata.builder(metadata); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index 01d25da00685c..cd3f49d9bc261 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.mapper.MapperService.MergeReason; import org.elasticsearch.index.mapper.MetadataFieldMapper.TypeParser; import org.elasticsearch.search.internal.SearchContext; @@ -302,8 +303,8 @@ public ObjectMapper findNestedObjectMapper(int nestedDocId, SearchContext sc, Le return nestedObjectMapper; } - public DocumentMapper merge(Mapping mapping) { - Mapping merged = this.mapping.merge(mapping); + public DocumentMapper merge(Mapping mapping, MergeReason reason) { + Mapping merged = this.mapping.merge(mapping, reason); return new DocumentMapper(mapperService, merged); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java b/server/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java index 9c28e3d5519f0..d34767676589a 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java @@ -249,6 +249,10 @@ public String name() { return this.name; } + public String pathMatch() { + return pathMatch; + } + public boolean match(String path, String name, XContentFieldType xcontentFieldType) { if (pathMatch != null && !matchType.matches(pathMatch, path)) { return false; diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 205f9eed2ca59..9f7e317148f58 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -86,6 +86,10 @@ public enum MergeReason { * Create or update a mapping. */ MAPPING_UPDATE, + /** + * Merge mappings from a composable index template. + */ + INDEX_TEMPLATE, /** * Recovery of an existing mapping, for instance because of a restart, * if a shard was moved to a different node or for administrative @@ -336,10 +340,11 @@ private synchronized DocumentMapper internalMerge(DocumentMapper mapper, MergeRe DocumentMapper oldMapper = this.mapper; DocumentMapper newMapper; if (oldMapper != null) { - newMapper = oldMapper.merge(mapper.mapping()); + newMapper = oldMapper.merge(mapper.mapping(), reason); } else { newMapper = mapper; } + newMapper.root().fixRedundantIncludes(); // check basic sanity of the new mapping List objectMappers = new ArrayList<>(); @@ -372,24 +377,15 @@ private synchronized DocumentMapper internalMerge(DocumentMapper mapper, MergeRe ContextMapping.validateContextPaths(indexSettings.getIndexVersionCreated(), fieldMappers, newFieldTypes::get); - if (reason == MergeReason.MAPPING_UPDATE || reason == MergeReason.MAPPING_UPDATE_PREFLIGHT) { - // this check will only be performed on the master node when there is - // a call to the update mapping API. For all other cases like - // the master node restoring mappings from disk or data nodes - // deserializing cluster state that was sent by the master node, - // this check will be skipped. + if (reason != MergeReason.MAPPING_RECOVERY) { + // These checks will only be performed on the master node when an index is created, or + // there is a call to the update mapping API. For all other cases like the master node + // restoring mappings from disk or data nodes deserializing cluster state that was sent + // by the master node, these checks will be skipped. // Also, don't take metadata mappers into account for the field limit check checkTotalFieldsLimit(objectMappers.size() + fieldMappers.size() - metadataMappers.length + fieldAliasMappers.size() ); checkFieldNameSoftLimit(objectMappers, fieldMappers, fieldAliasMappers); - } - - if (reason == MergeReason.MAPPING_UPDATE || reason == MergeReason.MAPPING_UPDATE_PREFLIGHT) { - // this check will only be performed on the master node when there is - // a call to the update mapping API. For all other cases like - // the master node restoring mappings from disk or data nodes - // deserializing cluster state that was sent by the master node, - // this check will be skipped. checkNestedFieldsLimit(fullPathObjectMappers); checkDepthLimit(fullPathObjectMappers.keySet()); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java b/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java index 0cc730c33cdf6..a9e02b359c17f 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java @@ -25,6 +25,8 @@ import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.index.mapper.MapperService.MergeReason; import java.io.IOException; import java.io.UncheckedIOException; @@ -86,21 +88,44 @@ public T metadataMapper(Class clazz) { return (T) metadataMappersMap.get(clazz); } - /** @see DocumentMapper#merge(Mapping) */ - public Mapping merge(Mapping mergeWith) { - RootObjectMapper mergedRoot = root.merge(mergeWith.root); + /** + * Merges a new mapping into the existing one. + * + * @param mergeWith the new mapping to merge into this one. + * @param reason the reason this merge was initiated. + * @return the resulting merged mapping. + */ + public Mapping merge(Mapping mergeWith, MergeReason reason) { + RootObjectMapper mergedRoot = root.merge(mergeWith.root, reason); + + // When merging metadata fields as part of applying an index template, new field definitions + // completely overwrite existing ones instead of being merged. This behavior matches how we + // merge leaf fields in the 'properties' section of the mapping. Map, MetadataFieldMapper> mergedMetadataMappers = new HashMap<>(metadataMappersMap); for (MetadataFieldMapper metaMergeWith : mergeWith.metadataMappers) { MetadataFieldMapper mergeInto = mergedMetadataMappers.get(metaMergeWith.getClass()); MetadataFieldMapper merged; - if (mergeInto == null) { + if (mergeInto == null || reason == MergeReason.INDEX_TEMPLATE) { merged = metaMergeWith; } else { merged = (MetadataFieldMapper) mergeInto.merge(metaMergeWith); } mergedMetadataMappers.put(merged.getClass(), merged); } - Map mergedMeta = mergeWith.meta == null ? meta : mergeWith.meta; + + // If we are merging the _meta object as part of applying an index template, then the new object + // is deep-merged into the existing one to allow individual keys to be added or overwritten. For + // standard mapping updates, the new _meta object completely replaces the old one. + Map mergedMeta; + if (mergeWith.meta == null) { + mergedMeta = meta; + } else if (meta == null || reason != MergeReason.INDEX_TEMPLATE) { + mergedMeta = mergeWith.meta; + } else { + mergedMeta = new HashMap<>(mergeWith.meta); + XContentHelper.mergeDefaults(mergedMeta, meta); + } + return new Mapping(indexCreated, mergedRoot, mergedMetadataMappers.values().toArray(new MetadataFieldMapper[0]), mergedMeta); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java index c1f3cf1d64a1a..85e02b985a984 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java @@ -22,6 +22,7 @@ import org.apache.lucene.search.Query; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.Version; +import org.elasticsearch.common.Explicit; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.CopyOnWriteHashMap; import org.elasticsearch.common.logging.DeprecationLogger; @@ -29,6 +30,7 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.mapper.MapperService.MergeReason; import java.io.IOException; import java.util.ArrayList; @@ -61,41 +63,79 @@ public enum Dynamic { public static class Nested { - public static final Nested NO = new Nested(false, false, false); + public static final Nested NO = new Nested(false, new Explicit<>(false, false), new Explicit<>(false, false)); - public static Nested newNested(boolean includeInParent, boolean includeInRoot) { + public static Nested newNested() { + return new Nested(true, new Explicit<>(false, false), new Explicit<>(false, false)); + } + + public static Nested newNested(Explicit includeInParent, Explicit includeInRoot) { return new Nested(true, includeInParent, includeInRoot); } private final boolean nested; + private Explicit includeInParent; + private Explicit includeInRoot; - private final boolean includeInParent; - - private final boolean includeInRoot; - - private Nested(boolean nested, boolean includeInParent, boolean includeInRoot) { + private Nested(boolean nested, Explicit includeInParent, Explicit includeInRoot) { this.nested = nested; this.includeInParent = includeInParent; this.includeInRoot = includeInRoot; } + public void merge(Nested mergeWith, MergeReason reason) { + if (isNested()) { + if (!mergeWith.isNested()) { + throw new IllegalArgumentException("cannot change object mapping from nested to non-nested"); + } + } else { + if (mergeWith.isNested()) { + throw new IllegalArgumentException("cannot change object mapping from non-nested to nested"); + } + } + + if (reason == MergeReason.INDEX_TEMPLATE) { + if (mergeWith.includeInParent.explicit()) { + includeInParent = mergeWith.includeInParent; + } + if (mergeWith.includeInRoot.explicit()) { + includeInRoot = mergeWith.includeInRoot; + } + } else { + if (includeInParent.value() != mergeWith.includeInParent.value()) { + throw new MapperException("the [include_in_parent] parameter can't be updated on a nested object mapping"); + } + if (includeInRoot.value() != mergeWith.includeInRoot.value()) { + throw new MapperException("the [include_in_root] parameter can't be updated on a nested object mapping"); + } + } + } + public boolean isNested() { return nested; } public boolean isIncludeInParent() { - return includeInParent; + return includeInParent.value(); } public boolean isIncludeInRoot() { - return includeInRoot; + return includeInRoot.value(); + } + + public void setIncludeInParent(boolean value) { + includeInParent = new Explicit<>(value, true); + } + + public void setIncludeInRoot(boolean value) { + includeInRoot = new Explicit<>(value, true); } } @SuppressWarnings("rawtypes") public static class Builder extends Mapper.Builder { - protected boolean enabled = Defaults.ENABLED; + protected Explicit enabled = new Explicit<>(true, false); protected Nested nested = Defaults.NESTED; @@ -110,7 +150,7 @@ public Builder(String name) { } public T enabled(boolean enabled) { - this.enabled = enabled; + this.enabled = new Explicit<>(enabled, true); return builder; } @@ -150,7 +190,7 @@ public ObjectMapper build(BuilderContext context) { return objectMapper; } - protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, + protected ObjectMapper createMapper(String name, String fullPath, Explicit enabled, Nested nested, Dynamic dynamic, Map mappers, @Nullable Settings settings) { return new ObjectMapper(name, fullPath, enabled, nested, dynamic, mappers, settings); } @@ -208,8 +248,8 @@ protected static boolean parseObjectOrDocumentTypeProperties(String fieldName, O @SuppressWarnings("rawtypes") protected static void parseNested(String name, Map node, ObjectMapper.Builder builder) { boolean nested = false; - boolean nestedIncludeInParent = false; - boolean nestedIncludeInRoot = false; + Explicit nestedIncludeInParent = new Explicit<>(false, false); + Explicit nestedIncludeInRoot = new Explicit<>(false, false); Object fieldNode = node.get("type"); if (fieldNode!=null) { String type = fieldNode.toString(); @@ -224,18 +264,19 @@ protected static void parseNested(String name, Map node, ObjectM } fieldNode = node.get("include_in_parent"); if (fieldNode != null) { - nestedIncludeInParent = XContentMapValues.nodeBooleanValue(fieldNode, name + ".include_in_parent"); + boolean includeInParent = XContentMapValues.nodeBooleanValue(fieldNode, name + ".include_in_parent"); + nestedIncludeInParent = new Explicit<>(includeInParent, true); node.remove("include_in_parent"); } fieldNode = node.get("include_in_root"); if (fieldNode != null) { - nestedIncludeInRoot = XContentMapValues.nodeBooleanValue(fieldNode, name + ".include_in_root"); + boolean includeInRoot = XContentMapValues.nodeBooleanValue(fieldNode, name + ".include_in_root"); + nestedIncludeInRoot = new Explicit<>(includeInRoot, true); node.remove("include_in_root"); } if (nested) { builder.nested = Nested.newNested(nestedIncludeInParent, nestedIncludeInRoot); } - } @SuppressWarnings("rawtypes") @@ -303,7 +344,7 @@ protected static void parseProperties(ObjectMapper.Builder objBuilder, Map enabled; private final Nested nested; @@ -315,7 +356,7 @@ protected static void parseProperties(ObjectMapper.Builder objBuilder, Map mappers; - ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, + ObjectMapper(String name, String fullPath, Explicit enabled, Nested nested, Dynamic dynamic, Map mappers, Settings settings) { super(name); assert settings != null; @@ -372,7 +413,7 @@ public String typeName() { } public boolean isEnabled() { - return this.enabled; + return this.enabled.value(); } public Mapper getMapper(String field) { @@ -439,64 +480,62 @@ public boolean parentObjectMapperAreNested(MapperService mapperService) { @Override public ObjectMapper merge(Mapper mergeWith) { + return merge(mergeWith, MergeReason.MAPPING_UPDATE); + } + + public ObjectMapper merge(Mapper mergeWith, MergeReason reason) { if (!(mergeWith instanceof ObjectMapper)) { - throw new IllegalArgumentException("Can't merge a non object mapping [" + mergeWith.name() - + "] with an object mapping [" + name() + "]"); + throw new IllegalArgumentException("can't merge a non object mapping [" + mergeWith.name() + "] with an object mapping"); } ObjectMapper mergeWithObject = (ObjectMapper) mergeWith; ObjectMapper merged = clone(); - merged.doMerge(mergeWithObject); + merged.doMerge(mergeWithObject, reason); return merged; } - protected void doMerge(final ObjectMapper mergeWith) { - if (nested().isNested()) { - if (!mergeWith.nested().isNested()) { - throw new IllegalArgumentException("object mapping [" + name() + "] can't be changed from nested to non-nested"); - } - } else { - if (mergeWith.nested().isNested()) { - throw new IllegalArgumentException("object mapping [" + name() + "] can't be changed from non-nested to nested"); - } - } + protected void doMerge(final ObjectMapper mergeWith, MergeReason reason) { + nested().merge(mergeWith.nested(), reason); if (mergeWith.dynamic != null) { this.dynamic = mergeWith.dynamic; } - checkObjectMapperParameters(mergeWith); + if (reason == MergeReason.INDEX_TEMPLATE) { + if (mergeWith.enabled.explicit()) { + this.enabled = mergeWith.enabled; + } + } else if (isEnabled() != mergeWith.isEnabled()) { + throw new MapperException("the [enabled] parameter can't be updated for the object mapping [" + name() + "]"); + } for (Mapper mergeWithMapper : mergeWith) { Mapper mergeIntoMapper = mappers.get(mergeWithMapper.simpleName()); Mapper merged; if (mergeIntoMapper == null) { - // no mapping, simply add it merged = mergeWithMapper; + } else if (mergeIntoMapper instanceof ObjectMapper) { + ObjectMapper objectMapper = (ObjectMapper) mergeIntoMapper; + merged = objectMapper.merge(mergeWithMapper, reason); } else { - // root mappers can only exist here for backcompat, and are merged in Mapping - merged = mergeIntoMapper.merge(mergeWithMapper); + assert mergeIntoMapper instanceof FieldMapper || mergeIntoMapper instanceof FieldAliasMapper; + if (mergeWithMapper instanceof ObjectMapper) { + throw new IllegalArgumentException("can't merge a non object mapping [" + + mergeWithMapper.name() + "] with an object mapping"); + } + + // If we're merging template mappings when creating an index, then a field definition always + // replaces an existing one. + if (reason == MergeReason.INDEX_TEMPLATE) { + merged = mergeWithMapper; + } else { + merged = mergeIntoMapper.merge(mergeWithMapper); + } } putMapper(merged); } } - private void checkObjectMapperParameters(final ObjectMapper mergeWith) { - if (isEnabled() != mergeWith.isEnabled()) { - throw new MapperException("The [enabled] parameter can't be updated for the object mapping [" + name() + "]."); - } - - if (nested().isIncludeInParent() != mergeWith.nested().isIncludeInParent()) { - throw new MapperException("The [include_in_parent] parameter can't be updated for the nested object mapping [" + - name() + "]."); - } - - if (nested().isIncludeInRoot() != mergeWith.nested().isIncludeInRoot()) { - throw new MapperException("The [include_in_root] parameter can't be updated for the nested object mapping [" + - name() + "]."); - } - } - @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { toXContent(builder, params, null); @@ -520,8 +559,8 @@ public void toXContent(XContentBuilder builder, Params params, ToXContent custom if (dynamic != null) { builder.field("dynamic", dynamic.name().toLowerCase(Locale.ROOT)); } - if (enabled != Defaults.ENABLED) { - builder.field("enabled", enabled); + if (isEnabled() != Defaults.ENABLED) { + builder.field("enabled", enabled.value()); } if (custom != null) { diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java index 15c7f568aa657..51bfab935364d 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.mapper.ParseContext.Document; +import org.elasticsearch.index.mapper.MapperService.MergeReason; import java.util.List; @@ -125,7 +126,7 @@ public void addDynamicMappingsUpdate(Mapping update) { if (dynamicMappingsUpdate == null) { dynamicMappingsUpdate = update; } else { - dynamicMappingsUpdate = dynamicMappingsUpdate.merge(update); + dynamicMappingsUpdate = dynamicMappingsUpdate.merge(update, MergeReason.MAPPING_UPDATE); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java index f23b885082be1..8a1876293e99f 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java @@ -29,12 +29,14 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.mapper.DynamicTemplate.XContentFieldType; +import org.elasticsearch.index.mapper.MapperService.MergeReason; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -79,39 +81,11 @@ public Builder dynamicTemplates(Collection templates) { @Override public RootObjectMapper build(BuilderContext context) { - fixRedundantIncludes(this, true); return (RootObjectMapper) super.build(context); } - /** - * Removes redundant root includes in {@link ObjectMapper.Nested} trees to avoid duplicate - * fields on the root mapper when {@code isIncludeInRoot} is {@code true} for a node that is - * itself included into a parent node, for which either {@code isIncludeInRoot} is - * {@code true} or which is transitively included in root by a chain of nodes with - * {@code isIncludeInParent} returning {@code true}. - * @param omb Builder whose children to check. - * @param parentIncluded True iff node is a child of root or a node that is included in - * root - */ - @SuppressWarnings("rawtypes") - private static void fixRedundantIncludes(ObjectMapper.Builder omb, boolean parentIncluded) { - for (Object mapper : omb.mappersBuilders) { - if (mapper instanceof ObjectMapper.Builder) { - ObjectMapper.Builder child = (ObjectMapper.Builder) mapper; - Nested nested = child.nested; - boolean isNested = nested.isNested(); - boolean includeInRootViaParent = parentIncluded && isNested && nested.isIncludeInParent(); - boolean includedInRoot = isNested && nested.isIncludeInRoot(); - if (includeInRootViaParent && includedInRoot) { - child.nested = Nested.newNested(true, false); - } - fixRedundantIncludes(child, includeInRootViaParent || includedInRoot); - } - } - } - @Override - protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, + protected ObjectMapper createMapper(String name, String fullPath, Explicit enabled, Nested nested, Dynamic dynamic, Map mappers, @Nullable Settings settings) { assert !nested.isNested(); return new RootObjectMapper(name, enabled, dynamic, mappers, @@ -121,6 +95,34 @@ protected ObjectMapper createMapper(String name, String fullPath, boolean enable } } + /** + * Removes redundant root includes in {@link ObjectMapper.Nested} trees to avoid duplicate + * fields on the root mapper when {@code isIncludeInRoot} is {@code true} for a node that is + * itself included into a parent node, for which either {@code isIncludeInRoot} is + * {@code true} or which is transitively included in root by a chain of nodes with + * {@code isIncludeInParent} returning {@code true}. + */ + public void fixRedundantIncludes() { + fixRedundantIncludes(this, true); + } + + private static void fixRedundantIncludes(ObjectMapper objectMapper, boolean parentIncluded) { + for (Mapper mapper : objectMapper) { + if (mapper instanceof ObjectMapper) { + ObjectMapper child = (ObjectMapper) mapper; + Nested nested = child.nested(); + boolean isNested = nested.isNested(); + boolean includeInRootViaParent = parentIncluded && isNested && nested.isIncludeInParent(); + boolean includedInRoot = isNested && nested.isIncludeInRoot(); + if (includeInRootViaParent && includedInRoot) { + nested.setIncludeInParent(true); + nested.setIncludeInRoot(false); + } + fixRedundantIncludes(child, includeInRootViaParent || includedInRoot); + } + } + } + public static class TypeParser extends ObjectMapper.TypeParser { @Override @@ -209,7 +211,7 @@ protected boolean processField(RootObjectMapper.Builder builder, String fieldNam private Explicit numericDetection; private Explicit dynamicTemplates; - RootObjectMapper(String name, boolean enabled, Dynamic dynamic, Map mappers, + RootObjectMapper(String name, Explicit enabled, Dynamic dynamic, Map mappers, Explicit dynamicDateTimeFormatters, Explicit dynamicTemplates, Explicit dateDetection, Explicit numericDetection, Settings settings) { super(name, name, enabled, Nested.NO, dynamic, mappers, settings); @@ -244,6 +246,10 @@ public DateFormatter[] dynamicDateTimeFormatters() { return dynamicDateTimeFormatters.value(); } + public DynamicTemplate[] dynamicTemplates() { + return dynamicTemplates.value(); + } + @SuppressWarnings("rawtypes") public Mapper.Builder findTemplateBuilder(ParseContext context, String name, XContentFieldType matchType) { return findTemplateBuilder(context, name, matchType.defaultMappingType(), matchType); @@ -282,25 +288,41 @@ public DynamicTemplate findTemplate(ContentPath path, String name, XContentField } @Override - public RootObjectMapper merge(Mapper mergeWith) { - return (RootObjectMapper) super.merge(mergeWith); + public RootObjectMapper merge(Mapper mergeWith, MergeReason reason) { + return (RootObjectMapper) super.merge(mergeWith, reason); } @Override - protected void doMerge(ObjectMapper mergeWith) { - super.doMerge(mergeWith); + protected void doMerge(ObjectMapper mergeWith, MergeReason reason) { + super.doMerge(mergeWith, reason); RootObjectMapper mergeWithObject = (RootObjectMapper) mergeWith; if (mergeWithObject.numericDetection.explicit()) { this.numericDetection = mergeWithObject.numericDetection; } + if (mergeWithObject.dateDetection.explicit()) { this.dateDetection = mergeWithObject.dateDetection; } + if (mergeWithObject.dynamicDateTimeFormatters.explicit()) { this.dynamicDateTimeFormatters = mergeWithObject.dynamicDateTimeFormatters; } + if (mergeWithObject.dynamicTemplates.explicit()) { - this.dynamicTemplates = mergeWithObject.dynamicTemplates; + if (reason == MergeReason.INDEX_TEMPLATE) { + Map templatesByKey = new LinkedHashMap<>(); + for (DynamicTemplate template : this.dynamicTemplates.value()) { + templatesByKey.put(template.name(), template); + } + for (DynamicTemplate template : mergeWithObject.dynamicTemplates.value()) { + templatesByKey.put(template.name(), template); + } + + DynamicTemplate[] mergedTemplates = templatesByKey.values().toArray(new DynamicTemplate[0]); + this.dynamicTemplates = new Explicit<>(mergedTemplates, true); + } else { + this.dynamicTemplates = mergeWithObject.dynamicTemplates; + } } } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamTests.java index a8681c4139954..e8552e84101cb 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamTests.java @@ -27,7 +27,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -169,66 +168,17 @@ public void testReplaceBackingIndexThrowsExceptionIfReplacingWriteIndex() { expectThrows(IllegalArgumentException.class, () -> original.replaceBackingIndex(indices.get(writeIndexPosition), newBackingIndex)); } - public void testInsertTimestampFieldMapping() { - TimestampField timestampField = new TimestampField("@timestamp", Map.of("type", "date", "meta", Map.of("x", "y"))); - - Map mappings = Map.of("_doc", Map.of("properties", new HashMap<>(Map.of("my_field", Map.of("type", "keyword"))))); - timestampField.insertTimestampFieldMapping(mappings); - Map expectedMapping = Map.of("_doc", Map.of("properties", Map.of("my_field", Map.of("type", "keyword"), - "@timestamp", Map.of("type", "date", "meta", Map.of("x", "y"))))); - assertThat(mappings, equalTo(expectedMapping)); - - // ensure that existing @timestamp definitions get overwritten: - mappings = Map.of("_doc", Map.of("properties", new HashMap<>(Map.of("my_field", Map.of("type", "keyword"), - "@timestamp", new HashMap<>(Map.of("type", "keyword")) )))); - timestampField.insertTimestampFieldMapping(mappings); - expectedMapping = Map.of("_doc", Map.of("properties", Map.of("my_field", Map.of("type", "keyword"), "@timestamp", - Map.of("type", "date", "meta", Map.of("x", "y"))))); + public void testGetTimestampFieldMapping() { + TimestampField field = new TimestampField("@timestamp", Map.of("type", "date", "meta", Map.of("x", "y"))); + Map mappings = field.getTimestampFieldMapping(); + Map expectedMapping = Map.of("_doc", Map.of("properties", + Map.of("@timestamp", Map.of("type", "date", "meta", Map.of("x", "y"))))); assertThat(mappings, equalTo(expectedMapping)); - } - - public void testInsertNestedTimestampFieldMapping() { - TimestampField timestampField = new TimestampField("event.attr.@timestamp", Map.of("type", "date", "meta", Map.of("x", "y"))); - Map mappings = Map.of("_doc", Map.of("properties", Map.of("event", Map.of("properties", Map.of("attr", - Map.of("properties", new HashMap<>(Map.of("my_field", Map.of("type", "keyword"))))))))); - timestampField.insertTimestampFieldMapping(mappings); - Map expectedMapping = Map.of("_doc", Map.of("properties", Map.of("event", Map.of("properties", Map.of("attr", - Map.of("properties", new HashMap<>(Map.of("my_field", Map.of("type", "keyword"), - "@timestamp", Map.of("type", "date", "meta", Map.of("x", "y")))))))))); - assertThat(mappings, equalTo(expectedMapping)); - - // ensure that existing @timestamp definitions get overwritten: - mappings = Map.of("_doc", Map.of("properties", Map.of("event", Map.of("properties", Map.of("attr", - Map.of("properties", new HashMap<>(Map.of("my_field", Map.of("type", "keyword"), - "@timestamp", new HashMap<>(Map.of("type", "keyword")) )))))))); - timestampField.insertTimestampFieldMapping(mappings); + TimestampField nestedField = new TimestampField("event.attr.@timestamp", Map.of("type", "date", "meta", Map.of("x", "y"))); + mappings = nestedField.getTimestampFieldMapping(); expectedMapping = Map.of("_doc", Map.of("properties", Map.of("event", Map.of("properties", Map.of("attr", - Map.of("properties", new HashMap<>(Map.of("my_field", Map.of("type", "keyword"), - "@timestamp", Map.of("type", "date", "meta", Map.of("x", "y")))))))))); - assertThat(mappings, equalTo(expectedMapping)); - - // no event and attr parent objects - mappings = Map.of("_doc", Map.of("properties", new HashMap<>())); - timestampField.insertTimestampFieldMapping(mappings); - expectedMapping = Map.of("_doc", Map.of("properties", Map.of("event", Map.of("properties", Map.of("attr", - Map.of("properties", new HashMap<>(Map.of("@timestamp", Map.of("type", "date", "meta", Map.of("x", "y")))))))))); - assertThat(mappings, equalTo(expectedMapping)); - - // no attr parent object - mappings = Map.of("_doc", Map.of("properties", Map.of("event", Map.of("properties", new HashMap<>())))); - timestampField.insertTimestampFieldMapping(mappings); - expectedMapping = Map.of("_doc", Map.of("properties", Map.of("event", Map.of("properties", Map.of("attr", - Map.of("properties", new HashMap<>(Map.of("@timestamp", Map.of("type", "date", "meta", Map.of("x", "y")))))))))); - assertThat(mappings, equalTo(expectedMapping)); - - // Empty attr parent object - mappings = Map.of("_doc", Map.of("properties", Map.of("event", Map.of("properties", - Map.of("attr", Map.of("properties", new HashMap<>())))))); - timestampField.insertTimestampFieldMapping(mappings); - expectedMapping = Map.of("_doc", Map.of("properties", Map.of("event", Map.of("properties", Map.of("attr", - Map.of("properties", new HashMap<>(Map.of("@timestamp", Map.of("type", "date", "meta", Map.of("x", "y")))))))))); + Map.of("properties", Map.of("@timestamp", Map.of("type", "date", "meta", Map.of("x", "y"))))))))); assertThat(mappings, equalTo(expectedMapping)); } - } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java index ba1120fce627d..ab49aea8fa188 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -73,7 +73,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -929,311 +928,6 @@ public void testDeprecateTranslogRetentionSettings() { + "and [index.translog.retention.size] are deprecated and effectively ignored. They will be removed in a future version."); } - @SuppressWarnings("unchecked") - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/57393") - public void testMappingsMergingIsSmart() throws Exception { - Template ctt1 = new Template(null, - new CompressedXContent("{\"_doc\":{\"_source\":{\"enabled\": false},\"_meta\":{\"ct1\":{\"ver\": \"text\"}}," + - "\"properties\":{\"foo\":{\"type\":\"text\",\"ignore_above\":7,\"analyzer\":\"english\"}}}}"), null); - Template ctt2 = new Template(null, - new CompressedXContent("{\"_doc\":{\"_meta\":{\"ct1\":{\"ver\": \"keyword\"},\"ct2\":\"potato\"}," + - "\"properties\":{\"foo\":{\"type\":\"keyword\",\"ignore_above\":13}}}}"), null); - - ComponentTemplate ct1 = new ComponentTemplate(ctt1, null, null); - ComponentTemplate ct2 = new ComponentTemplate(ctt2, null, null); - - boolean shouldBeText = randomBoolean(); - List composedOf = shouldBeText ? Arrays.asList("ct2", "ct1") : Arrays.asList("ct1", "ct2"); - logger.info("--> the {} analyzer should win ({})", shouldBeText ? "text" : "keyword", composedOf); - ComposableIndexTemplate template = new ComposableIndexTemplate(Collections.singletonList("index"), - null, composedOf, null, null, null, null); - - ClusterState state = ClusterState.builder(ClusterState.EMPTY_STATE) - .metadata(Metadata.builder(Metadata.EMPTY_METADATA) - .put("ct1", ct1) - .put("ct2", ct2) - .put("index-template", template) - .build()) - .build(); - - Map resolved = - MetadataCreateIndexService.resolveV2Mappings("{\"_doc\":{\"_meta\":{\"ct2\":\"eggplant\"}," + - "\"properties\":{\"bar\":{\"type\":\"text\"}}}}", state, - "index-template", new NamedXContentRegistry(Collections.emptyList())); - - assertThat("expected exactly one type but was: " + resolved, resolved.size(), equalTo(1)); - Map innerResolved = (Map) resolved.get(MapperService.SINGLE_MAPPING_NAME); - assertThat("was: " + innerResolved, innerResolved.size(), equalTo(3)); - - Map nonProperties = new HashMap<>(innerResolved); - nonProperties.remove("properties"); - Map expectedNonProperties = new HashMap<>(); - expectedNonProperties.put("_source", Collections.singletonMap("enabled", false)); - Map meta = new HashMap<>(); - meta.put("ct2", "eggplant"); - if (shouldBeText) { - meta.put("ct1", Collections.singletonMap("ver", "text")); - } else { - meta.put("ct1", Collections.singletonMap("ver", "keyword")); - } - expectedNonProperties.put("_meta", meta); - assertThat(nonProperties, equalTo(expectedNonProperties)); - - Map innerInnerResolved = (Map) innerResolved.get("properties"); - assertThat(innerInnerResolved.size(), equalTo(2)); - assertThat(innerInnerResolved.get("bar"), equalTo(Collections.singletonMap("type", "text"))); - Map fooMappings = new HashMap<>(); - if (shouldBeText) { - fooMappings.put("type", "text"); - fooMappings.put("ignore_above", 7); - fooMappings.put("analyzer", "english"); - } else { - fooMappings.put("type", "keyword"); - fooMappings.put("ignore_above", 13); - } - assertThat(innerInnerResolved.get("foo"), equalTo(fooMappings)); - } - - @SuppressWarnings("unchecked") - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/57393") - public void testMappingsMergingHandlesDots() throws Exception { - Template ctt1 = new Template(null, - new CompressedXContent("{\"_doc\":{\"properties\":{\"foo\":{\"properties\":{\"bar\":{\"type\": \"long\"}}}}}}"), null); - Template ctt2 = new Template(null, - new CompressedXContent("{\"_doc\":{\"properties\":{\"foo.bar\":{\"type\": \"text\",\"analyzer\":\"english\"}}}}"), null); - - ComponentTemplate ct1 = new ComponentTemplate(ctt1, null, null); - ComponentTemplate ct2 = new ComponentTemplate(ctt2, null, null); - - ComposableIndexTemplate template = new ComposableIndexTemplate(Collections.singletonList("index"), - null, Arrays.asList("ct2", "ct1"), null, null, null, null); - - ClusterState state = ClusterState.builder(ClusterState.EMPTY_STATE) - .metadata(Metadata.builder(Metadata.EMPTY_METADATA) - .put("ct1", ct1) - .put("ct2", ct2) - .put("index-template", template) - .build()) - .build(); - - Map resolved = - MetadataCreateIndexService.resolveV2Mappings("{}", state, - "index-template", new NamedXContentRegistry(Collections.emptyList())); - - assertThat("expected exactly one type but was: " + resolved, resolved.size(), equalTo(1)); - Map innerResolved = (Map) resolved.get(MapperService.SINGLE_MAPPING_NAME); - assertThat("was: " + innerResolved, innerResolved.size(), equalTo(1)); - - Map innerInnerResolved = (Map) innerResolved.get("properties"); - assertThat(innerInnerResolved.size(), equalTo(1)); - assertThat(innerInnerResolved.get("foo"), - equalTo(Collections.singletonMap("properties", Collections.singletonMap("bar", Collections.singletonMap("type", "long"))))); - } - - public void testMappingsMergingThrowsOnConflictDots() throws Exception { - Template ctt1 = new Template(null, - new CompressedXContent("{\"_doc\":{\"properties\":{\"foo\":{\"properties\":{\"bar\":{\"type\": \"long\"}}}}}}"), null); - Template ctt2 = new Template(null, - new CompressedXContent("{\"_doc\":{\"properties\":{\"foo.bar\":{\"type\": \"text\",\"analyzer\":\"english\"}}}}"), null); - - ComponentTemplate ct1 = new ComponentTemplate(ctt1, null, null); - ComponentTemplate ct2 = new ComponentTemplate(ctt2, null, null); - - ComposableIndexTemplate template = new ComposableIndexTemplate(Collections.singletonList("index"), - null, Arrays.asList("ct2", "ct1"), null, null, null, null); - - ClusterState state = ClusterState.builder(ClusterState.EMPTY_STATE) - .metadata(Metadata.builder(Metadata.EMPTY_METADATA) - .put("ct1", ct1) - .put("ct2", ct2) - .put("index-template", template) - .build()) - .build(); - - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> MetadataCreateIndexService.resolveV2Mappings("{}", state, - "index-template", new NamedXContentRegistry(Collections.emptyList()))); - - assertThat(e.getMessage(), containsString("mapping fields [foo.bar] cannot be replaced during template composition")); - } - - @SuppressWarnings("unchecked") - public void testDedupTemplateDynamicTemplates() throws Exception { - Template template = new Template(null, - new CompressedXContent("{\"_doc\":{\"_source\":{\"enabled\": false}, \"dynamic_templates\": [" + - "{\n" + - " \"docker.container.labels\": {\n" + - " \"mapping\": {\n" + - " \"type\": \"keyword\"\n" + - " },\n" + - " \"match_mapping_type\": \"string\",\n" + - " \"path_match\": \"labels.*\"\n" + - " }\n" + - " },\n" + - " {\n" + - " \"docker.container.labels\": {\n" + - " \"mapping\": {\n" + - " \"type\": \"keyword\"\n" + - " },\n" + - " \"match_mapping_type\": \"string\",\n" + - " \"path_match\": \"docker.container.labels.*\"\n" + - " }\n" + - "}]}}"), null); - - ComposableIndexTemplate indexTemplate = new ComposableIndexTemplate(Collections.singletonList("index"), - template, null, null, null, null); - - ClusterState state = ClusterState.builder(ClusterState.EMPTY_STATE) - .metadata(Metadata.builder(Metadata.EMPTY_METADATA) - .put("index-template", indexTemplate) - .build()) - .build(); - - Map resolved = - MetadataCreateIndexService.resolveV2Mappings("{}", state, - "index-template", new NamedXContentRegistry(Collections.emptyList())); - - Map doc = (Map) resolved.get(MapperService.SINGLE_MAPPING_NAME); - List> dynamicTemplates = (List>) doc.get("dynamic_templates"); - assertThat(dynamicTemplates.size(), is(1)); - Map dynamicMapping = (Map) dynamicTemplates.get(0).get("docker.container.labels"); - assertThat(dynamicMapping, is(notNullValue())); - assertThat("last mapping with the same name must override previously defined mappings with the same name", - dynamicMapping.get("path_match"), is("docker.container.labels.*")); - } - - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/57393") - public void testDedupRequestDynamicTemplates() throws Exception { - String requestMappingJson = "{\"_doc\":{\"_source\":{\"enabled\": false}, \"dynamic_templates\": [" + - "{\n" + - " \"docker.container.labels\": {\n" + - " \"mapping\": {\n" + - " \"type\": \"keyword\"\n" + - " },\n" + - " \"match_mapping_type\": \"string\",\n" + - " \"path_match\": \"labels.*\"\n" + - " }\n" + - " },\n" + - " {\n" + - " \"docker.container.labels\": {\n" + - " \"mapping\": {\n" + - " \"type\": \"keyword\"\n" + - " },\n" + - " \"match_mapping_type\": \"string\",\n" + - " \"path_match\": \"source.request.*\"\n" + - " }\n" + - "}]}}"; - - String templateMappingJson = "{\"_doc\":{\"_source\":{\"enabled\": false}, \"dynamic_templates\": [" + - "{\n" + - " \"docker.container.labels\": {\n" + - " \"mapping\": {\n" + - " \"type\": \"text\",\n" + - " \"copy_to\": \"text_labels\"\n" + - " },\n" + - " \"match_mapping_type\": \"string\",\n" + - " \"path_match\": \"source.template.*\"\n" + - " }\n" + - " }\n" + - "]}}"; - Template template = new Template(null, new CompressedXContent(templateMappingJson), null); - - ComposableIndexTemplate indexTemplate = new ComposableIndexTemplate(Collections.singletonList("index"), - template, null, null, null, null); - - ClusterState state = ClusterState.builder(ClusterState.EMPTY_STATE) - .metadata(Metadata.builder(Metadata.EMPTY_METADATA) - .put("index-template", indexTemplate) - .build()) - .build(); - - Map resolved = - MetadataCreateIndexService.resolveV2Mappings(requestMappingJson, state, - "index-template", new NamedXContentRegistry(Collections.emptyList())); - - Map doc = (Map) resolved.get(MapperService.SINGLE_MAPPING_NAME); - List> dynamicTemplates = (List>) doc.get("dynamic_templates"); - assertThat(dynamicTemplates.size(), is(1)); - Map dynamicMapping = (Map) dynamicTemplates.get(0).get("docker.container.labels"); - assertThat(dynamicMapping, is(notNullValue())); - assertThat("last mapping with the same name must override previously defined mappings with the same name", - dynamicMapping.get("path_match"), is("source.request.*")); - Map mapping = (Map) dynamicMapping.get("mapping"); - assertThat("the dynamic template defined in the request must not be merged with the dynamic template with the " + - "same name defined in the index template", mapping.size(), is(1)); - assertThat(mapping.get("type"), is("keyword")); - } - - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/57393") - public void testMultipleComponentTemplatesDefineSameDynamicTemplate() throws Exception { - String ct1Mapping = "{\"_doc\":{\"_source\":{\"enabled\": false}, \"dynamic_templates\": [" + - "{\n" + - " \"docker.container.labels\": {\n" + - " \"mapping\": {\n" + - " \"type\": \"text\",\n" + - " \"copy_to\": \"text_labels\"\n" + - " },\n" + - " \"match_mapping_type\": \"string\",\n" + - " \"path_match\": \"source.first.ct.*\"\n" + - " }\n" + - " },\n" + - "{\n" + - " \"other.labels\": {\n" + - " \"mapping\": {\n" + - " \"type\": \"keyword\"\n" + - " },\n" + - " \"match_mapping_type\": \"string\",\n" + - " \"path_match\": \"source.first.ct.other.labels*\"\n" + - " }\n" + - " }\n" + - "]}}"; - String ct2Mapping = "{\"_doc\":{\"_source\":{\"enabled\": false}, \"dynamic_templates\": [" + - "{\n" + - " \"docker.container.labels\": {\n" + - " \"mapping\": {\n" + - " \"type\": \"keyword\"\n" + - " },\n" + - " \"match_mapping_type\": \"string\",\n" + - " \"path_match\": \"source.second.ct.*\"\n" + - " }\n" + - " }\n" + - "]}}"; - - Template ctt1 = new Template(null, new CompressedXContent(ct1Mapping), null); - Template ctt2 = new Template(null, new CompressedXContent(ct2Mapping), null); - ComponentTemplate ct1 = new ComponentTemplate(ctt1, null, null); - ComponentTemplate ct2 = new ComponentTemplate(ctt2, null, null); - - ComposableIndexTemplate template = new ComposableIndexTemplate(Collections.singletonList("index"), - null, Arrays.asList("ct1", "ct2"), null, null, null); - - ClusterState state = ClusterState.builder(ClusterState.EMPTY_STATE) - .metadata(Metadata.builder(Metadata.EMPTY_METADATA) - .put("ct1", ct1) - .put("ct2", ct2) - .put("index-template", template) - .build()) - .build(); - - Map resolved = - MetadataCreateIndexService.resolveV2Mappings("{}", state, - "index-template", new NamedXContentRegistry(Collections.emptyList())); - - Map doc = (Map) resolved.get(MapperService.SINGLE_MAPPING_NAME); - List> dynamicTemplates = (List>) doc.get("dynamic_templates"); - assertThat(dynamicTemplates.size(), is(2)); - Map dockerLabelsDynamicTemplate = dynamicTemplates.get(0).get("docker.container.labels") != null ? - dynamicTemplates.get(0) : dynamicTemplates.get(1); - Map dynamicMapping = (Map) dockerLabelsDynamicTemplate.get("docker.container.labels"); - assertThat(dynamicMapping, is(notNullValue())); - assertThat("dynamic template defined in the last defined component template must override the previously defined dynamic templates", - dynamicMapping.get("path_match"), is("source.second.ct.*")); - Map mapping = (Map) dynamicMapping.get("mapping"); - assertThat("the dynamic template defined in the second component template must not be merged with the dynamic template with the " + - "same name defined in the first component template", mapping.size(), is(1)); - assertThat(mapping.get("type"), is("keyword")); - } - private IndexTemplateMetadata addMatchingTemplate(Consumer configurator) { IndexTemplateMetadata.Builder builder = templateMetadataBuilder("template1", "te*"); configurator.accept(builder); diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java index 8e96dc489c969..9cb58f436c420 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateServiceTests.java @@ -706,7 +706,7 @@ public void testResolveConflictingMappings() throws Exception { List.of("ct_low", "ct_high"), 0L, 1L, null, null); state = service.addIndexTemplateV2(state, true, "my-template", it); - List mappings = MetadataIndexTemplateService.resolveMappings(state, "my-template"); + List mappings = MetadataIndexTemplateService.collectMappings(state, "my-template"); assertNotNull(mappings); assertThat(mappings.size(), equalTo(3)); @@ -769,7 +769,7 @@ public void testResolveMappings() throws Exception { List.of("ct_low", "ct_high"), 0L, 1L, null, null); state = service.addIndexTemplateV2(state, true, "my-template", it); - List mappings = MetadataIndexTemplateService.resolveMappings(state, "my-template"); + List mappings = MetadataIndexTemplateService.collectMappings(state, "my-template"); assertNotNull(mappings); assertThat(mappings.size(), equalTo(3)); @@ -973,7 +973,7 @@ public void testIndexTemplateFailsToOverrideComponentTemplateMappingField() thr assertNotNull(e.getCause().getCause()); assertThat(e.getCause().getCause().getMessage(), - containsString("mapping fields [field2] cannot be replaced during template composition")); + containsString("can't merge a non object mapping [field2] with an object mapping")); } /** @@ -1037,7 +1037,7 @@ public void testUpdateComponentTemplateFailsIfResolvedIndexTemplatesWouldBeInval assertNotNull(e.getCause().getCause()); assertThat(e.getCause().getCause().getMessage(), - containsString("mapping fields [field2] cannot be replaced during template composition")); + containsString("can't merge a non object mapping [field2] with an object mapping")); } public void testPutExistingComponentTemplateIsNoop() throws Exception { diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java index 73a679ad863af..7677e55449afa 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java @@ -416,7 +416,7 @@ public void testMergeText() throws Exception { DocumentMapper update = indexService.mapperService().parse("_doc", new CompressedXContent(mappingUpdate)); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> mapper.merge(update.mapping())); + () -> mapper.merge(update.mapping(), MergeReason.MAPPING_UPDATE)); assertEquals("mapper [date] cannot be changed from type [date] to [text]", e.getMessage()); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DocumentMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DocumentMapperTests.java index 200546e3e3557..e935b045815eb 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DocumentMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DocumentMapperTests.java @@ -27,9 +27,11 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.mapper.MapperService.MergeReason; import org.elasticsearch.test.ESSingleNodeTestCase; import java.io.IOException; +import java.util.Map; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -41,8 +43,7 @@ public class DocumentMapperTests extends ESSingleNodeTestCase { - public void test1Merge() throws Exception { - + public void testAddFields() throws Exception { String stage1Mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("person").startObject("properties") .startObject("name").field("type", "text").endObject() .endObject().endObject().endObject()); @@ -56,14 +57,15 @@ public void test1Merge() throws Exception { .endObject().endObject().endObject()); DocumentMapper stage2 = parser.parse("person", new CompressedXContent(stage2Mapping)); - DocumentMapper merged = stage1.merge(stage2.mapping()); + MergeReason reason = randomFrom(MergeReason.MAPPING_UPDATE, MergeReason.INDEX_TEMPLATE); + DocumentMapper merged = stage1.merge(stage2.mapping(), reason); // stage1 mapping should not have been modified assertThat(stage1.mappers().getMapper("age"), nullValue()); assertThat(stage1.mappers().getMapper("obj1.prop1"), nullValue()); // but merged should assertThat(merged.mappers().getMapper("age"), notNullValue()); assertThat(merged.mappers().getMapper("obj1.prop1"), notNullValue()); - } +} public void testMergeObjectDynamic() throws Exception { DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser(); @@ -76,7 +78,7 @@ public void testMergeObjectDynamic() throws Exception { DocumentMapper withDynamicMapper = parser.parse("type1", new CompressedXContent(withDynamicMapping)); assertThat(withDynamicMapper.root().dynamic(), equalTo(ObjectMapper.Dynamic.FALSE)); - DocumentMapper merged = mapper.merge(withDynamicMapper.mapping()); + DocumentMapper merged = mapper.merge(withDynamicMapper.mapping(), MergeReason.MAPPING_UPDATE); assertThat(merged.root().dynamic(), equalTo(ObjectMapper.Dynamic.FALSE)); } @@ -90,19 +92,20 @@ public void testMergeObjectAndNested() throws Exception { .startObject("obj").field("type", "nested").endObject() .endObject().endObject().endObject()); DocumentMapper nestedMapper = parser.parse("type1", new CompressedXContent(nestedMapping)); + MergeReason reason = randomFrom(MergeReason.MAPPING_UPDATE, MergeReason.INDEX_TEMPLATE); try { - objectMapper.merge(nestedMapper.mapping()); + objectMapper.merge(nestedMapper.mapping(), reason); fail(); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), containsString("object mapping [obj] can't be changed from non-nested to nested")); + assertThat(e.getMessage(), containsString("cannot change object mapping from non-nested to nested")); } try { - nestedMapper.merge(objectMapper.mapping()); + nestedMapper.merge(objectMapper.mapping(), reason); fail(); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), containsString("object mapping [obj] can't be changed from nested to non-nested")); + assertThat(e.getMessage(), containsString("cannot change object mapping from nested to non-nested")); } } @@ -126,7 +129,7 @@ public void testMergeSearchAnalyzer() throws Exception { .endObject().endObject() .endObject().endObject()); - mapperService.merge("type", new CompressedXContent(mapping2), MapperService.MergeReason.MAPPING_UPDATE); + mapperService.merge("type", new CompressedXContent(mapping2), MergeReason.MAPPING_UPDATE); assertThat(mapperService.fieldType("field").searchAnalyzer().name(), equalTo("keyword")); } @@ -149,13 +152,14 @@ public void testChangeSearchAnalyzerToDefault() throws Exception { .endObject().endObject() .endObject().endObject()); - mapperService.merge("type", new CompressedXContent(mapping2), MapperService.MergeReason.MAPPING_UPDATE); + mapperService.merge("type", new CompressedXContent(mapping2), MergeReason.MAPPING_UPDATE); assertThat(mapperService.fieldType("field").searchAnalyzer().name(), equalTo("standard")); } public void testConcurrentMergeTest() throws Throwable { final MapperService mapperService = createIndex("test").mapperService(); - mapperService.merge("test", new CompressedXContent("{\"test\":{}}"), MapperService.MergeReason.MAPPING_UPDATE); + MergeReason reason = randomFrom(MergeReason.MAPPING_UPDATE, MergeReason.INDEX_TEMPLATE); + mapperService.merge("test", new CompressedXContent("{\"test\":{}}"), reason); final DocumentMapper documentMapper = mapperService.documentMapper(); DocumentFieldMappers dfm = documentMapper.mappers(); @@ -184,7 +188,7 @@ public void run() { Mapping update = doc.dynamicMappingsUpdate(); assert update != null; lastIntroducedFieldName.set(fieldName); - mapperService.merge("test", new CompressedXContent(update.toString()), MapperService.MergeReason.MAPPING_UPDATE); + mapperService.merge("test", new CompressedXContent(update.toString()), MergeReason.MAPPING_UPDATE); } } catch (Exception e) { error.set(e); @@ -220,6 +224,7 @@ public void run() { } public void testDoNotRepeatOriginalMapping() throws IOException { + MergeReason reason = randomFrom(MergeReason.MAPPING_UPDATE, MergeReason.INDEX_TEMPLATE); CompressedXContent mapping = new CompressedXContent(BytesReference.bytes(XContentFactory.jsonBuilder().startObject() .startObject("type") .startObject("_source") @@ -227,7 +232,7 @@ public void testDoNotRepeatOriginalMapping() throws IOException { .endObject() .endObject().endObject())); MapperService mapperService = createIndex("test").mapperService(); - mapperService.merge("type", mapping, MapperService.MergeReason.MAPPING_UPDATE); + mapperService.merge("type", mapping, reason); CompressedXContent update = new CompressedXContent(BytesReference.bytes(XContentFactory.jsonBuilder().startObject() .startObject("type") @@ -237,12 +242,32 @@ public void testDoNotRepeatOriginalMapping() throws IOException { .endObject() .endObject() .endObject().endObject())); - DocumentMapper mapper = mapperService.merge("type", update, MapperService.MergeReason.MAPPING_UPDATE); + DocumentMapper mapper = mapperService.merge("type", update, reason); assertNotNull(mapper.mappers().getMapper("foo")); assertFalse(mapper.sourceMapper().enabled()); } + public void testMergeMetadataFieldsForIndexTemplates() throws IOException { + CompressedXContent mapping = new CompressedXContent(BytesReference.bytes(XContentFactory.jsonBuilder().startObject() + .startObject("type") + .startObject("_source") + .field("enabled", false) + .endObject() + .endObject().endObject())); + MapperService mapperService = createIndex("test").mapperService(); + mapperService.merge("type", mapping, MergeReason.INDEX_TEMPLATE); + + CompressedXContent update = new CompressedXContent(BytesReference.bytes(XContentFactory.jsonBuilder().startObject() + .startObject("type") + .startObject("_source") + .field("enabled", true) + .endObject() + .endObject().endObject())); + DocumentMapper mapper = mapperService.merge("type", update, MergeReason.INDEX_TEMPLATE); + assertTrue(mapper.sourceMapper().enabled()); + } + public void testMergeMeta() throws IOException { DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser(); @@ -270,7 +295,8 @@ public void testMergeMeta() throws IOException { .endObject()); DocumentMapper updatedMapper = parser.parse("test", new CompressedXContent(updateMapping)); - assertThat(initMapper.merge(updatedMapper.mapping()).meta().get("foo"), equalTo("bar")); + DocumentMapper mergedMapper = initMapper.merge(updatedMapper.mapping(), MergeReason.MAPPING_UPDATE); + assertThat(mergedMapper.meta().get("foo"), equalTo("bar")); updateMapping = Strings .toString(XContentFactory.jsonBuilder() @@ -283,6 +309,51 @@ public void testMergeMeta() throws IOException { .endObject()); updatedMapper = parser.parse("test", new CompressedXContent(updateMapping)); - assertThat(initMapper.merge(updatedMapper.mapping()).meta().get("foo"), equalTo("new_bar")); + mergedMapper = initMapper.merge(updatedMapper.mapping(), MergeReason.MAPPING_UPDATE); + assertThat(mergedMapper.meta().get("foo"), equalTo("new_bar")); + } + + public void testMergeMetaForIndexTemplate() throws IOException { + DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser(); + + String initMapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("_meta") + .field("field", "value") + .startObject("object") + .field("field1", "value1") + .field("field2", "value2") + .endObject() + .endObject() + .endObject()); + DocumentMapper initMapper = parser.parse(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(initMapping)); + + Map expected = Map.of("field", "value", + "object", Map.of("field1", "value1", "field2", "value2")); + assertThat(initMapper.meta(), equalTo(expected)); + + String updateMapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("name").field("type", "text").endObject() + .endObject() + .endObject()); + DocumentMapper updatedMapper = parser.parse(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(updateMapping)); + DocumentMapper mergedMapper = initMapper.merge(updatedMapper.mapping(), MergeReason.INDEX_TEMPLATE); + assertThat(mergedMapper.meta(), equalTo(expected)); + + updateMapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("_meta") + .field("field", "value") + .startObject("object") + .field("field2", "new_value") + .field("field3", "value3") + .endObject() + .endObject() + .endObject()); + updatedMapper = parser.parse(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(updateMapping)); + mergedMapper = mergedMapper.merge(updatedMapper.mapping(), MergeReason.INDEX_TEMPLATE); + + expected = Map.of("field", "value", + "object", Map.of("field1", "value1", "field2", "new_value", "field3", "value3")); + assertThat(mergedMapper.meta(), equalTo(expected)); } } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java index e0322c082cf57..4bf18211a3723 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java @@ -341,6 +341,7 @@ public void testObject() throws Exception { // original mapping not modified assertEquals(mapping, serialize(mapper)); // but we have an update + String serializedUpdate = serialize(update); assertEquals(Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") .startObject("foo").startObject("properties").startObject("bar").startObject("properties").startObject("baz") .field("type", "text") diff --git a/server/src/test/java/org/elasticsearch/index/mapper/MapperMergeValidatorTests.java b/server/src/test/java/org/elasticsearch/index/mapper/MapperMergeValidatorTests.java index 1a5b8fa0d3403..7b6f06f38875c 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MapperMergeValidatorTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MapperMergeValidatorTests.java @@ -20,6 +20,7 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.Explicit; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESTestCase; @@ -165,14 +166,16 @@ public void testFieldAliasWithDifferentNestedScopes() { .build(); private static ObjectMapper createObjectMapper(String name) { - return new ObjectMapper(name, name, true, + return new ObjectMapper(name, name, + new Explicit<>(true, false), ObjectMapper.Nested.NO, ObjectMapper.Dynamic.FALSE, emptyMap(), SETTINGS); } private static ObjectMapper createNestedObjectMapper(String name) { - return new ObjectMapper(name, name, true, - ObjectMapper.Nested.newNested(false, false), + return new ObjectMapper(name, name, + new Explicit<>(true, false), + ObjectMapper.Nested.newNested(), ObjectMapper.Dynamic.FALSE, emptyMap(), SETTINGS); } } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java index 661306aea4f50..3f5d5876b7b6c 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java @@ -386,17 +386,21 @@ public void testMultiRootAndNested1() throws Exception { * lead to duplicate fields on the root document. */ public void testMultipleLevelsIncludeRoot1() throws Exception { + MapperService mapperService = createIndex("test").mapperService(); + String mapping = Strings.toString(XContentFactory.jsonBuilder() - .startObject().startObject("type").startObject("properties") + .startObject().startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("properties") .startObject("nested1").field("type", "nested").field("include_in_root", true) .field("include_in_parent", true).startObject("properties") .startObject("nested2").field("type", "nested").field("include_in_root", true) .field("include_in_parent", true) .endObject().endObject().endObject() .endObject().endObject().endObject()); + MergeReason mergeReason = randomFrom(MergeReason.MAPPING_UPDATE, MergeReason.INDEX_TEMPLATE); - DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser() - .parse("type", new CompressedXContent(mapping)); + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(mapping), mergeReason); + DocumentMapper docMapper = mapperService.documentMapper(); ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() @@ -418,8 +422,11 @@ public void testMultipleLevelsIncludeRoot1() throws Exception { * {@code false} and {@code include_in_root} set to {@code true}. */ public void testMultipleLevelsIncludeRoot2() throws Exception { + MapperService mapperService = createIndex("test").mapperService(); + String mapping = Strings.toString(XContentFactory.jsonBuilder() - .startObject().startObject("type").startObject("properties") + .startObject().startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("properties") .startObject("nested1").field("type", "nested") .field("include_in_root", true).field("include_in_parent", true).startObject("properties") .startObject("nested2").field("type", "nested") @@ -428,9 +435,10 @@ public void testMultipleLevelsIncludeRoot2() throws Exception { .field("include_in_root", true).field("include_in_parent", true) .endObject().endObject().endObject().endObject().endObject() .endObject().endObject().endObject()); + MergeReason mergeReason = randomFrom(MergeReason.MAPPING_UPDATE, MergeReason.INDEX_TEMPLATE); - DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser() - .parse("type", new CompressedXContent(mapping)); + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(mapping), mergeReason); + DocumentMapper docMapper = mapperService.documentMapper(); ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() @@ -445,6 +453,66 @@ public void testMultipleLevelsIncludeRoot2() throws Exception { assertThat(fields.size(), equalTo(new HashSet<>(fields).size())); } + /** + * Same as {@link NestedObjectMapperTests#testMultipleLevelsIncludeRoot1()} but tests that + * the redundant includes are removed even if each individual mapping doesn't contain the + * redundancy, only the merged mapping does. + */ + public void testMultipleLevelsIncludeRootWithMerge() throws Exception { + MapperService mapperService = createIndex("test").mapperService(); + + String firstMapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("properties") + .startObject("nested1") + .field("type", "nested") + .field("include_in_root", true) + .startObject("properties") + .startObject("nested2") + .field("type", "nested") + .field("include_in_root", true) + .field("include_in_parent", true) + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject()); + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(firstMapping), MergeReason.INDEX_TEMPLATE); + + String secondMapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("properties") + .startObject("nested1") + .field("type", "nested") + .field("include_in_root", true) + .field("include_in_parent", true) + .startObject("properties") + .startObject("nested2") + .field("type", "nested") + .field("include_in_root", true) + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject()); + + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(secondMapping), MergeReason.INDEX_TEMPLATE); + DocumentMapper docMapper = mapperService.documentMapper(); + + ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", + BytesReference.bytes(XContentFactory.jsonBuilder() + .startObject().startArray("nested1") + .startObject().startArray("nested2").startObject().field("foo", "bar") + .endObject().endArray().endObject().endArray() + .endObject()), + XContentType.JSON)); + + final Collection fields = doc.rootDoc().getFields(); + assertThat(fields.size(), equalTo(new HashSet<>(fields).size())); + } + public void testNestedArrayStrict() throws Exception { String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") .startObject("nested1").field("type", "nested").field("dynamic", "strict").startObject("properties") @@ -758,7 +826,7 @@ public void testMergeNestedMappings() throws IOException { // cannot update `include_in_parent` dynamically MapperException e1 = expectThrows(MapperException.class, () -> mapperService.merge("type", new CompressedXContent(mapping1), MergeReason.MAPPING_UPDATE)); - assertEquals("The [include_in_parent] parameter can't be updated for the nested object mapping [nested1].", e1.getMessage()); + assertEquals("the [include_in_parent] parameter can't be updated on a nested object mapping", e1.getMessage()); String mapping2 = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") .startObject("nested1").field("type", "nested").field("include_in_root", true) @@ -767,6 +835,6 @@ public void testMergeNestedMappings() throws IOException { // cannot update `include_in_root` dynamically MapperException e2 = expectThrows(MapperException.class, () -> mapperService.merge("type", new CompressedXContent(mapping2), MergeReason.MAPPING_UPDATE)); - assertEquals("The [include_in_root] parameter can't be updated for the nested object mapping [nested1].", e2.getMessage()); + assertEquals("the [include_in_root] parameter can't be updated on a nested object mapping", e2.getMessage()); } } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java index 0967bcff81f80..2b96686587fff 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.mapper; import org.elasticsearch.Version; +import org.elasticsearch.common.Explicit; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.mapper.FieldMapper.CopyTo; import org.elasticsearch.index.mapper.FieldMapper.MultiFields; @@ -32,6 +33,7 @@ import static java.util.Collections.emptyMap; import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_VERSION_CREATED; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.notNullValue; public class ObjectMapperMergeTests extends ESTestCase { @@ -83,26 +85,44 @@ public void testMergeWhenDisablingField() { // WHEN merging mappings // THEN a MapperException is thrown with an excepted message MapperException e = expectThrows(MapperException.class, () -> rootObjectMapper.merge(mergeWith)); - assertEquals("The [enabled] parameter can't be updated for the object mapping [foo].", e.getMessage()); + assertEquals("the [enabled] parameter can't be updated for the object mapping [foo]", e.getMessage()); } - public void testMergeWhenEnablingField() { - // GIVEN a mapping with "disabled" field enabled + public void testMergeEnabled() { ObjectMapper mergeWith = createMapping(true, true, true, false); - // WHEN merging mappings - // THEN a MapperException is thrown with an excepted message MapperException e = expectThrows(MapperException.class, () -> rootObjectMapper.merge(mergeWith)); - assertEquals("The [enabled] parameter can't be updated for the object mapping [disabled].", e.getMessage()); + assertEquals("the [enabled] parameter can't be updated for the object mapping [disabled]", e.getMessage()); + + ObjectMapper result = rootObjectMapper.merge(mergeWith, MapperService.MergeReason.INDEX_TEMPLATE); + assertTrue(result.isEnabled()); } - public void testDisableRootMapper() { + public void testMergeEnabledForRootMapper() { String type = MapperService.SINGLE_MAPPING_NAME; ObjectMapper firstMapper = createRootObjectMapper(type, true, Collections.emptyMap()); ObjectMapper secondMapper = createRootObjectMapper(type, false, Collections.emptyMap()); MapperException e = expectThrows(MapperException.class, () -> firstMapper.merge(secondMapper)); - assertEquals("The [enabled] parameter can't be updated for the object mapping [" + type + "].", e.getMessage()); + assertEquals("the [enabled] parameter can't be updated for the object mapping [" + type + "]", e.getMessage()); + + ObjectMapper result = firstMapper.merge(secondMapper, MapperService.MergeReason.INDEX_TEMPLATE); + assertFalse(result.isEnabled()); + } + + public void testMergeNested() { + String type = MapperService.SINGLE_MAPPING_NAME; + ObjectMapper firstMapper = createNestedMapper(type, + ObjectMapper.Nested.newNested(new Explicit<>(true, true), new Explicit<>(true, true))); + ObjectMapper secondMapper = createNestedMapper(type, + ObjectMapper.Nested.newNested(new Explicit<>(false, true), new Explicit<>(false, false))); + + MapperException e = expectThrows(MapperException.class, () -> firstMapper.merge(secondMapper)); + assertThat(e.getMessage(), containsString("[include_in_parent] parameter can't be updated on a nested object mapping")); + + ObjectMapper result = firstMapper.merge(secondMapper, MapperService.MergeReason.INDEX_TEMPLATE); + assertFalse(result.nested().isIncludeInParent()); + assertTrue(result.nested().isIncludeInRoot()); } private static RootObjectMapper createRootObjectMapper(String name, boolean enabled, Map mappers) { @@ -118,13 +138,21 @@ private static RootObjectMapper createRootObjectMapper(String name, boolean enab private static ObjectMapper createObjectMapper(String name, boolean enabled, Map mappers) { final Settings indexSettings = Settings.builder().put(SETTING_VERSION_CREATED, Version.CURRENT).build(); final Mapper.BuilderContext context = new Mapper.BuilderContext(indexSettings, new ContentPath()); - final ObjectMapper mapper = new ObjectMapper.Builder(name).enabled(enabled).build(context); + final ObjectMapper mapper = new ObjectMapper.Builder<>(name).enabled(enabled).build(context); mappers.values().forEach(mapper::putMapper); return mapper; } + private static ObjectMapper createNestedMapper(String name, ObjectMapper.Nested nested) { + final Settings indexSettings = Settings.builder().put(SETTING_VERSION_CREATED, Version.CURRENT).build(); + final Mapper.BuilderContext context = new Mapper.BuilderContext(indexSettings, new ContentPath()); + return new ObjectMapper.Builder<>(name) + .nested(nested) + .build(context); + } + private static TextFieldMapper createTextFieldMapper(String name) { final TextFieldType fieldType = new TextFieldType(name); return new TextFieldMapper(name, TextFieldMapper.Defaults.FIELD_TYPE, fieldType, -1, diff --git a/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperTests.java index 496513a9af85d..bf2ccc1ca6c42 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperTests.java @@ -162,6 +162,7 @@ public void testFieldPropertiesArray() throws Exception { } public void testMerge() throws IOException { + MergeReason reason = randomFrom(MergeReason.values()); String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject() .startObject("type") .startObject("properties") @@ -171,16 +172,164 @@ public void testMerge() throws IOException { .endObject() .endObject().endObject()); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper mapper = mapperService.merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); + DocumentMapper mapper = mapperService.merge("type", new CompressedXContent(mapping), reason); assertNull(mapper.root().dynamic()); String update = Strings.toString(XContentFactory.jsonBuilder().startObject() .startObject("type") .field("dynamic", "strict") .endObject().endObject()); - mapper = mapperService.merge("type", new CompressedXContent(update), MergeReason.MAPPING_UPDATE); + mapper = mapperService.merge("type", new CompressedXContent(update), reason); assertEquals(Dynamic.STRICT, mapper.root().dynamic()); } + public void testMergeEnabledForIndexTemplates() throws IOException { + String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("object") + .field("type", "object") + .field("enabled", false) + .endObject() + .endObject().endObject()); + MapperService mapperService = createIndex("test").mapperService(); + DocumentMapper mapper = mapperService.merge("type", new CompressedXContent(mapping), MergeReason.INDEX_TEMPLATE); + assertNull(mapper.root().dynamic()); + + // If we don't explicitly set 'enabled', then the mapping should not change. + String update = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("object") + .field("type", "object") + .field("dynamic", false) + .endObject() + .endObject().endObject()); + mapper = mapperService.merge("type", new CompressedXContent(update), MergeReason.INDEX_TEMPLATE); + + ObjectMapper objectMapper = mapper.objectMappers().get("object"); + assertNotNull(objectMapper); + assertFalse(objectMapper.isEnabled()); + + // Setting 'enabled' to true is allowed, and updates the mapping. + update = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("object") + .field("type", "object") + .field("enabled", true) + .endObject() + .endObject().endObject()); + mapper = mapperService.merge("type", new CompressedXContent(update), MergeReason.INDEX_TEMPLATE); + + objectMapper = mapper.objectMappers().get("object"); + assertNotNull(objectMapper); + assertTrue(objectMapper.isEnabled()); + } + + public void testFieldReplacementForIndexTemplates() throws IOException { + MapperService mapperService = createIndex("test").mapperService(); + String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("object") + .startObject("properties") + .startObject("field1") + .field("type", "keyword") + .endObject() + .startObject("field2") + .field("type", "text") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject()); + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(mapping), MergeReason.INDEX_TEMPLATE); + + String update = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("object") + .startObject("properties") + .startObject("field2") + .field("type", "integer") + .endObject() + .startObject("field3") + .field("type", "text") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject()); + DocumentMapper mapper = mapperService.merge(MapperService.SINGLE_MAPPING_NAME, + new CompressedXContent(update), MergeReason.INDEX_TEMPLATE); + + String expected = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("properties") + .startObject("object") + .startObject("properties") + .startObject("field1") + .field("type", "keyword") + .endObject() + .startObject("field2") + .field("type", "integer") + .endObject() + .startObject("field3") + .field("type", "text") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject()); + + assertEquals(expected, mapper.mappingSource().toString()); + } + + public void testDisallowFieldReplacementForIndexTemplates() throws IOException { + MapperService mapperService = createIndex("test").mapperService(); + String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("object") + .startObject("properties") + .startObject("field1") + .field("type", "object") + .endObject() + .startObject("field2") + .field("type", "text") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject()); + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(mapping), MergeReason.INDEX_TEMPLATE); + + String firstUpdate = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("object") + .startObject("properties") + .startObject("field2") + .field("type", "nested") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject()); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> mapperService.merge( + MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(firstUpdate), MergeReason.INDEX_TEMPLATE)); + assertThat(e.getMessage(), containsString("can't merge a non object mapping [object.field2] with an object mapping")); + + String secondUpdate = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("object") + .startObject("properties") + .startObject("field1") + .field("type", "text") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject()); + e = expectThrows(IllegalArgumentException.class, () -> mapperService.merge( + MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(secondUpdate), MergeReason.INDEX_TEMPLATE)); + assertThat(e.getMessage(), containsString("can't merge a non object mapping [object.field1] with an object mapping")); + } + public void testEmptyName() throws Exception { String mapping = Strings.toString(XContentFactory.jsonBuilder() .startObject() diff --git a/server/src/test/java/org/elasticsearch/index/mapper/RootObjectMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/RootObjectMapperTests.java index d4e8864926eb4..f5d7459f517d7 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/RootObjectMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/RootObjectMapperTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.index.mapper.MapperService.MergeReason; import org.elasticsearch.test.ESSingleNodeTestCase; +import java.io.IOException; import java.util.Arrays; import static org.elasticsearch.test.VersionUtils.randomVersionBetween; @@ -39,6 +40,7 @@ public class RootObjectMapperTests extends ESSingleNodeTestCase { public void testNumericDetection() throws Exception { + MergeReason reason = randomFrom(MergeReason.MAPPING_UPDATE, MergeReason.INDEX_TEMPLATE); String mapping = Strings.toString(XContentFactory.jsonBuilder() .startObject() .startObject("type") @@ -46,7 +48,7 @@ public void testNumericDetection() throws Exception { .endObject() .endObject()); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper mapper = mapperService.merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); + DocumentMapper mapper = mapperService.merge("type", new CompressedXContent(mapping), reason); assertEquals(mapping, mapper.mappingSource().toString()); // update with a different explicit value @@ -56,7 +58,7 @@ public void testNumericDetection() throws Exception { .field("numeric_detection", true) .endObject() .endObject()); - mapper = mapperService.merge("type", new CompressedXContent(mapping2), MergeReason.MAPPING_UPDATE); + mapper = mapperService.merge("type", new CompressedXContent(mapping2), reason); assertEquals(mapping2, mapper.mappingSource().toString()); // update with an implicit value: no change @@ -65,11 +67,12 @@ public void testNumericDetection() throws Exception { .startObject("type") .endObject() .endObject()); - mapper = mapperService.merge("type", new CompressedXContent(mapping3), MergeReason.MAPPING_UPDATE); + mapper = mapperService.merge("type", new CompressedXContent(mapping3), reason); assertEquals(mapping2, mapper.mappingSource().toString()); } public void testDateDetection() throws Exception { + MergeReason reason = randomFrom(MergeReason.MAPPING_UPDATE, MergeReason.INDEX_TEMPLATE); String mapping = Strings.toString(XContentFactory.jsonBuilder() .startObject() .startObject("type") @@ -77,7 +80,7 @@ public void testDateDetection() throws Exception { .endObject() .endObject()); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper mapper = mapperService.merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); + DocumentMapper mapper = mapperService.merge("type", new CompressedXContent(mapping), reason); assertEquals(mapping, mapper.mappingSource().toString()); // update with a different explicit value @@ -87,7 +90,7 @@ public void testDateDetection() throws Exception { .field("date_detection", false) .endObject() .endObject()); - mapper = mapperService.merge("type", new CompressedXContent(mapping2), MergeReason.MAPPING_UPDATE); + mapper = mapperService.merge("type", new CompressedXContent(mapping2), reason); assertEquals(mapping2, mapper.mappingSource().toString()); // update with an implicit value: no change @@ -96,11 +99,12 @@ public void testDateDetection() throws Exception { .startObject("type") .endObject() .endObject()); - mapper = mapperService.merge("type", new CompressedXContent(mapping3), MergeReason.MAPPING_UPDATE); + mapper = mapperService.merge("type", new CompressedXContent(mapping3), reason); assertEquals(mapping2, mapper.mappingSource().toString()); } public void testDateFormatters() throws Exception { + MergeReason reason = randomFrom(MergeReason.MAPPING_UPDATE, MergeReason.INDEX_TEMPLATE); String mapping = Strings.toString(XContentFactory.jsonBuilder() .startObject() .startObject("type") @@ -108,7 +112,7 @@ public void testDateFormatters() throws Exception { .endObject() .endObject()); MapperService mapperService = createIndex("test").mapperService(); - DocumentMapper mapper = mapperService.merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); + DocumentMapper mapper = mapperService.merge("type", new CompressedXContent(mapping), reason); assertEquals(mapping, mapper.mappingSource().toString()); // no update if formatters are not set explicitly @@ -117,7 +121,7 @@ public void testDateFormatters() throws Exception { .startObject("type") .endObject() .endObject()); - mapper = mapperService.merge("type", new CompressedXContent(mapping2), MergeReason.MAPPING_UPDATE); + mapper = mapperService.merge("type", new CompressedXContent(mapping2), reason); assertEquals(mapping, mapper.mappingSource().toString()); String mapping3 = Strings.toString(XContentFactory.jsonBuilder() @@ -126,7 +130,7 @@ public void testDateFormatters() throws Exception { .field("dynamic_date_formats", Arrays.asList()) .endObject() .endObject()); - mapper = mapperService.merge("type", new CompressedXContent(mapping3), MergeReason.MAPPING_UPDATE); + mapper = mapperService.merge("type", new CompressedXContent(mapping3), reason); assertEquals(mapping3, mapper.mappingSource().toString()); } @@ -169,6 +173,81 @@ public void testDynamicTemplates() throws Exception { assertEquals(mapping3, mapper.mappingSource().toString()); } + public void testDynamicTemplatesForIndexTemplate() throws IOException { + String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startArray("dynamic_templates") + .startObject() + .startObject("first_template") + .field("path_match", "first") + .startObject("mapping") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + .startObject() + .startObject("second_template") + .field("path_match", "second") + .startObject("mapping") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + .endArray() + .endObject()); + MapperService mapperService = createIndex("test").mapperService(); + mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(mapping), MergeReason.INDEX_TEMPLATE); + + // There should be no update if templates are not set. + mapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startObject("properties") + .startObject("field") + .field("type", "integer") + .endObject() + .endObject() + .endObject()); + DocumentMapper mapper = mapperService.merge(MapperService.SINGLE_MAPPING_NAME, + new CompressedXContent(mapping), MergeReason.INDEX_TEMPLATE); + + DynamicTemplate[] templates = mapper.root().dynamicTemplates(); + assertEquals(2, templates.length); + assertEquals("first_template", templates[0].name()); + assertEquals("first", templates[0].pathMatch()); + assertEquals("second_template", templates[1].name()); + assertEquals("second", templates[1].pathMatch()); + + // Dynamic templates should be appended and deduplicated. + mapping = Strings.toString(XContentFactory.jsonBuilder().startObject() + .startArray("dynamic_templates") + .startObject() + .startObject("third_template") + .field("path_match", "third") + .startObject("mapping") + .field("type", "integer") + .endObject() + .endObject() + .endObject() + .startObject() + .startObject("second_template") + .field("path_match", "second_updated") + .startObject("mapping") + .field("type", "double") + .endObject() + .endObject() + .endObject() + .endArray() + .endObject()); + mapper = mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(mapping), MergeReason.INDEX_TEMPLATE); + + templates = mapper.root().dynamicTemplates(); + assertEquals(3, templates.length); + assertEquals("first_template", templates[0].name()); + assertEquals("first", templates[0].pathMatch()); + assertEquals("second_template", templates[1].name()); + assertEquals("second_updated", templates[1].pathMatch()); + assertEquals("third_template", templates[2].name()); + assertEquals("third", templates[2].pathMatch()); + } + public void testIllegalFormatField() throws Exception { String dynamicMapping = Strings.toString(XContentFactory.jsonBuilder() .startObject() diff --git a/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java index cbfbb4a05c973..fe64fefb2846b 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.index.mapper.MapperService.MergeReason; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.InternalSettingsPlugin; @@ -124,10 +125,10 @@ private void assertConflicts(String mapping1, String mapping2, DocumentMapperPar DocumentMapper docMapper = parser.parse("type", new CompressedXContent(mapping1)); docMapper = parser.parse("type", docMapper.mappingSource()); if (conflicts.length == 0) { - docMapper.merge(parser.parse("type", new CompressedXContent(mapping2)).mapping()); + docMapper.merge(parser.parse("type", new CompressedXContent(mapping2)).mapping(), MergeReason.MAPPING_UPDATE); } else { try { - docMapper.merge(parser.parse("type", new CompressedXContent(mapping2)).mapping()); + docMapper.merge(parser.parse("type", new CompressedXContent(mapping2)).mapping(), MergeReason.MAPPING_UPDATE); fail(); } catch (IllegalArgumentException e) { for (String conflict : conflicts) { diff --git a/server/src/test/java/org/elasticsearch/index/mapper/UpdateMappingTests.java b/server/src/test/java/org/elasticsearch/index/mapper/UpdateMappingTests.java index 5fe52f62e8046..1c7da89a28e83 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/UpdateMappingTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/UpdateMappingTests.java @@ -174,13 +174,13 @@ public void testRejectFieldDefinedTwice() throws IOException { mapperService1.merge("type", new CompressedXContent(mapping1), MergeReason.MAPPING_UPDATE); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> mapperService1.merge("type", new CompressedXContent(mapping2), MergeReason.MAPPING_UPDATE)); - assertThat(e.getMessage(), equalTo("Can't merge a non object mapping [foo] with an object mapping [foo]")); + assertThat(e.getMessage(), equalTo("can't merge a non object mapping [foo] with an object mapping")); MapperService mapperService2 = createIndex("test2").mapperService(); mapperService2.merge("type", new CompressedXContent(mapping2), MergeReason.MAPPING_UPDATE); e = expectThrows(IllegalArgumentException.class, () -> mapperService2.merge("type", new CompressedXContent(mapping1), MergeReason.MAPPING_UPDATE)); - assertThat(e.getMessage(), equalTo("mapper [foo] cannot be changed from type [long] to [ObjectMapper]")); + assertThat(e.getMessage(), equalTo("can't merge a non object mapping [foo] with an object mapping")); } public void testMappingVersion() { diff --git a/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java b/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java index 8a20cb2d4c8cd..58a65840559fc 100644 --- a/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java @@ -210,7 +210,7 @@ public MappedFieldType fieldMapper(String name) { @Override public ObjectMapper getObjectMapper(String name) { BuilderContext context = new BuilderContext(this.getIndexSettings().getSettings(), new ContentPath()); - return new ObjectMapper.Builder<>(name).nested(Nested.newNested(false, false)).build(context); + return new ObjectMapper.Builder<>(name).nested(Nested.newNested()).build(context); } }; } diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 0dc71af0733e1..47ef3456d3bbd 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -316,7 +316,7 @@ public boolean shouldCache(Query query) { String fieldName = (String) invocation.getArguments()[0]; if (fieldName.startsWith(NESTEDFIELD_PREFIX)) { BuilderContext context = new BuilderContext(indexSettings.getSettings(), new ContentPath()); - return new ObjectMapper.Builder<>(fieldName).nested(Nested.newNested(false, false)).build(context); + return new ObjectMapper.Builder<>(fieldName).nested(Nested.newNested()).build(context); } return null; }); diff --git a/x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectFieldMapperTests.java b/x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectFieldMapperTests.java index e8e4116f08d31..4f9dda15bdf0a 100644 --- a/x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectFieldMapperTests.java +++ b/x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/xpack/flattened/mapper/FlatObjectFieldMapperTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.index.mapper.FieldNamesFieldMapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MapperService.MergeReason; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.plugins.Plugin; @@ -367,7 +368,8 @@ public void testDepthLimit() throws IOException { .endObject()); DocumentMapper newMapper = mapper.merge( - parser.parse("type", new CompressedXContent(newMapping)).mapping()); + parser.parse("type", new CompressedXContent(newMapping)).mapping(), + MergeReason.MAPPING_UPDATE); expectThrows(MapperParsingException.class, () -> newMapper.parse(new SourceToParse("test", "1", doc, XContentType.JSON))); @@ -430,7 +432,8 @@ public void testIgnoreAbove() throws IOException { .endObject()); DocumentMapper newMapper = mapper.merge( - parser.parse("type", new CompressedXContent(newMapping)).mapping()); + parser.parse("type", new CompressedXContent(newMapping)).mapping(), + MergeReason.MAPPING_UPDATE); ParsedDocument newParsedDoc = newMapper.parse(new SourceToParse("test", "1", doc, XContentType.JSON)); IndexableField[] newFields = newParsedDoc.rootDoc().getFields("field"); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java index 4cb7b397dc23c..4ffadc27f337d 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java @@ -275,8 +275,8 @@ public void onResponse(Boolean mappingsUpdated) { public void onFailure(Exception e) { if (ExceptionsHelper.unwrapCause(e) instanceof IllegalArgumentException) { // the underlying error differs depending on which way around the clashing fields are seen - Matcher matcher = Pattern.compile("(?:mapper|Can't merge a non object mapping) \\[(.*)\\] (?:cannot be changed " + - "from type \\[.*\\] to|with an object mapping) \\[.*\\]").matcher(e.getMessage()); + Matcher matcher = Pattern.compile("can't merge a non object mapping \\[(.*)\\] with an object mapping") + .matcher(e.getMessage()); if (matcher.matches()) { String msg = Messages.getMessage(Messages.JOB_CONFIG_MAPPING_TYPE_CLASH, matcher.group(1)); actionListener.onFailure(ExceptionsHelper.badRequestException(msg, e));