diff --git a/docs/reference/docs/bulk.asciidoc b/docs/reference/docs/bulk.asciidoc index 9c3b4101576eb..036e151d3f02d 100644 --- a/docs/reference/docs/bulk.asciidoc +++ b/docs/reference/docs/bulk.asciidoc @@ -283,6 +283,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=bulk-index] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=bulk-id] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=bulk-require-alias] + +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=bulk-dynamic-templates] -- `delete`:: @@ -311,6 +313,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=bulk-index] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=bulk-id] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=bulk-require-alias] + +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=bulk-dynamic-templates] -- `update`:: @@ -738,3 +742,39 @@ The API returns the following result. } ---- // TESTRESPONSE[s/"index_uuid": "aAsFqTI0Tc2W0LCWgPNrOA"/"index_uuid": $body.$_path/] + + +[discrete] +[[bulk-dynamic-templates]] +===== Example with dynamic templates parameter + +The below example creates a dynamic template, then performs a bulk request +consisting of index/create requests with the `dynamic_templates` parameter. + +[source,console] +---- +PUT my-index/ +{ + "mappings": { + "dynamic_templates": [ + { + "geo_point": { + "mapping": { + "type" : "geo_point" + } + } + } + ] + } +} + +POST /_bulk +{ "index" : { "_index" : "my_index", "_id" : "1", "dynamic_templates": {"work_location": "geo_point"}} } +{ "field" : "value1", "work_location": "41.12,-71.34", "raw_location": "41.12,-71.34"} +{ "create" : { "_index" : "my_index", "_id" : "2", "dynamic_templates": {"home_location": "geo_point"}} } +{ "field" : "value2", "home_location": "41.12,-71.34"} +---- + +The bulk request creates two new fields `work_location` and `home_location` with type `geo_point` according +to the `dynamic_templates` parameter; however, the `raw_location` field is created using default dynamic mapping +rules, as a `text` field in that case since it is supplied as a string in the JSON document. diff --git a/docs/reference/mapping/dynamic/templates.asciidoc b/docs/reference/mapping/dynamic/templates.asciidoc index 48c75ef6ae4fa..1a3e4629f74ce 100644 --- a/docs/reference/mapping/dynamic/templates.asciidoc +++ b/docs/reference/mapping/dynamic/templates.asciidoc @@ -14,6 +14,10 @@ name * <> operate on the full dotted path to the field +* if a dynamic template does not define `match_mapping_type` nor `match` nor +`path_match`, then it won't match any field, but it can be referred to by +name in `dynamic_templates` as part of a . + Use the `{name}` and `{dynamic_type}` <> in the mapping specification as placeholders. diff --git a/docs/reference/rest-api/common-parms.asciidoc b/docs/reference/rest-api/common-parms.asciidoc index b39d3085477dd..954a0017ff58a 100644 --- a/docs/reference/rest-api/common-parms.asciidoc +++ b/docs/reference/rest-api/common-parms.asciidoc @@ -626,6 +626,15 @@ If `true`, the destination must be an <>. Defaults `false`. end::require-alias[] +tag::bulk-dynamic-templates[] +`dynamic_templates`:: +(Optional, map) +A map from the full name of fields to the name of <. +Defaults to an empty map. If a name matches a dynamic template, then that template will be +applied regardless of other match predicates defined in the template. And if a field is +already defined in the mapping, then this parameter won't be used. +end::bulk-dynamic-templates[] + tag::node-filter[] ``:: (Optional, string) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/bulk/11_dynamic_templates.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/bulk/11_dynamic_templates.yml new file mode 100644 index 0000000000000..90fea59fd47f7 --- /dev/null +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/bulk/11_dynamic_templates.yml @@ -0,0 +1,171 @@ +--- +"Dynamic templates": + - skip: + version: " - 7.99.99" + reason: "Dynamic templates parameter is added to bulk requests in 8.0" + + - do: + indices.create: + index: test_index + body: + mappings: + dynamic_templates: + - location: + mapping: + type: geo_point + - my_location: + match: my* + mapping: + type: geo_point + - string: + mapping: + type: keyword + - do: + bulk: + refresh: true + body: + - index: + _index: test_index + _id: id_1 + dynamic_templates: + location: location + - { "location": [ -71.34, 41.12 ]} + - index: + _index: test_index + _id: id_2 + dynamic_templates: + location: location + - { "location": "41.12,-71.34"} + - match: { errors: false } + - match: { items.0.index.result: created } + - match: { items.1.index.result: created } + + - do: + search: + index: test_index + body: + query: + geo_bounding_box: + location: + top_left: + lat: 42 + lon: -72 + bottom_right: + lat: 40 + lon: -74 + - match: { hits.total.value: 2 } + - match: { hits.hits.0._id: id_1 } + - match: { hits.hits.1._id: id_2 } + + - do: + bulk: + refresh: true + body: + - index: + _index: test_index + _id: id_3 + - { "my_location": "41.12,-71.34" } # matches the field name defined in the `my_location` template + - index: + _index: test_index + _id: id_4 + dynamic_templates: + my_location: my_location + - { "my_location": "41.12,-71.34" } # use dynamic_templates parameter + - do: + search: + index: test_index + body: + query: + geo_bounding_box: + my_location: + top_left: + lat: 42 + lon: -72 + bottom_right: + lat: 40 + lon: -74 + - match: { hits.total.value: 2 } + - match: { hits.hits.0._id: id_3 } + - match: { hits.hits.1._id: id_4 } + + - do: + bulk: + refresh: true + body: + - index: + _index: test_index + _id: id_5 + dynamic_templates: + location: foo_bar # ok as fields are defined + - { "location": [ -71.34, 41.12 ]} + - index: + _index: test_index + _id: id_6 + dynamic_templates: + my_location: foo_bar # ok as fields are defined + - { "my_location": "41.12,-71.34" } + - index: + _index: test_index + _id: id_7 + dynamic_templates: + location: bar_foo # ok as fields are defined + - { "location": "41.12,-71.34" } + - match: { errors: false } + - match: { items.0.index.result: created } + - match: { items.1.index.result: created } + - match: { items.2.index.result: created } + + - do: + bulk: + refresh: true + body: + - index: + _index: test_index + _id: id_8 + dynamic_templates: + foo_location: bar_foo + - { "foo_location": [ -71.34, 41.12 ] } # failed because dynamic template is not found + - index: + _index: test_index + _id: id_9 + dynamic_templates: + foo_location: foo_bar + - { "foo_location": "41.12,-71.34" } # failed because dynamic template is not found + - index: + _index: test_index + _id: id_10 + dynamic_templates: + new_location: foo + - { "location": "41.12,-71.34"} # ok as fields are defined + - match: { errors: true } + - match: { items.0.index.status: 400 } + - match: { items.0.index.error.type: mapper_parsing_exception } + - match: { items.0.index.error.reason: "Can't find dynamic template for dynamic template name [bar_foo] of field [foo_location]"} + - match: { items.1.index.status: 400 } + - match: { items.1.index.error.type: mapper_parsing_exception } + - match: { items.1.index.error.reason: "Can't find dynamic template for dynamic template name [foo_bar] of field [foo_location]"} + - match: { items.2.index.status: 201 } + - match: { items.2.index.result: created } + + # Dynamic template has a wrong type + - do: + bulk: + body: + - index: + _index: test_index + _id: id_11 + dynamic_templates: + foo: string + - { "foo.bar": "hello world" } # failed because dynamic template has a wrong type + - index: + _index: test_index + _id: id_12 + dynamic_templates: + foo.bar: string + - { "foo.bar": "hello world" } # ok + - match: { errors: true } + - match: { items.0.index.status: 400 } + - match: { items.0.index.error.type: mapper_parsing_exception } + - match: { items.0.index.error.reason: "Field [foo] must be an object; but it's configured as [keyword] in dynamic template [string]"} + - match: { items.1.index.status: 201 } + - match: { items.1.index.result: created } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java index 11a9ec5ece649..5eba50509cd4e 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java @@ -9,28 +9,43 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; +import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.metadata.MappingMetadata; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.query.GeoBoundingBoxQueryBuilder; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalSettingsPlugin; import org.hamcrest.Matchers; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; import static org.elasticsearch.index.mapper.MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; public class DynamicMappingIT extends ESIntegTestCase { @@ -156,4 +171,119 @@ public void testMappingVersionAfterDynamicMappingUpdate() throws Exception { client().prepareIndex("test").setId("1").setSource("field", "text").get(); assertBusy(() -> assertThat(clusterService.state().metadata().index("test").getMappingVersion(), equalTo(1 + previousVersion))); } + + public void testBulkRequestWithDynamicTemplates() throws Exception { + final XContentBuilder mappings = XContentFactory.jsonBuilder(); + mappings.startObject(); + { + mappings.startArray("dynamic_templates"); + { + mappings.startObject(); + mappings.startObject("location"); + { + if (randomBoolean()) { + mappings.field("match", "location"); + } + if (randomBoolean()) { + mappings.field("match_mapping_type", "string"); + } + mappings.startObject("mapping"); + { + mappings.field("type", "geo_point"); + } + mappings.endObject(); + } + mappings.endObject(); + mappings.endObject(); + } + mappings.endArray(); + } + mappings.endObject(); + assertAcked(client().admin().indices().prepareCreate("test").setMapping(mappings)); + List requests = new ArrayList<>(); + requests.add(new IndexRequest("test").id("1").source("location", "41.12,-71.34") + .setDynamicTemplates(Map.of("location", "location"))); + requests.add(new IndexRequest("test").id("2").source( + XContentFactory.jsonBuilder() + .startObject() + .startObject("location").field("lat", 41.12).field("lon", -71.34).endObject() + .endObject()) + .setDynamicTemplates(Map.of("location", "location"))); + requests.add(new IndexRequest("test").id("3").source("address.location", "41.12,-71.34") + .setDynamicTemplates(Map.of("address.location", "location"))); + requests.add(new IndexRequest("test").id("4").source("location", new double[]{-71.34, 41.12}) + .setDynamicTemplates(Map.of("location", "location"))); + requests.add(new IndexRequest("test").id("5").source("array_of_numbers", new double[]{-71.34, 41.12})); + + Randomness.shuffle(requests); + BulkRequest bulkRequest = new BulkRequest().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + requests.forEach(bulkRequest::add); + final BulkResponse bulkResponse = client().bulk(bulkRequest).actionGet(); + assertFalse(bulkResponse.hasFailures()); + + SearchResponse searchResponse = client().prepareSearch("test") + .setQuery(new GeoBoundingBoxQueryBuilder("location").setCorners(new GeoPoint(42, -72), new GeoPoint(40, -74))) + .get(); + assertSearchHits(searchResponse, "1", "2", "4"); + searchResponse = client().prepareSearch("test") + .setQuery(new GeoBoundingBoxQueryBuilder("address.location").setCorners(new GeoPoint(42, -72), new GeoPoint(40, -74))) + .get(); + assertSearchHits(searchResponse, "3"); + } + + public void testBulkRequestWithNotFoundDynamicTemplate() throws Exception { + assertAcked(client().admin().indices().prepareCreate("test")); + final XContentBuilder mappings = XContentFactory.jsonBuilder(); + mappings.startObject(); + { + mappings.startArray("dynamic_templates"); + { + if (randomBoolean()) { + mappings.startObject(); + mappings.startObject("location"); + { + if (randomBoolean()) { + mappings.field("match", "location"); + } + if (randomBoolean()) { + mappings.field("match_mapping_type", "string"); + } + mappings.startObject("mapping"); + { + mappings.field("type", "geo_point"); + } + mappings.endObject(); + } + mappings.endObject(); + mappings.endObject(); + } + } + mappings.endArray(); + } + mappings.endObject(); + + BulkRequest bulkRequest = new BulkRequest().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + bulkRequest.add( + new IndexRequest("test").id("1").source( + XContentFactory.jsonBuilder() + .startObject() + .field("my_location", "41.12,-71.34") + .endObject()) + .setDynamicTemplates(Map.of("my_location", "foo_bar")), + new IndexRequest("test").id("2").source( + XContentFactory.jsonBuilder() + .startObject() + .field("address.location", "41.12,-71.34") + .endObject()) + .setDynamicTemplates(Map.of("address.location", "bar_foo")) + ); + final BulkResponse bulkItemResponses = client().bulk(bulkRequest).actionGet(); + assertTrue(bulkItemResponses.hasFailures()); + assertThat(bulkItemResponses.getItems()[0].getFailure().getCause(), instanceOf(MapperParsingException.class)); + assertThat(bulkItemResponses.getItems()[0].getFailureMessage(), + containsString("Can't find dynamic template for dynamic template name [foo_bar] of field [my_location]")); + assertThat(bulkItemResponses.getItems()[1].getFailure().getCause(), instanceOf(MapperParsingException.class)); + assertThat(bulkItemResponses.getItems()[1].getFailureMessage(), + containsString("Can't find dynamic template for dynamic template name [bar_foo] of field [address.location]")); + } } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java index 40c0080d298de..8d0774cc2b60a 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java @@ -52,6 +52,7 @@ public final class BulkRequestParser { private static final ParseField IF_SEQ_NO = new ParseField("if_seq_no"); private static final ParseField IF_PRIMARY_TERM = new ParseField("if_primary_term"); private static final ParseField REQUIRE_ALIAS = new ParseField(DocWriteRequest.REQUIRE_ALIAS); + private static final ParseField DYNAMIC_TEMPLATES = new ParseField("dynamic_templates"); // TODO: Remove this parameter once the BulkMonitoring endpoint has been removed private final boolean errorOnType; @@ -155,6 +156,7 @@ public void parse( int retryOnConflict = 0; String pipeline = defaultPipeline; boolean requireAlias = defaultRequireAlias != null && defaultRequireAlias; + Map dynamicTemplates = Map.of(); // at this stage, next token can either be END_OBJECT (and use default index and type, with auto generated id) // or START_OBJECT which will have another set of parameters @@ -205,7 +207,10 @@ public void parse( } } else if (token == XContentParser.Token.START_ARRAY) { throw new IllegalArgumentException("Malformed action/metadata line [" + line + - "], expected a simple value for field [" + currentFieldName + "] but found [" + token + "]"); + "], expected a simple value for field [" + currentFieldName + "] but found [" + token + "]"); + } else if (token == XContentParser.Token.START_OBJECT && + DYNAMIC_TEMPLATES.match(currentFieldName, parser.getDeprecationHandler())) { + dynamicTemplates = parser.mapStrings(); } else if (token == XContentParser.Token.START_OBJECT && SOURCE.match(currentFieldName, parser.getDeprecationHandler())) { fetchSourceContext = FetchSourceContext.fromXContent(parser); @@ -220,6 +225,10 @@ public void parse( } if ("delete".equals(action)) { + if (dynamicTemplates.isEmpty() == false) { + throw new IllegalArgumentException( + "Delete request in line [" + line + "] does not accept " + DYNAMIC_TEMPLATES.getPreferredName()); + } deleteRequestConsumer.accept(new DeleteRequest(index).id(id).routing(routing) .version(version).versionType(versionType).setIfSeqNo(ifSeqNo).setIfPrimaryTerm(ifPrimaryTerm)); } else { @@ -237,6 +246,7 @@ public void parse( .version(version).versionType(versionType) .setPipeline(pipeline).setIfSeqNo(ifSeqNo).setIfPrimaryTerm(ifPrimaryTerm) .source(sliceTrimmingCarriageReturn(data, from, nextMarker, xContentType), xContentType) + .setDynamicTemplates(dynamicTemplates) .setRequireAlias(requireAlias), type); } else { indexRequestConsumer.accept(new IndexRequest(index).id(id).routing(routing) @@ -251,12 +261,18 @@ public void parse( .version(version).versionType(versionType) .create(true).setPipeline(pipeline).setIfSeqNo(ifSeqNo).setIfPrimaryTerm(ifPrimaryTerm) .source(sliceTrimmingCarriageReturn(data, from, nextMarker, xContentType), xContentType) + .setDynamicTemplates(dynamicTemplates) .setRequireAlias(requireAlias), type); } else if ("update".equals(action)) { if (version != Versions.MATCH_ANY || versionType != VersionType.INTERNAL) { throw new IllegalArgumentException("Update requests do not support versioning. " + "Please use `if_seq_no` and `if_primary_term` instead"); } + // TODO: support dynamic_templates in update requests + if (dynamicTemplates.isEmpty() == false) { + throw new IllegalArgumentException( + "Update request in line [" + line + "] does not accept " + DYNAMIC_TEMPLATES.getPreferredName()); + } UpdateRequest updateRequest = new UpdateRequest().index(index).id(id).routing(routing) .retryOnConflict(retryOnConflict) .setIfSeqNo(ifSeqNo).setIfPrimaryTerm(ifPrimaryTerm) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 521f0bb862ef9..fe66520587dc8 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -281,8 +281,9 @@ static boolean executeBulkItemRequest(BulkPrimaryExecutionContext context, Updat request.ifSeqNo(), request.ifPrimaryTerm()); } else { final IndexRequest request = context.getRequestToExecute(); - result = primary.applyIndexOperationOnPrimary(version, request.versionType(), new SourceToParse( - request.index(), request.id(), request.source(), request.getContentType(), request.routing()), + final SourceToParse sourceToParse = new SourceToParse(request.index(), request.id(), request.source(), + request.getContentType(), request.routing(), request.getDynamicTemplates()); + result = primary.applyIndexOperationOnPrimary(version, request.versionType(), sourceToParse, request.ifSeqNo(), request.ifPrimaryTerm(), request.getAutoGeneratedTimestamp(), request.isRetry()); } if (result.getResultType() == Engine.Result.Type.MAPPING_UPDATE_REQUIRED) { @@ -479,8 +480,8 @@ private static Engine.Result performOpOnReplica(DocWriteResponse primaryResponse case INDEX: final IndexRequest indexRequest = (IndexRequest) docWriteRequest; final ShardId shardId = replica.shardId(); - final SourceToParse sourceToParse = new SourceToParse(shardId.getIndexName(), indexRequest.id(), - indexRequest.source(), indexRequest.getContentType(), indexRequest.routing()); + final SourceToParse sourceToParse = new SourceToParse(shardId.getIndexName(), indexRequest.id(), indexRequest.source(), + indexRequest.getContentType(), indexRequest.routing(), Map.of()); result = replica.applyIndexOperationOnReplica(primaryResponse.getSeqNo(), primaryResponse.getPrimaryTerm(), primaryResponse.getVersion(), indexRequest.getAutoGeneratedTimestamp(), indexRequest.isRetry(), sourceToParse); break; diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java index f91bad1532104..3c164f7eb0c25 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -109,6 +109,8 @@ public class IndexRequest extends ReplicatedWriteRequest implement private long ifSeqNo = UNASSIGNED_SEQ_NO; private long ifPrimaryTerm = UNASSIGNED_PRIMARY_TERM; + private Map dynamicTemplates = Map.of(); + public IndexRequest(StreamInput in) throws IOException { this(null, in); } @@ -146,6 +148,9 @@ public IndexRequest(@Nullable ShardId shardId, StreamInput in) throws IOExceptio } else { requireAlias = false; } + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + dynamicTemplates = in.readMap(StreamInput::readString, StreamInput::readString); + } } public IndexRequest() { @@ -655,6 +660,13 @@ private void writeBody(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_7_10_0)) { out.writeBoolean(requireAlias); } + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + out.writeMap(dynamicTemplates, StreamOutput::writeString, StreamOutput::writeString); + } else { + if (dynamicTemplates.isEmpty() == false) { + throw new IllegalArgumentException("[dynamic_templates] parameter requires all nodes on " + Version.V_8_0_0 + " or later"); + } + } } @Override @@ -712,4 +724,21 @@ public IndexRequest setRequireAlias(boolean requireAlias) { this.requireAlias = requireAlias; return this; } + + /** + * Specifies a map from the full path of field names to the name of dynamic mapping templates + */ + public IndexRequest setDynamicTemplates(Map dynamicTemplates) { + this.dynamicTemplates = Objects.requireNonNull(dynamicTemplates); + return this; + } + + /** + * Returns a map from the full path of field names to the name of dynamic mapping templates. + * + * @see #setDynamicTemplates(Map) + */ + public Map getDynamicTemplates() { + return dynamicTemplates; + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/SingleDocDirectoryReader.java b/server/src/main/java/org/elasticsearch/index/engine/SingleDocDirectoryReader.java index e25ee4bcd1f94..02ef823d79dc7 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/SingleDocDirectoryReader.java +++ b/server/src/main/java/org/elasticsearch/index/engine/SingleDocDirectoryReader.java @@ -37,6 +37,7 @@ import org.elasticsearch.index.translog.Translog; import java.io.IOException; +import java.util.Map; import java.util.concurrent.atomic.AtomicReference; /** @@ -140,7 +141,8 @@ private LeafReader getDelegate() { private LeafReader createInMemoryLeafReader() { assert Thread.holdsLock(this); final ParsedDocument parsedDocs = mapper.parse(new SourceToParse(shardId.getIndexName(), operation.id(), - operation.source(), XContentHelper.xContentType(operation.source()), operation.routing())); + operation.source(), XContentHelper.xContentType(operation.source()), operation.routing(), Map.of())); + parsedDocs.updateSeqID(operation.seqNo(), operation.primaryTerm()); parsedDocs.version().setLongValue(operation.version()); final IndexWriterConfig writeConfig = new IndexWriterConfig(analyzer).setOpenMode(IndexWriterConfig.OpenMode.CREATE); diff --git a/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java b/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java index c53545e8613bd..c5b61cd0b8b6d 100644 --- a/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java +++ b/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java @@ -224,7 +224,7 @@ private GetResult innerGetLoadFromStoredFields(String id, String[] storedFields, // Slow path: recreate stored fields from original source assert source != null : "original source in translog must exist"; SourceToParse sourceToParse = new SourceToParse(shardId.getIndexName(), id, source, XContentHelper.xContentType(source), - fieldVisitor.routing()); + fieldVisitor.routing(), Map.of()); ParsedDocument doc = indexShard.mapperService().documentMapper().parse(sourceToParse); assert doc.dynamicMappingsUpdate() == null : "mapping updates should not be required on already-indexed doc"; // update special fields diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index 222a4392c97e1..5f4c98ddce3ea 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -589,7 +589,8 @@ private static void parseArray(ParseContext context, ObjectMapper parentMapper, // TODO: shouldn't this skip, not parse? parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName); } else { - Mapper objectMapperFromTemplate = dynamic.getDynamicFieldsBuilder().createObjectMapperFromTemplate(context, arrayFieldName); + Mapper objectMapperFromTemplate = + dynamic.getDynamicFieldsBuilder().createObjectMapperFromTemplate(context, arrayFieldName); if (objectMapperFromTemplate == null) { parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName); } else { @@ -757,10 +758,18 @@ private static Tuple getDynamicParentMapper(ParseContext return new Tuple<>(pathsAdded, parent); } else { //objects are created under properties even with dynamic: runtime, as the runtime section only holds leaf fields - mapper = (ObjectMapper) dynamic.getDynamicFieldsBuilder().createDynamicObjectMapper(context, paths[i]); + final Mapper fieldMapper = dynamic.getDynamicFieldsBuilder().createDynamicObjectMapper(context, paths[i]); + if (fieldMapper instanceof ObjectMapper == false) { + assert context.sourceToParse().dynamicTemplates().containsKey(currentPath) : + "dynamic templates [" + context.sourceToParse().dynamicTemplates() + "]"; + throw new MapperParsingException("Field [" + currentPath + "] must be an object; " + + "but it's configured as [" + fieldMapper.typeName() + "] in dynamic template [" + + context.sourceToParse().dynamicTemplates().get(currentPath) + "]"); + } + mapper = (ObjectMapper) fieldMapper; if (mapper.nested() != ObjectMapper.Nested.NO) { throw new MapperParsingException("It is forbidden to create dynamic nested objects ([" - + context.path().pathAsText(paths[i]) + "]) through `copy_to` or dots in field names"); + + currentPath + "]) through `copy_to` or dots in field names"); } context.addDynamicMapper(mapper); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DynamicFieldsBuilder.java b/server/src/main/java/org/elasticsearch/index/mapper/DynamicFieldsBuilder.java index 9c678449fcc38..328c9a12495f8 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DynamicFieldsBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DynamicFieldsBuilder.java @@ -183,7 +183,7 @@ private static boolean applyMatchingTemplate(ParseContext context, String name, DynamicTemplate.XContentFieldType matchType, DateFormatter dateFormatter) throws IOException { - DynamicTemplate dynamicTemplate = context.root().findTemplate(context.path(), name, matchType); + DynamicTemplate dynamicTemplate = context.findDynamicTemplate(name, matchType); if (dynamicTemplate == null) { return false; } @@ -209,7 +209,7 @@ private static boolean applyMatchingTemplate(ParseContext context, private static Mapper.Builder findTemplateBuilderForObject(ParseContext context, String name) { DynamicTemplate.XContentFieldType matchType = DynamicTemplate.XContentFieldType.OBJECT; - DynamicTemplate dynamicTemplate = context.root().findTemplate(context.path(), name, matchType); + DynamicTemplate dynamicTemplate = context.findDynamicTemplate(name, matchType); if (dynamicTemplate == null) { return null; } 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 5851586441e25..ebf845851d2a2 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DynamicTemplate.java @@ -181,21 +181,28 @@ static DynamicTemplate parse(String name, Map conf) throws Mappe throw new IllegalArgumentException("Illegal dynamic template parameter: [" + propName + "]"); } } - - if (match == null && pathMatch == null && matchMappingType == null) { - throw new MapperParsingException("template must have match, path_match or match_mapping_type set " + conf.toString()); - } if (mapping == null) { throw new MapperParsingException("template [" + name + "] must have either mapping or runtime set"); } - XContentFieldType xcontentFieldType = null; - if (matchMappingType != null && matchMappingType.equals("*") == false) { - xcontentFieldType = XContentFieldType.fromString(matchMappingType); - if (runtime && xcontentFieldType.supportsRuntimeField() == false) { + final XContentFieldType[] xContentFieldTypes; + if ("*".equals(matchMappingType) || (matchMappingType == null && (match != null || pathMatch != null))) { + if (runtime) { + xContentFieldTypes = Arrays.stream(XContentFieldType.values()) + .filter(XContentFieldType::supportsRuntimeField) + .toArray(XContentFieldType[]::new); + } else { + xContentFieldTypes = XContentFieldType.values(); + } + } else if (matchMappingType != null) { + final XContentFieldType xContentFieldType = XContentFieldType.fromString(matchMappingType); + if (runtime && xContentFieldType.supportsRuntimeField() == false) { throw new MapperParsingException("Dynamic template [" + name + "] defines a runtime field but type [" - + xcontentFieldType + "] is not supported as runtime field"); + + xContentFieldType + "] is not supported as runtime field"); } + xContentFieldTypes = new XContentFieldType[]{xContentFieldType}; + } else { + xContentFieldTypes = new XContentFieldType[0]; } final MatchType matchType = MatchType.fromString(matchPattern); @@ -213,7 +220,7 @@ static DynamicTemplate parse(String name, Map conf) throws Mappe } } - return new DynamicTemplate(name, pathMatch, pathUnmatch, match, unmatch, xcontentFieldType, matchType, mapping, runtime); + return new DynamicTemplate(name, pathMatch, pathUnmatch, match, unmatch, xContentFieldTypes, matchType, mapping, runtime); } private final String name; @@ -222,19 +229,20 @@ static DynamicTemplate parse(String name, Map conf) throws Mappe private final String match; private final String unmatch; private final MatchType matchType; - private final XContentFieldType xcontentFieldType; + private final XContentFieldType[] xContentFieldTypes; private final Map mapping; private final boolean runtimeMapping; private DynamicTemplate(String name, String pathMatch, String pathUnmatch, String match, String unmatch, - XContentFieldType xcontentFieldType, MatchType matchType, Map mapping, boolean runtimeMapping) { + XContentFieldType[] xContentFieldTypes, MatchType matchType, Map mapping, + boolean runtimeMapping) { this.name = name; this.pathMatch = pathMatch; this.pathUnmatch = pathUnmatch; this.match = match; this.unmatch = unmatch; this.matchType = matchType; - this.xcontentFieldType = xcontentFieldType; + this.xContentFieldTypes = xContentFieldTypes; this.mapping = mapping; this.runtimeMapping = runtimeMapping; } @@ -247,20 +255,28 @@ public String pathMatch() { return pathMatch; } - public boolean match(String path, String name, XContentFieldType xcontentFieldType) { + public String match() { + return match; + } + + public boolean match(String templateName, String path, String fieldName, XContentFieldType xcontentFieldType) { + // If the template name parameter is specified, then we will check only the name of the template and ignore other matches. + if (templateName != null) { + return templateName.equals(name); + } if (pathMatch != null && matchType.matches(pathMatch, path) == false) { return false; } - if (match != null && matchType.matches(match, name) == false) { + if (match != null && matchType.matches(match, fieldName) == false) { return false; } if (pathUnmatch != null && matchType.matches(pathUnmatch, path)) { return false; } - if (unmatch != null && matchType.matches(unmatch, name)) { + if (unmatch != null && matchType.matches(unmatch, fieldName)) { return false; } - if (this.xcontentFieldType != null && this.xcontentFieldType != xcontentFieldType) { + if (Arrays.stream(xContentFieldTypes).noneMatch(xcontentFieldType::equals)) { return false; } if (runtimeMapping && xcontentFieldType.supportsRuntimeField() == false) { @@ -339,8 +355,8 @@ String getName() { return name; } - XContentFieldType getXContentFieldType() { - return xcontentFieldType; + XContentFieldType[] getXContentFieldTypes() { + return xContentFieldTypes; } Map getMapping() { @@ -362,10 +378,13 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (pathUnmatch != null) { builder.field("path_unmatch", pathUnmatch); } - if (xcontentFieldType != null) { - builder.field("match_mapping_type", xcontentFieldType); - } else if (match == null && pathMatch == null) { + // We have more than one types when (1) `match_mapping_type` is "*", and (2) match and/or path_match are defined but + // not `match_mapping_type`. In the latter the template implicitly accepts all types and we don't need to serialize + // the `match_mapping_type` values. + if (xContentFieldTypes.length > 1 && match == null && pathMatch == null) { builder.field("match_mapping_type", "*"); + } else if (xContentFieldTypes.length == 1) { + builder.field("match_mapping_type", xContentFieldTypes[0]); } if (matchType != MatchType.SIMPLE) { builder.field("match_pattern", matchType); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index bc4ee21c98a3d..75f280e2a315e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -668,4 +668,27 @@ public final T parseExternalValue(Class clazz) { * Get dynamic runtime fields created while parsing. */ public abstract List getDynamicRuntimeFields(); + + /** + * Find a dynamic mapping template for the given field and its matching type + * + * @param fieldName the name of the field + * @param matchType the expecting matchType of the field + * @return the matching template; otherwise returns null + * @throws MapperParsingException if the given field has a dynamic template name specified, but no template matches that name. + */ + public final DynamicTemplate findDynamicTemplate(String fieldName, DynamicTemplate.XContentFieldType matchType) { + final String pathAsString = path().pathAsText(fieldName); + final String matchTemplateName = sourceToParse().dynamicTemplates().get(pathAsString); + for (DynamicTemplate template : root().dynamicTemplates()) { + if (template.match(matchTemplateName, pathAsString, fieldName, matchType)) { + return template; + } + } + if (matchTemplateName != null) { + throw new MapperParsingException( + "Can't find dynamic template for dynamic template name [" + matchTemplateName + "] of field [" + pathAsString + "]"); + } + return null; + } } 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 860af525e48fe..b56b9cadb40a1 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java @@ -289,16 +289,6 @@ RuntimeField getRuntimeField(String name) { return runtimeFields.get(name); } - public DynamicTemplate findTemplate(ContentPath path, String name, XContentFieldType matchType) { - final String pathAsString = path.pathAsText(name); - for (DynamicTemplate dynamicTemplate : dynamicTemplates.value()) { - if (dynamicTemplate.match(pathAsString, name, matchType)) { - return dynamicTemplate; - } - } - return null; - } - @Override public RootObjectMapper merge(Mapper mergeWith, MergeReason reason) { return (RootObjectMapper) super.merge(mergeWith, reason); @@ -400,18 +390,9 @@ private static void validateDynamicTemplate(Mapper.TypeParser.ParserContext pars return; } - final XContentFieldType[] types; - if (template.getXContentFieldType() != null) { - types = new XContentFieldType[]{template.getXContentFieldType()}; - } else if (template.isRuntimeMapping()) { - types = Arrays.stream(XContentFieldType.values()).filter(XContentFieldType::supportsRuntimeField) - .toArray(XContentFieldType[]::new); - } else { - types = XContentFieldType.values(); - } + final XContentFieldType[] types = template.getXContentFieldTypes(); Exception lastError = null; - boolean dynamicTemplateInvalid = true; for (XContentFieldType fieldType : types) { String dynamicType = template.isRuntimeMapping() ? fieldType.defaultRuntimeMappingType() : fieldType.defaultMappingType(); @@ -420,42 +401,34 @@ private static void validateDynamicTemplate(Mapper.TypeParser.ParserContext pars if (template.isRuntimeMapping()) { RuntimeField.Parser parser = parserContext.runtimeFieldParser(mappingType); if (parser == null) { - lastError = new IllegalArgumentException("No runtime field found for type [" + mappingType + "]"); - continue; + throw new IllegalArgumentException("No runtime field found for type [" + mappingType + "]"); } validate(template, dynamicType, (name, mapping) -> parser.parse(name, mapping, parserContext)); } else { Mapper.TypeParser typeParser = parserContext.typeParser(mappingType); if (typeParser == null) { - lastError = new IllegalArgumentException("No mapper found for type [" + mappingType + "]"); - continue; + throw new IllegalArgumentException("No mapper found for type [" + mappingType + "]"); } validate(template, dynamicType, (name, mapping) -> typeParser.parse(name, mapping, parserContext).build(new ContentPath(1))); } - dynamicTemplateInvalid = false; + lastError = null; // ok, the template is valid for at least one type break; - } catch(Exception e) { + } catch (Exception e) { lastError = e; } } - - final boolean failInvalidDynamicTemplates = parserContext.indexVersionCreated().onOrAfter(Version.V_8_0_0); - if (dynamicTemplateInvalid) { + if (lastError != null) { String format = "dynamic template [%s] has invalid content [%s], " + "attempted to validate it with the following match_mapping_type: %s"; String message = String.format(Locale.ROOT, format, template.getName(), Strings.toString(template), Arrays.toString(types)); + final boolean failInvalidDynamicTemplates = parserContext.indexVersionCreated().onOrAfter(Version.V_8_0_0); if (failInvalidDynamicTemplates) { throw new IllegalArgumentException(message, lastError); } else { - final String deprecationMessage; - if (lastError != null) { - deprecationMessage = String.format(Locale.ROOT, "%s, last error: [%s]", message, lastError.getMessage()); - } else { - deprecationMessage = message; - } - DEPRECATION_LOGGER.deprecate(DeprecationCategory.TEMPLATES, "invalid_dynamic_template", deprecationMessage); + DEPRECATION_LOGGER.deprecate(DeprecationCategory.TEMPLATES, "invalid_dynamic_template", + "{}, last error: [{}]", message, lastError.getMessage()); } } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/SourceToParse.java b/server/src/main/java/org/elasticsearch/index/mapper/SourceToParse.java index 3a2a30f9ef0ac..25afdef5600cc 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SourceToParse.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SourceToParse.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.XContentType; +import java.util.Map; import java.util.Objects; public class SourceToParse { @@ -27,7 +28,10 @@ public class SourceToParse { private final XContentType xContentType; - public SourceToParse(String index, String id, BytesReference source, XContentType xContentType, @Nullable String routing) { + private final Map dynamicTemplates; + + public SourceToParse(String index, String id, BytesReference source, XContentType xContentType, @Nullable String routing, + Map dynamicTemplates) { this.index = Objects.requireNonNull(index); this.id = Objects.requireNonNull(id); // we always convert back to byte array, since we store it and Field only supports bytes.. @@ -35,10 +39,11 @@ public SourceToParse(String index, String id, BytesReference source, XContentTyp this.source = new BytesArray(Objects.requireNonNull(source).toBytesRef()); this.xContentType = Objects.requireNonNull(xContentType); this.routing = routing; + this.dynamicTemplates = Objects.requireNonNull(dynamicTemplates); } public SourceToParse(String index, String id, BytesReference source, XContentType xContentType) { - this(index, id, source, xContentType, null); + this(index, id, source, xContentType, null, Map.of()); } public BytesReference source() { @@ -57,6 +62,13 @@ public String id() { return this.routing; } + /** + * Returns a map from the full path (i.e. foo.bar) of field names to the names of dynamic mapping templates. + */ + public Map dynamicTemplates() { + return dynamicTemplates; + } + public XContentType getXContentType() { return this.xContentType; } diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 48082f1ed86ab..e26aa650be02a 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -849,6 +849,8 @@ public static Engine.Index prepareIndex(DocumentMapperForType docMapper, SourceT long autoGeneratedIdTimestamp, boolean isRetry, long ifSeqNo, long ifPrimaryTerm) { long startTime = System.nanoTime(); + assert source.dynamicTemplates().isEmpty() || origin == Engine.Operation.Origin.PRIMARY : + "dynamic_templates parameter can only be associated with primary operations"; ParsedDocument doc = docMapper.getDocumentMapper().parse(source); if (docMapper.getMapping() != null) { doc.addDynamicMappingsUpdate(docMapper.getMapping()); @@ -1543,7 +1545,7 @@ private Engine.Result applyTranslogOperation(Engine engine, Translog.Operation o result = applyIndexOperation(engine, index.seqNo(), index.primaryTerm(), index.version(), versionType, UNASSIGNED_SEQ_NO, 0, index.getAutoGeneratedIdTimestamp(), true, origin, new SourceToParse(shardId.getIndexName(), index.id(), index.source(), - XContentHelper.xContentType(index.source()), index.routing())); + XContentHelper.xContentType(index.source()), index.routing(), Map.of())); break; case DELETE: final Translog.Delete delete = (Translog.Delete) operation; diff --git a/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java b/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java index cb5748b409a81..a37773e2482f2 100644 --- a/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java +++ b/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java @@ -324,7 +324,7 @@ private static ParsedDocument parseDocument(IndexShard indexShard, String index, MapperService mapperService = indexShard.mapperService(); DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(); ParsedDocument parsedDocument = docMapper.getDocumentMapper().parse( - new SourceToParse(index, "_id_for_tv_api", doc, xContentType, routing)); + new SourceToParse(index, "_id_for_tv_api", doc, xContentType, routing, Map.of())); if (docMapper.getMapping() != null) { parsedDocument.addDynamicMappingsUpdate(docMapper.getMapping()); } diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java index 195a884b7571d..f9c88731065bb 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java @@ -38,6 +38,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -320,4 +321,56 @@ public void testBulkTerminatedByNewline() throws Exception { XContentType.JSON); assertEquals(3, bulkRequestWithNewLine.numberOfActions()); } + + public void testDynamicTemplates() throws Exception { + BytesArray data = new BytesArray( + "{ \"index\":{\"_index\":\"test\",\"dynamic_templates\":{\"baz\":\"t1\", \"foo.bar\":\"t2\"}}}\n" + + "{ \"field1\" : \"value1\" }\n" + + + "{ \"delete\" : { \"_index\" : \"test\", \"_id\" : \"2\" } }\n" + + + "{ \"create\" : {\"_index\":\"test\",\"dynamic_templates\":{\"bar\":\"t1\"}}}\n" + + "{ \"field1\" : \"value3\" }\n" + + + "{ \"create\" : {\"dynamic_templates\":{\"foo.bar\":\"xyz\"}}}\n" + + "{ \"field1\" : \"value3\" }\n" + + + "{ \"index\" : {\"dynamic_templates\":{}}\n" + + "{ \"field1\" : \"value3\" }\n" + ); + BulkRequest bulkRequest = new BulkRequest().add(data, null, XContentType.JSON); + assertThat(bulkRequest.requests, hasSize(5)); + assertThat(((IndexRequest) bulkRequest.requests.get(0)).getDynamicTemplates(), + equalTo(Map.of("baz", "t1", "foo.bar", "t2"))); + assertThat(((IndexRequest) bulkRequest.requests.get(2)).getDynamicTemplates(), + equalTo(Map.of("bar", "t1"))); + assertThat(((IndexRequest) bulkRequest.requests.get(3)).getDynamicTemplates(), + equalTo(Map.of("foo.bar", "xyz"))); + assertThat(((IndexRequest) bulkRequest.requests.get(4)).getDynamicTemplates(), + equalTo(Map.of())); + } + + public void testInvalidDynamicTemplates() { + BytesArray deleteWithDynamicTemplates = new BytesArray( + "{ \"delete\" : { \"_index\" : \"test\", \"_id\" : \"2\", \"dynamic_templates\":{\"baz\":\"t1\"}} }\n"); + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, + () -> new BulkRequest().add(deleteWithDynamicTemplates, null, XContentType.JSON)); + assertThat(error.getMessage(), equalTo("Delete request in line [1] does not accept dynamic_templates")); + + BytesArray updateWithDynamicTemplates = new BytesArray( + "{ \"update\" : {\"dynamic_templates\":{\"foo.bar\":\"xyz\"}}}\n" + + "{ \"field1\" : \"value3\" }\n"); + error = expectThrows(IllegalArgumentException.class, + () -> new BulkRequest().add(updateWithDynamicTemplates, null, XContentType.JSON)); + assertThat(error.getMessage(), equalTo("Update request in line [2] does not accept dynamic_templates")); + + BytesArray invalidDynamicTemplates = new BytesArray( + "{ \"index\":{\"_index\":\"test\",\"dynamic_templates\":[]}\n" + + "{ \"field1\" : \"value1\" }\n" + ); + error = expectThrows(IllegalArgumentException.class, + () -> new BulkRequest().add(invalidDynamicTemplates, null, XContentType.JSON)); + assertThat(error.getMessage(), equalTo("Malformed action/metadata line [1], " + + "expected a simple value for field [dynamic_templates] but found [START_ARRAY]")); + } } diff --git a/server/src/test/java/org/elasticsearch/action/index/IndexRequestTests.java b/server/src/test/java/org/elasticsearch/action/index/IndexRequestTests.java index 452d997d5005f..1bca0c580d321 100644 --- a/server/src/test/java/org/elasticsearch/action/index/IndexRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/index/IndexRequestTests.java @@ -22,13 +22,18 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.Arrays; import java.util.HashSet; +import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import static org.hamcrest.Matchers.anEmptyMap; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; @@ -145,7 +150,11 @@ public void testIndexResponse() { public void testIndexRequestXContentSerialization() throws IOException { IndexRequest indexRequest = new IndexRequest("foo").id("1"); boolean isRequireAlias = randomBoolean(); + Map dynamicTemplates = IntStream.range(0, randomIntBetween(0, 10)) + .boxed() + .collect(Collectors.toMap(n -> "field-" + n, n -> "name-" + n)); indexRequest.source("{}", XContentType.JSON); + indexRequest.setDynamicTemplates(dynamicTemplates); indexRequest.setRequireAlias(isRequireAlias); assertEquals(XContentType.JSON, indexRequest.getContentType()); @@ -156,6 +165,7 @@ public void testIndexRequestXContentSerialization() throws IOException { assertEquals(XContentType.JSON, serialized.getContentType()); assertEquals(new BytesArray("{}"), serialized.source()); assertEquals(isRequireAlias, serialized.isRequireAlias()); + assertThat(serialized.getDynamicTemplates(), equalTo(dynamicTemplates)); } // reindex makes use of index requests without a source so this needs to be handled @@ -174,6 +184,51 @@ public void testSerializationOfEmptyRequestWorks() throws IOException { } } + public void testSerializeDynamicTemplates() throws Exception { + IndexRequest indexRequest = new IndexRequest("foo").id("1"); + indexRequest.source("{}", XContentType.JSON); + // Empty dynamic templates + { + if (randomBoolean()) { + indexRequest.setDynamicTemplates(Map.of()); + } + Version ver = VersionUtils.randomCompatibleVersion(random(), Version.CURRENT); + BytesStreamOutput out = new BytesStreamOutput(); + out.setVersion(ver); + indexRequest.writeTo(out); + StreamInput in = StreamInput.wrap(out.bytes().toBytesRef().bytes); + in.setVersion(ver); + IndexRequest serialized = new IndexRequest(in); + assertThat(serialized.getDynamicTemplates(), anEmptyMap()); + } + // old version + { + Map dynamicTemplates = IntStream.range(0, randomIntBetween(1, 10)) + .boxed().collect(Collectors.toMap(n -> "field-" + n, n -> "name-" + n)); + indexRequest.setDynamicTemplates(dynamicTemplates); + Version ver = VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, VersionUtils.getPreviousVersion(Version.V_8_0_0)); + BytesStreamOutput out = new BytesStreamOutput(); + out.setVersion(ver); + IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> indexRequest.writeTo(out)); + assertThat(error.getMessage(), + equalTo("[dynamic_templates] parameter requires all nodes on " + Version.V_8_0_0 + " or later")); + } + // new version + { + Map dynamicTemplates = IntStream.range(0, randomIntBetween(0, 10)) + .boxed().collect(Collectors.toMap(n -> "field-" + n, n -> "name-" + n)); + indexRequest.setDynamicTemplates(dynamicTemplates); + Version ver = VersionUtils.randomVersionBetween(random(), Version.V_8_0_0, Version.CURRENT); + BytesStreamOutput out = new BytesStreamOutput(); + out.setVersion(ver); + indexRequest.writeTo(out); + StreamInput in = StreamInput.wrap(out.bytes().toBytesRef().bytes); + in.setVersion(ver); + IndexRequest serialized = new IndexRequest(in); + assertThat(serialized.getDynamicTemplates(), equalTo(dynamicTemplates)); + } + } + public void testToStringSizeLimit() throws UnsupportedEncodingException { IndexRequest request = new IndexRequest("index"); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java index 249c942142499..74798ada41912 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java @@ -9,6 +9,8 @@ package org.elasticsearch.index.mapper; import org.apache.lucene.document.Field; +import org.apache.lucene.document.LatLonDocValuesField; +import org.apache.lucene.document.LatLonPoint; import org.apache.lucene.document.StringField; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BytesRef; @@ -36,11 +38,13 @@ import static org.elasticsearch.test.StreamsUtils.copyToBytesFromClasspath; import static org.elasticsearch.test.StreamsUtils.copyToStringFromClasspath; +import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.sameInstance; public class DocumentParserTests extends MapperServiceTestCase { @@ -1037,6 +1041,161 @@ public void testDynamicDottedFieldNameLongArrayWithParentTemplate() throws Excep assertThat(bazMapper, instanceOf(NumberFieldMapper.class)); } + public void testWithDynamicTemplates() throws Exception { + DocumentMapper mapper = createDocumentMapper(topMapping(b -> { + b.startArray("dynamic_templates"); + { + b.startObject(); + { + b.startObject("points"); + { + b.field("match", "none"); // do not map anything + b.startObject("mapping"); + { + b.field("type", "geo_point"); + } + b.endObject(); + } + b.endObject(); + } + b.endObject(); + } + b.endArray(); + })); + + String field = randomFrom("loc", "foo.loc", "foo.bar.loc"); + + ParsedDocument doc = mapper.parse(source("1", b -> b.field(field, "41.12,-71.34"), null, Map.of(field, "points"))); + IndexableField[] fields = doc.rootDoc().getFields(field); + assertThat(fields, arrayWithSize(2)); + assertThat(fields[0].fieldType(), sameInstance(LatLonDocValuesField.TYPE)); + assertThat(fields[1].fieldType(), sameInstance(LatLonPoint.TYPE)); + + doc = mapper.parse(source("1", b -> b.field(field, new double[]{-71.34, 41.12}), null, Map.of(field, "points"))); + fields = doc.rootDoc().getFields(field); + assertThat(fields, arrayWithSize(2)); + assertThat(fields[0].fieldType(), sameInstance(LatLonDocValuesField.TYPE)); + assertThat(fields[1].fieldType(), sameInstance(LatLonPoint.TYPE)); + + doc = mapper.parse(source("1", b -> { + b.startObject(field); + b.field("lat", "-71.34"); + b.field("lon", 41.12); + b.endObject(); + }, null, Map.of(field, "points"))); + fields = doc.rootDoc().getFields(field); + assertThat(fields, arrayWithSize(2)); + assertThat(fields[0].fieldType(), sameInstance(LatLonDocValuesField.TYPE)); + assertThat(fields[1].fieldType(), sameInstance(LatLonPoint.TYPE)); + + doc = mapper.parse(source("1", b -> b.field(field, new String[]{"41.12,-71.34", "43,-72.34"}), null, Map.of(field, "points"))); + fields = doc.rootDoc().getFields(field); + assertThat(fields, arrayWithSize(4)); + assertThat(fields[0].fieldType(), sameInstance(LatLonDocValuesField.TYPE)); + assertThat(fields[1].fieldType(), sameInstance(LatLonDocValuesField.TYPE)); + assertThat(fields[2].fieldType(), sameInstance(LatLonPoint.TYPE)); + assertThat(fields[3].fieldType(), sameInstance(LatLonPoint.TYPE)); + + doc = mapper.parse(source("1", b -> { + b.startArray(field); + b.startObject(); + b.field("lat", -71.34); + b.field("lon", 41.12); + b.endObject(); + + b.startObject(); + b.field("lat", -71.34); + b.field("lon", 41.12); + b.endObject(); + b.endArray(); + }, null, Map.of(field, "points"))); + fields = doc.rootDoc().getFields(field); + assertThat(fields, arrayWithSize(4)); + assertThat(fields[0].fieldType(), sameInstance(LatLonDocValuesField.TYPE)); + assertThat(fields[1].fieldType(), sameInstance(LatLonDocValuesField.TYPE)); + assertThat(fields[2].fieldType(), sameInstance(LatLonPoint.TYPE)); + assertThat(fields[3].fieldType(), sameInstance(LatLonPoint.TYPE)); + + doc = mapper.parse(source("1", b -> { + b.startObject("address"); + b.field("home", "43,-72.34"); + b.endObject(); + }, null, Map.of("address.home", "points"))); + fields = doc.rootDoc().getFields("address.home"); + assertThat(fields, arrayWithSize(2)); + assertThat(fields[0].fieldType(), sameInstance(LatLonDocValuesField.TYPE)); + assertThat(fields[1].fieldType(), sameInstance(LatLonPoint.TYPE)); + } + + public void testDynamicTemplatesNotFound() throws Exception { + DocumentMapper mapper = createDocumentMapper(topMapping(b -> { + b.startArray("dynamic_templates"); + { + b.startObject(); + { + b.startObject("booleans"); + { + b.field("match", "none"); + b.startObject("mapping"); + { + b.field("type", "boolean"); + b.field("store", false); + b.field("doc_values", false); + } + b.endObject(); + } + b.endObject(); + } + b.endObject(); + } + b.endArray(); + })); + String field = randomFrom("foo", "foo.bar", "foo.bar.baz"); + ParsedDocument doc = mapper.parse(source("1", b -> b.field(field, "true"), null, Map.of(field, "booleans"))); + IndexableField[] fields = doc.rootDoc().getFields(field); + assertThat(fields, arrayWithSize(1)); + assertThat(fields[0].fieldType(), sameInstance(BooleanFieldMapper.Defaults.FIELD_TYPE)); + MapperParsingException error = expectThrows(MapperParsingException.class, () -> + mapper.parse(source("1", b -> b.field(field, "hello"), null, Map.of(field, "foo_bar")))); + assertThat(error.getMessage(), + containsString("Can't find dynamic template for dynamic template name [foo_bar] of field [" + field + "]")); + } + + public void testWrongTypeDynamicTemplate() throws Exception { + DocumentMapper mapper = createDocumentMapper(topMapping(b -> { + b.startArray("dynamic_templates"); + { + b.startObject(); + { + b.startObject("booleans"); + { + b.field("match", "none"); + b.startObject("mapping"); + { + b.field("type", "boolean"); + b.field("store", false); + b.field("doc_values", false); + } + b.endObject(); + } + b.endObject(); + } + b.endObject(); + } + b.endArray(); + })); + String field = randomFrom("foo.bar", "foo.bar.baz"); + MapperParsingException error = expectThrows(MapperParsingException.class, + () -> mapper.parse(source("1", b -> b.field(field, "true"), null, Map.of("foo", "booleans")))); + assertThat(error.getMessage(), + containsString("Field [foo] must be an object; but it's configured as [boolean] in dynamic template [booleans]")); + + ParsedDocument doc = mapper.parse(source("1", b -> b.field(field, "true"), null, Map.of(field, "booleans"))); + IndexableField[] fields = doc.rootDoc().getFields(field); + assertThat(fields, arrayWithSize(1)); + assertThat(fields[0].fieldType(), sameInstance(BooleanFieldMapper.Defaults.FIELD_TYPE)); + } + public void testDynamicDottedFieldNameLongArrayWithExistingParent() throws Exception { DocumentMapper mapper = createDocumentMapper(fieldMapping(b -> b.field("type", "object"))); ParsedDocument doc = mapper.parse(source(b -> b.startArray("field.bar.baz").value(0).value(1).endArray())); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DynamicTemplateTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DynamicTemplateTests.java index a4aaeebf139fc..13f18f600d551 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DynamicTemplateTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DynamicTemplateTests.java @@ -22,6 +22,8 @@ import java.util.List; import java.util.Map; +import static org.hamcrest.Matchers.equalTo; + public class DynamicTemplateTests extends ESTestCase { public void testMappingTypeTypeNotSet() { @@ -162,7 +164,7 @@ public void testMatchAllTemplate() { templateDef.put("match_mapping_type", "*"); templateDef.put("mapping", Collections.singletonMap("store", true)); DynamicTemplate template = DynamicTemplate.parse("my_template", templateDef); - assertTrue(template.match("a.b", "b", randomFrom(XContentFieldType.values()))); + assertTrue(template.match(null, "a.b", "b", randomFrom(XContentFieldType.values()))); assertFalse(template.isRuntimeMapping()); } @@ -172,13 +174,13 @@ public void testMatchAllTemplateRuntime() { templateDef.put("runtime", Collections.emptyMap()); DynamicTemplate template = DynamicTemplate.parse("my_template", templateDef); assertTrue(template.isRuntimeMapping()); - assertTrue(template.match("a.b", "b", XContentFieldType.BOOLEAN)); - assertTrue(template.match("a.b", "b", XContentFieldType.DATE)); - assertTrue(template.match("a.b", "b", XContentFieldType.STRING)); - assertTrue(template.match("a.b", "b", XContentFieldType.DOUBLE)); - assertTrue(template.match("a.b", "b", XContentFieldType.LONG)); - assertFalse(template.match("a.b", "b", XContentFieldType.OBJECT)); - assertFalse(template.match("a.b", "b", XContentFieldType.BINARY)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.BOOLEAN)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.DATE)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.STRING)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.DOUBLE)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.LONG)); + assertFalse(template.match(null, "a.b", "b", XContentFieldType.OBJECT)); + assertFalse(template.match(null, "a.b", "b", XContentFieldType.BINARY)); } public void testMatchAllTypesTemplateRuntime() { @@ -187,13 +189,13 @@ public void testMatchAllTypesTemplateRuntime() { templateDef.put("runtime", Collections.emptyMap()); DynamicTemplate template = DynamicTemplate.parse("my_template", templateDef); assertTrue(template.isRuntimeMapping()); - assertTrue(template.match("a.b", "b", XContentFieldType.BOOLEAN)); - assertTrue(template.match("a.b", "b", XContentFieldType.DATE)); - assertTrue(template.match("a.b", "b", XContentFieldType.STRING)); - assertTrue(template.match("a.b", "b", XContentFieldType.DOUBLE)); - assertTrue(template.match("a.b", "b", XContentFieldType.LONG)); - assertFalse(template.match("a.b", "b", XContentFieldType.OBJECT)); - assertFalse(template.match("a.b", "b", XContentFieldType.BINARY)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.BOOLEAN)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.DATE)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.STRING)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.DOUBLE)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.LONG)); + assertFalse(template.match(null, "a.b", "b", XContentFieldType.OBJECT)); + assertFalse(template.match(null, "a.b", "b", XContentFieldType.BINARY)); } public void testMatchTypeTemplate() { @@ -201,8 +203,8 @@ public void testMatchTypeTemplate() { templateDef.put("match_mapping_type", "string"); templateDef.put("mapping", Collections.singletonMap("store", true)); DynamicTemplate template = DynamicTemplate.parse("my_template", templateDef); - assertTrue(template.match("a.b", "b", XContentFieldType.STRING)); - assertFalse(template.match("a.b", "b", XContentFieldType.BOOLEAN)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.STRING)); + assertFalse(template.match(null, "a.b", "b", XContentFieldType.BOOLEAN)); assertFalse(template.isRuntimeMapping()); } @@ -211,8 +213,8 @@ public void testMatchTypeTemplateRuntime() { templateDef.put("match_mapping_type", "string"); templateDef.put("runtime", Collections.emptyMap()); DynamicTemplate template = DynamicTemplate.parse("my_template", templateDef); - assertTrue(template.match("a.b", "b", XContentFieldType.STRING)); - assertFalse(template.match("a.b", "b", XContentFieldType.BOOLEAN)); + assertTrue(template.match(null, "a.b", "b", XContentFieldType.STRING)); + assertFalse(template.match(null, "a.b", "b", XContentFieldType.BOOLEAN)); assertTrue(template.isRuntimeMapping()); } @@ -249,6 +251,9 @@ public void testSerialization() throws Exception { // name-based template templateDef = new HashMap<>(); + if (randomBoolean()) { + templateDef.put("match_mapping_type", "*"); + } templateDef.put("match", "*name"); templateDef.put("unmatch", "first_name"); templateDef.put("mapping", Collections.singletonMap("store", true)); @@ -261,6 +266,9 @@ public void testSerialization() throws Exception { templateDef = new HashMap<>(); templateDef.put("path_match", "*name"); templateDef.put("path_unmatch", "first_name"); + if (randomBoolean()) { + templateDef.put("match_mapping_type", "*"); + } templateDef.put("mapping", Collections.singletonMap("store", true)); template = DynamicTemplate.parse("my_template", templateDef); builder = JsonXContent.contentBuilder(); @@ -272,11 +280,22 @@ public void testSerialization() throws Exception { templateDef = new HashMap<>(); templateDef.put("match", "^a$"); templateDef.put("match_pattern", "regex"); + if (randomBoolean()) { + templateDef.put("match_mapping_type", "*"); + } templateDef.put("mapping", Collections.singletonMap("store", true)); template = DynamicTemplate.parse("my_template", templateDef); builder = JsonXContent.contentBuilder(); template.toXContent(builder, ToXContent.EMPTY_PARAMS); assertEquals("{\"match\":\"^a$\",\"match_pattern\":\"regex\",\"mapping\":{\"store\":true}}", Strings.toString(builder)); + + // empty condition + templateDef = new HashMap<>(); + templateDef.put("mapping", Collections.singletonMap("store", true)); + template = DynamicTemplate.parse("my_template", templateDef); + builder = JsonXContent.contentBuilder(); + template.toXContent(builder, ToXContent.EMPTY_PARAMS); + assertThat(Strings.toString(builder), equalTo("{\"mapping\":{\"store\":true}}")); } public void testSerializationRuntimeMappings() throws Exception { @@ -293,6 +312,9 @@ public void testSerializationRuntimeMappings() throws Exception { templateDef = new HashMap<>(); templateDef.put("match", "*name"); templateDef.put("unmatch", "first_name"); + if (randomBoolean()) { + templateDef.put("match_mapping_type", "*"); + } templateDef.put("runtime", Collections.singletonMap("type", "new_type")); template = DynamicTemplate.parse("my_template", templateDef); builder = JsonXContent.contentBuilder(); @@ -303,6 +325,9 @@ public void testSerializationRuntimeMappings() throws Exception { templateDef = new HashMap<>(); templateDef.put("path_match", "*name"); templateDef.put("path_unmatch", "first_name"); + if (randomBoolean()) { + templateDef.put("match_mapping_type", "*"); + } templateDef.put("runtime", Collections.emptyMap()); template = DynamicTemplate.parse("my_template", templateDef); builder = JsonXContent.contentBuilder(); @@ -313,6 +338,9 @@ public void testSerializationRuntimeMappings() throws Exception { // regex matching templateDef = new HashMap<>(); templateDef.put("match", "^a$"); + if (randomBoolean()) { + templateDef.put("match_mapping_type", "*"); + } templateDef.put("match_pattern", "regex"); templateDef.put("runtime", Collections.emptyMap()); template = DynamicTemplate.parse("my_template", templateDef); @@ -320,4 +348,49 @@ public void testSerializationRuntimeMappings() throws Exception { template.toXContent(builder, ToXContent.EMPTY_PARAMS); assertEquals("{\"match\":\"^a$\",\"match_pattern\":\"regex\",\"runtime\":{}}", Strings.toString(builder)); } + + public void testMatchTemplateName() throws Exception { + // match_mapping_type + { + Map templateDef = new HashMap<>(); + templateDef.put("match_mapping_type", "string"); + if (randomBoolean()) { + templateDef.put("runtime", Collections.emptyMap()); + } else { + templateDef.put("mapping", Map.of()); + } + DynamicTemplate template = DynamicTemplate.parse("my_template", templateDef); + assertTrue(template.match("my_template", "a", "a.b", randomFrom(XContentFieldType.values()))); + assertFalse(template.match("not_template_name", "a", "a.b", XContentFieldType.BOOLEAN)); + + assertTrue(template.match(null, "a", "a.b", XContentFieldType.STRING)); + assertFalse(template.match(null, "a", "a.b", XContentFieldType.BOOLEAN)); + } + // match name + { + Map templateDef = new HashMap<>(); + templateDef.put("match", "foo*"); + templateDef.put("mapping", Map.of()); + DynamicTemplate template = DynamicTemplate.parse("my_template", templateDef); + assertTrue(template.match("my_template", "foo.bar", "foo", randomFrom(XContentFieldType.values()))); + assertTrue(template.match(null, "foo.bar", "foo", randomFrom(XContentFieldType.values()))); + assertFalse(template.match("not_template_name", "foo.bar", "foo", randomFrom(XContentFieldType.values()))); + assertTrue(template.match("my_template", "foo.bar", "not_match_name", randomFrom(XContentFieldType.values()))); + assertFalse(template.match(null, "foo.bar", "not_match_name", randomFrom(XContentFieldType.values()))); + } + // no match condition + { + Map templateDef = new HashMap<>(); + if (randomBoolean()) { + templateDef.put("runtime", Collections.emptyMap()); + } else { + templateDef.put("mapping", Map.of()); + } + DynamicTemplate template = DynamicTemplate.parse("my_template", templateDef); + assertTrue(template.match("my_template", "foo.bar", "bar", randomFrom(XContentFieldType.values()))); + assertFalse(template.match(null, "foo.bar", "foo", randomFrom(XContentFieldType.values()))); + assertFalse(template.match("not_template_name", "foo.bar", "bar", randomFrom(XContentFieldType.values()))); + assertTrue(template.match("my_template", "foo.bar", "bar", randomFrom(XContentFieldType.values()))); + } + } } 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 7007166126983..df12c6309468b 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/RootObjectMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/RootObjectMapperTests.java @@ -10,17 +10,21 @@ import org.elasticsearch.Version; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; 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 java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.Map; import static org.elasticsearch.test.VersionUtils.randomVersionBetween; +import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; @@ -678,4 +682,32 @@ public void testRuntimeSectionRemainingField() throws IOException { MapperParsingException e = expectThrows(MapperParsingException.class, () -> createMapperService(mapping)); assertEquals("Failed to parse mapping: unknown parameter [unsupported] on mapper [field] of type [keyword]", e.getMessage()); } + + public void testTemplateWithoutMatchPredicates() throws Exception { + XContentBuilder mapping = XContentFactory.jsonBuilder(); + mapping.startObject(); + { + mapping.startObject(MapperService.SINGLE_MAPPING_NAME); + mapping.startArray("dynamic_templates"); + { + mapping.startObject(); + mapping.startObject("geo_point"); + { + mapping.startObject("mapping"); + mapping.field("type", "geo_point"); + mapping.endObject(); + } + mapping.endObject(); + mapping.endObject(); + } + mapping.endArray(); + mapping.endObject(); + } + mapping.endObject(); + MapperService mapperService = createMapperService(mapping); + ParsedDocument doc = mapperService.documentMapper().parse(new SourceToParse("test", "1", + new BytesArray("{\"foo\": \"41.12,-71.34\", \"bar\": \"41.12,-71.34\"}"), XContentType.JSON, null, Map.of("foo", "geo_point"))); + assertThat(doc.rootDoc().getFields("foo"), arrayWithSize(2)); + assertThat(doc.rootDoc().getFields("bar"), arrayWithSize(1)); + } } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/RoutingFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/RoutingFieldMapperTests.java index 6d2043d1f4269..9183cbbf1f7b7 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/RoutingFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/RoutingFieldMapperTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.xcontent.XContentType; import java.io.IOException; +import java.util.Map; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -37,7 +38,7 @@ public void testRoutingMapper() throws Exception { .startObject() .field("field", "value") .endObject()), - XContentType.JSON, "routing_value")); + XContentType.JSON, "routing_value", Map.of())); assertThat(doc.rootDoc().get("_routing"), equalTo("routing_value")); assertThat(doc.rootDoc().get("field"), equalTo("value")); diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java b/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java index 6c0a0e3685ae0..33f4db118ba4b 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java @@ -101,7 +101,7 @@ public Engine.Operation convertToEngineOp(Translog.Operation operation, Engine.O final String indexName = mapperService.index().getName(); final Engine.Index engineIndex = IndexShard.prepareIndex(docMapper(MapperService.SINGLE_MAPPING_NAME), new SourceToParse(indexName, index.id(), index.source(), XContentHelper.xContentType(index.source()), - index.routing()), index.seqNo(), index.primaryTerm(), + index.routing(), Map.of()), index.seqNo(), index.primaryTerm(), index.version(), versionType, origin, index.getAutoGeneratedIdTimestamp(), true, SequenceNumbers.UNASSIGNED_SEQ_NO, SequenceNumbers.UNASSIGNED_PRIMARY_TERM); return engineIndex; diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java index cd1358b0d4f26..50b6132738d4e 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java @@ -234,7 +234,15 @@ protected final SourceToParse source(String id, CheckedConsumer build, + @Nullable String routing, Map dynamicTemplates) throws IOException { + XContentBuilder builder = JsonXContent.contentBuilder().startObject(); + build.accept(builder); + builder.endObject(); + return new SourceToParse("test", id, BytesReference.bytes(builder), XContentType.JSON, routing, dynamicTemplates); } protected final SourceToParse source(String source) { diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 2011ff08ffbe1..191de1057f510 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -81,6 +81,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -732,7 +733,7 @@ protected Engine.IndexResult indexDoc(IndexShard shard, String id, String source String routing) throws IOException { SourceToParse sourceToParse = new SourceToParse( - shard.shardId().getIndexName(), id, new BytesArray(source), xContentType, routing); + shard.shardId().getIndexName(), id, new BytesArray(source), xContentType, routing, Map.of()); Engine.IndexResult result; if (shard.routingEntry().primary()) { result = shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, sourceToParse, diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 6a4be02caafa8..91fc8db228264 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -74,6 +74,7 @@ import java.io.IOException; import java.nio.file.Path; import java.util.Collections; +import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.function.Function; @@ -334,7 +335,7 @@ public IndexShard reindex(DirectoryReader reader, MappingMetadata mapping) throw assert source != null : "_source is null but should have been filtered out at snapshot time"; Engine.Result result = targetShard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, new SourceToParse(index, id, source, XContentHelper.xContentType(source), - rootFieldsVisitor.routing()), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, + rootFieldsVisitor.routing(), Map.of()), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); if (result.getResultType() != Engine.Result.Type.SUCCESS) { throw new IllegalStateException("failed applying post restore operation result: " + result