From edbfb3597df91496c7830e944cf09aa129d154f8 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Wed, 10 Aug 2022 11:35:07 -0400 Subject: [PATCH 1/4] Include runtime fields in total fields count We have a check that enforces the total number of fields needs to be below a certain (configurable) threshold. Before runtime fields did not contribute to the count. This patch makes all runtime fields contribute to the count, runtime fields: - that were explicitly defined in mapping by a user - as well as runtime fields that were dynamically created by dynamic mappings Closes #88265 --- .../mapping/mapping-settings-limit.asciidoc | 3 +- .../index/mapper/DynamicMappingIT.java | 64 +++++++++++++++++++ .../index/mapper/DocumentParserContext.java | 21 +++--- .../index/mapper/MappingLookup.java | 5 +- 4 files changed, 83 insertions(+), 10 deletions(-) diff --git a/docs/reference/mapping/mapping-settings-limit.asciidoc b/docs/reference/mapping/mapping-settings-limit.asciidoc index 0f94a376f4041..c499ca7675f2c 100644 --- a/docs/reference/mapping/mapping-settings-limit.asciidoc +++ b/docs/reference/mapping/mapping-settings-limit.asciidoc @@ -4,7 +4,8 @@ Use the following settings to limit the number of field mappings (created manual `index.mapping.total_fields.limit`:: The maximum number of fields in an index. Field and object mappings, as well as - field aliases count towards this limit. The default value is `1000`. + field aliases count towards this limit. Mapped runtime fields count towards this + limit as well. The default value is `1000`. + [IMPORTANT] ==== 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 4e2b3fb952164..3b0be9933165a 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java @@ -18,6 +18,7 @@ import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.MappingMetadata; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Randomness; @@ -213,6 +214,69 @@ public void onFailure(Exception e) { } } + public void testTotalFieldsLimitWithRunTimefields() { + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.getKey(), 4) + .build(); + + String mapping = """ + { + "dynamic":"runtime", + "runtime": { + "my_object.rfield1": { + "type": "keyword" + }, + "rfield2": { + "type": "keyword" + } + }, + "properties": { + "field3" : { + "type": "keyword" + } + } + } + """; + + client().admin().indices().prepareCreate("index1").setSettings(indexSettings).setMapping(mapping).get(); + ensureGreen("index1"); + + { + // introduction of a new object with 2 sub-fields fails + final IndexRequestBuilder indexRequestBuilder = client().prepareIndex("index1") + .setId("1") + .setSource("my_object2", Map.of("field1", "value1", "field2", "value2")); + Exception exc = expectThrows(MapperParsingException.class, () -> indexRequestBuilder.get(TimeValue.timeValueSeconds(10))); + assertThat(exc.getMessage(), Matchers.containsString("failed to parse")); + assertThat(exc.getCause(), instanceOf(IllegalArgumentException.class)); + assertThat( + exc.getCause().getMessage(), + Matchers.containsString("Limit of total fields [4] has been exceeded while adding new fields [2]") + ); + } + + { + // introduction of a single field succeeds + client().prepareIndex("index1").setId("2").setSource("field4", 100).get(); + } + + { + // remove 2 runtime field mappings + assertAcked(client().admin().indices().preparePutMapping("index1").setSource(""" + { + "runtime": { + "my_object.rfield1": null, + "rfield2" : null + } + } + """, XContentType.JSON)); + + // introduction of a new object with 2 sub-fields succeeds + client().prepareIndex("index1").setId("2").setSource("my_object2", Map.of("field1", "value1", "field2", "value2")).get(); + } + } + public void testMappingVersionAfterDynamicMappingUpdate() throws Exception { createIndex("test"); final ClusterService clusterService = internalCluster().clusterService(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java index 9b6353b862b24..9e69e277caa68 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java @@ -226,14 +226,7 @@ public final String documentDescription() { * Add a new mapper dynamically created while parsing. */ public final void addDynamicMapper(Mapper mapper) { - // eagerly check field name limit here to avoid OOM errors - // only check fields that are not already mapped or tracked in order to avoid hitting field limit too early via double-counting - // note that existing fields can also receive dynamic mapping updates (e.g. constant_keyword to fix the value) - if (mappingLookup.getMapper(mapper.name()) == null - && mappingLookup.objectMappers().containsKey(mapper.name()) == false - && newFieldsSeen.add(mapper.name())) { - mappingLookup.checkFieldLimit(indexSettings().getMappingTotalFieldsLimit(), newFieldsSeen.size()); - } + checkFieldLimit(mapper.name()); if (mapper instanceof ObjectMapper objectMapper) { dynamicObjectMappers.put(objectMapper.name(), objectMapper); // dynamic object mappers may have been obtained from applying a dynamic template, in which case their definition may contain @@ -254,6 +247,17 @@ public final void addDynamicMapper(Mapper mapper) { dynamicMappers.add(mapper); } + private void checkFieldLimit(String fieldName) { + // eagerly check field name limit here to avoid OOM errors + // only check fields that are not already mapped or tracked in order to avoid hitting field limit too early via double-counting + // note that existing fields can also receive dynamic mapping updates (e.g. constant_keyword to fix the value) + if (mappingLookup.getMapper(fieldName) == null + && mappingLookup.objectMappers().containsKey(fieldName) == false + && newFieldsSeen.add(fieldName)) { + mappingLookup.checkFieldLimit(indexSettings().getMappingTotalFieldsLimit(), newFieldsSeen.size()); + } + } + /** * Get dynamic mappers created as a result of parsing an incoming document. Responsible for exposing all the newly created * fields that need to be merged into the existing mappings. Used to create the required mapping update at the end of document parsing. @@ -279,6 +283,7 @@ final ObjectMapper getDynamicObjectMapper(String name) { * Add a new runtime field dynamically created while parsing. */ public final void addDynamicRuntimeField(RuntimeField runtimeField) { + checkFieldLimit(runtimeField.name()); dynamicRuntimeFields.add(runtimeField); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java index 19f02f9ad0b84..a45fa7ff0e248 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -47,6 +47,7 @@ private CacheKey() {} /** Full field name to mapper */ private final Map fieldMappers; private final Map objectMappers; + private final int runtimeFieldMappersCount; private final NestedLookup nestedLookup; private final FieldTypeLookup fieldTypeLookup; private final FieldTypeLookup indexTimeLookup; // for index-time scripts, a lookup that does not include runtime fields @@ -180,6 +181,7 @@ private MappingLookup( // make all fields into compact+fast immutable maps this.fieldMappers = Map.copyOf(fieldMappers); this.objectMappers = Map.copyOf(objects); + this.runtimeFieldMappersCount = runtimeFields.size(); this.indexAnalyzersMap = Map.copyOf(indexAnalyzersMap); this.completionFields = Set.copyOf(completionFields); this.indexTimeScriptMappers = List.copyOf(indexTimeScriptMappers); @@ -262,7 +264,8 @@ private void checkFieldLimit(long limit) { } void checkFieldLimit(long limit, int additionalFieldsToAdd) { - if (fieldMappers.size() + objectMappers.size() + additionalFieldsToAdd - mapping.getSortedMetadataMappers().length > limit) { + if (fieldMappers.size() + objectMappers.size() + runtimeFieldMappersCount + additionalFieldsToAdd - mapping + .getSortedMetadataMappers().length > limit) { throw new IllegalArgumentException( "Limit of total fields [" + limit From 8a7bdd8210d529c34fefd15059fe0868269957a7 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Wed, 10 Aug 2022 11:47:25 -0400 Subject: [PATCH 2/4] Update docs/changelog/89251.yaml --- docs/changelog/89251.yaml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 docs/changelog/89251.yaml diff --git a/docs/changelog/89251.yaml b/docs/changelog/89251.yaml new file mode 100644 index 0000000000000..a3285d7b467a5 --- /dev/null +++ b/docs/changelog/89251.yaml @@ -0,0 +1,6 @@ +pr: 89251 +summary: Include runtime fields in total fields count +area: Mapping +type: bug +issues: + - 88265 From 1ee42705cb6ad4c4c5b6083a33c042a1b8269912 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Thu, 11 Aug 2022 16:23:14 -0400 Subject: [PATCH 3/4] Address Luca's feedback 1 --- .../index/mapper/DynamicMappingIT.java | 16 +++++++------ .../index/mapper/DocumentParserContext.java | 24 +++++++++---------- .../index/mapper/MapperServiceTests.java | 7 ++++++ 3 files changed, 27 insertions(+), 20 deletions(-) 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 3b0be9933165a..d31475a172056 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java @@ -214,7 +214,7 @@ public void onFailure(Exception e) { } } - public void testTotalFieldsLimitWithRunTimefields() { + public void testTotalFieldsLimitWithRuntimeFields() { Settings indexSettings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.getKey(), 4) @@ -243,10 +243,10 @@ public void testTotalFieldsLimitWithRunTimefields() { ensureGreen("index1"); { - // introduction of a new object with 2 sub-fields fails + // introduction of a new object with 2 new sub-fields fails final IndexRequestBuilder indexRequestBuilder = client().prepareIndex("index1") .setId("1") - .setSource("my_object2", Map.of("field1", "value1", "field2", "value2")); + .setSource("field3", "value3", "my_object2", Map.of("new_field1", "value1", "new_field2", "value2")); Exception exc = expectThrows(MapperParsingException.class, () -> indexRequestBuilder.get(TimeValue.timeValueSeconds(10))); assertThat(exc.getMessage(), Matchers.containsString("failed to parse")); assertThat(exc.getCause(), instanceOf(IllegalArgumentException.class)); @@ -257,8 +257,8 @@ public void testTotalFieldsLimitWithRunTimefields() { } { - // introduction of a single field succeeds - client().prepareIndex("index1").setId("2").setSource("field4", 100).get(); + // introduction of a new single field succeeds + client().prepareIndex("index1").setId("2").setSource("field3", "value3", "new_field4", 100).get(); } { @@ -272,8 +272,10 @@ public void testTotalFieldsLimitWithRunTimefields() { } """, XContentType.JSON)); - // introduction of a new object with 2 sub-fields succeeds - client().prepareIndex("index1").setId("2").setSource("my_object2", Map.of("field1", "value1", "field2", "value2")).get(); + // introduction of a new object with 2 new sub-fields succeeds + client().prepareIndex("index1") + .setId("1") + .setSource("field3", "value3", "my_object2", Map.of("new_field1", "value1", "new_field2", "value2")); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java index 9e69e277caa68..f1d5b5245b2c7 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java @@ -226,7 +226,14 @@ public final String documentDescription() { * Add a new mapper dynamically created while parsing. */ public final void addDynamicMapper(Mapper mapper) { - checkFieldLimit(mapper.name()); + // eagerly check field name limit here to avoid OOM errors + // only check fields that are not already mapped or tracked in order to avoid hitting field limit too early via double-counting + // note that existing fields can also receive dynamic mapping updates (e.g. constant_keyword to fix the value) + if (mappingLookup.getMapper(mapper.name()) == null + && mappingLookup.objectMappers().containsKey(mapper.name()) == false + && newFieldsSeen.add(mapper.name())) { + mappingLookup.checkFieldLimit(indexSettings().getMappingTotalFieldsLimit(), newFieldsSeen.size()); + } if (mapper instanceof ObjectMapper objectMapper) { dynamicObjectMappers.put(objectMapper.name(), objectMapper); // dynamic object mappers may have been obtained from applying a dynamic template, in which case their definition may contain @@ -247,17 +254,6 @@ public final void addDynamicMapper(Mapper mapper) { dynamicMappers.add(mapper); } - private void checkFieldLimit(String fieldName) { - // eagerly check field name limit here to avoid OOM errors - // only check fields that are not already mapped or tracked in order to avoid hitting field limit too early via double-counting - // note that existing fields can also receive dynamic mapping updates (e.g. constant_keyword to fix the value) - if (mappingLookup.getMapper(fieldName) == null - && mappingLookup.objectMappers().containsKey(fieldName) == false - && newFieldsSeen.add(fieldName)) { - mappingLookup.checkFieldLimit(indexSettings().getMappingTotalFieldsLimit(), newFieldsSeen.size()); - } - } - /** * Get dynamic mappers created as a result of parsing an incoming document. Responsible for exposing all the newly created * fields that need to be merged into the existing mappings. Used to create the required mapping update at the end of document parsing. @@ -283,7 +279,9 @@ final ObjectMapper getDynamicObjectMapper(String name) { * Add a new runtime field dynamically created while parsing. */ public final void addDynamicRuntimeField(RuntimeField runtimeField) { - checkFieldLimit(runtimeField.name()); + if (newFieldsSeen.add(runtimeField.name())) { + mappingLookup.checkFieldLimit(indexSettings().getMappingTotalFieldsLimit(), newFieldsSeen.size()); + } dynamicRuntimeFields.add(runtimeField); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java b/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java index f3771510d8da9..38d7567ce40e3 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java @@ -70,6 +70,13 @@ public void testTotalFieldsLimit() throws Throwable { () -> merge(mapperService, mapping(b -> b.startObject("newfield").field("type", "long").endObject())) ); assertTrue(e.getMessage(), e.getMessage().contains("Limit of total fields [" + totalFieldsLimit + "] has been exceeded")); + + // adding one more runtime field should trigger exception + e = expectThrows( + IllegalArgumentException.class, + () -> merge(mapperService, runtimeMapping(b -> b.startObject("newfield").field("type", "long").endObject())) + ); + assertTrue(e.getMessage(), e.getMessage().contains("Limit of total fields [" + totalFieldsLimit + "] has been exceeded")); } private void createMappingSpecifyingNumberOfFields(XContentBuilder b, int numberOfFields) throws IOException { From dbf20223bce7a5340f803eb59188a9d7d5aa5418 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Fri, 12 Aug 2022 10:05:57 -0400 Subject: [PATCH 4/4] Address Luca's feedback 2 --- .../elasticsearch/index/mapper/DocumentParserContext.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java index f1d5b5245b2c7..da4b8673c362b 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java @@ -277,8 +277,11 @@ final ObjectMapper getDynamicObjectMapper(String name) { /** * Add a new runtime field dynamically created while parsing. + * We use the same set for both new indexed and new runtime fields, + * because for dynamic mappings, a new field can be either mapped + * as runtime or indexed, but never both. */ - public final void addDynamicRuntimeField(RuntimeField runtimeField) { + final void addDynamicRuntimeField(RuntimeField runtimeField) { if (newFieldsSeen.add(runtimeField.name())) { mappingLookup.checkFieldLimit(indexSettings().getMappingTotalFieldsLimit(), newFieldsSeen.size()); }