From 528bfc6b488bdfa99aa777f4c88d5874e0fae073 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 20 Nov 2020 15:38:02 -0500 Subject: [PATCH 01/47] Bust the request cache when the mapping changes This makes sure that we only serve a hit from the request cache if it was build using the same mapping and that the entire search phase is processed using the same mapping. These are both bugs that Elasticsearch has had for a long, long time but now that we have runtime fields they are much more likely to matter because runtime fields can be changed *drastically* from moment to moment. It does by adding a counter to the local mapping that is incremented whenever the mapping changes. This counter and the mapping is stored under a single `volatile` variable that is updated to point to the new mapping whenever the mapping changes. When the search phase begins we read the contents of the `volatile` and use it to power the entire phase, and including the version counter into the cache key. Before this change we'd go back to the `volatile` over and over again, potentially getting a new version of the mapping every time. This was convenient but made it possible that we'd see the mapping update half way through the search phase which could cause trouble. Mechanically, this creates a `snapshot()` method on `MapperService` to do the volatile read of the current mapping. The snapshot does all of the reading we need from here on out. I kept all of the methods on `MapperService` that read directly from the mapping but deprecated all of them because using them can lead to seeing the mapping updates when you don't expect them. We can slowly remove the usages in follow up changes. This change is big enough as is. Sadly, the `Mapper`s are also mutable. Luckilly, we only mutate them before sticking them behind the `volatile` reference. Hopefully. There is a lot of code there and it is hard to be sure. But it looks like we *intend* to do this. We should make them properly immutable in a follow up change. Closes #62033 --- .../ChildrenToParentAggregatorTests.java | 16 +- .../ParentToChildrenAggregatorTests.java | 16 +- .../org/elasticsearch/index/IndexService.java | 2 +- .../index/mapper/MapperService.java | 594 +++++++++++++++--- .../index/query/QueryShardContext.java | 63 +- .../elasticsearch/indices/IndicesService.java | 5 +- .../search/DefaultSearchContext.java | 5 - .../elasticsearch/search/NestedDocuments.java | 20 +- .../search/fetch/FetchPhase.java | 2 +- .../internal/FilteredSearchContext.java | 6 - .../search/internal/SearchContext.java | 3 - .../search/internal/ShardSearchRequest.java | 3 +- .../mapper/FieldNamesFieldTypeTests.java | 11 +- .../index/mapper/MapperServiceTests.java | 16 +- .../index/mapper/NestedDocumentsTests.java | 6 +- .../index/query/QueryShardContextTests.java | 68 +- .../index/query/RangeQueryRewriteTests.java | 6 +- .../terms/SignificantTextAggregatorTests.java | 12 + .../metrics/MinAggregatorTests.java | 21 +- .../ScriptedMetricAggregatorTests.java | 10 +- .../fetch/subphase/FieldFetcherTests.java | 22 +- .../internal/ShardSearchRequestTests.java | 3 +- .../AbstractSuggestionBuilderTestCase.java | 18 +- .../aggregations/AggregatorTestCase.java | 51 +- .../test/AbstractBuilderTestCase.java | 2 +- .../elasticsearch/test/TestSearchContext.java | 6 - .../DocumentSubsetBitsetCacheTests.java | 10 +- ...ityIndexReaderWrapperIntegrationTests.java | 33 +- .../test/runtime_fields/10_keyword.yml | 26 + 29 files changed, 748 insertions(+), 308 deletions(-) diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java index 967db37f25002..4d37c2c76e30a 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java @@ -32,11 +32,8 @@ import org.apache.lucene.search.TermInSetQuery; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.IdFieldMapper; @@ -273,20 +270,19 @@ private static SortedDocValuesField createJoinField(String parentType, String id } @Override - protected MapperService mapperServiceMock() { + protected MapperService.Snapshot mapperSnapshotMock() { ParentJoinFieldMapper joinFieldMapper = createJoinFieldMapper(); - MapperService mapperService = mock(MapperService.class); MetaJoinFieldMapper.MetaJoinFieldType metaJoinFieldType = mock(MetaJoinFieldMapper.MetaJoinFieldType.class); + MapperService.Snapshot mapperSnapshot = mock(MapperService.Snapshot.class); when(metaJoinFieldType.getJoinField()).thenReturn("join_field"); - when(mapperService.fieldType("_parent_join")).thenReturn(metaJoinFieldType); - when(mapperService.fieldType("join_field")).thenReturn(joinFieldMapper.fieldType()); - when(mapperService.fieldType("join_field#" + PARENT_TYPE)) + when(mapperSnapshot.fieldType("_parent_join")).thenReturn(metaJoinFieldType); + when(mapperSnapshot.fieldType("join_field")).thenReturn(joinFieldMapper.fieldType()); + when(mapperSnapshot.fieldType("join_field#" + PARENT_TYPE)) .thenReturn(new ParentIdFieldMapper.ParentIdFieldType("join_field#" + PARENT_TYPE, false)); - return mapperService; + return mapperSnapshot; } private static ParentJoinFieldMapper createJoinFieldMapper() { - Settings settings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build(); return new ParentJoinFieldMapper.Builder("join_field") .addRelation(PARENT_TYPE, Collections.singleton(CHILD_TYPE)) .build(new ContentPath(0)); diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregatorTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregatorTests.java index 343713edabfca..b72911f3eb316 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregatorTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregatorTests.java @@ -33,11 +33,8 @@ import org.apache.lucene.search.TermInSetQuery; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.IdFieldMapper; @@ -225,20 +222,19 @@ private static SortedDocValuesField createJoinField(String parentType, String id } @Override - protected MapperService mapperServiceMock() { + protected MapperService.Snapshot mapperSnapshotMock() { ParentJoinFieldMapper joinFieldMapper = createJoinFieldMapper(); - MapperService mapperService = mock(MapperService.class); MetaJoinFieldMapper.MetaJoinFieldType metaJoinFieldType = mock(MetaJoinFieldMapper.MetaJoinFieldType.class); + MapperService.Snapshot mapperSnapshot = mock(MapperService.Snapshot.class); when(metaJoinFieldType.getJoinField()).thenReturn("join_field"); - when(mapperService.fieldType("_parent_join")).thenReturn(metaJoinFieldType); - when(mapperService.fieldType("join_field")).thenReturn(joinFieldMapper.fieldType()); - when(mapperService.fieldType("join_field#" + PARENT_TYPE)) + when(mapperSnapshot.fieldType("_parent_join")).thenReturn(metaJoinFieldType); + when(mapperSnapshot.fieldType("join_field")).thenReturn(joinFieldMapper.fieldType()); + when(mapperSnapshot.fieldType("join_field#" + PARENT_TYPE)) .thenReturn(new ParentIdFieldMapper.ParentIdFieldType("join_field#" + PARENT_TYPE, false)); - return mapperService; + return mapperSnapshot; } private static ParentJoinFieldMapper createJoinFieldMapper() { - Settings settings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build(); return new ParentJoinFieldMapper.Builder("join_field") .addRelation(PARENT_TYPE, Collections.singleton(CHILD_TYPE)) .build(new ContentPath(0)); diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index bec23def8b1c3..4d68725f30d4e 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -596,7 +596,7 @@ public QueryShardContext newQueryShardContext( final SearchIndexNameMatcher indexNameMatcher = new SearchIndexNameMatcher(index().getName(), clusterAlias, clusterService, expressionResolver); return new QueryShardContext( - shardId, indexSettings, bigArrays, indexCache.bitsetFilterCache(), indexFieldData::getForField, mapperService(), + shardId, indexSettings, bigArrays, indexCache.bitsetFilterCache(), indexFieldData::getForField, mapperService().snapshot(), similarityService(), scriptService, xContentRegistry, namedWriteableRegistry, client, searcher, nowInMillis, clusterAlias, indexNameMatcher, allowExpensiveQueries, valuesSourceRegistry, runtimeMappings); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 3901c4135e7d0..842cbf70be7b2 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -42,11 +42,13 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.analysis.CharFilterFactory; +import org.elasticsearch.index.analysis.FieldNameAnalyzer; import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.ReloadableCustomAnalyzer; import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.index.analysis.TokenizerFactory; +import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.IndicesModule; @@ -56,7 +58,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -65,6 +66,11 @@ import java.util.function.Function; import java.util.function.Supplier; +import static java.util.Collections.emptySet; +import static java.util.Collections.singleton; +import static java.util.Collections.unmodifiableSet; +import static java.util.Objects.requireNonNull; + public class MapperService extends AbstractIndexComponent implements Closeable { /** @@ -108,11 +114,14 @@ public enum MergeReason { private final DocumentMapperParser documentMapperParser; private final DocumentParser documentParser; private final Version indexVersionCreated; - private final MapperAnalyzerWrapper indexAnalyzer; + private final Analyzer indexAnalyzer; private final MapperRegistry mapperRegistry; private final Supplier parserContextSupplier; - private volatile DocumentMapper mapper; + /** + * The current mapping accessed through {@link #snapshot()} and {@link #indexAnalyzer()}. + */ + private volatile AbstractSnapshot snapshot = new EmptySnapshot(this); public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, NamedXContentRegistry xContentRegistry, SimilarityService similarityService, MapperRegistry mapperRegistry, @@ -120,8 +129,13 @@ public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, ScriptService scriptService) { super(indexSettings); this.indexVersionCreated = indexSettings.getIndexVersionCreated(); + indexAnalyzer = new DelegatingAnalyzerWrapper(Analyzer.PER_FIELD_REUSE_STRATEGY) { + @Override + protected Analyzer getWrappedAnalyzer(String fieldName) { + return snapshot.indexAnalyzer(); + } + }; this.indexAnalyzers = indexAnalyzers; - this.indexAnalyzer = new MapperAnalyzerWrapper(); this.mapperRegistry = mapperRegistry; Function parserContextFunction = dateFormatter -> new Mapper.TypeParser.ParserContext(similarityService::getSimilarity, mapperRegistry.getMapperParsers()::get, @@ -135,8 +149,21 @@ public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, this::getMetadataMappers, parserContextSupplier, metadataMapperParsers); } + /** + * {@code volatile} read of an immutable snapshot of the current mapping. + */ + public Snapshot snapshot() { + return snapshot; + } + + /** + * Does this index contain nested documents? + * @deprecated Get a {@link #snapshot} and call {@link Snapshot#hasNested} on + * it as many times as needed. + */ + @Deprecated public boolean hasNested() { - return this.mapper != null && this.mapper.hasNestedObjects(); + return snapshot.hasNested(); } public IndexAnalyzers getIndexAnalyzers() { @@ -156,19 +183,8 @@ DocumentParser documentParser() { } Map, MetadataFieldMapper> getMetadataMappers() { - final DocumentMapper existingMapper = mapper; - final Map metadataMapperParsers = - mapperRegistry.getMetadataMapperParsers(indexSettings.getIndexVersionCreated()); Map, MetadataFieldMapper> metadataMappers = new LinkedHashMap<>(); - if (existingMapper == null) { - for (MetadataFieldMapper.TypeParser parser : metadataMapperParsers.values()) { - MetadataFieldMapper metadataFieldMapper = parser.getDefault(parserContext()); - metadataMappers.put(metadataFieldMapper.getClass(), metadataFieldMapper); - } - - } else { - metadataMappers.putAll(existingMapper.mapping().metadataMappersMap); - } + snapshot.collectMetadataMappers(metadataMappers); return metadataMappers; } @@ -190,30 +206,30 @@ public boolean updateMapping(final IndexMetadata currentIndexMetadata, final Ind + " but was " + newIndexMetadata.getIndex(); if (currentIndexMetadata != null && currentIndexMetadata.getMappingVersion() == newIndexMetadata.getMappingVersion()) { - assertMappingVersion(currentIndexMetadata, newIndexMetadata, this.mapper); + assertMappingVersion(currentIndexMetadata, newIndexMetadata, snapshot); return false; } - final DocumentMapper updatedMapper; + final Snapshot updatedSnapshot; try { - updatedMapper = internalMerge(newIndexMetadata, MergeReason.MAPPING_RECOVERY); + updatedSnapshot = internalMerge(newIndexMetadata, MergeReason.MAPPING_RECOVERY); } catch (Exception e) { logger.warn(() -> new ParameterizedMessage("[{}] failed to apply mappings", index()), e); throw e; } - if (updatedMapper == null) { + if (updatedSnapshot == null) { return false; } boolean requireRefresh = false; - assertMappingVersion(currentIndexMetadata, newIndexMetadata, updatedMapper); + assertMappingVersion(currentIndexMetadata, newIndexMetadata, updatedSnapshot); MappingMetadata mappingMetadata = newIndexMetadata.mapping(); CompressedXContent incomingMappingSource = mappingMetadata.source(); - String op = mapper != null ? "updated" : "added"; + String op = snapshot != null ? "updated" : "added"; if (logger.isDebugEnabled() && incomingMappingSource.compressed().length < 512) { logger.debug("[{}] {} mapping, source [{}]", index(), op, incomingMappingSource.string()); } else if (logger.isTraceEnabled()) { @@ -226,9 +242,10 @@ public boolean updateMapping(final IndexMetadata currentIndexMetadata, final Ind // refresh mapping can happen when the parsing/merging of the mapping from the metadata doesn't result in the same // mapping, in this case, we send to the master to refresh its own version of the mappings (to conform with the // merge version of it, which it does when refreshing the mappings), and warn log it. - if (documentMapper().mappingSource().equals(incomingMappingSource) == false) { + DocumentMapper documentMapper = snapshot.documentMapper(); + if (documentMapper.mappingSource().equals(incomingMappingSource) == false) { logger.debug("[{}] parsed mapping, and got different sources\noriginal:\n{}\nparsed:\n{}", - index(), incomingMappingSource, documentMapper().mappingSource()); + index(), incomingMappingSource, documentMapper.mappingSource()); requireRefresh = true; } @@ -238,11 +255,11 @@ public boolean updateMapping(final IndexMetadata currentIndexMetadata, final Ind private void assertMappingVersion( final IndexMetadata currentIndexMetadata, final IndexMetadata newIndexMetadata, - final DocumentMapper updatedMapper) throws IOException { + final Snapshot updatedSnapshot) throws IOException { if (Assertions.ENABLED && currentIndexMetadata != null) { if (currentIndexMetadata.getMappingVersion() == newIndexMetadata.getMappingVersion()) { // if the mapping version is unchanged, then there should not be any updates and all mappings should be the same - assert updatedMapper == mapper; + assert updatedSnapshot == snapshot; MappingMetadata mapping = newIndexMetadata.mapping(); if (mapping != null) { @@ -251,7 +268,7 @@ private void assertMappingVersion( assert currentSource.equals(newSource) : "expected current mapping [" + currentSource + "] for type [" + mapping.type() + "] " + "to be the same as new mapping [" + newSource + "]"; - final CompressedXContent mapperSource = new CompressedXContent(Strings.toString(mapper)); + final CompressedXContent mapperSource = new CompressedXContent(Strings.toString(snapshot.documentMapper())); assert currentSource.equals(mapperSource) : "expected current mapping [" + currentSource + "] for type [" + mapping.type() + "] " + "to be the same as new mapping [" + mapperSource + "]"; @@ -264,11 +281,11 @@ private void assertMappingVersion( assert currentMappingVersion < newMappingVersion : "expected current mapping version [" + currentMappingVersion + "] " + "to be less than new mapping version [" + newMappingVersion + "]"; - assert updatedMapper != null; + assert updatedSnapshot != null; final MappingMetadata currentMapping = currentIndexMetadata.mapping(); if (currentMapping != null) { final CompressedXContent currentSource = currentMapping.source(); - final CompressedXContent newSource = updatedMapper.mappingSource(); + final CompressedXContent newSource = updatedSnapshot.documentMapper().mappingSource(); assert currentSource.equals(newSource) == false : "expected current mapping [" + currentSource + "] to be different than new mapping [" + newSource + "]"; } @@ -286,10 +303,11 @@ public void merge(IndexMetadata indexMetadata, MergeReason reason) { } public DocumentMapper merge(String type, CompressedXContent mappingSource, MergeReason reason) { - return internalMerge(type, mappingSource, reason); + Snapshot updatedSnapshot = internalMerge(type, mappingSource, reason); + return updatedSnapshot == null ? null : updatedSnapshot.documentMapper(); } - private synchronized DocumentMapper internalMerge(IndexMetadata indexMetadata, MergeReason reason) { + private synchronized Snapshot internalMerge(IndexMetadata indexMetadata, MergeReason reason) { assert reason != MergeReason.MAPPING_UPDATE_PREFLIGHT; MappingMetadata mappingMetadata = indexMetadata.mapping(); if (mappingMetadata != null) { @@ -298,7 +316,7 @@ private synchronized DocumentMapper internalMerge(IndexMetadata indexMetadata, M return null; } - private synchronized DocumentMapper internalMerge(String type, CompressedXContent mappings, MergeReason reason) { + private synchronized Snapshot internalMerge(String type, CompressedXContent mappings, MergeReason reason) { DocumentMapper documentMapper; @@ -311,36 +329,30 @@ private synchronized DocumentMapper internalMerge(String type, CompressedXConten return internalMerge(documentMapper, reason); } - private synchronized DocumentMapper internalMerge(DocumentMapper mapper, MergeReason reason) { + private synchronized Snapshot internalMerge(DocumentMapper mapper, MergeReason reason) { assert mapper != null; // compute the merged DocumentMapper - DocumentMapper oldMapper = this.mapper; - DocumentMapper newMapper; - if (oldMapper != null) { - newMapper = oldMapper.merge(mapper.mapping(), reason); - } else { - newMapper = mapper; - } - newMapper.root().fixRedundantIncludes(); - newMapper.validate(indexSettings, reason != MergeReason.MAPPING_RECOVERY); + MappedSnapshot newSnapshot = this.snapshot.merge(mapper, reason); + newSnapshot.documentMapper().root().fixRedundantIncludes(); + newSnapshot.documentMapper().validate(indexSettings, reason != MergeReason.MAPPING_RECOVERY); if (reason == MergeReason.MAPPING_UPDATE_PREFLIGHT) { - return newMapper; + return newSnapshot; } // commit the change - this.mapper = newMapper; - assert assertSerialization(newMapper); + this.snapshot = newSnapshot; + assert assertSerialization(newSnapshot); - return newMapper; + return newSnapshot; } - private boolean assertSerialization(DocumentMapper mapper) { + private boolean assertSerialization(MappedSnapshot snapshot) { // capture the source now, it may change due to concurrent parsing - final CompressedXContent mappingSource = mapper.mappingSource(); - DocumentMapper newMapper = parse(mapper.type(), mappingSource); + final CompressedXContent mappingSource = snapshot.mapper.mappingSource(); + DocumentMapper newMapper = parse(snapshot.mapper.type(), mappingSource); if (newMapper.mappingSource().equals(mappingSource) == false) { throw new IllegalStateException("DocumentMapper serialization result is different from source. \n--> Source [" @@ -354,11 +366,9 @@ public DocumentMapper parse(String mappingType, CompressedXContent mappingSource return documentMapperParser.parse(mappingType, mappingSource); } - /** - * Return the document mapper, or {@code null} if no mapping has been put yet. - */ + @Deprecated public DocumentMapper documentMapper() { - return mapper; + return snapshot.documentMapper(); } /** @@ -379,8 +389,9 @@ public static boolean isMappingSourceTyped(String type, Map mapp */ private String resolveDocumentType(String type) { if (MapperService.SINGLE_MAPPING_NAME.equals(type)) { - if (mapper != null) { - return mapper.type(); + Snapshot currentSnapshot = snapshot; + if (currentSnapshot.documentMapper() != null) { + return currentSnapshot.documentMapper().type(); } } return type; @@ -389,64 +400,60 @@ private String resolveDocumentType(String type) { /** * Returns the document mapper for this MapperService. If no mapper exists, * creates one and returns that. + * @deprecated Get a {@link #snapshot} and call {@link Snapshot#documentMapperWithAutoCreate} on + * it as many times as needed. */ + @Deprecated public DocumentMapperForType documentMapperWithAutoCreate() { - DocumentMapper mapper = documentMapper(); - if (mapper != null) { - return new DocumentMapperForType(mapper, null); - } - mapper = parse(SINGLE_MAPPING_NAME, null); - return new DocumentMapperForType(mapper, mapper.mapping()); + return snapshot.documentMapperWithAutoCreate(); } /** * Given the full name of a field, returns its {@link MappedFieldType}. + * @deprecated Get a {@link #snapshot} and call {@link Snapshot#fieldType} on + * it as many times as needed. */ + @Deprecated public MappedFieldType fieldType(String fullName) { - if (fullName.equals(TypeFieldType.NAME)) { - return new TypeFieldType(this.mapper == null ? "_doc" : this.mapper.type()); - } - return this.mapper == null ? null : this.mapper.mappers().fieldTypes().get(fullName); + return snapshot.fieldType(fullName); } /** * Returns all the fields that match the given pattern. If the pattern is prefixed with a type * then the fields will be returned with a type prefix. + * @deprecated Get a {@link #snapshot} and call {@link Snapshot#simpleMatchToFullName} on + * it as many times as needed. */ + @Deprecated public Set simpleMatchToFullName(String pattern) { - if (Regex.isSimpleMatchPattern(pattern) == false) { - // no wildcards - return Collections.singleton(pattern); - } - return this.mapper == null ? Collections.emptySet() : this.mapper.mappers().fieldTypes().simpleMatchToFullName(pattern); + return snapshot.simpleMatchToFullName(pattern); } /** - * Given a field name, returns its possible paths in the _source. For example, - * the 'source path' for a multi-field is the path to its parent field. + * All mapped field types with eager global ordinals. + * @deprecated Get a {@link #snapshot} and call {@link Snapshot#getEagerGlobalOrdinalsFields} on + * it as many times as needed. */ - public Set sourcePath(String fullName) { - return this.mapper == null ? Collections.emptySet() : this.mapper.mappers().fieldTypes().sourcePaths(fullName); + @Deprecated + public Iterable getEagerGlobalOrdinalsFields() { + return snapshot.getEagerGlobalOrdinalsFields(); } /** - * Returns all mapped field types. + * Get the named object mapper. + * @deprecated Get a {@link #snapshot} and call {@link Snapshot#getObjectMapper} on + * it as many times as needed. */ - public Iterable getEagerGlobalOrdinalsFields() { - return this.mapper == null ? Collections.emptySet() : - this.mapper.mappers().fieldTypes().filter(MappedFieldType::eagerGlobalOrdinals); - } - + @Deprecated public ObjectMapper getObjectMapper(String name) { - return this.mapper == null ? null : this.mapper.mappers().objectMappers().get(name); + return snapshot.getObjectMapper(name); } + /** + * An analyzer that performs a volatile read on the mapping find correct {@link FieldNameAnalyzer}. + */ public Analyzer indexAnalyzer() { - return this.indexAnalyzer; - } - - public boolean containsBrokenAnalysis(String field) { - return this.indexAnalyzer.containsBrokenAnalysis(field); + return indexAnalyzer; } @Override @@ -478,23 +485,6 @@ public boolean isMetadataField(String field) { return mapperRegistry.isMetadataField(indexVersionCreated, field); } - /** An analyzer wrapper that can lookup fields within the index mappings */ - final class MapperAnalyzerWrapper extends DelegatingAnalyzerWrapper { - - MapperAnalyzerWrapper() { - super(Analyzer.PER_FIELD_REUSE_STRATEGY); - } - - @Override - protected Analyzer getWrappedAnalyzer(String fieldName) { - return mapper.mappers().indexAnalyzer(); - } - - boolean containsBrokenAnalysis(String field) { - return mapper.mappers().indexAnalyzer().containsBrokenAnalysis(field); - } - } - public synchronized List reloadSearchAnalyzers(AnalysisRegistry registry) throws IOException { logger.info("reloading search analyzers"); // refresh indexAnalyzers and search analyzers @@ -514,4 +504,400 @@ public synchronized List reloadSearchAnalyzers(AnalysisRegistry registry } return reloadedAnalyzers; } + + /** + * An immutable snapshot of the current mapping. + */ + public interface Snapshot { + /** + * Given the full name of a field, returns its {@link MappedFieldType}. + */ + MappedFieldType fieldType(String fullName); + + boolean hasNested(); + + /** + * Get the named object mapper + */ + ObjectMapper getObjectMapper(String name); + + /** + * Returns all the fields that match the given pattern. If the pattern is prefixed with a type + * then the fields will be returned with a type prefix. + */ + Set simpleMatchToFullName(String pattern); + + /** + * Given a field name, returns its possible paths in the _source. For example, + * the 'source path' for a multi-field is the path to its parent field. + */ + Set sourcePath(String fullName); + + /** + * The document mapper for this MapperService. If no mapper exists, + * creates one and returns that. + */ + DocumentMapperForType documentMapperWithAutoCreate(); + + /** + * All mapped field types with eager global ordinals. + */ + Iterable getEagerGlobalOrdinalsFields(); + + FieldNameAnalyzer indexAnalyzer(); + + /** + * Does the index analyzer for this field have token filters that may produce + * backwards offsets in term vectors + */ + boolean containsBrokenAnalysis(String field); + + /** + * Get the actual mapping. + * @deprecated Prefer any other method. {@link DocumentMapper} doesn't support + * runtime fields and is otherwise tightly coupled to the internals + * of mappings. + */ + @Deprecated + DocumentMapper documentMapper(); + + /** + * Current version of the of the mapping. Increments if the mapping + * changes locally. Distinct from + * {@link IndexMetadata#getMappingVersion()} because it purely + * considers the local mapping changes. + */ + long version(); + + ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException; + + /** + * The context used to parse field. + */ + ParserContext parserContext(); + + /** + * @return Whether a field is a metadata field. + * this method considers all mapper plugins + */ + boolean isMetadataField(String field); + + IndexAnalyzers getIndexAnalyzers(); + } + + private abstract static class AbstractSnapshot implements Snapshot { + protected final MapperService mapperService; + + private AbstractSnapshot(MapperService mapperService) { + this.mapperService = requireNonNull(mapperService); + } + + /** + * The context used to parse field. + */ + public final ParserContext parserContext() { + // Safe to plumb through to the Snapshot because it is immutable + return mapperService.parserContext(); + } + + /** + * @return Whether a field is a metadata field. + * this method considers all mapper plugins + */ + public final boolean isMetadataField(String field) { + // Safe to plumb through to the Snapshot because it is immutable + return mapperService.isMetadataField(field); + } + + public final IndexAnalyzers getIndexAnalyzers() { + // Safe to plumb through to the Snapshot because it is immutable + return mapperService.getIndexAnalyzers(); + } + + abstract MappedSnapshot merge(DocumentMapper mapper, MergeReason reason); + + abstract void collectMetadataMappers(Map, MetadataFieldMapper> metadataMappers); + } + + private static class EmptySnapshot extends AbstractSnapshot { + EmptySnapshot(MapperService mapperService) { + super(mapperService); + } + + @Override + public MappedFieldType fieldType(String fullName) { + if (fullName.equals(TypeFieldType.NAME)) { + return new TypeFieldType("_doc"); + } + return null; + } + + @Override + public boolean hasNested() { + return false; + } + + @Override + public ObjectMapper getObjectMapper(String name) { + return null; + } + + @Override + public Set simpleMatchToFullName(String pattern) { + return Regex.isSimpleMatchPattern(pattern) ? emptySet() : singleton(pattern); + } + + @Override + public Set sourcePath(String fullName) { + return emptySet(); + } + + @Override + public DocumentMapperForType documentMapperWithAutoCreate() { + DocumentMapper auto = mapperService.parse(SINGLE_MAPPING_NAME, null); + return new DocumentMapperForType(auto, auto.mapping()); + } + + @Override + public Iterable getEagerGlobalOrdinalsFields() { + return emptySet(); + } + + @Override + public FieldNameAnalyzer indexAnalyzer() { + return null; + } + + @Override + public boolean containsBrokenAnalysis(String field) { + return false; + } + + @Override + public DocumentMapper documentMapper() { + return null; + } + + @Override + public long version() { + return 0; + } + + @Override + public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { + return null; + } + + @Override + void collectMetadataMappers(Map, MetadataFieldMapper> metadataMappers) { + Map metadataMapperParsers = mapperService.mapperRegistry.getMetadataMapperParsers( + mapperService.indexSettings.getIndexVersionCreated() + ); + for (MetadataFieldMapper.TypeParser parser : metadataMapperParsers.values()) { + MetadataFieldMapper metadataFieldMapper = parser.getDefault(parserContext()); + metadataMappers.put(metadataFieldMapper.getClass(), metadataFieldMapper); + } + } + + @Override + MappedSnapshot merge(DocumentMapper mapper, MergeReason reason) { + return new MappedSnapshot(mapperService, mapper, 1); + } + } + + static class MappedSnapshot extends AbstractSnapshot { + private final DocumentMapper mapper; + /** + * Current version of the of the mapping. Increments if the mapping + * changes locally. Distinct from + * {@link IndexMetadata#getMappingVersion()} because it purely + * considers the local mapping changes. + */ + private final long version; + + MappedSnapshot(MapperService mapperService, DocumentMapper mapper, long version) { + super(mapperService); + this.mapper = requireNonNull(mapper); + this.version = version; + } + + @Override + public MappedFieldType fieldType(String fullName) { + if (fullName.equals(TypeFieldType.NAME)) { + return new TypeFieldType(mapper.type()); + } + return mapper.mappers().fieldTypes().get(fullName); + } + + @Override + public DocumentMapper documentMapper() { + return mapper; + } + + @Override + public boolean hasNested() { + return mapper.hasNestedObjects(); + } + + @Override + public ObjectMapper getObjectMapper(String name) { + return mapper.mappers().objectMappers().get(name); + } + + @Override + public Set simpleMatchToFullName(String pattern) { + return Regex.isSimpleMatchPattern(pattern) ? mapper.mappers().fieldTypes().simpleMatchToFullName(pattern) : singleton(pattern); + } + + @Override + public Set sourcePath(String fullName) { + return mapper.mappers().fieldTypes().sourcePaths(fullName); + } + + @Override + public DocumentMapperForType documentMapperWithAutoCreate() { + return new DocumentMapperForType(mapper, null); + } + + @Override + public Iterable getEagerGlobalOrdinalsFields() { + return mapper.mappers().fieldTypes().filter(MappedFieldType::eagerGlobalOrdinals); + } + + @Override + public FieldNameAnalyzer indexAnalyzer() { + return mapper.mappers().indexAnalyzer(); + } + + @Override + public boolean containsBrokenAnalysis(String field) { + return mapper.mappers().indexAnalyzer().containsBrokenAnalysis(field); + } + + @Override + public long version() { + return version; + } + + @Override + public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { + return mapper.parse(source); + } + + @Override + protected void collectMetadataMappers(Map, MetadataFieldMapper> metadataMappers) { + metadataMappers.putAll(mapper.mapping().metadataMappersMap); + } + + @Override + MappedSnapshot merge(DocumentMapper newMapper, MergeReason reason) { + DocumentMapper merged = mapper.merge(newMapper.mapping(), reason); + if (merged.mappingSource().equals(mapper.mappingSource())) { + return this; + } + return new MappedSnapshot(mapperService, merged, version + 1); + } + } + + /** + * A mapping snapshot with the "central" methods that are useful for testing. + */ + public static class StubSnapshot implements Snapshot { + private final Function lookup; + private final Supplier> fields; + + public StubSnapshot(Function lookup) { + this.lookup = lookup; + this.fields = () -> { + throw new UnsupportedOperationException(); + }; + } + + public StubSnapshot(Map lookup) { + this.lookup = lookup::get; + this.fields = lookup::keySet; + } + + @Override + public MappedFieldType fieldType(String fullName) { + return lookup.apply(fullName); + } + + @Override + public Set simpleMatchToFullName(String pattern) { + if (Regex.isSimpleMatchPattern(pattern) == false) { + return singleton(pattern); + } + if (Regex.isMatchAllPattern(pattern)) { + return unmodifiableSet(fields.get()); + } + throw new UnsupportedOperationException(); + } + + @Override + public boolean hasNested() { + return false; + } + + @Override + public ObjectMapper getObjectMapper(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public Set sourcePath(String fullName) { + throw new UnsupportedOperationException(); + } + + @Override + public DocumentMapperForType documentMapperWithAutoCreate() { + throw new UnsupportedOperationException(); + } + + @Override + public Iterable getEagerGlobalOrdinalsFields() { + throw new UnsupportedOperationException(); + } + + @Override + public FieldNameAnalyzer indexAnalyzer() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean containsBrokenAnalysis(String field) { + throw new UnsupportedOperationException(); + } + + @Override + public DocumentMapper documentMapper() { + throw new UnsupportedOperationException(); + } + + @Override + public long version() { + throw new UnsupportedOperationException(); + } + + @Override + public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { + throw new UnsupportedOperationException(); + } + + @Override + public ParserContext parserContext() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isMetadataField(String field) { + throw new UnsupportedOperationException(); + } + + @Override + public IndexAnalyzers getIndexAnalyzers() { + throw new UnsupportedOperationException(); + } + + } } diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index f1cb793833b56..7a78e11b6d700 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -46,7 +46,6 @@ import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.mapper.ContentPath; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; @@ -63,6 +62,7 @@ import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptFactory; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.transport.RemoteClusterAware; @@ -88,7 +88,7 @@ public class QueryShardContext extends QueryRewriteContext { private final ScriptService scriptService; private final IndexSettings indexSettings; private final BigArrays bigArrays; - private final MapperService mapperService; + private final MapperService.Snapshot mapperSnapshot; private final SimilarityService similarityService; private final BitsetFilterCache bitsetFilterCache; private final TriFunction, IndexFieldData> indexFieldDataService; @@ -117,7 +117,7 @@ public QueryShardContext( BigArrays bigArrays, BitsetFilterCache bitsetFilterCache, TriFunction, IndexFieldData> indexFieldDataLookup, - MapperService mapperService, + MapperService.Snapshot mapperSnapshot, SimilarityService similarityService, ScriptService scriptService, NamedXContentRegistry xContentRegistry, @@ -137,7 +137,7 @@ public QueryShardContext( bigArrays, bitsetFilterCache, indexFieldDataLookup, - mapperService, + mapperSnapshot, similarityService, scriptService, xContentRegistry, @@ -152,13 +152,13 @@ public QueryShardContext( ), allowExpensiveQueries, valuesSourceRegistry, - parseRuntimeMappings(runtimeMappings, mapperService) + parseRuntimeMappings(runtimeMappings, mapperSnapshot) ); } public QueryShardContext(QueryShardContext source) { this(source.shardId, source.indexSettings, source.bigArrays, source.bitsetFilterCache, source.indexFieldDataService, - source.mapperService, source.similarityService, source.scriptService, source.getXContentRegistry(), + source.mapperSnapshot, source.similarityService, source.scriptService, source.getXContentRegistry(), source.getWriteableRegistry(), source.client, source.searcher, source.nowInMillis, source.indexNameMatcher, source.fullyQualifiedIndex, source.allowExpensiveQueries, source.valuesSourceRegistry, source.runtimeMappings); } @@ -168,7 +168,7 @@ private QueryShardContext(int shardId, BigArrays bigArrays, BitsetFilterCache bitsetFilterCache, TriFunction, IndexFieldData> indexFieldDataLookup, - MapperService mapperService, + MapperService.Snapshot mapperSnapshot, SimilarityService similarityService, ScriptService scriptService, NamedXContentRegistry xContentRegistry, @@ -184,7 +184,7 @@ private QueryShardContext(int shardId, super(xContentRegistry, namedWriteableRegistry, client, nowInMillis); this.shardId = shardId; this.similarityService = similarityService; - this.mapperService = mapperService; + this.mapperSnapshot = mapperSnapshot; this.bigArrays = bigArrays; this.bitsetFilterCache = bitsetFilterCache; this.indexFieldDataService = indexFieldDataLookup; @@ -208,7 +208,7 @@ private void reset() { } public Similarity getSearchSimilarity() { - return similarityService != null ? similarityService.similarity(mapperService::fieldType) : null; + return similarityService != null ? similarityService.similarity(mapperSnapshot::fieldType) : null; } public List defaultFields() { @@ -253,20 +253,19 @@ public Map copyNamedQueries() { } public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { - return mapperService.documentMapper() == null ? null : mapperService.documentMapper().parse(source); + return mapperSnapshot.parseDocument(source); } public FieldNameAnalyzer getFieldNameIndexAnalyzer() { - DocumentMapper documentMapper = mapperService.documentMapper(); - return documentMapper == null ? null : documentMapper.mappers().indexAnalyzer(); + return mapperSnapshot.indexAnalyzer(); } public boolean hasNested() { - return mapperService.hasNested(); + return mapperSnapshot.hasNested(); } public boolean hasMappings() { - return mapperService.documentMapper() != null; + return mapperSnapshot.documentMapper() != null; } /** @@ -275,13 +274,13 @@ public boolean hasMappings() { */ public Set simpleMatchToIndexNames(String pattern) { if (runtimeMappings.isEmpty()) { - return mapperService.simpleMatchToFullName(pattern); + return mapperSnapshot.simpleMatchToFullName(pattern); } if (Regex.isSimpleMatchPattern(pattern) == false) { // no wildcards return Collections.singleton(pattern); } - Set matches = new HashSet<>(mapperService.simpleMatchToFullName(pattern)); + Set matches = new HashSet<>(mapperSnapshot.simpleMatchToFullName(pattern)); for (String name : runtimeMappings.keySet()) { if (Regex.simpleMatch(pattern, name)) { matches.add(name); @@ -312,23 +311,23 @@ public boolean isFieldMapped(String name) { private MappedFieldType fieldType(String name) { MappedFieldType fieldType = runtimeMappings.get(name); - return fieldType == null ? mapperService.fieldType(name) : fieldType; + return fieldType == null ? mapperSnapshot.fieldType(name) : fieldType; } public ObjectMapper getObjectMapper(String name) { - return mapperService.getObjectMapper(name); + return mapperSnapshot.getObjectMapper(name); } public boolean isMetadataField(String field) { - return mapperService.isMetadataField(field); + return mapperSnapshot.isMetadataField(field); } public Set sourcePath(String fullName) { - return mapperService.sourcePath(fullName); + return mapperSnapshot.sourcePath(fullName); } public boolean isSourceEnabled() { - return mapperService.documentMapper().sourceMapper().enabled(); + return mapperSnapshot.documentMapper().sourceMapper().enabled(); } /** @@ -336,7 +335,7 @@ public boolean isSourceEnabled() { * Generally used to handle unmapped fields in the context of sorting. */ public MappedFieldType buildAnonymousFieldType(String type) { - Mapper.TypeParser.ParserContext parserContext = mapperService.parserContext(); + Mapper.TypeParser.ParserContext parserContext = mapperSnapshot.parserContext(); Mapper.TypeParser typeParser = parserContext.typeParser(type); if (typeParser == null) { throw new IllegalArgumentException("No mapper found for type [" + type + "]"); @@ -350,11 +349,11 @@ public MappedFieldType buildAnonymousFieldType(String type) { } public IndexAnalyzers getIndexAnalyzers() { - return mapperService.getIndexAnalyzers(); + return mapperSnapshot.getIndexAnalyzers(); } public Analyzer getIndexAnalyzer() { - return mapperService.indexAnalyzer(); + return mapperSnapshot.indexAnalyzer(); } public ValuesSourceRegistry getValuesSourceRegistry() { @@ -374,7 +373,7 @@ MappedFieldType failIfFieldMappingNotFound(String name, MappedFieldType fieldMap return fieldMapping; } else if (mapUnmappedFieldAsString) { TextFieldMapper.Builder builder - = new TextFieldMapper.Builder(name, () -> mapperService.getIndexAnalyzers().getDefaultIndexAnalyzer()); + = new TextFieldMapper.Builder(name, () -> mapperSnapshot.getIndexAnalyzers().getDefaultIndexAnalyzer()); return builder.build(new ContentPath(1)).fieldType(); } else { throw new QueryShardException(this, "No field mapping can be found for the field with name [{}]", name); @@ -386,7 +385,7 @@ MappedFieldType failIfFieldMappingNotFound(String name, MappedFieldType fieldMap * backwards offsets in term vectors */ public boolean containsBrokenAnalysis(String field) { - return mapperService.containsBrokenAnalysis(field); + return mapperSnapshot.containsBrokenAnalysis(field); } private SearchLookup lookup = null; @@ -585,13 +584,21 @@ public BigArrays bigArrays() { private static Map parseRuntimeMappings( Map runtimeMappings, - MapperService mapperService + MapperService.Snapshot mapperSnapshot ) { Map runtimeFieldTypes = new HashMap<>(); if (runtimeMappings.isEmpty() == false) { - RuntimeFieldType.parseRuntimeFields(new HashMap<>(runtimeMappings), mapperService.parserContext(), + RuntimeFieldType.parseRuntimeFields(new HashMap<>(runtimeMappings), mapperSnapshot.parserContext(), runtimeFieldType -> runtimeFieldTypes.put(runtimeFieldType.name(), runtimeFieldType)); } return Collections.unmodifiableMap(runtimeFieldTypes); } + + public NestedDocuments getNestedDocuments() { + return new NestedDocuments(mapperSnapshot.documentMapper(), bitsetFilterCache::getBitSetProducer); + } + + public long localMappingVersion() { + return mapperSnapshot.version(); + } } diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index 0c422da7e9ae5..bc83030665340 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -1384,7 +1384,8 @@ public void loadIntoContext(ShardSearchRequest request, SearchContext context, Q final DirectoryReader directoryReader = context.searcher().getDirectoryReader(); boolean[] loadedFromCache = new boolean[] { true }; - BytesReference bytesReference = cacheShardLevelResult(context.indexShard(), directoryReader, request.cacheKey(), + BytesReference cacheKey = request.cacheKey(context.getQueryShardContext().localMappingVersion()); + BytesReference bytesReference = cacheShardLevelResult(context.indexShard(), directoryReader, cacheKey, out -> { queryPhase.execute(context); context.queryResult().writeToNoId(out); @@ -1405,7 +1406,7 @@ public void loadIntoContext(ShardSearchRequest request, SearchContext context, Q // key invalidate the result in the thread that caused the timeout. This will end up to be simpler and eventually correct since // running a search that times out concurrently will likely timeout again if it's run while we have this `stale` result in the // cache. One other option is to not cache requests with a timeout at all... - indicesRequestCache.invalidate(new IndexShardCacheEntity(context.indexShard()), directoryReader, request.cacheKey()); + indicesRequestCache.invalidate(new IndexShardCacheEntity(context.indexShard()), directoryReader, cacheKey); if (logger.isTraceEnabled()) { logger.trace("Query timed out, invalidating cache entry for request on shard [{}]:\n {}", request.shardId(), request.source()); diff --git a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java index 56cfd7c6ec1c3..42dd27143ebd2 100644 --- a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java @@ -700,11 +700,6 @@ public QuerySearchResult queryResult() { return queryResult; } - @Override - public NestedDocuments getNestedDocuments() { - return new NestedDocuments(indexService.mapperService(), bitsetFilterCache()::getBitSetProducer); - } - @Override public FetchPhase fetchPhase() { return fetchPhase; diff --git a/server/src/main/java/org/elasticsearch/search/NestedDocuments.java b/server/src/main/java/org/elasticsearch/search/NestedDocuments.java index f3094351583f1..0f150571dbb55 100644 --- a/server/src/main/java/org/elasticsearch/search/NestedDocuments.java +++ b/server/src/main/java/org/elasticsearch/search/NestedDocuments.java @@ -30,7 +30,7 @@ import org.apache.lucene.search.join.BitSetProducer; import org.apache.lucene.util.BitSet; import org.elasticsearch.common.lucene.search.Queries; -import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.ObjectMapper; import java.io.IOException; @@ -47,24 +47,24 @@ public class NestedDocuments { private final Map childObjectFilters = new HashMap<>(); private final Map childObjectMappers = new HashMap<>(); private final BitSetProducer parentDocumentFilter; - private final MapperService mapperService; + private final DocumentMapper documentMapper; /** * Create a new NestedDocuments object for an index - * @param mapperService the index's MapperService + * @param documentMapper a snapshot of the index's mapping * @param filterProducer a function to build BitSetProducers from filter queries */ - public NestedDocuments(MapperService mapperService, Function filterProducer) { - this.mapperService = mapperService; - if (mapperService.hasNested() == false) { + public NestedDocuments(DocumentMapper documentMapper, Function filterProducer) { + this.documentMapper = documentMapper; + if (documentMapper == null || documentMapper.hasNestedObjects() == false) { this.parentDocumentFilter = null; } else { this.parentDocumentFilter = filterProducer.apply(Queries.newNonNestedFilter()); - for (ObjectMapper mapper : mapperService.documentMapper().getNestedParentMappers()) { + for (ObjectMapper mapper : documentMapper.getNestedParentMappers()) { parentObjectFilters.put(mapper.name(), filterProducer.apply(mapper.nestedTypeFilter())); } - for (ObjectMapper mapper : mapperService.documentMapper().getNestedMappers()) { + for (ObjectMapper mapper : documentMapper.getNestedMappers()) { childObjectFilters.put(mapper.name(), null); childObjectMappers.put(mapper.name(), mapper); } @@ -98,7 +98,7 @@ private Weight getNestedChildWeight(LeafReaderContext ctx, String path) throws I * Given an object path, returns whether or not any of its parents are plain objects */ public boolean hasNonNestedParent(String path) { - return mapperService.documentMapper().hasNonNestedParent(path); + return documentMapper.hasNonNestedParent(path); } private class HasNestedDocuments implements LeafNestedDocuments { @@ -185,7 +185,7 @@ private SearchHit.NestedIdentity loadNestedIdentity() throws IOException { int parentNameLength; String path = findObjectPath(doc); while (path != null) { - String parent = mapperService.documentMapper().getNestedParent(path); + String parent = documentMapper.getNestedParent(path); // We have to pull a new scorer for each document here, because we advance from // the last parent which will be behind the doc Scorer childScorer = getNestedChildWeight(ctx, path).scorer(ctx); diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index d95b691c3c103..7a83782e16015 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -112,7 +112,7 @@ public void execute(SearchContext context) { SearchHit[] hits = new SearchHit[context.docIdsToLoadSize()]; List processors = getProcessors(context.shardTarget(), fetchContext); - NestedDocuments nestedDocuments = context.getNestedDocuments(); + NestedDocuments nestedDocuments = context.getQueryShardContext().getNestedDocuments(); int currentReaderIndex = -1; LeafReaderContext currentReaderContext = null; diff --git a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index f3c6bcebc9255..b4d731a00f49f 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -30,7 +30,6 @@ import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.SearchExtBuilder; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.SearchContextAggregations; @@ -396,11 +395,6 @@ public FetchSearchResult fetchResult() { return in.fetchResult(); } - @Override - public NestedDocuments getNestedDocuments() { - return in.getNestedDocuments(); - } - @Override public FetchPhase fetchPhase() { return in.fetchPhase(); diff --git a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java index 6d85f0fd69894..8645e2d6b2514 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -33,7 +33,6 @@ import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.RescoreDocIds; import org.elasticsearch.search.SearchExtBuilder; import org.elasticsearch.search.SearchShardTarget; @@ -314,8 +313,6 @@ public final void assignRescoreDocIds(RescoreDocIds rescoreDocIds) { public abstract QuerySearchResult queryResult(); - public abstract NestedDocuments getNestedDocuments(); - public abstract FetchPhase fetchPhase(); public abstract FetchSearchResult fetchResult(); diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index af2e46a65dffb..9d2e6d8785d2e 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -396,9 +396,10 @@ public TimeValue keepAlive() { /** * Returns the cache key for this shard search request, based on its content */ - public BytesReference cacheKey() throws IOException { + public BytesReference cacheKey(long localMappingVersion) throws IOException { BytesStreamOutput out = scratch.get(); try { + out.writeLong(localMappingVersion); this.innerWriteTo(out, true); // copy it over since we don't want to share the thread-local bytes in #scratch return out.copyBytes(); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java index 7e93b0e79c63c..2296ff8307d91 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java @@ -29,11 +29,9 @@ import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.test.ESTestCase; -import java.util.Collections; +import java.util.Map; import static java.util.Collections.emptyMap; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class FieldNamesFieldTypeTests extends ESTestCase { @@ -45,10 +43,9 @@ public void testTermQuery() { Settings settings = settings(Version.CURRENT).build(); IndexSettings indexSettings = new IndexSettings( new IndexMetadata.Builder("foo").settings(settings).numberOfShards(1).numberOfReplicas(0).build(), settings); - MapperService mapperService = mock(MapperService.class); - when(mapperService.fieldType("_field_names")).thenReturn(fieldNamesFieldType); - when(mapperService.fieldType("field_name")).thenReturn(fieldType); - when(mapperService.simpleMatchToFullName("field_name")).thenReturn(Collections.singleton("field_name")); + MapperService.Snapshot mapperService = new MapperService.StubSnapshot( + Map.of("_field_names", fieldNamesFieldType, "field_name", fieldType) + ); QueryShardContext queryShardContext = new QueryShardContext(0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService, 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 3d76675038a4f..bacb980ea29e5 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java @@ -29,6 +29,7 @@ import java.io.IOException; import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @@ -37,9 +38,20 @@ public class MapperServiceTests extends MapperServiceTestCase { public void testPreflightUpdateDoesNotChangeMapping() throws Throwable { final MapperService mapperService = createMapperService(mapping(b -> {})); merge(mapperService, MergeReason.MAPPING_UPDATE_PREFLIGHT, mapping(b -> createMappingSpecifyingNumberOfFields(b, 1))); - assertThat("field was not created by preflight check", mapperService.fieldType("field0"), nullValue()); + assertThat("field was not created by preflight check", mapperService.snapshot().fieldType("field0"), nullValue()); merge(mapperService, MergeReason.MAPPING_UPDATE, mapping(b -> createMappingSpecifyingNumberOfFields(b, 1))); - assertThat("field was not created by mapping update", mapperService.fieldType("field0"), notNullValue()); + assertThat("field was not created by mapping update", mapperService.snapshot().fieldType("field0"), notNullValue()); + } + + public void testSnapshot() throws IOException { + MapperService service = createMapperService(mapping(b -> {})); + MapperService.Snapshot oldSnapshot = service.snapshot(); + assertThat(oldSnapshot.fieldType("cat"), nullValue()); + + merge(service, mapping(b -> b.startObject("cat").field("type", "keyword").endObject())); + MapperService.Snapshot newSnapshot = service.snapshot(); + assertThat(newSnapshot.fieldType("cat"), not(nullValue())); + assertThat(oldSnapshot.fieldType("cat"), nullValue()); } /** diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java index 16280eea7ce9a..c930dafce3d2a 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java @@ -56,7 +56,7 @@ public void testSimpleNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService, QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService.snapshot().documentMapper(), QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -153,7 +153,7 @@ public void testMultiLevelNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService, QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService.snapshot().documentMapper(), QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -264,7 +264,7 @@ public void testNestedObjectWithinNonNestedObject() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService, QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService.snapshot().documentMapper(), QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); diff --git a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java index 1a0b7f339997f..af4a83d3b614d 100644 --- a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java @@ -39,7 +39,6 @@ import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -56,12 +55,13 @@ import org.elasticsearch.index.mapper.IndexFieldMapper; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.Mapper; +import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MockFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.mapper.TestRuntimeField; import org.elasticsearch.index.mapper.TextFieldMapper; -import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.plugins.MapperPlugin; @@ -86,6 +86,7 @@ import java.util.function.Supplier; import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonMap; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.notNullValue; @@ -371,18 +372,6 @@ private static QueryShardContext createQueryShardContext( Map runtimeMappings, List mapperPlugins ) { - MapperService mapperService = createMapperService(indexUuid, fieldTypeLookup, mapperPlugins); - final long nowInMillis = randomNonNegativeLong(); - return new QueryShardContext( - 0, mapperService.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE, null, - (mappedFieldType, idxName, searchLookup) -> mappedFieldType.fielddataBuilder(idxName, searchLookup).build(null, null), - mapperService, null, null, NamedXContentRegistry.EMPTY, new NamedWriteableRegistry(Collections.emptyList()), - null, null, () -> nowInMillis, clusterAlias, null, () -> true, null, runtimeMappings); - } - - private static MapperService createMapperService(String indexUuid, - Map fieldTypeLookup, - List mapperPlugins) { IndexMetadata.Builder indexMetadataBuilder = new IndexMetadata.Builder("index"); indexMetadataBuilder.settings(Settings.builder().put("index.version.created", Version.CURRENT) .put("index.number_of_shards", 1) @@ -390,29 +379,50 @@ private static MapperService createMapperService(String indexUuid, .put(IndexMetadata.SETTING_INDEX_UUID, indexUuid) ); IndexMetadata indexMetadata = indexMetadataBuilder.build(); + IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); + MapperService.Snapshot mapperSnapshot = createMapperSnapshot(indexSettings, fieldTypeLookup, mapperPlugins); + final long nowInMillis = randomNonNegativeLong(); + return new QueryShardContext( + 0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, + (mappedFieldType, idxName, searchLookup) -> mappedFieldType.fielddataBuilder(idxName, searchLookup).build(null, null), + mapperSnapshot, null, null, NamedXContentRegistry.EMPTY, new NamedWriteableRegistry(Collections.emptyList()), + null, null, () -> nowInMillis, clusterAlias, null, () -> true, null, runtimeMappings); + } + + private static MapperService.Snapshot createMapperSnapshot( + IndexSettings indexSettings, + Map fieldTypeLookup, + List mapperPlugins + ) { IndexAnalyzers indexAnalyzers = new IndexAnalyzers( - Collections.singletonMap("default", new NamedAnalyzer("default", AnalyzerScope.INDEX, null)), - Collections.emptyMap(), Collections.emptyMap() + singletonMap("default", new NamedAnalyzer("default", AnalyzerScope.INDEX, null)), + emptyMap(), + emptyMap() ); - IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); IndicesModule indicesModule = new IndicesModule(mapperPlugins); MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); - SimilarityService similarityService = new SimilarityService(indexSettings, null, Collections.emptyMap()); - return new MapperService(indexSettings, indexAnalyzers, NamedXContentRegistry.EMPTY, similarityService, - mapperRegistry, () -> { - throw new UnsupportedOperationException(); - }, () -> true, null) { + Supplier queryShardContextSupplier = () -> { throw new UnsupportedOperationException(); }; + Mapper.TypeParser.ParserContext parseContext = new Mapper.TypeParser.ParserContext( + null, + mapperRegistry.getMapperParsers()::get, + mapperRegistry.getRuntimeFieldTypeParsers()::get, + indexSettings.getIndexVersionCreated(), + queryShardContextSupplier, + null, + null, + indexAnalyzers, + indexSettings, + () -> true + ); + return new MapperService.StubSnapshot(fieldTypeLookup) { @Override - public MappedFieldType fieldType(String name) { - return fieldTypeLookup.get(name); + public IndexAnalyzers getIndexAnalyzers() { + return indexAnalyzers; } @Override - public Set simpleMatchToFullName(String pattern) { - if (Regex.isMatchAllPattern(pattern)) { - return Collections.unmodifiableSet(fieldTypeLookup.keySet()); - } - throw new UnsupportedOperationException(); + public ParserContext parserContext() { + return parseContext; } }; } diff --git a/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java b/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java index 9ab1e80f09bdc..1b904cb93f3ef 100644 --- a/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java @@ -42,7 +42,7 @@ public void testRewriteMissingField() throws Exception { IndexService indexService = createIndex("test"); IndexReader reader = new MultiReader(); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE, - null, null, indexService.mapperService(), null, null, xContentRegistry(), writableRegistry(), + null, null, indexService.mapperService().snapshot(), null, null, xContentRegistry(), writableRegistry(), null, new IndexSearcher(reader), null, null, null, () -> true, null, emptyMap()); RangeQueryBuilder range = new RangeQueryBuilder("foo"); assertEquals(Relation.DISJOINT, range.getRelation(context)); @@ -60,7 +60,7 @@ public void testRewriteMissingReader() throws Exception { indexService.mapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, null, - indexService.mapperService(), null, null, xContentRegistry(), writableRegistry(), + indexService.mapperService().snapshot(), null, null, xContentRegistry(), writableRegistry(), null, null, null, null, null, () -> true, null, emptyMap()); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // can't make assumptions on a missing reader, so it must return INTERSECT @@ -80,7 +80,7 @@ public void testRewriteEmptyReader() throws Exception { new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); IndexReader reader = new MultiReader(); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE, - null, null, indexService.mapperService(), null, null, xContentRegistry(), writableRegistry(), + null, null, indexService.mapperService().snapshot(), null, null, xContentRegistry(), writableRegistry(), null, new IndexSearcher(reader), null, null, null, () -> true, null, emptyMap()); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // no values -> DISJOINT diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java index 090990c2f6092..87bd248939eca 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java @@ -34,8 +34,10 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.index.analysis.FieldNameAnalyzer; import org.elasticsearch.index.mapper.BinaryFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType; import org.elasticsearch.search.aggregations.AggregationBuilder; @@ -56,6 +58,7 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.sampler; import static org.elasticsearch.search.aggregations.AggregationBuilders.significantText; import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.when; public class SignificantTextAggregatorTests extends AggregatorTestCase { @@ -360,4 +363,13 @@ public void testSignificanceOnTextArrays() throws IOException { } } } + + @Override + protected MapperService.Snapshot mapperSnapshotMock() { + MapperService.Snapshot mapperSnapshot = super.mapperSnapshotMock(); + when(mapperSnapshot.indexAnalyzer()).thenReturn( + new FieldNameAnalyzer(Map.of("text", new StandardAnalyzer(), "typeTestFieldName", new StandardAnalyzer())) + ); + return mapperSnapshot; + } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MinAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MinAggregatorTests.java index 0886ad91015f1..0318a74d35bd2 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MinAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MinAggregatorTests.java @@ -50,6 +50,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.mapper.IpFieldMapper; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; @@ -165,10 +166,22 @@ protected ScriptService getMockScriptService() { } @Override - protected QueryShardContext queryShardContextMock(IndexSearcher searcher, MapperService mapperService, - IndexSettings indexSettings, CircuitBreakerService circuitBreakerService, - BigArrays bigArrays) { - this.queryShardContext = super.queryShardContextMock(searcher, mapperService, indexSettings, circuitBreakerService, bigArrays); + protected QueryShardContext queryShardContextMock( + IndexSearcher searcher, + MapperService.Snapshot mapperSnapshot, + IndexSettings indexSettings, + CircuitBreakerService circuitBreakerService, + BitsetFilterCache bitsetFilterCache, + BigArrays bigArrays + ) { + this.queryShardContext = super.queryShardContextMock( + searcher, + mapperSnapshot, + indexSettings, + circuitBreakerService, + bitsetFilterCache, + bigArrays + ); return queryShardContext; } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorTests.java index e14b8d502d2e6..128e9c57d6fe2 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.QueryShardContext; @@ -552,9 +553,10 @@ protected A createAggregator( */ @Override protected QueryShardContext queryShardContextMock(IndexSearcher searcher, - MapperService mapperService, + MapperService.Snapshot mapperSnapshot, IndexSettings indexSettings, CircuitBreakerService circuitBreakerService, + BitsetFilterCache bitsetFilterCache, BigArrays bigArrays) { MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, SCRIPTS, Collections.emptyMap()); Map engines = Collections.singletonMap(scriptEngine.getType(), scriptEngine); @@ -563,9 +565,9 @@ protected QueryShardContext queryShardContextMock(IndexSearcher searcher, 0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, - getIndexFieldDataLookup(mapperService, circuitBreakerService), - mapperService, + bitsetFilterCache, + getIndexFieldDataLookup(indexSettings.getIndex().getName(), circuitBreakerService), + mapperSnapshot, null, scriptService, xContentRegistry(), diff --git a/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java b/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java index 967489a23e984..255df703c93bc 100644 --- a/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java @@ -439,7 +439,25 @@ private static QueryShardContext createQueryShardContext(MapperService mapperSer .put(IndexMetadata.SETTING_INDEX_UUID, "uuid").build(); IndexMetadata indexMetadata = new IndexMetadata.Builder("index").settings(settings).build(); IndexSettings indexSettings = new IndexSettings(indexMetadata, settings); - return new QueryShardContext(0, indexSettings, null, null, null, mapperService, null, null, null, null, null, null, null, null, - null, null, null, emptyMap()); + return new QueryShardContext( + 0, + indexSettings, + null, + null, + null, + mapperService.snapshot(), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + emptyMap() + ); } } diff --git a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java index 812eca890c608..31fdf49a39ef0 100644 --- a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java @@ -159,7 +159,8 @@ private static void assertEquals(ShardSearchRequest orig, ShardSearchRequest cop assertEquals(orig.numberOfShards(), copy.numberOfShards()); assertArrayEquals(orig.indexRoutings(), copy.indexRoutings()); assertEquals(orig.preference(), copy.preference()); - assertEquals(orig.cacheKey(), copy.cacheKey()); + long localMappingVersion = randomLong(); + assertEquals(orig.cacheKey(localMappingVersion), copy.cacheKey(localMappingVersion)); assertNotSame(orig, copy); assertEquals(orig.getAliasFilter(), copy.getAliasFilter()); assertEquals(orig.indexBoost(), copy.indexBoost(), 0.0f); diff --git a/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java b/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java index f9892be86028a..2ba12cdec3232 100644 --- a/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java @@ -55,6 +55,7 @@ import java.io.IOException; import java.util.Collections; import java.util.HashMap; +import java.util.Map; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; @@ -163,10 +164,8 @@ public void testBuild() throws IOException { Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index(randomAlphaOfLengthBetween(1, 10), "_na_"), indexSettings); - MapperService mapperService = mock(MapperService.class); ScriptService scriptService = mock(ScriptService.class); MappedFieldType fieldType = mockFieldType(suggestionBuilder.field()); - when(mapperService.fieldType(any(String.class))).thenReturn(fieldType); IndexAnalyzers indexAnalyzers = new IndexAnalyzers( new HashMap<>() { @Override @@ -176,11 +175,15 @@ public NamedAnalyzer get(Object key) { }, Collections.emptyMap(), Collections.emptyMap()); - when(mapperService.getIndexAnalyzers()).thenReturn(indexAnalyzers); + MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot(fieldName -> fieldType) { + public IndexAnalyzers getIndexAnalyzers() { + return indexAnalyzers; + } + }; when(scriptService.compile(any(Script.class), any())).then(invocation -> new TestTemplateService.MockTemplateScript.Factory( ((Script) invocation.getArguments()[0]).getIdOrCode())); QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, - null, mapperService, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, + null, mapperSnapshot, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, System::currentTimeMillis, null, null, () -> true, null, emptyMap()); SuggestionContext suggestionContext = suggestionBuilder.build(mockShardContext); @@ -214,13 +217,10 @@ public void testBuildWithUnmappedField() { Settings indexSettings = builder.build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index(randomAlphaOfLengthBetween(1, 10), "_na_"), indexSettings); - MapperService mapperService = mock(MapperService.class); - ScriptService scriptService = mock(ScriptService.class); + MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot(Map.of()); - when(mapperService.getNamedAnalyzer(any(String.class))).then( - invocation -> new NamedAnalyzer((String) invocation.getArguments()[0], AnalyzerScope.INDEX, new SimpleAnalyzer())); QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, - null, mapperService, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, + null, mapperSnapshot, null, null, xContentRegistry(), namedWriteableRegistry, null, null, System::currentTimeMillis, null, null, () -> true, null, emptyMap()); if (randomBoolean()) { mockShardContext.setAllowUnmappedFields(randomBoolean()); diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index ddbaf9e72c5ab..172ae7a8f273f 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -100,7 +100,6 @@ import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.script.ScriptService; -import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.MultiBucketConsumerService.MultiBucketConsumer; @@ -177,12 +176,11 @@ protected Map getFieldAliases(MappedFieldType... fieldT return Collections.emptyMap(); } - private static void registerFieldTypes(SearchContext searchContext, MapperService mapperService, - Map fieldNameToType) { + private static void registerFieldTypes(MapperService.Snapshot mapperSnapshot, Map fieldNameToType) { for (Map.Entry entry : fieldNameToType.entrySet()) { String fieldName = entry.getKey(); MappedFieldType fieldType = entry.getValue(); - when(mapperService.fieldType(fieldName)).thenReturn(fieldType); + when(mapperSnapshot.fieldType(fieldName)).thenReturn(fieldType); } } @@ -273,13 +271,16 @@ public boolean shouldCache(Query query) { BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), circuitBreakerService).withCircuitBreaking(); when(searchContext.bigArrays()).thenReturn(bigArrays); - // TODO: now just needed for top_hits, this will need to be revised for other agg unit tests: - MapperService mapperService = mapperServiceMock(); - when(mapperService.getIndexSettings()).thenReturn(indexSettings); - when(mapperService.hasNested()).thenReturn(false); - when(mapperService.indexAnalyzer()).thenReturn(new StandardAnalyzer()); // for significant text - QueryShardContext queryShardContext = - queryShardContextMock(contextIndexSearcher, mapperService, indexSettings, circuitBreakerService, bigArrays); + MapperService.Snapshot mapperSnapshot = mapperSnapshotMock(); + when(mapperSnapshot.hasNested()).thenReturn(false); + QueryShardContext queryShardContext = queryShardContextMock( + contextIndexSearcher, + mapperSnapshot, + indexSettings, + circuitBreakerService, + searchContext.bitsetFilterCache(), + bigArrays + ); when(searchContext.getQueryShardContext()).thenReturn(queryShardContext); when(queryShardContext.getObjectMapper(anyString())).thenAnswer(invocation -> { String fieldName = (String) invocation.getArguments()[0]; @@ -289,18 +290,13 @@ public boolean shouldCache(Query query) { return null; }); - NestedDocuments nestedDocuments = new NestedDocuments(mapperService, searchContext.bitsetFilterCache()::getBitSetProducer); - when(searchContext.getNestedDocuments()) - .thenReturn(nestedDocuments); - Map fieldNameToType = new HashMap<>(); fieldNameToType.putAll(Arrays.stream(fieldTypes) .filter(Objects::nonNull) .collect(Collectors.toMap(MappedFieldType::name, Function.identity()))); fieldNameToType.putAll(getFieldAliases(fieldTypes)); - registerFieldTypes(searchContext, mapperService, - fieldNameToType); + registerFieldTypes(mapperSnapshot, fieldNameToType); doAnswer(invocation -> { /* Store the release-ables so we can release them at the end of the test case. This is important because aggregations don't * close their sub-aggregations. This is fairly similar to what the production code does. */ @@ -324,22 +320,23 @@ protected IndexSettings createIndexSettings() { /** * sub-tests that need a more complex mock can overwrite this */ - protected MapperService mapperServiceMock() { - return mock(MapperService.class); + protected MapperService.Snapshot mapperSnapshotMock() { + return mock(MapperService.Snapshot.class); } /** * sub-tests that need a more complex mock can overwrite this */ protected QueryShardContext queryShardContextMock(IndexSearcher searcher, - MapperService mapperService, + MapperService.Snapshot mapperSnapshot, IndexSettings indexSettings, CircuitBreakerService circuitBreakerService, + BitsetFilterCache bitsetFilterCache, BigArrays bigArrays) { - return new QueryShardContext(0, indexSettings, bigArrays, null, - getIndexFieldDataLookup(mapperService, circuitBreakerService), - mapperService, null, getMockScriptService(), xContentRegistry(), + return new QueryShardContext(0, indexSettings, bigArrays, bitsetFilterCache, + getIndexFieldDataLookup(indexSettings.getIndex().getName(), circuitBreakerService), + mapperSnapshot, null, getMockScriptService(), xContentRegistry(), writableRegistry(), null, searcher, System::currentTimeMillis, null, null, () -> true, valuesSourceRegistry, emptyMap()); } @@ -348,9 +345,10 @@ mapperService, null, getMockScriptService(), xContentRegistry(), * Sub-tests that need a more complex index field data provider can override this */ protected TriFunction, IndexFieldData> getIndexFieldDataLookup( - MapperService mapperService, CircuitBreakerService circuitBreakerService) { - return (fieldType, s, searchLookup) -> fieldType.fielddataBuilder( - mapperService.getIndexSettings().getIndex().getName(), searchLookup) + String indexName, + CircuitBreakerService circuitBreakerService + ) { + return (fieldType, s, searchLookup) -> fieldType.fielddataBuilder(indexName, searchLookup) .build(new IndexFieldDataCache.None(), circuitBreakerService); } @@ -603,7 +601,6 @@ protected List unsupportedMappedFieldTypes() { */ public void testSupportedFieldTypes() throws IOException { MapperRegistry mapperRegistry = new IndicesModule(Collections.emptyList()).getMapperRegistry(); - Settings settings = Settings.builder().put("index.version.created", Version.CURRENT.id).build(); String fieldName = "typeTestFieldName"; List supportedVSTypes = getSupportedValuesSourceTypes(); List unsupportedMappedFieldTypes = unsupportedMappedFieldTypes(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java index f44cc47924576..c2d84fec864f5 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java @@ -412,7 +412,7 @@ public void close() throws IOException { QueryShardContext createShardContext(IndexSearcher searcher) { return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, bitsetFilterCache, - indexFieldDataService::getForField, mapperService, similarityService, scriptService, xContentRegistry, + indexFieldDataService::getForField, mapperService.snapshot(), similarityService, scriptService, xContentRegistry, namedWriteableRegistry, this.client, searcher, () -> nowInMillis, null, indexNameMatcher(), () -> true, null, emptyMap()); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java index d05a375410266..ecc982c33d85e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java @@ -32,7 +32,6 @@ import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.SearchExtBuilder; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.SearchContextAggregations; @@ -511,11 +510,6 @@ public FetchSearchResult fetchResult() { return null; } - @Override - public NestedDocuments getNestedDocuments() { - return new NestedDocuments(indexService.mapperService(), bitsetFilterCache()::getBitSetProducer); - } - @Override public FetchPhase fetchPhase() { return null; diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java index 55a56949b7dce..2e3fbb7936cc3 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java @@ -536,7 +536,7 @@ public void close() throws IOException { } } - private TestIndexContext testIndex(MapperService mapperService, Client client) throws IOException { + private TestIndexContext testIndex(MapperService.Snapshot mapperSnapshot, Client client) throws IOException { TestIndexContext context = null; final long nowInMillis = randomNonNegativeLong(); @@ -564,7 +564,7 @@ private TestIndexContext testIndex(MapperService mapperService, Client client) t final LeafReaderContext leaf = directoryReader.leaves().get(0); final QueryShardContext shardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, null, mapperService, null, null, xContentRegistry(), writableRegistry(), + null, null, mapperSnapshot, null, null, xContentRegistry(), writableRegistry(), client, new IndexSearcher(directoryReader), () -> nowInMillis, null, null, () -> true, null, emptyMap()); context = new TestIndexContext(directory, iw, directoryReader, shardContext, leaf); @@ -585,9 +585,7 @@ null, null, mapperService, null, null, xContentRegistry(), writableRegistry(), } private void runTestOnIndices(int numberIndices, CheckedConsumer, Exception> body) throws Exception { - final MapperService mapperService = mock(MapperService.class); - when(mapperService.fieldType(Mockito.anyString())).thenAnswer(invocation -> { - final String fieldName = (String) invocation.getArguments()[0]; + MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot(fieldName -> { if (fieldName.equals(MISSING_FIELD_NAME)) { return null; } else { @@ -601,7 +599,7 @@ private void runTestOnIndices(int numberIndices, CheckedConsumer context = new ArrayList<>(numberIndices); try { for (int i = 0; i < numberIndices; i++) { - context.add(testIndex(mapperService, client)); + context.add(testIndex(mapperSnapshot, client)); } body.accept(context); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java index 74042155ffd93..af8e313941188 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java @@ -37,7 +37,6 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.license.XPackLicenseState.Feature; -import org.elasticsearch.mock.orig.Mockito; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.internal.ContextIndexSearcher; import org.elasticsearch.test.AbstractBuilderTestCase; @@ -49,7 +48,6 @@ import org.elasticsearch.xpack.core.security.authz.permission.FieldPermissions; import org.elasticsearch.xpack.core.security.user.User; -import java.util.Collections; import java.util.HashSet; import java.util.Set; import java.util.concurrent.Executors; @@ -59,7 +57,6 @@ import static java.util.Collections.singletonMap; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; -import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -68,15 +65,10 @@ public class SecurityIndexReaderWrapperIntegrationTests extends AbstractBuilderT public void testDLS() throws Exception { ShardId shardId = new ShardId("_index", "_na_", 0); - MapperService mapperService = mock(MapperService.class); - ScriptService scriptService = mock(ScriptService.class); - when(mapperService.documentMapper()).thenReturn(null); - when(mapperService.simpleMatchToFullName(anyString())) - .then(invocationOnMock -> Collections.singletonList((String) invocationOnMock.getArguments()[0])); - when(mapperService.fieldType(Mockito.anyString())).then(invocation -> { - final String fieldName = (String) invocation.getArguments()[0]; - return new KeywordFieldMapper.KeywordFieldType(fieldName); - }); + MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot( + fieldName -> new KeywordFieldMapper.KeywordFieldType(fieldName) + ); + ScriptService scriptService = mock(ScriptService.class); final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); final SecurityContext securityContext = new SecurityContext(Settings.EMPTY, threadContext); @@ -91,7 +83,7 @@ public void testDLS() throws Exception { when(client.settings()).thenReturn(Settings.EMPTY); final long nowInMillis = randomNonNegativeLong(); QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, null, mapperService, null, null, xContentRegistry(), writableRegistry(), + null, null, mapperSnapshot, null, null, xContentRegistry(), writableRegistry(), client, null, () -> nowInMillis, null, null, () -> true, null, emptyMap()); QueryShardContext queryShardContext = spy(realQueryShardContext); DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY, Executors.newSingleThreadExecutor()); @@ -182,15 +174,10 @@ protected IndicesAccessControl getIndicesAccessControl() { public void testDLSWithLimitedPermissions() throws Exception { ShardId shardId = new ShardId("_index", "_na_", 0); - MapperService mapperService = mock(MapperService.class); - ScriptService scriptService = mock(ScriptService.class); - when(mapperService.documentMapper()).thenReturn(null); - when(mapperService.simpleMatchToFullName(anyString())) - .then(invocationOnMock -> Collections.singletonList((String) invocationOnMock.getArguments()[0])); - when(mapperService.fieldType(Mockito.anyString())).then(invocation -> { - final String fieldName = (String) invocation.getArguments()[0]; - return new KeywordFieldMapper.KeywordFieldType(fieldName); - }); + MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot( + fieldName -> new KeywordFieldMapper.KeywordFieldType(fieldName) + ); + ScriptService scriptService = mock(ScriptService.class); final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); final SecurityContext securityContext = new SecurityContext(Settings.EMPTY, threadContext); @@ -223,7 +210,7 @@ public void testDLSWithLimitedPermissions() throws Exception { when(client.settings()).thenReturn(Settings.EMPTY); final long nowInMillis = randomNonNegativeLong(); QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, null, mapperService, null, null, xContentRegistry(), writableRegistry(), + null, null, mapperSnapshot, null, null, xContentRegistry(), writableRegistry(), client, null, () -> nowInMillis, null, null, () -> true, null, emptyMap()); QueryShardContext queryShardContext = spy(realQueryShardContext); DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY, Executors.newSingleThreadExecutor()); diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/runtime_fields/10_keyword.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/runtime_fields/10_keyword.yml index 700bef6c3002f..a32c2487241d0 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/runtime_fields/10_keyword.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/runtime_fields/10_keyword.yml @@ -128,6 +128,7 @@ setup: search: index: sensor body: + size: 0 aggs: dow: terms: @@ -138,6 +139,31 @@ setup: - match: {aggregations.dow.buckets.1.key: Monday} - match: {aggregations.dow.buckets.1.doc_count: 1} + # Update the mapping and make sure the cache doesn't still have the old results + - do: + indices.put_mapping: + index: sensor + body: + runtime: + day_of_week: + type: keyword + script: | + emit(doc['timestamp'].value.dayOfWeekEnum.getDisplayName(TextStyle.SHORT, Locale.ROOT)); + - do: + search: + index: sensor + body: + size: 0 + aggs: + dow: + terms: + field: day_of_week + - match: {hits.total.value: 6} + - match: {aggregations.dow.buckets.0.key: Fri} + - match: {aggregations.dow.buckets.0.doc_count: 1} + - match: {aggregations.dow.buckets.1.key: Mon} + - match: {aggregations.dow.buckets.1.doc_count: 1} + --- "term query": - do: From 8c808e79fda2075bddd7c37f54c815753a1459ab Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 24 Nov 2020 09:59:43 -0500 Subject: [PATCH 02/47] Refactor --- .../elasticsearch/index/mapper/MapperService.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 842cbf70be7b2..ec945db5f3139 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -183,9 +183,7 @@ DocumentParser documentParser() { } Map, MetadataFieldMapper> getMetadataMappers() { - Map, MetadataFieldMapper> metadataMappers = new LinkedHashMap<>(); - snapshot.collectMetadataMappers(metadataMappers); - return metadataMappers; + return snapshot.getMetadataMappers(); } /** @@ -616,7 +614,7 @@ public final IndexAnalyzers getIndexAnalyzers() { abstract MappedSnapshot merge(DocumentMapper mapper, MergeReason reason); - abstract void collectMetadataMappers(Map, MetadataFieldMapper> metadataMappers); + abstract Map, MetadataFieldMapper> getMetadataMappers(); } private static class EmptySnapshot extends AbstractSnapshot { @@ -689,14 +687,16 @@ public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingEx } @Override - void collectMetadataMappers(Map, MetadataFieldMapper> metadataMappers) { + Map, MetadataFieldMapper> getMetadataMappers() { Map metadataMapperParsers = mapperService.mapperRegistry.getMetadataMapperParsers( mapperService.indexSettings.getIndexVersionCreated() ); + Map, MetadataFieldMapper> metadataMappers = new LinkedHashMap<>(); for (MetadataFieldMapper.TypeParser parser : metadataMapperParsers.values()) { MetadataFieldMapper metadataFieldMapper = parser.getDefault(parserContext()); metadataMappers.put(metadataFieldMapper.getClass(), metadataFieldMapper); } + return metadataMappers; } @Override @@ -785,8 +785,8 @@ public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingEx } @Override - protected void collectMetadataMappers(Map, MetadataFieldMapper> metadataMappers) { - metadataMappers.putAll(mapper.mapping().metadataMappersMap); + Map, MetadataFieldMapper> getMetadataMappers() { + return new LinkedHashMap<>(mapper.mapping().metadataMappersMap); } @Override From 3fc61e8bbaf06775ea5b740b97dedd7032f83766 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 24 Nov 2020 10:05:49 -0500 Subject: [PATCH 03/47] Refactor again --- .../index/mapper/MapperService.java | 29 +++++++++++++++---- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index ec945db5f3139..d1428dbd0e5c9 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -227,14 +227,16 @@ public boolean updateMapping(final IndexMetadata currentIndexMetadata, final Ind MappingMetadata mappingMetadata = newIndexMetadata.mapping(); CompressedXContent incomingMappingSource = mappingMetadata.source(); - String op = snapshot != null ? "updated" : "added"; if (logger.isDebugEnabled() && incomingMappingSource.compressed().length < 512) { - logger.debug("[{}] {} mapping, source [{}]", index(), op, incomingMappingSource.string()); + logger.debug("[{}] {} mapping, source [{}]", index(), snapshot.updateOperationName(), incomingMappingSource.string()); } else if (logger.isTraceEnabled()) { - logger.trace("[{}] {} mapping, source [{}]", index(), op, incomingMappingSource.string()); + logger.trace("[{}] {} mapping, source [{}]", index(), snapshot.updateOperationName(), incomingMappingSource.string()); } else { - logger.debug("[{}] {} mapping (source suppressed due to length, use TRACE level if needed)", - index(), op); + logger.debug( + "[{}] {} mapping (source suppressed due to length, use TRACE level if needed)", + index(), + snapshot.updateOperationName() + ); } // refresh mapping can happen when the parsing/merging of the mapping from the metadata doesn't result in the same @@ -615,6 +617,13 @@ public final IndexAnalyzers getIndexAnalyzers() { abstract MappedSnapshot merge(DocumentMapper mapper, MergeReason reason); abstract Map, MetadataFieldMapper> getMetadataMappers(); + + /** + * The name of the operation to log when merging new mappings. If the + * mapping is empty it'll be {@code add} and if it isn't then it'll be + * {@code update}. + */ + abstract String updateOperationName(); } private static class EmptySnapshot extends AbstractSnapshot { @@ -703,6 +712,11 @@ Map, MetadataFieldMapper> getMetadataMapper MappedSnapshot merge(DocumentMapper mapper, MergeReason reason) { return new MappedSnapshot(mapperService, mapper, 1); } + + @Override + protected String updateOperationName() { + return "added"; + } } static class MappedSnapshot extends AbstractSnapshot { @@ -797,6 +811,11 @@ MappedSnapshot merge(DocumentMapper newMapper, MergeReason reason) { } return new MappedSnapshot(mapperService, merged, version + 1); } + + @Override + protected String updateOperationName() { + return "updated"; + } } /** From 1dedc0a8b809632d09b22bd7be78abfb730e3ca2 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 24 Nov 2020 10:38:39 -0500 Subject: [PATCH 04/47] Javadoc --- .../elasticsearch/index/mapper/MapperService.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index d1428dbd0e5c9..21a7dcfbc1f55 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -585,6 +585,18 @@ public interface Snapshot { IndexAnalyzers getIndexAnalyzers(); } + /** + * Superclass for {@link Snapshot} implementations hosted by {@link MapperService}. + * This contains a few package private methods which are important for how + * {@linkplain MapperService} handles snapshots but aren't exposed outside + * of it. + *

+ * You may ask "Why have both the interface and the abstract class?" And that is + * a reasonable question. Mostly we have two things so there is an obvious + * separation of "internal" stuff and "external" stuff. Which makes testing + * a little simpler because most tests just rely on the {@linkplain Snapshot} + * interface. + */ private abstract static class AbstractSnapshot implements Snapshot { protected final MapperService mapperService; From fddfdb09a29a15d7fcf51faf3d963f67a9c7e538 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 24 Nov 2020 10:57:44 -0500 Subject: [PATCH 05/47] Moar javadoc --- .../elasticsearch/index/mapper/MapperService.java | 10 +++++++++- .../index/query/QueryShardContext.java | 13 +++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 21a7dcfbc1f55..5b94307c18991 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -451,7 +451,9 @@ public ObjectMapper getObjectMapper(String name) { /** * An analyzer that performs a volatile read on the mapping find correct {@link FieldNameAnalyzer}. + * @deprecated Prefer {@link #snapshot()} and then {@link Snapshot#indexAnalyzer()}. */ + @Deprecated public Analyzer indexAnalyzer() { return indexAnalyzer; } @@ -638,6 +640,9 @@ public final IndexAnalyzers getIndexAnalyzers() { abstract String updateOperationName(); } + /** + * {@link Snapshot} of an "empty" mapping. + */ private static class EmptySnapshot extends AbstractSnapshot { EmptySnapshot(MapperService mapperService) { super(mapperService); @@ -731,7 +736,10 @@ protected String updateOperationName() { } } - static class MappedSnapshot extends AbstractSnapshot { + /** + * Snapshot of a non-empty mapping. + */ + private static class MappedSnapshot extends AbstractSnapshot { private final DocumentMapper mapper; /** * Current version of the of the mapping. Increments if the mapping diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index 7a78e11b6d700..972fdee49d49a 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -29,6 +29,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.TriFunction; @@ -252,6 +253,9 @@ public Map copyNamedQueries() { return Map.copyOf(namedQueries); } + /** + * Parse a document with current mapping. + */ public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { return mapperSnapshot.parseDocument(source); } @@ -594,10 +598,19 @@ private static Map parseRuntimeMappings( return Collections.unmodifiableMap(runtimeFieldTypes); } + /** + * Build a loader for nested documents. + */ public NestedDocuments getNestedDocuments() { return new NestedDocuments(mapperSnapshot.documentMapper(), bitsetFilterCache::getBitSetProducer); } + /** + * Current version of the of the mapping. Increments if the mapping + * changes locally. Distinct from + * {@link IndexMetadata#getMappingVersion()} because it purely + * considers the local mapping changes. + */ public long localMappingVersion() { return mapperSnapshot.version(); } From 58af6feea5553003cf5e12b38248f438fabf5c52 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 24 Nov 2020 17:08:04 -0500 Subject: [PATCH 06/47] Add example in core --- .../search.aggregation/220_filters_bucket.yml | 69 +++++++++++++++++++ 1 file changed, 69 insertions(+) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/220_filters_bucket.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/220_filters_bucket.yml index e0183f0c54f66..72b9eba61f76e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/220_filters_bucket.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/220_filters_bucket.yml @@ -272,3 +272,72 @@ setup: the_filter: filters: filters: [] + + +--- +"cache": + - skip: + version: " - 7.99.99" + reason: cache fixed in 8.0.0 to be backported to 7.11.0 + + - do: + bulk: + refresh: true + body: + - index: + _index: test_1 + _id: 100 + - int_field: 1 + double_field: 1.0 + string_field: foo bar + + - do: + search: + index: test_1 + body: + size: 0 + aggs: + f: + filters: + filters: + foo: + match: + string_field: foo + foo_bar: + match: + string_field: foo bar + - match: { hits.total.value: 5 } + - length: { aggregations.f.buckets: 2 } + - match: { aggregations.f.buckets.foo.doc_count: 4 } + - match: { aggregations.f.buckets.foo_bar.doc_count: 1 } + + # Modify the mapping configuration that generates queries. This should bust the cache. + - do: + indices.put_mapping: + index: test_1 + body: + properties: + string_field: + type: keyword + split_queries_on_whitespace: true + + # This should be entirely fresh because updating the mapping busted the cache. + - do: + search: + index: test_1 + body: + size: 0 + aggs: + f: + filters: + filters: + foo: + match: + string_field: foo + foo_bar: + match: + string_field: foo bar + - match: { hits.total.value: 5 } + - length: { aggregations.f.buckets: 2 } + - match: { aggregations.f.buckets.foo.doc_count: 4 } + - match: { aggregations.f.buckets.foo_bar.doc_count: 4 } From 177e4d8509ab33d37408e9e90d8fb35372cefd57 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 1 Dec 2020 09:42:51 -0500 Subject: [PATCH 07/47] WIP --- .../index/mapper/MapperService.java | 119 ++++++++++++------ .../index/query/QueryShardContext.java | 6 +- .../index/mapper/NestedDocumentsTests.java | 6 +- .../aggregations/AggregatorTestCase.java | 2 - 4 files changed, 90 insertions(+), 43 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 3aace990cd933..3a54226b3b9a9 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -20,6 +20,8 @@ package org.elasticsearch.index.mapper; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.join.BitSetProducer; import org.elasticsearch.Assertions; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -51,6 +53,7 @@ import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.NestedDocuments; import java.io.Closeable; import java.io.IOException; @@ -115,7 +118,7 @@ public enum MergeReason { private final Supplier parserContextSupplier; /** - * The current mapping accessed through {@link #snapshot()} and {@link #indexAnalyzer()}. + * The current mapping accessed through {@link #snapshot()} and {@link #indexAnalyzer(String, Function)}. */ private volatile AbstractSnapshot snapshot = new EmptySnapshot(this); @@ -198,7 +201,7 @@ public boolean updateMapping(final IndexMetadata currentIndexMetadata, final Ind return false; } - final Snapshot updatedSnapshot; + final AbstractSnapshot updatedSnapshot; try { updatedSnapshot = internalMerge(newIndexMetadata, MergeReason.MAPPING_RECOVERY); } catch (Exception e) { @@ -245,7 +248,7 @@ public boolean updateMapping(final IndexMetadata currentIndexMetadata, final Ind private void assertMappingVersion( final IndexMetadata currentIndexMetadata, final IndexMetadata newIndexMetadata, - final Snapshot updatedSnapshot) throws IOException { + final AbstractSnapshot updatedSnapshot) throws IOException { if (Assertions.ENABLED && currentIndexMetadata != null) { if (currentIndexMetadata.getMappingVersion() == newIndexMetadata.getMappingVersion()) { // if the mapping version is unchanged, then there should not be any updates and all mappings should be the same @@ -293,11 +296,11 @@ public void merge(IndexMetadata indexMetadata, MergeReason reason) { } public DocumentMapper merge(String type, CompressedXContent mappingSource, MergeReason reason) { - Snapshot updatedSnapshot = internalMerge(type, mappingSource, reason); - return updatedSnapshot == null ? null : updatedSnapshot.documentMapper(); + AbstractSnapshot updatedSnapshot = internalMerge(type, mappingSource, reason); + return updatedSnapshot == null ? null : updatedSnapshot.documentMapper(); // TODO return Snapshot here } - private synchronized Snapshot internalMerge(IndexMetadata indexMetadata, MergeReason reason) { + private synchronized AbstractSnapshot internalMerge(IndexMetadata indexMetadata, MergeReason reason) { assert reason != MergeReason.MAPPING_UPDATE_PREFLIGHT; MappingMetadata mappingMetadata = indexMetadata.mapping(); if (mappingMetadata != null) { @@ -306,7 +309,7 @@ private synchronized Snapshot internalMerge(IndexMetadata indexMetadata, MergeRe return null; } - private synchronized Snapshot internalMerge(String type, CompressedXContent mappings, MergeReason reason) { + private synchronized AbstractSnapshot internalMerge(String type, CompressedXContent mappings, MergeReason reason) { DocumentMapper documentMapper; @@ -319,7 +322,7 @@ private synchronized Snapshot internalMerge(String type, CompressedXContent mapp return internalMerge(documentMapper, reason); } - private synchronized Snapshot internalMerge(DocumentMapper mapper, MergeReason reason) { + private synchronized AbstractSnapshot internalMerge(DocumentMapper mapper, MergeReason reason) { assert mapper != null; @@ -379,7 +382,7 @@ public static boolean isMappingSourceTyped(String type, Map mapp */ private String resolveDocumentType(String type) { if (MapperService.SINGLE_MAPPING_NAME.equals(type)) { - Snapshot currentSnapshot = snapshot; + AbstractSnapshot currentSnapshot = snapshot; if (currentSnapshot.documentMapper() != null) { return currentSnapshot.documentMapper().type(); } @@ -444,7 +447,7 @@ public ObjectMapper getObjectMapper(String name) { * @param field the field name * @param unindexedFieldAnalyzer a function to return an Analyzer for a field with no * directly associated index-time analyzer - * @deprecated Prefer {@link #snapshot()} and then {@link Snapshot#indexAnalyzer()}. + * @deprecated Prefer {@link #snapshot()} and then {@link Snapshot#indexAnalyzer}. */ @Deprecated public NamedAnalyzer indexAnalyzer(String field, Function unindexedFieldAnalyzer) { @@ -553,15 +556,6 @@ public interface Snapshot { */ boolean containsBrokenAnalysis(String field); - /** - * Get the actual mapping. - * @deprecated Prefer any other method. {@link DocumentMapper} doesn't support - * runtime fields and is otherwise tightly coupled to the internals - * of mappings. - */ - @Deprecated - DocumentMapper documentMapper(); - /** * Current version of the of the mapping. Increments if the mapping * changes locally. Distinct from @@ -584,6 +578,21 @@ public interface Snapshot { boolean isMetadataField(String field); IndexAnalyzers getIndexAnalyzers(); + + /** + * Build a loader for nested documents. + */ + NestedDocuments getNestedDocuments(Function filterProducer); + + /** + * Are there mappings for this index? + */ + boolean hasMappings(); + + /** + * Is source enabled on this index? + */ + boolean sourceEnabled(); } /** @@ -631,6 +640,8 @@ public final IndexAnalyzers getIndexAnalyzers() { abstract Map, MetadataFieldMapper> getMetadataMappers(); + abstract DocumentMapper documentMapper(); + /** * The name of the operation to log when merging new mappings. If the * mapping is empty it'll be {@code add} and if it isn't then it'll be @@ -696,11 +707,6 @@ public boolean containsBrokenAnalysis(String field) { return false; } - @Override - public DocumentMapper documentMapper() { - return null; - } - @Override public long version() { return 0; @@ -733,6 +739,26 @@ MappedSnapshot merge(DocumentMapper mapper, MergeReason reason) { protected String updateOperationName() { return "added"; } + + @Override + public NestedDocuments getNestedDocuments(Function filterProducer) { + return null; + } + + @Override + public boolean hasMappings() { + return false; + } + + @Override + public boolean sourceEnabled() { + return false; + } + + @Override + DocumentMapper documentMapper() { + return null; + } } /** @@ -762,11 +788,6 @@ public MappedFieldType fieldType(String fullName) { return mapper.mappers().fieldTypes().get(fullName); } - @Override - public DocumentMapper documentMapper() { - return mapper; - } - @Override public boolean hasNested() { return mapper.hasNestedObjects(); @@ -839,6 +860,25 @@ MappedSnapshot merge(DocumentMapper newMapper, MergeReason reason) { protected String updateOperationName() { return "updated"; } + + @Override + public NestedDocuments getNestedDocuments(Function filterProducer) { + return new NestedDocuments(mapper, filterProducer); + } + + @Override + public boolean hasMappings() { + return true; + } + + @Override + public boolean sourceEnabled() { + return mapper.sourceMapper().enabled(); + } + + DocumentMapper documentMapper() { + return mapper; + } } /** @@ -881,6 +921,16 @@ public boolean hasNested() { return false; } + @Override + public boolean hasMappings() { + return true; + } + + @Override + public boolean sourceEnabled() { + return true; + } + @Override public ObjectMapper getObjectMapper(String name) { throw new UnsupportedOperationException(); @@ -911,11 +961,6 @@ public boolean containsBrokenAnalysis(String field) { throw new UnsupportedOperationException(); } - @Override - public DocumentMapper documentMapper() { - throw new UnsupportedOperationException(); - } - @Override public long version() { throw new UnsupportedOperationException(); @@ -940,6 +985,10 @@ public boolean isMetadataField(String field) { public IndexAnalyzers getIndexAnalyzers() { throw new UnsupportedOperationException(); } - + + @Override + public NestedDocuments getNestedDocuments(Function filterProducer) { + throw new UnsupportedOperationException(); + } } } diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index c5a0a00db797d..7fa9fd8df43b9 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -267,7 +267,7 @@ public boolean hasNested() { } public boolean hasMappings() { - return mapperSnapshot.documentMapper() != null; + return mapperSnapshot.hasMappings(); } /** @@ -329,7 +329,7 @@ public Set sourcePath(String fullName) { } public boolean isSourceEnabled() { - return mapperSnapshot.documentMapper().sourceMapper().enabled(); + return mapperSnapshot.sourceEnabled(); } /** @@ -594,7 +594,7 @@ private static Map parseRuntimeMappings( * Build a loader for nested documents. */ public NestedDocuments getNestedDocuments() { - return new NestedDocuments(mapperSnapshot.documentMapper(), bitsetFilterCache::getBitSetProducer); + return mapperSnapshot.getNestedDocuments(bitsetFilterCache::getBitSetProducer); } /** diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java index c930dafce3d2a..ddce0e9819fde 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java @@ -56,7 +56,7 @@ public void testSimpleNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService.snapshot().documentMapper(), QueryBitSetProducer::new); + NestedDocuments nested = mapperService.snapshot().getNestedDocuments(QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -153,7 +153,7 @@ public void testMultiLevelNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService.snapshot().documentMapper(), QueryBitSetProducer::new); + NestedDocuments nested = mapperService.snapshot().getNestedDocuments(QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -264,7 +264,7 @@ public void testNestedObjectWithinNonNestedObject() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService.snapshot().documentMapper(), QueryBitSetProducer::new); + NestedDocuments nested = mapperService.snapshot().getNestedDocuments(QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index e95f8b8ed4716..8b9022d3ef47e 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -55,7 +55,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.settings.Settings; @@ -144,7 +143,6 @@ import static org.elasticsearch.test.InternalAggregationTestCase.DEFAULT_MAX_BUCKETS; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; From e59716408d1899689565f8e48dd1826af7a9ec81 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 1 Dec 2020 11:22:25 -0500 Subject: [PATCH 08/47] WIP --- .../index/mapper/MapperService.java | 112 -------------- .../mapper/FieldNamesFieldTypeTests.java | 4 +- .../index/query/QueryShardContextTests.java | 3 +- .../AbstractSuggestionBuilderTestCase.java | 5 +- .../index/mapper/StubSnapshot.java | 145 ++++++++++++++++++ .../aggregations/AggregatorTestCase.java | 3 + .../DocumentSubsetBitsetCacheTests.java | 3 +- ...ityIndexReaderWrapperIntegrationTests.java | 9 +- 8 files changed, 159 insertions(+), 125 deletions(-) create mode 100644 test/framework/src/main/java/org/elasticsearch/index/mapper/StubSnapshot.java diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 3a54226b3b9a9..c3b3f30dcb7b8 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -68,7 +68,6 @@ import static java.util.Collections.emptySet; import static java.util.Collections.singleton; -import static java.util.Collections.unmodifiableSet; import static java.util.Objects.requireNonNull; public class MapperService extends AbstractIndexComponent implements Closeable { @@ -880,115 +879,4 @@ DocumentMapper documentMapper() { return mapper; } } - - /** - * A mapping snapshot with the "central" methods that are useful for testing. - */ - public static class StubSnapshot implements Snapshot { - private final Function lookup; - private final Supplier> fields; - - public StubSnapshot(Function lookup) { - this.lookup = lookup; - this.fields = () -> { - throw new UnsupportedOperationException(); - }; - } - - public StubSnapshot(Map lookup) { - this.lookup = lookup::get; - this.fields = lookup::keySet; - } - - @Override - public MappedFieldType fieldType(String fullName) { - return lookup.apply(fullName); - } - - @Override - public Set simpleMatchToFullName(String pattern) { - if (Regex.isSimpleMatchPattern(pattern) == false) { - return singleton(pattern); - } - if (Regex.isMatchAllPattern(pattern)) { - return unmodifiableSet(fields.get()); - } - throw new UnsupportedOperationException(); - } - - @Override - public boolean hasNested() { - return false; - } - - @Override - public boolean hasMappings() { - return true; - } - - @Override - public boolean sourceEnabled() { - return true; - } - - @Override - public ObjectMapper getObjectMapper(String name) { - throw new UnsupportedOperationException(); - } - - @Override - public Set sourcePath(String fullName) { - throw new UnsupportedOperationException(); - } - - @Override - public DocumentMapperForType documentMapperWithAutoCreate() { - throw new UnsupportedOperationException(); - } - - @Override - public Iterable getEagerGlobalOrdinalsFields() { - throw new UnsupportedOperationException(); - } - - @Override - public NamedAnalyzer indexAnalyzer(String field, Function unindexedFieldAnalyzer) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean containsBrokenAnalysis(String field) { - throw new UnsupportedOperationException(); - } - - @Override - public long version() { - throw new UnsupportedOperationException(); - } - - @Override - public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { - throw new UnsupportedOperationException(); - } - - @Override - public ParserContext parserContext() { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isMetadataField(String field) { - throw new UnsupportedOperationException(); - } - - @Override - public IndexAnalyzers getIndexAnalyzers() { - throw new UnsupportedOperationException(); - } - - @Override - public NestedDocuments getNestedDocuments(Function filterProducer) { - throw new UnsupportedOperationException(); - } - } } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java index 2296ff8307d91..fbc2a3331cc5c 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java @@ -43,9 +43,7 @@ public void testTermQuery() { Settings settings = settings(Version.CURRENT).build(); IndexSettings indexSettings = new IndexSettings( new IndexMetadata.Builder("foo").settings(settings).numberOfShards(1).numberOfReplicas(0).build(), settings); - MapperService.Snapshot mapperService = new MapperService.StubSnapshot( - Map.of("_field_names", fieldNamesFieldType, "field_name", fieldType) - ); + MapperService.Snapshot mapperService = new StubSnapshot(Map.of("_field_names", fieldNamesFieldType, "field_name", fieldType)); QueryShardContext queryShardContext = new QueryShardContext(0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService, diff --git a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java index af4a83d3b614d..fa08d6661a491 100644 --- a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java @@ -60,6 +60,7 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MockFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.mapper.StubSnapshot; import org.elasticsearch.index.mapper.TestRuntimeField; import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.indices.IndicesModule; @@ -414,7 +415,7 @@ private static MapperService.Snapshot createMapperSnapshot( indexSettings, () -> true ); - return new MapperService.StubSnapshot(fieldTypeLookup) { + return new StubSnapshot(fieldTypeLookup) { @Override public IndexAnalyzers getIndexAnalyzers() { return indexAnalyzers; diff --git a/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java b/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java index 2ba12cdec3232..d365ed414c6d0 100644 --- a/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java @@ -39,6 +39,7 @@ import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.StubSnapshot; import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.index.mapper.TextSearchInfo; import org.elasticsearch.index.query.QueryShardContext; @@ -175,7 +176,7 @@ public NamedAnalyzer get(Object key) { }, Collections.emptyMap(), Collections.emptyMap()); - MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot(fieldName -> fieldType) { + MapperService.Snapshot mapperSnapshot = new StubSnapshot(fieldName -> fieldType) { public IndexAnalyzers getIndexAnalyzers() { return indexAnalyzers; } @@ -217,7 +218,7 @@ public void testBuildWithUnmappedField() { Settings indexSettings = builder.build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index(randomAlphaOfLengthBetween(1, 10), "_na_"), indexSettings); - MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot(Map.of()); + MapperService.Snapshot mapperSnapshot = new StubSnapshot(Map.of()); QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperSnapshot, null, null, xContentRegistry(), namedWriteableRegistry, null, null, diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/StubSnapshot.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/StubSnapshot.java new file mode 100644 index 0000000000000..d8093839208c6 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/StubSnapshot.java @@ -0,0 +1,145 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.mapper; + +import org.apache.lucene.search.Query; +import org.apache.lucene.search.join.BitSetProducer; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.analysis.NamedAnalyzer; +import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext; +import org.elasticsearch.search.NestedDocuments; + +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; + +import static java.util.Collections.singleton; +import static java.util.Collections.unmodifiableSet; + +/** + * A {@link MapperService.Snapshot} with the "central" methods that are useful for testing. + */ +public class StubSnapshot implements MapperService.Snapshot { + private final Function lookup; + private final Supplier> fields; + + public StubSnapshot(Function lookup) { + this.lookup = lookup; + this.fields = () -> { throw new UnsupportedOperationException(); }; + } + + public StubSnapshot(Map lookup) { + this.lookup = lookup::get; + this.fields = lookup::keySet; + } + + @Override + public MappedFieldType fieldType(String fullName) { + return lookup.apply(fullName); + } + + @Override + public Set simpleMatchToFullName(String pattern) { + if (Regex.isSimpleMatchPattern(pattern) == false) { + return singleton(pattern); + } + if (Regex.isMatchAllPattern(pattern)) { + return unmodifiableSet(fields.get()); + } + throw new UnsupportedOperationException(); + } + + @Override + public boolean hasNested() { + return false; + } + + @Override + public boolean hasMappings() { + return true; + } + + @Override + public boolean sourceEnabled() { + return true; + } + + @Override + public ObjectMapper getObjectMapper(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public Set sourcePath(String fullName) { + throw new UnsupportedOperationException(); + } + + @Override + public DocumentMapperForType documentMapperWithAutoCreate() { + throw new UnsupportedOperationException(); + } + + @Override + public Iterable getEagerGlobalOrdinalsFields() { + throw new UnsupportedOperationException(); + } + + @Override + public NamedAnalyzer indexAnalyzer(String field, Function unindexedFieldAnalyzer) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean containsBrokenAnalysis(String field) { + throw new UnsupportedOperationException(); + } + + @Override + public long version() { + throw new UnsupportedOperationException(); + } + + @Override + public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { + throw new UnsupportedOperationException(); + } + + @Override + public ParserContext parserContext() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isMetadataField(String field) { + throw new UnsupportedOperationException(); + } + + @Override + public IndexAnalyzers getIndexAnalyzers() { + throw new UnsupportedOperationException(); + } + + @Override + public NestedDocuments getNestedDocuments(Function filterProducer) { + throw new UnsupportedOperationException(); + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 8b9022d3ef47e..0630d692e708e 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -101,6 +101,7 @@ import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.MultiBucketConsumerService.MultiBucketConsumer; @@ -143,6 +144,7 @@ import static org.elasticsearch.test.InternalAggregationTestCase.DEFAULT_MAX_BUCKETS; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; @@ -279,6 +281,7 @@ public boolean shouldCache(Query query) { MapperService.Snapshot mapperSnapshot = mapperSnapshotMock(); when(mapperSnapshot.hasNested()).thenReturn(false); + when(mapperSnapshot.getNestedDocuments(any())).thenReturn(new NestedDocuments(null, null)); QueryShardContext queryShardContext = queryShardContextMock( contextIndexSearcher, mapperSnapshot, diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java index 2e3fbb7936cc3..1d7e57704d041 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.StubSnapshot; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.TermQueryBuilder; @@ -585,7 +586,7 @@ null, null, mapperSnapshot, null, null, xContentRegistry(), writableRegistry(), } private void runTestOnIndices(int numberIndices, CheckedConsumer, Exception> body) throws Exception { - MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot(fieldName -> { + MapperService.Snapshot mapperSnapshot = new StubSnapshot(fieldName -> { if (fieldName.equals(MISSING_FIELD_NAME)) { return null; } else { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java index af8e313941188..75d8c396cc560 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.StubSnapshot; import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.TermsQueryBuilder; @@ -65,9 +66,7 @@ public class SecurityIndexReaderWrapperIntegrationTests extends AbstractBuilderT public void testDLS() throws Exception { ShardId shardId = new ShardId("_index", "_na_", 0); - MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot( - fieldName -> new KeywordFieldMapper.KeywordFieldType(fieldName) - ); + MapperService.Snapshot mapperSnapshot = new StubSnapshot(fieldName -> new KeywordFieldMapper.KeywordFieldType(fieldName)); ScriptService scriptService = mock(ScriptService.class); final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); @@ -174,9 +173,7 @@ protected IndicesAccessControl getIndicesAccessControl() { public void testDLSWithLimitedPermissions() throws Exception { ShardId shardId = new ShardId("_index", "_na_", 0); - MapperService.Snapshot mapperSnapshot = new MapperService.StubSnapshot( - fieldName -> new KeywordFieldMapper.KeywordFieldType(fieldName) - ); + MapperService.Snapshot mapperSnapshot = new StubSnapshot(fieldName -> new KeywordFieldMapper.KeywordFieldType(fieldName)); ScriptService scriptService = mock(ScriptService.class); final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); From 35873854c68b2aa71e11acc879ab3e2e3815c19e Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 9 Dec 2020 13:47:07 -0500 Subject: [PATCH 09/47] ooops: --- .../aggregations/AggregatorTestCase.java | 31 ------------------- 1 file changed, 31 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index f63a67a87194f..2edc123f98994 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -368,37 +368,6 @@ protected MapperService.Snapshot mapperSnapshotMock() { } /** -<<<<<<< HEAD - * sub-tests that need a more complex mock can overwrite this - */ - protected QueryShardContext queryShardContextMock(IndexSearcher searcher, - MapperService.Snapshot mapperSnapshot, - IndexSettings indexSettings, - CircuitBreakerService circuitBreakerService, - BitsetFilterCache bitsetFilterCache, - BigArrays bigArrays) { - - return new QueryShardContext(0, indexSettings, bigArrays, bitsetFilterCache, - getIndexFieldDataLookup(indexSettings.getIndex().getName(), circuitBreakerService), - mapperSnapshot, null, getMockScriptService(), xContentRegistry(), - writableRegistry(), null, searcher, System::currentTimeMillis, null, null, () -> true, - valuesSourceRegistry, emptyMap()); - } - - /** - * Sub-tests that need a more complex index field data provider can override this - */ - protected TriFunction, IndexFieldData> getIndexFieldDataLookup( - String indexName, - CircuitBreakerService circuitBreakerService - ) { - return (fieldType, s, searchLookup) -> fieldType.fielddataBuilder(indexName, searchLookup) - .build(new IndexFieldDataCache.None(), circuitBreakerService); - } - - /** -======= ->>>>>>> master * Sub-tests that need scripting can override this method to provide a script service and pre-baked scripts */ protected ScriptService getMockScriptService() { From 82b2a53c6b5d927473ccfd86f1d7138310c9b365 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 9 Dec 2020 13:55:10 -0500 Subject: [PATCH 10/47] Compile plz --- .../org/elasticsearch/index/query/QueryShardContextTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java index ac95e8b370bb6..c144422e3d26d 100644 --- a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java @@ -413,7 +413,8 @@ private static MapperService.Snapshot createMapperSnapshot( null, indexAnalyzers, indexSettings, - () -> true + () -> true, + false ); return new StubSnapshot(fieldTypeLookup) { @Override From d30da9aae2a5e3a5d6c8238597d2fb93aa971c24 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 9 Dec 2020 14:28:30 -0500 Subject: [PATCH 11/47] Fix test --- .../search/aggregations/AggregatorTestCase.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 2edc123f98994..35c133217201a 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -44,6 +44,7 @@ import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.Accountable; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.NumericUtils; import org.elasticsearch.Version; @@ -269,11 +270,18 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche s, searchLookup) -> fieldType.fielddataBuilder(indexSettings.getIndex().getName(), searchLookup) .build(new IndexFieldDataCache.None(), breakerService); + BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(indexSettings, new BitsetFilterCache.Listener() { + @Override + public void onRemoval(ShardId shardId, Accountable accountable) {} + + @Override + public void onCache(ShardId shardId, Accountable accountable) {} + }); QueryShardContext queryShardContext = new QueryShardContext( 0, indexSettings, bigArrays, - null, + bitsetFilterCache, fieldDataBuilder, mapperSnapshot, null, @@ -291,7 +299,6 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche ); MultiBucketConsumer consumer = new MultiBucketConsumer(maxBucket, breakerService.getBreaker(CircuitBreaker.REQUEST)); - BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(indexSettings, mock(Listener.class)); return new ProductionAggregationContext( queryShardContext, query, From bcd8e52922f93dd3b6022182c2ba69def0c2c5f2 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 9 Dec 2020 15:14:46 -0500 Subject: [PATCH 12/47] Checkstyle --- .../elasticsearch/search/aggregations/AggregatorTestCase.java | 1 - 1 file changed, 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 35c133217201a..a6c5511e71b38 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -72,7 +72,6 @@ import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; -import org.elasticsearch.index.cache.bitset.BitsetFilterCache.Listener; import org.elasticsearch.index.cache.query.DisabledQueryCache; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexFieldDataCache; From 81b9824e5ea80d26faf9c7ac19bc115e259203ea Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 14 Dec 2020 12:26:43 -0500 Subject: [PATCH 13/47] WIP --- .../join/mapper/MetaJoinFieldMapper.java | 2 +- .../ChildrenToParentAggregatorTests.java | 43 +- .../ParentToChildrenAggregatorTests.java | 37 +- .../org/elasticsearch/index/IndexService.java | 23 +- .../index/cache/bitset/BitsetFilterCache.java | 4 +- .../index/mapper/DocumentMapper.java | 97 +-- .../index/mapper/FieldMapper.java | 2 +- .../index/mapper/FieldTypeLookup.java | 28 +- .../index/mapper/MapperService.java | 553 +++--------------- .../index/mapper/MappingLookup.java | 158 ++++- .../index/query/QueryShardContext.java | 73 +-- .../elasticsearch/indices/IndicesService.java | 2 +- .../elasticsearch/search/NestedDocuments.java | 20 +- .../search/internal/ShardSearchRequest.java | 5 +- .../MetadataRolloverServiceTests.java | 12 +- .../MetadataCreateIndexServiceTests.java | 2 +- .../index/mapper/DateFieldTypeTests.java | 6 +- .../mapper/DocumentFieldMapperTests.java | 139 ----- .../FieldAliasMapperValidationTests.java | 67 ++- .../mapper/FieldNamesFieldTypeTests.java | 6 +- .../index/mapper/FieldTypeLookupTests.java | 31 +- .../index/mapper/IndexFieldTypeTests.java | 2 +- .../index/mapper/MapperServiceTests.java | 16 +- .../index/mapper/MappingLookupTests.java | 137 ++++- .../index/mapper/NestedDocumentsTests.java | 6 +- .../index/mapper/NestedObjectMapperTests.java | 91 ++- .../index/mapper/NumberFieldTypeTests.java | 2 +- .../index/mapper/RangeFieldTypeTests.java | 2 +- .../index/query/QueryShardContextTests.java | 214 +++---- .../index/query/RangeQueryRewriteTests.java | 50 +- .../bucket/nested/NestedAggregatorTests.java | 30 +- .../nested/ReverseNestedAggregatorTests.java | 19 +- .../SignificantTermsAggregatorTests.java | 14 - .../terms/SignificantTextAggregatorTests.java | 28 - .../fetch/subphase/FieldFetcherTests.java | 3 +- .../highlight/HighlightBuilderTests.java | 2 +- .../internal/ShardSearchRequestTests.java | 6 +- .../rescore/QueryRescorerBuilderTests.java | 4 +- .../search/sort/AbstractSortTestCase.java | 2 +- .../AbstractSuggestionBuilderTestCase.java | 17 +- .../index/mapper/MappingLookupUtils.java | 47 ++ .../index/mapper/StubSnapshot.java | 145 ----- .../aggregations/AggregatorTestCase.java | 80 +-- .../test/AbstractBuilderTestCase.java | 24 +- .../DocumentSubsetBitsetCacheTests.java | 29 +- ...ityIndexReaderWrapperIntegrationTests.java | 18 +- .../mapper/FlattenedFieldLookupTests.java | 10 +- .../job/RollupIndexerIndexingTests.java | 2 +- .../mapper/WildcardFieldMapperTests.java | 2 +- 49 files changed, 953 insertions(+), 1359 deletions(-) delete mode 100644 server/src/test/java/org/elasticsearch/index/mapper/DocumentFieldMapperTests.java create mode 100644 test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java delete mode 100644 test/framework/src/main/java/org/elasticsearch/index/mapper/StubSnapshot.java diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/mapper/MetaJoinFieldMapper.java b/modules/parent-join/src/main/java/org/elasticsearch/join/mapper/MetaJoinFieldMapper.java index decb3fd2e2ca2..849ae094cab1f 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/mapper/MetaJoinFieldMapper.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/mapper/MetaJoinFieldMapper.java @@ -49,7 +49,7 @@ public static class MetaJoinFieldType extends StringFieldType { private final String joinField; - private MetaJoinFieldType(String joinField) { + public MetaJoinFieldType(String joinField) { super(NAME, false, false, false, TextSearchInfo.SIMPLE_MATCH_ONLY, Collections.emptyMap()); this.joinField = joinField; } diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java index 4d37c2c76e30a..cc3753f9c0c40 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java @@ -38,7 +38,6 @@ import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.shard.ShardId; @@ -66,9 +65,6 @@ import java.util.TreeMap; import java.util.function.Consumer; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - public class ChildrenToParentAggregatorTests extends AggregatorTestCase { private static final String CHILD_TYPE = "child_type"; @@ -269,25 +265,6 @@ private static SortedDocValuesField createJoinField(String parentType, String id return new SortedDocValuesField("join_field#" + parentType, new BytesRef(id)); } - @Override - protected MapperService.Snapshot mapperSnapshotMock() { - ParentJoinFieldMapper joinFieldMapper = createJoinFieldMapper(); - MetaJoinFieldMapper.MetaJoinFieldType metaJoinFieldType = mock(MetaJoinFieldMapper.MetaJoinFieldType.class); - MapperService.Snapshot mapperSnapshot = mock(MapperService.Snapshot.class); - when(metaJoinFieldType.getJoinField()).thenReturn("join_field"); - when(mapperSnapshot.fieldType("_parent_join")).thenReturn(metaJoinFieldType); - when(mapperSnapshot.fieldType("join_field")).thenReturn(joinFieldMapper.fieldType()); - when(mapperSnapshot.fieldType("join_field#" + PARENT_TYPE)) - .thenReturn(new ParentIdFieldMapper.ParentIdFieldType("join_field#" + PARENT_TYPE, false)); - return mapperSnapshot; - } - - private static ParentJoinFieldMapper createJoinFieldMapper() { - return new ParentJoinFieldMapper.Builder("join_field") - .addRelation(PARENT_TYPE, Collections.singleton(CHILD_TYPE)) - .build(new ContentPath(0)); - } - private void testCase(Query query, IndexSearcher indexSearcher, Consumer verify) throws IOException { @@ -295,7 +272,7 @@ private void testCase(Query query, IndexSearcher indexSearcher, Consumer getSearchPlugins() { return Collections.singletonList(new ParentJoinPlugin()); diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregatorTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregatorTests.java index b72911f3eb316..d6bfa71af290d 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregatorTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ParentToChildrenAggregatorTests.java @@ -36,17 +36,12 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.index.Index; -import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.join.ParentJoinPlugin; -import org.elasticsearch.join.mapper.MetaJoinFieldMapper; -import org.elasticsearch.join.mapper.ParentIdFieldMapper; -import org.elasticsearch.join.mapper.ParentJoinFieldMapper; import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregatorTestCase; @@ -63,9 +58,8 @@ import java.util.Map; import java.util.function.Consumer; +import static org.elasticsearch.join.aggregations.ChildrenToParentAggregatorTests.withJoinFields; import static org.hamcrest.Matchers.equalTo; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class ParentToChildrenAggregatorTests extends AggregatorTestCase { @@ -159,8 +153,12 @@ public void testParentChildAsSubAgg() throws IOException { expectedOddMin = Math.min(expectedOddMin, e.getValue().v2()); } } - StringTerms result = - searchAndReduce(indexSearcher, new MatchAllDocsQuery(), request, longField("number"), keywordField("kwd")); + StringTerms result = searchAndReduce( + indexSearcher, + new MatchAllDocsQuery(), + request, + withJoinFields(longField("number"), keywordField("kwd")) + ); StringTerms.Bucket evenBucket = result.getBucketByKey("even"); InternalChildren evenChildren = evenBucket.getAggregations().get("children"); @@ -221,25 +219,6 @@ private static SortedDocValuesField createJoinField(String parentType, String id return new SortedDocValuesField("join_field#" + parentType, new BytesRef(id)); } - @Override - protected MapperService.Snapshot mapperSnapshotMock() { - ParentJoinFieldMapper joinFieldMapper = createJoinFieldMapper(); - MetaJoinFieldMapper.MetaJoinFieldType metaJoinFieldType = mock(MetaJoinFieldMapper.MetaJoinFieldType.class); - MapperService.Snapshot mapperSnapshot = mock(MapperService.Snapshot.class); - when(metaJoinFieldType.getJoinField()).thenReturn("join_field"); - when(mapperSnapshot.fieldType("_parent_join")).thenReturn(metaJoinFieldType); - when(mapperSnapshot.fieldType("join_field")).thenReturn(joinFieldMapper.fieldType()); - when(mapperSnapshot.fieldType("join_field#" + PARENT_TYPE)) - .thenReturn(new ParentIdFieldMapper.ParentIdFieldType("join_field#" + PARENT_TYPE, false)); - return mapperSnapshot; - } - - private static ParentJoinFieldMapper createJoinFieldMapper() { - return new ParentJoinFieldMapper.Builder("join_field") - .addRelation(PARENT_TYPE, Collections.singleton(CHILD_TYPE)) - .build(new ContentPath(0)); - } - private void testCase(Query query, IndexSearcher indexSearcher, Consumer verify) throws IOException { @@ -247,7 +226,7 @@ private void testCase(Query query, IndexSearcher indexSearcher, Consumer w.write(mappingSource.compressed())); final Collection deleteTombstoneMetadataFields = Arrays.asList(VersionFieldMapper.NAME, IdFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); @@ -185,10 +183,6 @@ public IndexFieldMapper IndexFieldMapper() { return metadataMapper(IndexFieldMapper.class); } - public boolean hasNestedObjects() { - return mappers().hasNested(); - } - public MappingLookup mappers() { return this.fieldMappers; } @@ -212,91 +206,6 @@ public ParsedDocument createNoopTombstoneDoc(String index, String reason) throws return parsedDoc; } - /** - * Given an object path, checks to see if any of its parents are non-nested objects - */ - public boolean hasNonNestedParent(String path) { - ObjectMapper mapper = mappers().objectMappers().get(path); - if (mapper == null) { - return false; - } - while (mapper != null) { - if (mapper.nested().isNested() == false) { - return true; - } - if (path.contains(".") == false) { - return false; - } - path = path.substring(0, path.lastIndexOf(".")); - mapper = mappers().objectMappers().get(path); - } - return false; - } - - /** - * Returns all nested object mappers - */ - public List getNestedMappers() { - List childMappers = new ArrayList<>(); - for (ObjectMapper mapper : mappers().objectMappers().values()) { - if (mapper.nested().isNested() == false) { - continue; - } - childMappers.add(mapper); - } - return childMappers; - } - - /** - * Returns all nested object mappers which contain further nested object mappers - * - * Used by BitSetProducerWarmer - */ - public List getNestedParentMappers() { - List parents = new ArrayList<>(); - for (ObjectMapper mapper : mappers().objectMappers().values()) { - String nestedParentPath = getNestedParent(mapper.fullPath()); - if (nestedParentPath == null) { - continue; - } - ObjectMapper parent = mappers().objectMappers().get(nestedParentPath); - if (parent.nested().isNested()) { - parents.add(parent); - } - } - return parents; - } - - /** - * Given a nested object path, returns the path to its nested parent - * - * In particular, if a nested field `foo` contains an object field - * `bar.baz`, then calling this method with `foo.bar.baz` will return - * the path `foo`, skipping over the object-but-not-nested `foo.bar` - */ - public String getNestedParent(String path) { - ObjectMapper mapper = mappers().objectMappers().get(path); - if (mapper == null) { - return null; - } - if (path.contains(".") == false) { - return null; - } - do { - path = path.substring(0, path.lastIndexOf(".")); - mapper = mappers().objectMappers().get(path); - if (mapper == null) { - return null; - } - if (mapper.nested().isNested()) { - return path; - } - if (path.contains(".") == false) { - return null; - } - } while(true); - } - public DocumentMapper merge(Mapping mapping, MergeReason reason) { Mapping merged = this.mapping.merge(mapping, reason); return new DocumentMapper(this.indexSettings, this.indexAnalyzers, this.documentParser, merged); @@ -310,7 +219,7 @@ public void validate(IndexSettings settings, boolean checkLimits) { + "required for partitioned index [" + settings.getIndex().getName() + "]"); } } - if (settings.getIndexSortConfig().hasIndexSort() && hasNestedObjects()) { + if (settings.getIndexSortConfig().hasIndexSort() && fieldMappers.hasNested()) { throw new IllegalArgumentException("cannot have nested fields when index sort is activated"); } if (checkLimits) { diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java index b90fb8ff8fe81..afc1eb2baf6c3 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java @@ -309,7 +309,7 @@ protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, protected abstract String contentType(); - public final Map indexAnalyzers() { + public Map indexAnalyzers() { return indexAnalyzers; } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java index aeb7f75001954..487eb7a68ce4f 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java @@ -32,7 +32,7 @@ /** * An immutable container for looking up {@link MappedFieldType}s by their name. */ -final class FieldTypeLookup { +public final class FieldTypeLookup { private final Map fullNameToFieldType = new HashMap<>(); /** @@ -43,11 +43,16 @@ final class FieldTypeLookup { * For convenience, the set of copied fields includes the field itself. */ private final Map> fieldToCopiedFields = new HashMap<>(); + private final String type; private final DynamicKeyFieldTypeLookup dynamicKeyLookup; - FieldTypeLookup(Collection fieldMappers, - Collection fieldAliasMappers, - Collection runtimeFieldTypes) { + FieldTypeLookup( + String type, + Collection fieldMappers, + Collection fieldAliasMappers, + Collection runtimeFieldTypes + ) { + this.type = type; Map dynamicKeyMappers = new HashMap<>(); for (FieldMapper fieldMapper : fieldMappers) { @@ -88,7 +93,11 @@ final class FieldTypeLookup { /** * Returns the mapped field type for the given field name. */ - MappedFieldType get(String field) { + public MappedFieldType get(String field) { + if (field.equals(TypeFieldType.NAME)) { + return new TypeFieldType(type); + } + MappedFieldType fieldType = fullNameToFieldType.get(field); if (fieldType != null) { return fieldType; @@ -124,7 +133,10 @@ Set simpleMatchToFullName(String pattern) { * should be a concrete field and *not* an alias. * @return A set of paths in the _source that contain the field's values. */ - Set sourcePaths(String field) { + public Set sourcePaths(String field) { + if (isEmpty()) { + return Set.of(); + } String resolvedField = field; int lastDotIndex = field.lastIndexOf('.'); if (lastDotIndex > 0) { @@ -149,4 +161,8 @@ Iterable filter(Predicate predicate) { return () -> Stream.concat(fullNameToFieldType.values().stream(), dynamicKeyLookup.fieldTypes()) .distinct().filter(predicate).iterator(); } + + public boolean isEmpty() { + return fullNameToFieldType.isEmpty(); + } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 4886ec89e5ec4..b1a823f0456e7 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -20,15 +20,12 @@ package org.elasticsearch.index.mapper; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.join.BitSetProducer; import org.elasticsearch.Assertions; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.MappingMetadata; import org.elasticsearch.common.Strings; import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; @@ -47,17 +44,16 @@ import org.elasticsearch.index.analysis.ReloadableCustomAnalyzer; import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.index.analysis.TokenizerFactory; -import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.script.ScriptService; -import org.elasticsearch.search.NestedDocuments; import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -66,10 +62,6 @@ import java.util.function.Function; import java.util.function.Supplier; -import static java.util.Collections.emptySet; -import static java.util.Collections.singleton; -import static java.util.Objects.requireNonNull; - public class MapperService extends AbstractIndexComponent implements Closeable { /** @@ -116,10 +108,7 @@ public enum MergeReason { private final MapperRegistry mapperRegistry; private final Supplier parserContextSupplier; - /** - * The current mapping accessed through {@link #snapshot()} and {@link #indexAnalyzer(String, Function)}. - */ - private volatile AbstractSnapshot snapshot = new EmptySnapshot(this); + private volatile DocumentMapper mapper; public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, NamedXContentRegistry xContentRegistry, SimilarityService similarityService, MapperRegistry mapperRegistry, @@ -141,21 +130,8 @@ public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, this::getMetadataMappers, parserContextSupplier, metadataMapperParsers); } - /** - * {@code volatile} read of an immutable snapshot of the current mapping. - */ - public Snapshot snapshot() { - return snapshot; - } - - /** - * Does this index contain nested documents? - * @deprecated Get a {@link #snapshot} and call {@link Snapshot#hasNested} on - * it as many times as needed. - */ - @Deprecated public boolean hasNested() { - return snapshot.hasNested(); + return lookup().hasNested(); } public IndexAnalyzers getIndexAnalyzers() { @@ -175,7 +151,20 @@ DocumentParser documentParser() { } Map, MetadataFieldMapper> getMetadataMappers() { - return snapshot.getMetadataMappers(); + final DocumentMapper existingMapper = mapper; + final Map metadataMapperParsers = + mapperRegistry.getMetadataMapperParsers(indexSettings.getIndexVersionCreated()); + Map, MetadataFieldMapper> metadataMappers = new LinkedHashMap<>(); + if (existingMapper == null) { + for (MetadataFieldMapper.TypeParser parser : metadataMapperParsers.values()) { + MetadataFieldMapper metadataFieldMapper = parser.getDefault(parserContext()); + metadataMappers.put(metadataFieldMapper.getClass(), metadataFieldMapper); + } + + } else { + metadataMappers.putAll(existingMapper.mapping().metadataMappersMap); + } + return metadataMappers; } /** @@ -196,48 +185,45 @@ public boolean updateMapping(final IndexMetadata currentIndexMetadata, final Ind + " but was " + newIndexMetadata.getIndex(); if (currentIndexMetadata != null && currentIndexMetadata.getMappingVersion() == newIndexMetadata.getMappingVersion()) { - assertMappingVersion(currentIndexMetadata, newIndexMetadata, snapshot); + assertMappingVersion(currentIndexMetadata, newIndexMetadata, this.mapper); return false; } - final AbstractSnapshot updatedSnapshot; + final DocumentMapper updatedMapper; try { - updatedSnapshot = internalMerge(newIndexMetadata, MergeReason.MAPPING_RECOVERY); + updatedMapper = internalMerge(newIndexMetadata, MergeReason.MAPPING_RECOVERY); } catch (Exception e) { logger.warn(() -> new ParameterizedMessage("[{}] failed to apply mappings", index()), e); throw e; } - if (updatedSnapshot == null) { + if (updatedMapper == null) { return false; } boolean requireRefresh = false; - assertMappingVersion(currentIndexMetadata, newIndexMetadata, updatedSnapshot); + assertMappingVersion(currentIndexMetadata, newIndexMetadata, updatedMapper); MappingMetadata mappingMetadata = newIndexMetadata.mapping(); CompressedXContent incomingMappingSource = mappingMetadata.source(); + String op = mapper != null ? "updated" : "added"; if (logger.isDebugEnabled() && incomingMappingSource.compressed().length < 512) { - logger.debug("[{}] {} mapping, source [{}]", index(), snapshot.updateOperationName(), incomingMappingSource.string()); + logger.debug("[{}] {} mapping, source [{}]", index(), op, incomingMappingSource.string()); } else if (logger.isTraceEnabled()) { - logger.trace("[{}] {} mapping, source [{}]", index(), snapshot.updateOperationName(), incomingMappingSource.string()); + logger.trace("[{}] {} mapping, source [{}]", index(), op, incomingMappingSource.string()); } else { - logger.debug( - "[{}] {} mapping (source suppressed due to length, use TRACE level if needed)", - index(), - snapshot.updateOperationName() - ); + logger.debug("[{}] {} mapping (source suppressed due to length, use TRACE level if needed)", + index(), op); } // refresh mapping can happen when the parsing/merging of the mapping from the metadata doesn't result in the same // mapping, in this case, we send to the master to refresh its own version of the mappings (to conform with the // merge version of it, which it does when refreshing the mappings), and warn log it. - DocumentMapper documentMapper = snapshot.documentMapper(); - if (documentMapper.mappingSource().equals(incomingMappingSource) == false) { + if (documentMapper().mappingSource().equals(incomingMappingSource) == false) { logger.debug("[{}] parsed mapping, and got different sources\noriginal:\n{}\nparsed:\n{}", - index(), incomingMappingSource, documentMapper.mappingSource()); + index(), incomingMappingSource, documentMapper().mappingSource()); requireRefresh = true; } @@ -247,11 +233,11 @@ public boolean updateMapping(final IndexMetadata currentIndexMetadata, final Ind private void assertMappingVersion( final IndexMetadata currentIndexMetadata, final IndexMetadata newIndexMetadata, - final AbstractSnapshot updatedSnapshot) throws IOException { + final DocumentMapper updatedMapper) throws IOException { if (Assertions.ENABLED && currentIndexMetadata != null) { if (currentIndexMetadata.getMappingVersion() == newIndexMetadata.getMappingVersion()) { // if the mapping version is unchanged, then there should not be any updates and all mappings should be the same - assert updatedSnapshot == snapshot; + assert updatedMapper == mapper; MappingMetadata mapping = newIndexMetadata.mapping(); if (mapping != null) { @@ -260,7 +246,7 @@ private void assertMappingVersion( assert currentSource.equals(newSource) : "expected current mapping [" + currentSource + "] for type [" + mapping.type() + "] " + "to be the same as new mapping [" + newSource + "]"; - final CompressedXContent mapperSource = new CompressedXContent(Strings.toString(snapshot.documentMapper())); + final CompressedXContent mapperSource = new CompressedXContent(Strings.toString(mapper)); assert currentSource.equals(mapperSource) : "expected current mapping [" + currentSource + "] for type [" + mapping.type() + "] " + "to be the same as new mapping [" + mapperSource + "]"; @@ -273,11 +259,11 @@ private void assertMappingVersion( assert currentMappingVersion < newMappingVersion : "expected current mapping version [" + currentMappingVersion + "] " + "to be less than new mapping version [" + newMappingVersion + "]"; - assert updatedSnapshot != null; + assert updatedMapper != null; final MappingMetadata currentMapping = currentIndexMetadata.mapping(); if (currentMapping != null) { final CompressedXContent currentSource = currentMapping.source(); - final CompressedXContent newSource = updatedSnapshot.documentMapper().mappingSource(); + final CompressedXContent newSource = updatedMapper.mappingSource(); assert currentSource.equals(newSource) == false : "expected current mapping [" + currentSource + "] to be different than new mapping [" + newSource + "]"; } @@ -295,11 +281,10 @@ public void merge(IndexMetadata indexMetadata, MergeReason reason) { } public DocumentMapper merge(String type, CompressedXContent mappingSource, MergeReason reason) { - AbstractSnapshot updatedSnapshot = internalMerge(type, mappingSource, reason); - return updatedSnapshot == null ? null : updatedSnapshot.documentMapper(); // TODO return Snapshot here + return internalMerge(type, mappingSource, reason); } - private synchronized AbstractSnapshot internalMerge(IndexMetadata indexMetadata, MergeReason reason) { + private synchronized DocumentMapper internalMerge(IndexMetadata indexMetadata, MergeReason reason) { assert reason != MergeReason.MAPPING_UPDATE_PREFLIGHT; MappingMetadata mappingMetadata = indexMetadata.mapping(); if (mappingMetadata != null) { @@ -308,7 +293,7 @@ private synchronized AbstractSnapshot internalMerge(IndexMetadata indexMetadata, return null; } - private synchronized AbstractSnapshot internalMerge(String type, CompressedXContent mappings, MergeReason reason) { + private synchronized DocumentMapper internalMerge(String type, CompressedXContent mappings, MergeReason reason) { DocumentMapper documentMapper; @@ -321,30 +306,36 @@ private synchronized AbstractSnapshot internalMerge(String type, CompressedXCont return internalMerge(documentMapper, reason); } - private synchronized AbstractSnapshot internalMerge(DocumentMapper mapper, MergeReason reason) { + private synchronized DocumentMapper internalMerge(DocumentMapper mapper, MergeReason reason) { assert mapper != null; // compute the merged DocumentMapper - MappedSnapshot newSnapshot = this.snapshot.merge(mapper, reason); - newSnapshot.documentMapper().root().fixRedundantIncludes(); - newSnapshot.documentMapper().validate(indexSettings, reason != MergeReason.MAPPING_RECOVERY); + DocumentMapper oldMapper = this.mapper; + DocumentMapper newMapper; + if (oldMapper != null) { + newMapper = oldMapper.merge(mapper.mapping(), reason); + } else { + newMapper = mapper; + } + newMapper.root().fixRedundantIncludes(); + newMapper.validate(indexSettings, reason != MergeReason.MAPPING_RECOVERY); if (reason == MergeReason.MAPPING_UPDATE_PREFLIGHT) { - return newSnapshot; + return newMapper; } // commit the change - this.snapshot = newSnapshot; - assert assertSerialization(newSnapshot); + this.mapper = newMapper; + assert assertSerialization(newMapper); - return newSnapshot; + return newMapper; } - private boolean assertSerialization(MappedSnapshot snapshot) { + private boolean assertSerialization(DocumentMapper mapper) { // capture the source now, it may change due to concurrent parsing - final CompressedXContent mappingSource = snapshot.mapper.mappingSource(); - DocumentMapper newMapper = parse(snapshot.mapper.type(), mappingSource); + final CompressedXContent mappingSource = mapper.mappingSource(); + DocumentMapper newMapper = parse(mapper.type(), mappingSource); if (newMapper.mappingSource().equals(mappingSource) == false) { throw new IllegalStateException("DocumentMapper serialization result is different from source. \n--> Source [" @@ -358,9 +349,11 @@ public DocumentMapper parse(String mappingType, CompressedXContent mappingSource return documentMapperParser.parse(mappingType, mappingSource); } - @Deprecated + /** + * Return the document mapper, or {@code null} if no mapping has been put yet. + */ public DocumentMapper documentMapper() { - return snapshot.documentMapper(); + return mapper; } /** @@ -381,9 +374,8 @@ public static boolean isMappingSourceTyped(String type, Map mapp */ private String resolveDocumentType(String type) { if (MapperService.SINGLE_MAPPING_NAME.equals(type)) { - AbstractSnapshot currentSnapshot = snapshot; - if (currentSnapshot.documentMapper() != null) { - return currentSnapshot.documentMapper().type(); + if (mapper != null) { + return mapper.type(); } } return type; @@ -392,65 +384,56 @@ private String resolveDocumentType(String type) { /** * Returns the document mapper for this MapperService. If no mapper exists, * creates one and returns that. - * @deprecated Get a {@link #snapshot} and call {@link Snapshot#documentMapperWithAutoCreate} on - * it as many times as needed. */ - @Deprecated public DocumentMapperForType documentMapperWithAutoCreate() { - return snapshot.documentMapperWithAutoCreate(); + DocumentMapper mapper = documentMapper(); + if (mapper != null) { + return new DocumentMapperForType(mapper, null); + } + mapper = parse(SINGLE_MAPPING_NAME, null); + return new DocumentMapperForType(mapper, mapper.mapping()); } /** * Given the full name of a field, returns its {@link MappedFieldType}. - * @deprecated Get a {@link #snapshot} and call {@link Snapshot#fieldType} on - * it as many times as needed. */ - @Deprecated public MappedFieldType fieldType(String fullName) { - return snapshot.fieldType(fullName); + return lookup().fieldTypes().get(fullName); } /** * Returns all the fields that match the given pattern. If the pattern is prefixed with a type * then the fields will be returned with a type prefix. - * @deprecated Get a {@link #snapshot} and call {@link Snapshot#simpleMatchToFullName} on - * it as many times as needed. */ - @Deprecated public Set simpleMatchToFullName(String pattern) { - return snapshot.simpleMatchToFullName(pattern); + return lookup().simpleMatchToFullName(pattern); + } + + public MappingLookup lookup() { + DocumentMapper mapper = this.mapper; + return mapper == null ? MappingLookup.EMPTY : mapper.mappers(); } /** - * All mapped field types with eager global ordinals. - * @deprecated Get a {@link #snapshot} and call {@link Snapshot#getEagerGlobalOrdinalsFields} on - * it as many times as needed. + * Returns all mapped field types. */ - @Deprecated public Iterable getEagerGlobalOrdinalsFields() { - return snapshot.getEagerGlobalOrdinalsFields(); + return this.mapper == null ? Collections.emptySet() : + this.mapper.mappers().fieldTypes().filter(MappedFieldType::eagerGlobalOrdinals); } - /** - * Get the named object mapper. - * @deprecated Get a {@link #snapshot} and call {@link Snapshot#getObjectMapper} on - * it as many times as needed. - */ - @Deprecated public ObjectMapper getObjectMapper(String name) { - return snapshot.getObjectMapper(name); + return this.mapper == null ? null : this.mapper.mappers().objectMappers().get(name); } /** - * An analyzer that performs a volatile read on the mapping find correct {@link NamedAnalyzer} for the field. + * Return the index-time analyzer associated with a particular field * @param field the field name * @param unindexedFieldAnalyzer a function to return an Analyzer for a field with no * directly associated index-time analyzer - * @deprecated Prefer {@link #snapshot()} and then {@link Snapshot#indexAnalyzer}. */ - @Deprecated public NamedAnalyzer indexAnalyzer(String field, Function unindexedFieldAnalyzer) { - return snapshot.indexAnalyzer(field, unindexedFieldAnalyzer); + return lookup().indexAnalyzer(field, unindexedFieldAnalyzer); } @Override @@ -501,382 +484,4 @@ public synchronized List reloadSearchAnalyzers(AnalysisRegistry registry } return reloadedAnalyzers; } - - /** - * An immutable snapshot of the current mapping. - */ - public interface Snapshot { - /** - * Given the full name of a field, returns its {@link MappedFieldType}. - */ - MappedFieldType fieldType(String fullName); - - boolean hasNested(); - - /** - * Get the named object mapper - */ - ObjectMapper getObjectMapper(String name); - - /** - * Returns all the fields that match the given pattern. If the pattern is prefixed with a type - * then the fields will be returned with a type prefix. - */ - Set simpleMatchToFullName(String pattern); - - /** - * Given a field name, returns its possible paths in the _source. For example, - * the 'source path' for a multi-field is the path to its parent field. - */ - Set sourcePath(String fullName); - - /** - * The document mapper for this MapperService. If no mapper exists, - * creates one and returns that. - */ - DocumentMapperForType documentMapperWithAutoCreate(); - - /** - * All mapped field types with eager global ordinals. - */ - Iterable getEagerGlobalOrdinalsFields(); - - /** - * The index-time analyzer associated with a particular field. - * @param field the field name - * @param unindexedFieldAnalyzer a function to return an Analyzer for a field with no - * directly associated index-time analyzer - */ - NamedAnalyzer indexAnalyzer(String field, Function unindexedFieldAnalyzer); - - /** - * Does the index analyzer for this field have token filters that may produce - * backwards offsets in term vectors - */ - boolean containsBrokenAnalysis(String field); - - /** - * Current version of the of the mapping. Increments if the mapping - * changes locally. Distinct from - * {@link IndexMetadata#getMappingVersion()} because it purely - * considers the local mapping changes. - */ - long version(); - - ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException; - - /** - * The context used to parse field. - */ - ParserContext parserContext(); - - /** - * @return Whether a field is a metadata field. - * this method considers all mapper plugins - */ - boolean isMetadataField(String field); - - IndexAnalyzers getIndexAnalyzers(); - - /** - * Build a loader for nested documents. - */ - NestedDocuments getNestedDocuments(Function filterProducer); - - /** - * Are there mappings for this index? - */ - boolean hasMappings(); - - /** - * Is source enabled on this index? - */ - boolean sourceEnabled(); - } - - /** - * Superclass for {@link Snapshot} implementations hosted by {@link MapperService}. - * This contains a few package private methods which are important for how - * {@linkplain MapperService} handles snapshots but aren't exposed outside - * of it. - *

- * You may ask "Why have both the interface and the abstract class?" And that is - * a reasonable question. Mostly we have two things so there is an obvious - * separation of "internal" stuff and "external" stuff. Which makes testing - * a little simpler because most tests just rely on the {@linkplain Snapshot} - * interface. - */ - private abstract static class AbstractSnapshot implements Snapshot { - protected final MapperService mapperService; - - private AbstractSnapshot(MapperService mapperService) { - this.mapperService = requireNonNull(mapperService); - } - - /** - * The context used to parse field. - */ - public final ParserContext parserContext() { - // Safe to plumb through to the Snapshot because it is immutable - return mapperService.parserContext(); - } - - /** - * @return Whether a field is a metadata field. - * this method considers all mapper plugins - */ - public final boolean isMetadataField(String field) { - // Safe to plumb through to the Snapshot because it is immutable - return mapperService.isMetadataField(field); - } - - public final IndexAnalyzers getIndexAnalyzers() { - // Safe to plumb through to the Snapshot because it is immutable - return mapperService.getIndexAnalyzers(); - } - - abstract MappedSnapshot merge(DocumentMapper mapper, MergeReason reason); - - abstract Map, MetadataFieldMapper> getMetadataMappers(); - - abstract DocumentMapper documentMapper(); - - /** - * The name of the operation to log when merging new mappings. If the - * mapping is empty it'll be {@code add} and if it isn't then it'll be - * {@code update}. - */ - abstract String updateOperationName(); - } - - /** - * {@link Snapshot} of an "empty" mapping. - */ - private static class EmptySnapshot extends AbstractSnapshot { - EmptySnapshot(MapperService mapperService) { - super(mapperService); - } - - @Override - public MappedFieldType fieldType(String fullName) { - if (fullName.equals(TypeFieldType.NAME)) { - return new TypeFieldType("_doc"); - } - return null; - } - - @Override - public boolean hasNested() { - return false; - } - - @Override - public ObjectMapper getObjectMapper(String name) { - return null; - } - - @Override - public Set simpleMatchToFullName(String pattern) { - return Regex.isSimpleMatchPattern(pattern) ? emptySet() : singleton(pattern); - } - - @Override - public Set sourcePath(String fullName) { - return emptySet(); - } - - @Override - public DocumentMapperForType documentMapperWithAutoCreate() { - DocumentMapper auto = mapperService.parse(SINGLE_MAPPING_NAME, null); - return new DocumentMapperForType(auto, auto.mapping()); - } - - @Override - public Iterable getEagerGlobalOrdinalsFields() { - return emptySet(); - } - - @Override - public NamedAnalyzer indexAnalyzer(String field, Function unindexedFieldAnalyzer) { - return unindexedFieldAnalyzer.apply(field); - } - - @Override - public boolean containsBrokenAnalysis(String field) { - return false; - } - - @Override - public long version() { - return 0; - } - - @Override - public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { - return null; - } - - @Override - Map, MetadataFieldMapper> getMetadataMappers() { - Map metadataMapperParsers = mapperService.mapperRegistry.getMetadataMapperParsers( - mapperService.indexSettings.getIndexVersionCreated() - ); - Map, MetadataFieldMapper> metadataMappers = new LinkedHashMap<>(); - for (MetadataFieldMapper.TypeParser parser : metadataMapperParsers.values()) { - MetadataFieldMapper metadataFieldMapper = parser.getDefault(parserContext()); - metadataMappers.put(metadataFieldMapper.getClass(), metadataFieldMapper); - } - return metadataMappers; - } - - @Override - MappedSnapshot merge(DocumentMapper mapper, MergeReason reason) { - return new MappedSnapshot(mapperService, mapper, 1); - } - - @Override - protected String updateOperationName() { - return "added"; - } - - @Override - public NestedDocuments getNestedDocuments(Function filterProducer) { - return null; - } - - @Override - public boolean hasMappings() { - return false; - } - - @Override - public boolean sourceEnabled() { - return false; - } - - @Override - DocumentMapper documentMapper() { - return null; - } - } - - /** - * Snapshot of a non-empty mapping. - */ - private static class MappedSnapshot extends AbstractSnapshot { - private final DocumentMapper mapper; - /** - * Current version of the of the mapping. Increments if the mapping - * changes locally. Distinct from - * {@link IndexMetadata#getMappingVersion()} because it purely - * considers the local mapping changes. - */ - private final long version; - - MappedSnapshot(MapperService mapperService, DocumentMapper mapper, long version) { - super(mapperService); - this.mapper = requireNonNull(mapper); - this.version = version; - } - - @Override - public MappedFieldType fieldType(String fullName) { - if (fullName.equals(TypeFieldType.NAME)) { - return new TypeFieldType(mapper.type()); - } - return mapper.mappers().fieldTypes().get(fullName); - } - - @Override - public boolean hasNested() { - return mapper.hasNestedObjects(); - } - - @Override - public ObjectMapper getObjectMapper(String name) { - return mapper.mappers().objectMappers().get(name); - } - - @Override - public Set simpleMatchToFullName(String pattern) { - return Regex.isSimpleMatchPattern(pattern) ? mapper.mappers().fieldTypes().simpleMatchToFullName(pattern) : singleton(pattern); - } - - @Override - public Set sourcePath(String fullName) { - return mapper.mappers().fieldTypes().sourcePaths(fullName); - } - - @Override - public DocumentMapperForType documentMapperWithAutoCreate() { - return new DocumentMapperForType(mapper, null); - } - - @Override - public Iterable getEagerGlobalOrdinalsFields() { - return mapper.mappers().fieldTypes().filter(MappedFieldType::eagerGlobalOrdinals); - } - - @Override - public NamedAnalyzer indexAnalyzer(String field, Function unindexedFieldAnalyzer) { - return this.mapper.mappers().indexAnalyzer(field, unindexedFieldAnalyzer); - } - - @Override - public boolean containsBrokenAnalysis(String field) { - NamedAnalyzer analyzer = indexAnalyzer(field, u -> null); - if (analyzer == null) { - return false; - } - return analyzer.containsBrokenAnalysis(); - } - - @Override - public long version() { - return version; - } - - @Override - public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { - return mapper.parse(source); - } - - @Override - Map, MetadataFieldMapper> getMetadataMappers() { - return new LinkedHashMap<>(mapper.mapping().metadataMappersMap); - } - - @Override - MappedSnapshot merge(DocumentMapper newMapper, MergeReason reason) { - DocumentMapper merged = mapper.merge(newMapper.mapping(), reason); - if (merged.mappingSource().equals(mapper.mappingSource())) { - return this; - } - return new MappedSnapshot(mapperService, merged, version + 1); - } - - @Override - protected String updateOperationName() { - return "updated"; - } - - @Override - public NestedDocuments getNestedDocuments(Function filterProducer) { - return new NestedDocuments(mapper, filterProducer); - } - - @Override - public boolean hasMappings() { - return true; - } - - @Override - public boolean sourceEnabled() { - return mapper.sourceMapper().enabled(); - } - - DocumentMapper documentMapper() { - return mapper; - } - } } 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 ad68865ba101d..8897d3e7f1660 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.mapper; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.regex.Regex; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.NamedAnalyzer; @@ -28,19 +30,36 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.Function; import java.util.stream.Stream; public final class MappingLookup { + /** + * A lookup representing an empty mapping. + */ + public static final MappingLookup EMPTY = new MappingLookup( + "_doc", + List.of(), + List.of(), + List.of(), + List.of(), + 0, + souceToParse -> null, + writer -> writer.writeString("empty") + ); + /** Full field name to mapper */ private final Map fieldMappers; private final Map objectMappers; private final boolean hasNested; private final FieldTypeLookup fieldTypeLookup; private final int metadataFieldCount; + private final Function documentParser; private final Map indexAnalyzers = new HashMap<>(); + private final Writeable cacheKey; - public static MappingLookup fromMapping(Mapping mapping) { + public static MappingLookup fromMapping(Mapping mapping, Function documentParser, Writeable cacheKey) { List newObjectMappers = new ArrayList<>(); List newFieldMappers = new ArrayList<>(); List newFieldAliasMappers = new ArrayList<>(); @@ -52,8 +71,16 @@ public static MappingLookup fromMapping(Mapping mapping) { for (Mapper child : mapping.root) { collect(child, newObjectMappers, newFieldMappers, newFieldAliasMappers); } - return new MappingLookup(newFieldMappers, newObjectMappers, newFieldAliasMappers, - mapping.root.runtimeFieldTypes(), mapping.metadataMappers.length); + return new MappingLookup( + mapping.root().name(), + newFieldMappers, + newObjectMappers, + newFieldAliasMappers, + mapping.root.runtimeFieldTypes(), + mapping.metadataMappers.length, + documentParser, + cacheKey + ); } private static void collect(Mapper mapper, Collection objectMappers, @@ -74,11 +101,16 @@ private static void collect(Mapper mapper, Collection objectMapper } } - public MappingLookup(Collection mappers, + public MappingLookup(String type, + Collection mappers, Collection objectMappers, Collection aliasMappers, Collection runtimeFieldTypes, - int metadataFieldCount) { + int metadataFieldCount, + Function documentParser, + Writeable cacheKey) { + this.documentParser = documentParser; + this.cacheKey = cacheKey; Map fieldMappers = new HashMap<>(); Map objects = new HashMap<>(); @@ -113,7 +145,7 @@ public MappingLookup(Collection mappers, } } - this.fieldTypeLookup = new FieldTypeLookup(mappers, aliasMappers, runtimeFieldTypes); + this.fieldTypeLookup = new FieldTypeLookup(type, mappers, aliasMappers, runtimeFieldTypes); this.fieldMappers = Collections.unmodifiableMap(fieldMappers); this.objectMappers = Collections.unmodifiableMap(objects); @@ -129,7 +161,7 @@ public Mapper getMapper(String field) { return fieldMappers.get(field); } - FieldTypeLookup fieldTypes() { + public FieldTypeLookup fieldTypes() { return fieldTypeLookup; } @@ -147,7 +179,7 @@ public Iterable fieldMappers() { return fieldMappers.values(); } - public void checkLimits(IndexSettings settings) { + void checkLimits(IndexSettings settings) { checkFieldLimit(settings.getMappingTotalFieldsLimit()); checkObjectDepthLimit(settings.getMappingDepthLimit()); checkFieldNameLengthLimit(settings.getMappingFieldNameLengthLimit()); @@ -234,4 +266,114 @@ private static String parentObject(String field) { } return field.substring(0, lastDot); } + + public Set simpleMatchToFullName(String pattern) { + if (Regex.isSimpleMatchPattern(pattern) == false) { + // no wildcards + return Collections.singleton(pattern); + } + return fieldTypes().simpleMatchToFullName(pattern); + } + + public ParsedDocument parseDocument(SourceToParse source) { + return documentParser.apply(source); + } + + public boolean isEmpty() { + return objectMappers.isEmpty() && fieldTypeLookup.isEmpty(); + } + + public boolean isSourceEnabled() { + // NOCOMMIT implement me + return false; + } + + /** + * Returns all nested object mappers which contain further nested object mappers + * + * Used by BitSetProducerWarmer + */ + public List getNestedParentMappers() { + List parents = new ArrayList<>(); + for (ObjectMapper mapper : objectMappers.values()) { + String nestedParentPath = getNestedParent(mapper.fullPath()); + if (nestedParentPath == null) { + continue; + } + ObjectMapper parent = objectMappers.get(nestedParentPath); + if (parent.nested().isNested()) { + parents.add(parent); + } + } + return parents; + } + + /** + * Given a nested object path, returns the path to its nested parent + * + * In particular, if a nested field `foo` contains an object field + * `bar.baz`, then calling this method with `foo.bar.baz` will return + * the path `foo`, skipping over the object-but-not-nested `foo.bar` + */ + public String getNestedParent(String path) { + ObjectMapper mapper = objectMappers.get(path); + if (mapper == null) { + return null; + } + if (path.contains(".") == false) { + return null; + } + do { + path = path.substring(0, path.lastIndexOf(".")); + mapper = objectMappers.get(path); + if (mapper == null) { + return null; + } + if (mapper.nested().isNested()) { + return path; + } + if (path.contains(".") == false) { + return null; + } + } while(true); + } + + /** + * Returns all nested object mappers + */ + public List getNestedMappers() { + List childMappers = new ArrayList<>(); + for (ObjectMapper mapper : objectMappers.values()) { + if (mapper.nested().isNested() == false) { + continue; + } + childMappers.add(mapper); + } + return childMappers; + } + + /** + * Given an object path, checks to see if any of its parents are non-nested objects + */ + public boolean hasNonNestedParent(String path) { + ObjectMapper mapper = objectMappers.get(path); + if (mapper == null) { + return false; + } + while (mapper != null) { + if (mapper.nested().isNested() == false) { + return true; + } + if (path.contains(".") == false) { + return false; + } + path = path.substring(0, path.lastIndexOf(".")); + mapper = objectMappers.get(path); + } + return false; + } + + public Writeable cacheKey() { + return cacheKey; + } } diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index 7fa9fd8df43b9..1cbc8efa54b8d 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -30,11 +30,11 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.util.BigArrays; @@ -53,6 +53,7 @@ import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ObjectMapper; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.RuntimeFieldType; @@ -91,7 +92,8 @@ public class QueryShardContext extends QueryRewriteContext { private final ScriptService scriptService; private final IndexSettings indexSettings; private final BigArrays bigArrays; - private final MapperService.Snapshot mapperSnapshot; + private final MapperService mapperService; + private final MappingLookup mappingLookup; private final SimilarityService similarityService; private final BitsetFilterCache bitsetFilterCache; private final TriFunction, IndexFieldData> indexFieldDataService; @@ -120,7 +122,8 @@ public QueryShardContext( BigArrays bigArrays, BitsetFilterCache bitsetFilterCache, TriFunction, IndexFieldData> indexFieldDataLookup, - MapperService.Snapshot mapperSnapshot, + MapperService mapperService, + MappingLookup mappingLookup, SimilarityService similarityService, ScriptService scriptService, NamedXContentRegistry xContentRegistry, @@ -140,7 +143,8 @@ public QueryShardContext( bigArrays, bitsetFilterCache, indexFieldDataLookup, - mapperSnapshot, + mapperService, + mappingLookup, similarityService, scriptService, xContentRegistry, @@ -155,13 +159,13 @@ public QueryShardContext( ), allowExpensiveQueries, valuesSourceRegistry, - parseRuntimeMappings(runtimeMappings, mapperSnapshot) + parseRuntimeMappings(runtimeMappings, mapperService) ); } public QueryShardContext(QueryShardContext source) { this(source.shardId, source.indexSettings, source.bigArrays, source.bitsetFilterCache, source.indexFieldDataService, - source.mapperSnapshot, source.similarityService, source.scriptService, source.getXContentRegistry(), + source.mapperService, source.mappingLookup, source.similarityService, source.scriptService, source.getXContentRegistry(), source.getWriteableRegistry(), source.client, source.searcher, source.nowInMillis, source.indexNameMatcher, source.fullyQualifiedIndex, source.allowExpensiveQueries, source.valuesSourceRegistry, source.runtimeMappings); } @@ -171,7 +175,8 @@ private QueryShardContext(int shardId, BigArrays bigArrays, BitsetFilterCache bitsetFilterCache, TriFunction, IndexFieldData> indexFieldDataLookup, - MapperService.Snapshot mapperSnapshot, + MapperService mapperService, + MappingLookup mappingLookup, SimilarityService similarityService, ScriptService scriptService, NamedXContentRegistry xContentRegistry, @@ -187,7 +192,8 @@ private QueryShardContext(int shardId, super(xContentRegistry, namedWriteableRegistry, client, nowInMillis); this.shardId = shardId; this.similarityService = similarityService; - this.mapperSnapshot = mapperSnapshot; + this.mapperService = mapperService; + this.mappingLookup = mappingLookup; this.bigArrays = bigArrays; this.bitsetFilterCache = bitsetFilterCache; this.indexFieldDataService = indexFieldDataLookup; @@ -211,7 +217,7 @@ private void reset() { } public Similarity getSearchSimilarity() { - return similarityService != null ? similarityService.similarity(mapperSnapshot::fieldType) : null; + return similarityService != null ? similarityService.similarity(this::fieldType) : null; } public List defaultFields() { @@ -259,15 +265,15 @@ public Map copyNamedQueries() { * Parse a document with current mapping. */ public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { - return mapperSnapshot.parseDocument(source); + return mappingLookup.parseDocument(source); } public boolean hasNested() { - return mapperSnapshot.hasNested(); + return mappingLookup.hasNested(); } public boolean hasMappings() { - return mapperSnapshot.hasMappings(); + return false == mappingLookup.isEmpty(); } /** @@ -276,13 +282,13 @@ public boolean hasMappings() { */ public Set simpleMatchToIndexNames(String pattern) { if (runtimeMappings.isEmpty()) { - return mapperSnapshot.simpleMatchToFullName(pattern); + return mappingLookup.simpleMatchToFullName(pattern); } if (Regex.isSimpleMatchPattern(pattern) == false) { // no wildcards return Collections.singleton(pattern); } - Set matches = new HashSet<>(mapperSnapshot.simpleMatchToFullName(pattern)); + Set matches = new HashSet<>(mappingLookup.simpleMatchToFullName(pattern)); for (String name : runtimeMappings.keySet()) { if (Regex.simpleMatch(pattern, name)) { matches.add(name); @@ -313,23 +319,23 @@ public boolean isFieldMapped(String name) { private MappedFieldType fieldType(String name) { MappedFieldType fieldType = runtimeMappings.get(name); - return fieldType == null ? mapperSnapshot.fieldType(name) : fieldType; + return fieldType == null ? mappingLookup.fieldTypes().get(name) : fieldType; } public ObjectMapper getObjectMapper(String name) { - return mapperSnapshot.getObjectMapper(name); + return mappingLookup.objectMappers().get(name); } public boolean isMetadataField(String field) { - return mapperSnapshot.isMetadataField(field); + return mapperService.isMetadataField(field); } public Set sourcePath(String fullName) { - return mapperSnapshot.sourcePath(fullName); + return mappingLookup.fieldTypes().sourcePaths(fullName); } public boolean isSourceEnabled() { - return mapperSnapshot.sourceEnabled(); + return mappingLookup.isSourceEnabled(); } /** @@ -337,7 +343,7 @@ public boolean isSourceEnabled() { * Generally used to handle unmapped fields in the context of sorting. */ public MappedFieldType buildAnonymousFieldType(String type) { - Mapper.TypeParser.ParserContext parserContext = mapperSnapshot.parserContext(); + Mapper.TypeParser.ParserContext parserContext = mapperService.parserContext(); Mapper.TypeParser typeParser = parserContext.typeParser(type); if (typeParser == null) { throw new IllegalArgumentException("No mapper found for type [" + type + "]"); @@ -351,7 +357,7 @@ public MappedFieldType buildAnonymousFieldType(String type) { } public IndexAnalyzers getIndexAnalyzers() { - return mapperSnapshot.getIndexAnalyzers(); + return mapperService.getIndexAnalyzers(); } /** @@ -362,7 +368,7 @@ public Analyzer getIndexAnalyzer(Function unindexedFieldA return new DelegatingAnalyzerWrapper(Analyzer.PER_FIELD_REUSE_STRATEGY) { @Override protected Analyzer getWrappedAnalyzer(String fieldName) { - return mapperSnapshot.indexAnalyzer(fieldName, unindexedFieldAnalyzer); + return mappingLookup.indexAnalyzer(fieldName, unindexedFieldAnalyzer); } }; } @@ -383,7 +389,7 @@ MappedFieldType failIfFieldMappingNotFound(String name, MappedFieldType fieldMap if (fieldMapping != null || allowUnmappedFields) { return fieldMapping; } else if (mapUnmappedFieldAsString) { - TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name, mapperSnapshot.getIndexAnalyzers()); + TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name, getIndexAnalyzers()); return builder.build(new ContentPath(1)).fieldType(); } else { throw new QueryShardException(this, "No field mapping can be found for the field with name [{}]", name); @@ -395,7 +401,8 @@ MappedFieldType failIfFieldMappingNotFound(String name, MappedFieldType fieldMap * backwards offsets in term vectors */ public boolean containsBrokenAnalysis(String field) { - return mapperSnapshot.containsBrokenAnalysis(field); + NamedAnalyzer a = mappingLookup.indexAnalyzer(field, f -> null); + return a == null ? false : a.containsBrokenAnalysis(); } private SearchLookup lookup = null; @@ -578,13 +585,10 @@ public BigArrays bigArrays() { // TODO this is only used in agg land, maybe rem return bigArrays; } - private static Map parseRuntimeMappings( - Map runtimeMappings, - MapperService.Snapshot mapperSnapshot - ) { + private static Map parseRuntimeMappings(Map runtimeMappings, MapperService mapperService) { Map runtimeFieldTypes = new HashMap<>(); if (runtimeMappings.isEmpty() == false) { - RuntimeFieldType.parseRuntimeFields(new HashMap<>(runtimeMappings), mapperSnapshot.parserContext(), + RuntimeFieldType.parseRuntimeFields(new HashMap<>(runtimeMappings), mapperService.parserContext(), runtimeFieldType -> runtimeFieldTypes.put(runtimeFieldType.name(), runtimeFieldType)); } return Collections.unmodifiableMap(runtimeFieldTypes); @@ -594,16 +598,13 @@ private static Map parseRuntimeMappings( * Build a loader for nested documents. */ public NestedDocuments getNestedDocuments() { - return mapperSnapshot.getNestedDocuments(bitsetFilterCache::getBitSetProducer); + return new NestedDocuments(mappingLookup, bitsetFilterCache::getBitSetProducer); } /** - * Current version of the of the mapping. Increments if the mapping - * changes locally. Distinct from - * {@link IndexMetadata#getMappingVersion()} because it purely - * considers the local mapping changes. + * Cache key for current mapping. */ - public long localMappingVersion() { - return mapperSnapshot.version(); + public Writeable mappingKey() { + return mappingLookup.cacheKey(); } } diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index c47601b875e46..dcab2562d0e27 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -1394,7 +1394,7 @@ public void loadIntoContext(ShardSearchRequest request, SearchContext context, Q final DirectoryReader directoryReader = context.searcher().getDirectoryReader(); boolean[] loadedFromCache = new boolean[] { true }; - BytesReference cacheKey = request.cacheKey(context.getQueryShardContext().localMappingVersion()); + BytesReference cacheKey = request.cacheKey(context.getQueryShardContext().mappingKey()); BytesReference bytesReference = cacheShardLevelResult(context.indexShard(), directoryReader, cacheKey, out -> { queryPhase.execute(context); diff --git a/server/src/main/java/org/elasticsearch/search/NestedDocuments.java b/server/src/main/java/org/elasticsearch/search/NestedDocuments.java index 0f150571dbb55..33368750d856a 100644 --- a/server/src/main/java/org/elasticsearch/search/NestedDocuments.java +++ b/server/src/main/java/org/elasticsearch/search/NestedDocuments.java @@ -30,7 +30,7 @@ import org.apache.lucene.search.join.BitSetProducer; import org.apache.lucene.util.BitSet; import org.elasticsearch.common.lucene.search.Queries; -import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ObjectMapper; import java.io.IOException; @@ -47,24 +47,24 @@ public class NestedDocuments { private final Map childObjectFilters = new HashMap<>(); private final Map childObjectMappers = new HashMap<>(); private final BitSetProducer parentDocumentFilter; - private final DocumentMapper documentMapper; + private final MappingLookup mappingLookup; /** * Create a new NestedDocuments object for an index - * @param documentMapper a snapshot of the index's mapping + * @param mappingLookup a snapshot of the index's mapping * @param filterProducer a function to build BitSetProducers from filter queries */ - public NestedDocuments(DocumentMapper documentMapper, Function filterProducer) { - this.documentMapper = documentMapper; - if (documentMapper == null || documentMapper.hasNestedObjects() == false) { + public NestedDocuments(MappingLookup mappingLookup, Function filterProducer) { + this.mappingLookup = mappingLookup; + if (mappingLookup.hasNested() == false) { this.parentDocumentFilter = null; } else { this.parentDocumentFilter = filterProducer.apply(Queries.newNonNestedFilter()); - for (ObjectMapper mapper : documentMapper.getNestedParentMappers()) { + for (ObjectMapper mapper : mappingLookup.getNestedParentMappers()) { parentObjectFilters.put(mapper.name(), filterProducer.apply(mapper.nestedTypeFilter())); } - for (ObjectMapper mapper : documentMapper.getNestedMappers()) { + for (ObjectMapper mapper : mappingLookup.getNestedMappers()) { childObjectFilters.put(mapper.name(), null); childObjectMappers.put(mapper.name(), mapper); } @@ -98,7 +98,7 @@ private Weight getNestedChildWeight(LeafReaderContext ctx, String path) throws I * Given an object path, returns whether or not any of its parents are plain objects */ public boolean hasNonNestedParent(String path) { - return documentMapper.hasNonNestedParent(path); + return mappingLookup.hasNonNestedParent(path); } private class HasNestedDocuments implements LeafNestedDocuments { @@ -185,7 +185,7 @@ private SearchHit.NestedIdentity loadNestedIdentity() throws IOException { int parentNameLength; String path = findObjectPath(doc); while (path != null) { - String parent = documentMapper.getNestedParent(path); + String parent = mappingLookup.getNestedParent(path); // We have to pull a new scorer for each document here, because we advance from // the last parent which will be behind the doc Scorer childScorer = getNestedChildWeight(ctx, path).scorer(ctx); diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 157540377959b..88db0344ba5a1 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -36,6 +36,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.index.query.BoolQueryBuilder; @@ -396,10 +397,10 @@ public TimeValue keepAlive() { /** * Returns the cache key for this shard search request, based on its content */ - public BytesReference cacheKey(long localMappingVersion) throws IOException { + public BytesReference cacheKey(Writeable mappingKey) throws IOException { BytesStreamOutput out = scratch.get(); try { - out.writeLong(localMappingVersion); + mappingKey.writeTo(out); this.innerWriteTo(out, true); // copy it over since we don't want to share the thread-local bytes in #scratch return out.copyBytes(); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java index 131e00076b76b..158cd144472ba 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java @@ -562,8 +562,16 @@ protected String contentType() { return null; } }; - MappingLookup mappingLookup = - new MappingLookup(List.of(mockedTimestampField, dateFieldMapper), List.of(), List.of(), List.of(), 0); + MappingLookup mappingLookup = new MappingLookup( + "_doc", + List.of(mockedTimestampField, dateFieldMapper), + List.of(), + List.of(), + List.of(), + 0, + null, + null + ); ClusterService clusterService = ClusterServiceUtils.createClusterService(testThreadPool); Environment env = mock(Environment.class); diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java index 4c5f8c08abedc..b73fd58b23890 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -125,7 +125,7 @@ public void setupCreateIndexRequestAndAliasValidator() { .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build(); queryShardContext = new QueryShardContext(0, new IndexSettings(IndexMetadata.builder("test").settings(indexSettings).build(), indexSettings), - BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, xContentRegistry(), writableRegistry(), + BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, () -> randomNonNegativeLong(), null, null, () -> true, null, emptyMap()); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java index d7ce0a3460e23..940f6b9d288fe 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java @@ -167,7 +167,7 @@ public void testTermQuery() { .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build(); QueryShardContext context = new QueryShardContext(0, new IndexSettings(IndexMetadata.builder("foo").settings(indexSettings).build(), indexSettings), - BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, + BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null, null, () -> true, null, emptyMap()); MappedFieldType ft = new DateFieldType("field"); String date = "2015-10-12T14:10:55"; @@ -189,7 +189,7 @@ public void testRangeQuery() throws IOException { .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build(); QueryShardContext context = new QueryShardContext(0, new IndexSettings(IndexMetadata.builder("foo").settings(indexSettings).build(), indexSettings), - BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, xContentRegistry(), writableRegistry(), + BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null, null, () -> true, null, emptyMap()); MappedFieldType ft = new DateFieldType("field"); String date1 = "2015-10-12T14:10:55"; @@ -233,7 +233,7 @@ public void testRangeQueryWithIndexSort() { IndexSettings indexSettings = new IndexSettings(indexMetadata, settings); QueryShardContext context = new QueryShardContext(0, indexSettings, - BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, xContentRegistry(), writableRegistry(), + BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, () -> 0L, null, null, () -> true, null, emptyMap()); MappedFieldType ft = new DateFieldType("field"); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DocumentFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DocumentFieldMapperTests.java deleted file mode 100644 index d20ffbd784590..0000000000000 --- a/server/src/test/java/org/elasticsearch/index/mapper/DocumentFieldMapperTests.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.mapper; - -import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.analysis.TokenStream; -import org.apache.lucene.analysis.Tokenizer; -import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; -import org.apache.lucene.util.LuceneTestCase; -import org.elasticsearch.index.analysis.AnalyzerScope; -import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.query.QueryShardContext; - -import java.io.IOException; -import java.io.StringReader; -import java.util.Arrays; -import java.util.Collections; - -public class DocumentFieldMapperTests extends LuceneTestCase { - - private static class FakeAnalyzer extends Analyzer { - - private final String output; - - FakeAnalyzer(String output) { - this.output = output; - } - - @Override - protected TokenStreamComponents createComponents(String fieldName) { - Tokenizer tokenizer = new Tokenizer() { - boolean incremented = false; - final CharTermAttribute term = addAttribute(CharTermAttribute.class); - - @Override - public boolean incrementToken() { - if (incremented) { - return false; - } - term.setLength(0).append(output); - incremented = true; - return true; - } - }; - return new TokenStreamComponents(tokenizer); - } - - } - - static class FakeFieldType extends TermBasedFieldType { - - private FakeFieldType(String name) { - super(name, true, false, true, TextSearchInfo.SIMPLE_MATCH_ONLY, Collections.emptyMap()); - } - - @Override - public ValueFetcher valueFetcher(QueryShardContext context, String format) { - throw new UnsupportedOperationException(); - } - - @Override - public String typeName() { - return "fake"; - } - } - - static class FakeFieldMapper extends FieldMapper { - - final String indexedValue; - - FakeFieldMapper(FakeFieldType fieldType, String indexedValue) { - super(fieldType.name(), fieldType, - new NamedAnalyzer("fake", AnalyzerScope.INDEX, new FakeAnalyzer(indexedValue)), - MultiFields.empty(), CopyTo.empty()); - this.indexedValue = indexedValue; - } - - @Override - protected void parseCreateField(ParseContext context) { - } - - @Override - protected String contentType() { - return null; - } - - @Override - public Builder getMergeBuilder() { - return null; - } - } - - public void testAnalyzers() throws IOException { - FakeFieldType fieldType1 = new FakeFieldType("field1"); - FieldMapper fieldMapper1 = new FakeFieldMapper(fieldType1, "index1"); - - FakeFieldType fieldType2 = new FakeFieldType("field2"); - FieldMapper fieldMapper2 = new FakeFieldMapper(fieldType2, "index2"); - - MappingLookup mappingLookup = new MappingLookup( - Arrays.asList(fieldMapper1, fieldMapper2), - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList(), - 0); - - assertAnalyzes(mappingLookup.indexAnalyzer("field1", f -> null), "field1", "index1"); - assertAnalyzes(mappingLookup.indexAnalyzer("field2", f -> null), "field2", "index2"); - expectThrows(IllegalArgumentException.class, - () -> mappingLookup.indexAnalyzer("field3", f -> { - throw new IllegalArgumentException(); - }).tokenStream("field3", "blah")); - } - - private void assertAnalyzes(Analyzer analyzer, String field, String output) throws IOException { - try (TokenStream tok = analyzer.tokenStream(field, new StringReader(""))) { - CharTermAttribute term = tok.addAttribute(CharTermAttribute.class); - assertTrue(tok.incrementToken()); - assertEquals(output, term.toString()); - } - } -} diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java index 475d9b252fbad..280b73b491fbd 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java @@ -36,13 +36,19 @@ public void testDuplicateFieldAliasAndObject() { ObjectMapper objectMapper = createObjectMapper("some.path"); FieldAliasMapper aliasMapper = new FieldAliasMapper("path", "some.path", "field"); - MapperParsingException e = expectThrows(MapperParsingException.class, () -> - new MappingLookup( + MapperParsingException e = expectThrows( + MapperParsingException.class, + () -> new MappingLookup( + "_doc", Collections.emptyList(), singletonList(objectMapper), singletonList(aliasMapper), emptyList(), - 0)); + 0, + null, + null + ) + ); assertEquals("Alias [some.path] is defined both as an object and an alias", e.getMessage()); } @@ -51,14 +57,19 @@ public void testDuplicateFieldAliasAndConcreteField() { FieldMapper invalidField = new MockFieldMapper("invalid"); FieldAliasMapper invalidAlias = new FieldAliasMapper("invalid", "invalid", "field"); - MapperParsingException e = expectThrows(MapperParsingException.class, () -> - new MappingLookup( + MapperParsingException e = expectThrows( + MapperParsingException.class, + () -> new MappingLookup( + "_doc", Arrays.asList(field, invalidField), emptyList(), singletonList(invalidAlias), emptyList(), - 0)); - + 0, + null, + null + ) + ); assertEquals("Alias [invalid] is defined both as an alias and a concrete field", e.getMessage()); } @@ -68,11 +79,15 @@ public void testAliasThatRefersToAlias() { FieldAliasMapper invalidAlias = new FieldAliasMapper("invalid-alias", "invalid-alias", "alias"); MappingLookup mappers = new MappingLookup( + "_doc", singletonList(field), emptyList(), Arrays.asList(alias, invalidAlias), emptyList(), - 0); + 0, + null, + null + ); alias.validate(mappers); MapperParsingException e = expectThrows(MapperParsingException.class, () -> { @@ -88,11 +103,15 @@ public void testAliasThatRefersToItself() { MapperParsingException e = expectThrows(MapperParsingException.class, () -> { MappingLookup mappers = new MappingLookup( + "_doc", emptyList(), emptyList(), singletonList(invalidAlias), emptyList(), - 0); + 0, + null, + null + ); invalidAlias.validate(mappers); }); @@ -105,11 +124,15 @@ public void testAliasWithNonExistentPath() { MapperParsingException e = expectThrows(MapperParsingException.class, () -> { MappingLookup mappers = new MappingLookup( + "_doc", emptyList(), emptyList(), singletonList(invalidAlias), emptyList(), - 0); + 0, + null, + null + ); invalidAlias.validate(mappers); }); @@ -122,11 +145,15 @@ public void testFieldAliasWithNestedScope() { FieldAliasMapper aliasMapper = new FieldAliasMapper("alias", "nested.alias", "nested.field"); MappingLookup mappers = new MappingLookup( + "_doc", singletonList(createFieldMapper("nested", "field")), singletonList(objectMapper), singletonList(aliasMapper), emptyList(), - 0); + 0, + null, + null + ); aliasMapper.validate(mappers); } @@ -135,11 +162,15 @@ public void testFieldAliasWithDifferentObjectScopes() { FieldAliasMapper aliasMapper = new FieldAliasMapper("alias", "object2.alias", "object1.field"); MappingLookup mappers = new MappingLookup( + "_doc", List.of(createFieldMapper("object1", "field")), List.of(createObjectMapper("object1"), createObjectMapper("object2")), singletonList(aliasMapper), emptyList(), - 0); + 0, + null, + null + ); aliasMapper.validate(mappers); } @@ -149,11 +180,15 @@ public void testFieldAliasWithNestedTarget() { IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> { MappingLookup mappers = new MappingLookup( + "_doc", singletonList(createFieldMapper("nested", "field")), Collections.singletonList(objectMapper), singletonList(aliasMapper), emptyList(), - 0); + 0, + null, + null + ); aliasMapper.validate(mappers); }); @@ -168,11 +203,15 @@ public void testFieldAliasWithDifferentNestedScopes() { IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> { MappingLookup mappers = new MappingLookup( + "_doc", singletonList(createFieldMapper("nested1", "field")), List.of(createNestedObjectMapper("nested1"), createNestedObjectMapper("nested2")), singletonList(aliasMapper), emptyList(), - 0); + 0, + null, + null + ); aliasMapper.validate(mappers); }); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java index fbc2a3331cc5c..f868c75418e59 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java @@ -29,8 +29,6 @@ import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.test.ESTestCase; -import java.util.Map; - import static java.util.Collections.emptyMap; public class FieldNamesFieldTypeTests extends ESTestCase { @@ -43,10 +41,10 @@ public void testTermQuery() { Settings settings = settings(Version.CURRENT).build(); IndexSettings indexSettings = new IndexSettings( new IndexMetadata.Builder("foo").settings(settings).numberOfShards(1).numberOfReplicas(0).build(), settings); - MapperService.Snapshot mapperService = new StubSnapshot(Map.of("_field_names", fieldNamesFieldType, "field_name", fieldType)); + MappingLookup mappingLookup = MappingLookupUtils.fromTypes(fieldNamesFieldType, fieldType); QueryShardContext queryShardContext = new QueryShardContext(0, - indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService, + indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, null, mappingLookup, null, null, null, null, null, null, () -> 0L, null, null, () -> true, null, emptyMap()); Query termQuery = fieldNamesFieldType.termQuery("field_name", queryShardContext); assertEquals(new TermQuery(new Term(FieldNamesFieldMapper.CONTENT_TYPE, "field_name")), termQuery); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java index f2b63c145c0f9..ab302d0f83975 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java @@ -30,11 +30,12 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.Matchers.equalTo; public class FieldTypeLookupTests extends ESTestCase { public void testEmpty() { - FieldTypeLookup lookup = new FieldTypeLookup(Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); + FieldTypeLookup lookup = new FieldTypeLookup("_doc", Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); assertNull(lookup.get("foo")); Collection names = lookup.simpleMatchToFullName("foo"); assertNotNull(names); @@ -47,7 +48,7 @@ public void testFilter() { Collection fieldAliases = singletonList(new FieldAliasMapper("alias", "alias", "test")); Collection runtimeFields = List.of( new TestRuntimeField("runtime", "type"), new TestRuntimeField("field", "type")); - FieldTypeLookup fieldTypeLookup = new FieldTypeLookup(fieldMappers, fieldAliases, runtimeFields); + FieldTypeLookup fieldTypeLookup = new FieldTypeLookup("_doc", fieldMappers, fieldAliases, runtimeFields); assertEquals(3, size(fieldTypeLookup.filter(ft -> true))); for (MappedFieldType fieldType : fieldTypeLookup.filter(ft -> true)) { if (fieldType.name().equals("test")) { @@ -76,7 +77,7 @@ public void testFilter() { public void testAddNewField() { MockFieldMapper f = new MockFieldMapper("foo"); - FieldTypeLookup lookup = new FieldTypeLookup(Collections.singletonList(f), emptyList(), Collections.emptyList()); + FieldTypeLookup lookup = new FieldTypeLookup("_doc", Collections.singletonList(f), emptyList(), Collections.emptyList()); assertNull(lookup.get("bar")); assertEquals(f.fieldType(), lookup.get("foo")); assertEquals(1, size(lookup.filter(ft -> true))); @@ -86,7 +87,7 @@ public void testAddFieldAlias() { MockFieldMapper field = new MockFieldMapper("foo"); FieldAliasMapper alias = new FieldAliasMapper("alias", "alias", "foo"); - FieldTypeLookup lookup = new FieldTypeLookup(Collections.singletonList(field), Collections.singletonList(alias), + FieldTypeLookup lookup = new FieldTypeLookup("_doc", Collections.singletonList(field), Collections.singletonList(alias), Collections.emptyList()); MappedFieldType aliasType = lookup.get("alias"); @@ -100,7 +101,7 @@ public void testSimpleMatchToFullName() { FieldAliasMapper alias1 = new FieldAliasMapper("food", "food", "foo"); FieldAliasMapper alias2 = new FieldAliasMapper("barometer", "barometer", "bar"); - FieldTypeLookup lookup = new FieldTypeLookup(Arrays.asList(field1, field2), Arrays.asList(alias1, alias2), Collections.emptyList()); + FieldTypeLookup lookup = new FieldTypeLookup("_doc", List.of(field1, field2), List.of(alias1, alias2), List.of()); Collection names = lookup.simpleMatchToFullName("b*"); @@ -117,7 +118,7 @@ public void testSourcePathWithMultiFields() { .addMultiField(new MockFieldMapper.Builder("field.subfield2")) .build(new ContentPath()); - FieldTypeLookup lookup = new FieldTypeLookup(singletonList(field), emptyList(), emptyList()); + FieldTypeLookup lookup = new FieldTypeLookup("_doc", singletonList(field), emptyList(), emptyList()); assertEquals(Set.of("field"), lookup.sourcePaths("field")); assertEquals(Set.of("field"), lookup.sourcePaths("field.subfield1")); @@ -133,17 +134,25 @@ public void testSourcePathsWithCopyTo() { .copyTo("field") .build(new ContentPath()); - FieldTypeLookup lookup = new FieldTypeLookup(Arrays.asList(field, otherField), emptyList(), emptyList()); + FieldTypeLookup lookup = new FieldTypeLookup("_doc", Arrays.asList(field, otherField), emptyList(), emptyList()); assertEquals(Set.of("other_field", "field"), lookup.sourcePaths("field")); assertEquals(Set.of("other_field", "field"), lookup.sourcePaths("field.subfield1")); } + public void testTypeLookup() { + String type = randomAlphaOfLength(4); + assertThat( + ((TypeFieldType) new FieldTypeLookup(type, List.of(), List.of(), List.of()).get(TypeFieldType.NAME)).getType(), + equalTo(type) + ); + } + public void testRuntimeFieldsLookup() { MockFieldMapper concrete = new MockFieldMapper("concrete"); TestRuntimeField runtime = new TestRuntimeField("runtime", "type"); - FieldTypeLookup fieldTypeLookup = new FieldTypeLookup(List.of(concrete), emptyList(), List.of(runtime)); + FieldTypeLookup fieldTypeLookup = new FieldTypeLookup("_doc", List.of(concrete), emptyList(), List.of(runtime)); assertThat(fieldTypeLookup.get("concrete"), instanceOf(MockFieldMapper.FakeFieldType.class)); assertThat(fieldTypeLookup.get("runtime"), instanceOf(TestRuntimeField.class)); assertEquals(2, size(fieldTypeLookup.filter(ft -> true))); @@ -157,7 +166,7 @@ public void testRuntimeFieldOverrides() { TestRuntimeField subfieldOverride = new TestRuntimeField("object.subfield", "type"); TestRuntimeField runtime = new TestRuntimeField("runtime", "type"); - FieldTypeLookup fieldTypeLookup = new FieldTypeLookup(List.of(field, concrete, subfield), emptyList(), + FieldTypeLookup fieldTypeLookup = new FieldTypeLookup("_doc", List.of(field, concrete, subfield), emptyList(), List.of(fieldOverride, runtime, subfieldOverride)); assertThat(fieldTypeLookup.get("field"), instanceOf(TestRuntimeField.class)); assertThat(fieldTypeLookup.get("object.subfield"), instanceOf(TestRuntimeField.class)); @@ -172,7 +181,7 @@ public void testRuntimeFieldsSimpleMatchToFullName() { TestRuntimeField field2 = new TestRuntimeField("field2", "type"); TestRuntimeField subfield = new TestRuntimeField("object.subfield", "type"); - FieldTypeLookup fieldTypeLookup = new FieldTypeLookup(List.of(field1, concrete), emptyList(), List.of(field2, subfield)); + FieldTypeLookup fieldTypeLookup = new FieldTypeLookup("_doc", List.of(field1, concrete), emptyList(), List.of(field2, subfield)); { Set matches = fieldTypeLookup.simpleMatchToFullName("fie*"); assertEquals(2, matches.size()); @@ -194,7 +203,7 @@ public void testRuntimeFieldsSourcePaths() { TestRuntimeField field2 = new TestRuntimeField("field2", "type"); TestRuntimeField subfield = new TestRuntimeField("object.subfield", "type"); - FieldTypeLookup fieldTypeLookup = new FieldTypeLookup(List.of(field1, concrete), emptyList(), List.of(field2, subfield)); + FieldTypeLookup fieldTypeLookup = new FieldTypeLookup("_doc", List.of(field1, concrete), emptyList(), List.of(field2, subfield)); { Set sourcePaths = fieldTypeLookup.sourcePaths("field1"); assertEquals(1, sourcePaths.size()); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/IndexFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/IndexFieldTypeTests.java index b858929e3abc4..ee5c145bb822a 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/IndexFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/IndexFieldTypeTests.java @@ -69,7 +69,7 @@ private QueryShardContext createContext() { IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); Predicate indexNameMatcher = pattern -> Regex.simpleMatch(pattern, "index"); - return new QueryShardContext(0, indexSettings, null, null, null, null, null, null, xContentRegistry(), writableRegistry(), + return new QueryShardContext(0, indexSettings, null, null, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, System::currentTimeMillis, null, indexNameMatcher, () -> true, null, emptyMap()); } } 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 bacb980ea29e5..d708c80b293ba 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java @@ -38,20 +38,20 @@ public class MapperServiceTests extends MapperServiceTestCase { public void testPreflightUpdateDoesNotChangeMapping() throws Throwable { final MapperService mapperService = createMapperService(mapping(b -> {})); merge(mapperService, MergeReason.MAPPING_UPDATE_PREFLIGHT, mapping(b -> createMappingSpecifyingNumberOfFields(b, 1))); - assertThat("field was not created by preflight check", mapperService.snapshot().fieldType("field0"), nullValue()); + assertThat("field was not created by preflight check", mapperService.fieldType("field0"), nullValue()); merge(mapperService, MergeReason.MAPPING_UPDATE, mapping(b -> createMappingSpecifyingNumberOfFields(b, 1))); - assertThat("field was not created by mapping update", mapperService.snapshot().fieldType("field0"), notNullValue()); + assertThat("field was not created by mapping update", mapperService.fieldType("field0"), notNullValue()); } - public void testSnapshot() throws IOException { + public void testLookup() throws IOException { MapperService service = createMapperService(mapping(b -> {})); - MapperService.Snapshot oldSnapshot = service.snapshot(); - assertThat(oldSnapshot.fieldType("cat"), nullValue()); + MappingLookup oldLookup = service.lookup(); + assertThat(oldLookup.fieldTypes().get("cat"), nullValue()); merge(service, mapping(b -> b.startObject("cat").field("type", "keyword").endObject())); - MapperService.Snapshot newSnapshot = service.snapshot(); - assertThat(newSnapshot.fieldType("cat"), not(nullValue())); - assertThat(oldSnapshot.fieldType("cat"), nullValue()); + MappingLookup newLookup = service.lookup(); + assertThat(newLookup.fieldTypes().get("cat"), not(nullValue())); + assertThat(oldLookup.fieldTypes().get("cat"), nullValue()); } /** diff --git a/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java b/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java index b918819bcd29b..bd05916d47fdc 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java @@ -19,10 +19,20 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.Tokenizer; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.elasticsearch.Version; import org.elasticsearch.common.Explicit; +import org.elasticsearch.index.analysis.AnalyzerScope; +import org.elasticsearch.index.analysis.NamedAnalyzer; +import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.test.ESTestCase; +import java.io.IOException; +import java.io.StringReader; +import java.util.Arrays; import java.util.Collections; import static org.hamcrest.CoreMatchers.instanceOf; @@ -30,8 +40,8 @@ public class MappingLookupTests extends ESTestCase { public void testOnlyRuntimeField() { - MappingLookup mappingLookup = new MappingLookup(Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), - Collections.singletonList(new TestRuntimeField("test", "type")), 0); + MappingLookup mappingLookup = new MappingLookup("_doc", Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), + Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, null); assertEquals(0, size(mappingLookup.fieldMappers())); assertEquals(0, mappingLookup.objectMappers().size()); assertNull(mappingLookup.getMapper("test")); @@ -40,8 +50,8 @@ public void testOnlyRuntimeField() { public void testRuntimeFieldLeafOverride() { MockFieldMapper fieldMapper = new MockFieldMapper("test"); - MappingLookup mappingLookup = new MappingLookup(Collections.singletonList(fieldMapper), Collections.emptyList(), - Collections.emptyList(), Collections.singletonList(new TestRuntimeField("test", "type")), 0); + MappingLookup mappingLookup = new MappingLookup("_doc", Collections.singletonList(fieldMapper), Collections.emptyList(), + Collections.emptyList(), Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, null); assertThat(mappingLookup.getMapper("test"), instanceOf(MockFieldMapper.class)); assertEquals(1, size(mappingLookup.fieldMappers())); assertEquals(0, mappingLookup.objectMappers().size()); @@ -53,8 +63,16 @@ public void testSubfieldOverride() { MockFieldMapper fieldMapper = new MockFieldMapper("object.subfield"); ObjectMapper objectMapper = new ObjectMapper("object", "object", new Explicit<>(true, true), ObjectMapper.Nested.NO, ObjectMapper.Dynamic.TRUE, Collections.singletonMap("object.subfield", fieldMapper), Version.CURRENT); - MappingLookup mappingLookup = new MappingLookup(Collections.singletonList(fieldMapper), Collections.singletonList(objectMapper), - Collections.emptyList(), Collections.singletonList(new TestRuntimeField("object.subfield", "type")), 0); + MappingLookup mappingLookup = new MappingLookup( + "_doc", + Collections.singletonList(fieldMapper), + Collections.singletonList(objectMapper), + Collections.emptyList(), + Collections.singletonList(new TestRuntimeField("object.subfield", "type")), + 0, + null, + null + ); assertThat(mappingLookup.getMapper("object.subfield"), instanceOf(MockFieldMapper.class)); assertEquals(1, size(mappingLookup.fieldMappers())); assertEquals(1, mappingLookup.objectMappers().size()); @@ -62,6 +80,41 @@ public void testSubfieldOverride() { assertEquals(1, size(mappingLookup.fieldTypes().filter(ft -> true))); } + + public void testAnalyzers() throws IOException { + FakeFieldType fieldType1 = new FakeFieldType("field1"); + FieldMapper fieldMapper1 = new FakeFieldMapper(fieldType1, "index1"); + + FakeFieldType fieldType2 = new FakeFieldType("field2"); + FieldMapper fieldMapper2 = new FakeFieldMapper(fieldType2, "index2"); + + MappingLookup mappingLookup = new MappingLookup( + "_doc", + Arrays.asList(fieldMapper1, fieldMapper2), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + 0, + null, + null + ); + + assertAnalyzes(mappingLookup.indexAnalyzer("field1", f -> null), "field1", "index1"); + assertAnalyzes(mappingLookup.indexAnalyzer("field2", f -> null), "field2", "index2"); + expectThrows(IllegalArgumentException.class, + () -> mappingLookup.indexAnalyzer("field3", f -> { + throw new IllegalArgumentException(); + }).tokenStream("field3", "blah")); + } + + private void assertAnalyzes(Analyzer analyzer, String field, String output) throws IOException { + try (TokenStream tok = analyzer.tokenStream(field, new StringReader(""))) { + CharTermAttribute term = tok.addAttribute(CharTermAttribute.class); + assertTrue(tok.incrementToken()); + assertEquals(output, term.toString()); + } + } + private static int size(Iterable iterable) { int count = 0; for (Object obj : iterable) { @@ -69,4 +122,76 @@ private static int size(Iterable iterable) { } return count; } + + private static class FakeAnalyzer extends Analyzer { + + private final String output; + + FakeAnalyzer(String output) { + this.output = output; + } + + @Override + protected TokenStreamComponents createComponents(String fieldName) { + Tokenizer tokenizer = new Tokenizer() { + boolean incremented = false; + final CharTermAttribute term = addAttribute(CharTermAttribute.class); + + @Override + public boolean incrementToken() { + if (incremented) { + return false; + } + term.setLength(0).append(output); + incremented = true; + return true; + } + }; + return new TokenStreamComponents(tokenizer); + } + + } + + static class FakeFieldType extends TermBasedFieldType { + + private FakeFieldType(String name) { + super(name, true, false, true, TextSearchInfo.SIMPLE_MATCH_ONLY, Collections.emptyMap()); + } + + @Override + public ValueFetcher valueFetcher(QueryShardContext context, String format) { + throw new UnsupportedOperationException(); + } + + @Override + public String typeName() { + return "fake"; + } + } + + static class FakeFieldMapper extends FieldMapper { + + final String indexedValue; + + FakeFieldMapper(FakeFieldType fieldType, String indexedValue) { + super(fieldType.name(), fieldType, + new NamedAnalyzer("fake", AnalyzerScope.INDEX, new FakeAnalyzer(indexedValue)), + MultiFields.empty(), CopyTo.empty()); + this.indexedValue = indexedValue; + } + + @Override + protected void parseCreateField(ParseContext context) { + } + + @Override + protected String contentType() { + return null; + } + + @Override + public Builder getMergeBuilder() { + return null; + } + } } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java index ddce0e9819fde..9e79f62308924 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java @@ -56,7 +56,7 @@ public void testSimpleNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = mapperService.snapshot().getNestedDocuments(QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService.lookup(), QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -153,7 +153,7 @@ public void testMultiLevelNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = mapperService.snapshot().getNestedDocuments(QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService.lookup(), QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -264,7 +264,7 @@ public void testNestedObjectWithinNonNestedObject() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = mapperService.snapshot().getNestedDocuments(QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService.lookup(), QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java index b236801a1f5c7..9acfffb62f25f 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java @@ -63,14 +63,13 @@ public void testEmptyNested() throws Exception { } public void testSingleNested() throws Exception { + MappingLookup lookup = createDocumentMapper(mapping(b -> b.startObject("nested1").field("type", "nested").endObject())).mappers(); - DocumentMapper docMapper = createDocumentMapper(mapping(b -> b.startObject("nested1").field("type", "nested").endObject())); - - assertThat(docMapper.hasNestedObjects(), equalTo(true)); - ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); + assertThat(lookup.hasNested(), equalTo(true)); + ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); - ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference + ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference .bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -86,7 +85,7 @@ public void testSingleNested() throws Exception { assertThat(doc.docs().get(1).get("field"), equalTo("value")); - doc = docMapper.parse(new SourceToParse("test", "1", BytesReference + doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference .bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -109,7 +108,7 @@ public void testSingleNested() throws Exception { } public void testMultiNested() throws Exception { - DocumentMapper docMapper = createDocumentMapper(mapping(b -> { + MappingLookup lookup = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -120,19 +119,19 @@ public void testMultiNested() throws Exception { b.endObject(); } b.endObject(); - })); + })).mappers(); - assertThat(docMapper.hasNestedObjects(), equalTo(true)); - ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); + assertThat(lookup.hasNested(), equalTo(true)); + ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested1Mapper.nested().isIncludeInRoot(), equalTo(false)); - ObjectMapper nested2Mapper = docMapper.mappers().objectMappers().get("nested1.nested2"); + ObjectMapper nested2Mapper = lookup.objectMappers().get("nested1.nested2"); assertThat(nested2Mapper.nested().isNested(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(false)); - ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", + ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -174,7 +173,7 @@ public void testMultiNested() throws Exception { } public void testMultiObjectAndNested1() throws Exception { - DocumentMapper docMapper = createDocumentMapper(mapping(b -> { + MappingLookup lookup = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -190,19 +189,19 @@ public void testMultiObjectAndNested1() throws Exception { b.endObject(); } b.endObject(); - })); + })).mappers(); - assertThat(docMapper.hasNestedObjects(), equalTo(true)); - ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); + assertThat(lookup.hasNested(), equalTo(true)); + ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested1Mapper.nested().isIncludeInRoot(), equalTo(false)); - ObjectMapper nested2Mapper = docMapper.mappers().objectMappers().get("nested1.nested2"); + ObjectMapper nested2Mapper = lookup.objectMappers().get("nested1.nested2"); assertThat(nested2Mapper.nested().isNested(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(false)); - ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", + ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -244,7 +243,7 @@ public void testMultiObjectAndNested1() throws Exception { } public void testMultiObjectAndNested2() throws Exception { - DocumentMapper docMapper = createDocumentMapper(mapping(b -> { + MappingLookup lookup = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -261,19 +260,19 @@ public void testMultiObjectAndNested2() throws Exception { b.endObject(); } b.endObject(); - })); + })).mappers(); - assertThat(docMapper.hasNestedObjects(), equalTo(true)); - ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); + assertThat(lookup.hasNested(), equalTo(true)); + ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInParent(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInRoot(), equalTo(false)); - ObjectMapper nested2Mapper = docMapper.mappers().objectMappers().get("nested1.nested2"); + ObjectMapper nested2Mapper = lookup.objectMappers().get("nested1.nested2"); assertThat(nested2Mapper.nested().isNested(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(false)); - ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", + ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -315,7 +314,7 @@ public void testMultiObjectAndNested2() throws Exception { } public void testMultiRootAndNested1() throws Exception { - DocumentMapper docMapper = createDocumentMapper(mapping(b -> { + MappingLookup lookup = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -331,23 +330,23 @@ public void testMultiRootAndNested1() throws Exception { b.endObject(); } b.endObject(); - })); + })).mappers(); - assertEquals("nested1", docMapper.getNestedParent("nested1.nested2")); - assertNull(docMapper.getNestedParent("nonexistent")); - assertNull(docMapper.getNestedParent("nested1")); + assertEquals("nested1", lookup.getNestedParent("nested1.nested2")); + assertNull(lookup.getNestedParent("nonexistent")); + assertNull(lookup.getNestedParent("nested1")); - assertThat(docMapper.hasNestedObjects(), equalTo(true)); - ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); + assertThat(lookup.hasNested(), equalTo(true)); + ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested1Mapper.nested().isIncludeInRoot(), equalTo(false)); - ObjectMapper nested2Mapper = docMapper.mappers().objectMappers().get("nested1.nested2"); + ObjectMapper nested2Mapper = lookup.objectMappers().get("nested1.nested2"); assertThat(nested2Mapper.nested().isNested(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(true)); - ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", + ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -525,7 +524,7 @@ public void testMultipleLevelsIncludeRootWithMerge() throws Exception { } public void testNestedArrayStrict() throws Exception { - DocumentMapper docMapper = createDocumentMapper(mapping(b -> { + MappingLookup lookup = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -537,14 +536,14 @@ public void testNestedArrayStrict() throws Exception { b.endObject(); } b.endObject(); - })); + })).mappers(); - assertThat(docMapper.hasNestedObjects(), equalTo(true)); - ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); + assertThat(lookup.hasNested(), equalTo(true)); + ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.dynamic(), equalTo(Dynamic.STRICT)); - ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", + ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -618,7 +617,7 @@ public void testParentObjectMapperAreNested() throws Exception { } b.endObject(); })); - assertFalse(mapperService.documentMapper().hasNonNestedParent("comments.messages")); + assertFalse(mapperService.lookup().hasNonNestedParent("comments.messages")); mapperService = createMapperService(mapping(b -> { b.startObject("comments"); @@ -632,7 +631,7 @@ public void testParentObjectMapperAreNested() throws Exception { } b.endObject(); })); - assertTrue(mapperService.documentMapper().hasNonNestedParent("comments.messages")); + assertTrue(mapperService.lookup().hasNonNestedParent("comments.messages")); } public void testLimitNestedDocsDefaultSettings() throws Exception { @@ -777,17 +776,15 @@ public void testLimitNestedDocsMultipleNestedFields() throws Exception { } public void testReorderParent() throws IOException { - Version version = VersionUtils.randomIndexCompatibleVersion(random()); + MappingLookup lookup = createDocumentMapper(version, mapping(b -> b.startObject("nested1").field("type", "nested").endObject())) + .mappers(); - DocumentMapper docMapper - = createDocumentMapper(version, mapping(b -> b.startObject("nested1").field("type", "nested").endObject())); - - assertThat(docMapper.hasNestedObjects(), equalTo(true)); - ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); + assertThat(lookup.hasNested(), equalTo(true)); + ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); - ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", + ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldTypeTests.java index ac763d35a0c3f..c4ae1291a53e0 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldTypeTests.java @@ -474,7 +474,7 @@ public void doTestIndexSortRangeQueries(NumberType type, Supplier valueS IndexSearcher searcher = newSearcher(reader); QueryShardContext context = new QueryShardContext(0, indexSettings, - BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, xContentRegistry(), writableRegistry(), + BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, () -> 0L, null, null, () -> true, null, emptyMap()); final int iters = 10; diff --git a/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java index a4c026f6d9799..22e013bb9bedc 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java @@ -213,7 +213,7 @@ private QueryShardContext createContext() { Settings indexSettings = Settings.builder() .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAlphaOfLengthBetween(1, 10), indexSettings); - return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, + return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null, null, () -> true, null, emptyMap()); } diff --git a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java index c144422e3d26d..814b0acac95d7 100644 --- a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java @@ -36,7 +36,6 @@ import org.apache.lucene.store.Directory; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetadata; -import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; @@ -56,11 +55,11 @@ import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; -import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MappingLookupUtils; import org.elasticsearch.index.mapper.MockFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; -import org.elasticsearch.index.mapper.StubSnapshot; import org.elasticsearch.index.mapper.TestRuntimeField; import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.indices.IndicesModule; @@ -78,11 +77,12 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; @@ -93,6 +93,8 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class QueryShardContextTests extends ESTestCase { @@ -205,6 +207,7 @@ public void testIndexSortedOnField() { null, null, null, + null, NamedXContentRegistry.EMPTY, new NamedWriteableRegistry(Collections.emptyList()), null, @@ -223,100 +226,101 @@ public void testIndexSortedOnField() { } public void testFielddataLookupSelfReference() { - QueryShardContext queryShardContext = createQueryShardContext("uuid", null, fieldTypeLookup((field, leafLookup, docId) -> { - //simulate a runtime field that depends on itself e.g. field: doc['field'] - return leafLookup.doc().get(field).toString(); - })); + QueryShardContext queryShardContext = createQueryShardContext( + // simulate a runtime field that depends on itself e.g. field: doc['field'] + runtimeField("field", leafLookup -> leafLookup.doc().get("field").toString()) + ); IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> collect("field", queryShardContext)); assertEquals("Cyclic dependency detected while resolving runtime fields: field -> field", iae.getMessage()); } public void testFielddataLookupLooseLoop() { - QueryShardContext queryShardContext = createQueryShardContext("uuid", null, fieldTypeLookup((field, leafLookup, docId) -> { - //simulate a runtime field cycle: 1: doc['2'] 2: doc['3'] 3: doc['4'] 4: doc['1'] - if (field.equals("4")) { - return leafLookup.doc().get("1").toString(); - } - return leafLookup.doc().get(Integer.toString(Integer.parseInt(field) + 1)).toString(); - })); + QueryShardContext queryShardContext = createQueryShardContext( + // simulate a runtime field cycle: 1: doc['2'] 2: doc['3'] 3: doc['4'] 4: doc['1'] + runtimeField("1", leafLookup -> leafLookup.doc().get("2").get(0).toString()), + runtimeField("2", leafLookup -> leafLookup.doc().get("3").get(0).toString()), + runtimeField("3", leafLookup -> leafLookup.doc().get("4").get(0).toString()), + runtimeField("4", leafLookup -> leafLookup.doc().get("1").get(0).toString()) + ); IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> collect("1", queryShardContext)); assertEquals("Cyclic dependency detected while resolving runtime fields: 1 -> 2 -> 3 -> 4 -> 1", iae.getMessage()); } public void testFielddataLookupTerminatesInLoop() { - QueryShardContext queryShardContext = createQueryShardContext("uuid", null, fieldTypeLookup((field, leafLookup, docId) -> { - //simulate a runtime field cycle: 1: doc['2'] 2: doc['3'] 3: doc['4'] 4: doc['4'] - if (field.equals("4")) { - return leafLookup.doc().get("4").toString(); - } - return leafLookup.doc().get(Integer.toString(Integer.parseInt(field) + 1)).toString(); - })); + QueryShardContext queryShardContext = createQueryShardContext( + // simulate a runtime field cycle: 1: doc['2'] 2: doc['3'] 3: doc['4'] 4: doc['4'] + runtimeField("1", leafLookup -> leafLookup.doc().get("2").get(0).toString()), + runtimeField("2", leafLookup -> leafLookup.doc().get("3").get(0).toString()), + runtimeField("3", leafLookup -> leafLookup.doc().get("4").get(0).toString()), + runtimeField("4", leafLookup -> leafLookup.doc().get("4").get(0).toString()) + ); IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> collect("1", queryShardContext)); assertEquals("Cyclic dependency detected while resolving runtime fields: 1 -> 2 -> 3 -> 4 -> 4", iae.getMessage()); } public void testFielddataLookupSometimesLoop() throws IOException { - QueryShardContext queryShardContext = createQueryShardContext("uuid", null, fieldTypeLookup((field, leafLookup, docId) -> { - if (docId == 0) { - return field + "_" + docId; - } else { - assert docId == 1; - if (field.equals("field4")) { - return leafLookup.doc().get("field1").toString(); + QueryShardContext queryShardContext = createQueryShardContext( + // simulate a runtime field cycle in the second doc: 1: doc['2'] 2: doc['3'] 3: doc['4'] 4: doc['4'] + runtimeField("1", leafLookup -> leafLookup.doc().get("2").get(0).toString()), + runtimeField("2", leafLookup -> leafLookup.doc().get("3").get(0).toString()), + runtimeField("3", leafLookup -> leafLookup.doc().get("4").get(0).toString()), + runtimeField("4", (leafLookup, docId) -> { + if (docId == 0) { + return "escape!"; } - int i = Integer.parseInt(field.substring(field.length() - 1)); - return leafLookup.doc().get("field" + (i + 1)).toString(); - } - })); - List values = collect("field1", queryShardContext, new TermQuery(new Term("indexed_field", "first"))); - assertEquals(List.of("field1_0"), values); - IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> collect("field1", queryShardContext)); - assertEquals("Cyclic dependency detected while resolving runtime fields: field1 -> field2 -> field3 -> field4 -> field1", - iae.getMessage()); + return leafLookup.doc().get("4").get(0).toString(); + }) + ); + List values = collect("1", queryShardContext, new TermQuery(new Term("indexed_field", "first"))); + assertEquals(List.of("escape!"), values); + IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> collect("1", queryShardContext)); + assertEquals("Cyclic dependency detected while resolving runtime fields: 1 -> 2 -> 3 -> 4 -> 4", iae.getMessage()); } public void testFielddataLookupBeyondMaxDepth() { - QueryShardContext queryShardContext = createQueryShardContext("uuid", null, fieldTypeLookup((field, leafLookup, docId) -> { - int i = Integer.parseInt(field); - return leafLookup.doc().get(Integer.toString(i + 1)).toString(); - })); + QueryShardContext queryShardContext = createQueryShardContext( + runtimeField("1", leafLookup -> leafLookup.doc().get("2").get(0).toString()), + runtimeField("2", leafLookup -> leafLookup.doc().get("3").get(0).toString()), + runtimeField("3", leafLookup -> leafLookup.doc().get("4").get(0).toString()), + runtimeField("4", leafLookup -> leafLookup.doc().get("5").get(0).toString()), + runtimeField("5", leafLookup -> leafLookup.doc().get("6").get(0).toString()), + runtimeField("6", leafLookup -> "cat") + ); IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> collect("1", queryShardContext)); assertEquals("Field requires resolving too many dependent fields: 1 -> 2 -> 3 -> 4 -> 5 -> 6", iae.getMessage()); } public void testFielddataLookupReferencesBelowMaxDepth() throws IOException { - QueryShardContext queryShardContext = createQueryShardContext("uuid", null, fieldTypeLookup((field, leafLookup, docId) -> { - int i = Integer.parseInt(field.substring(field.length() - 1)); - if (i == 5) { - return "test"; - } else { - ScriptDocValues scriptDocValues = leafLookup.doc().get("field" + (i + 1)); - return scriptDocValues.get(0).toString() + docId; - } - })); - assertEquals(List.of("test0000", "test1111"), collect("field1", queryShardContext)); + QueryShardContext queryShardContext = createQueryShardContext( + runtimeField("1", leafLookup -> leafLookup.doc().get("2").get(0).toString()), + runtimeField("2", leafLookup -> leafLookup.doc().get("3").get(0).toString()), + runtimeField("3", leafLookup -> leafLookup.doc().get("4").get(0).toString()), + runtimeField("4", leafLookup -> leafLookup.doc().get("5").get(0).toString()), + runtimeField("5", (leafLookup, docId) -> "cat on doc " + docId) + ); + assertEquals(List.of("cat on doc 0", "cat on doc 1"), collect("1", queryShardContext)); } public void testFielddataLookupOneFieldManyReferences() throws IOException { int numFields = randomIntBetween(5, 20); - QueryShardContext queryShardContext = createQueryShardContext("uuid", null, fieldTypeLookup((field, leafLookup, docId) -> { - if (field.equals("field")) { - StringBuilder value = new StringBuilder(); - for (int i = 0; i < numFields; i++) { - value.append(leafLookup.doc().get("field" + i).get(0)); - } - return value.toString(); - } else { - return "test" + docId; + List fields = new ArrayList<>(numFields + 1); + fields.add(runtimeField("root", leafLookup -> { + StringBuilder value = new StringBuilder(); + for (int i = 0; i < numFields; i++) { + value.append(leafLookup.doc().get(i).get(0)); } + return value.toString(); })); - StringBuilder expectedFirstDoc = new StringBuilder(); - StringBuilder expectedSecondDoc = new StringBuilder(); + StringBuilder expected = new StringBuilder(); for (int i = 0; i < numFields; i++) { - expectedFirstDoc.append("test0"); - expectedSecondDoc.append("test1"); + String fieldValue = Integer.toString(i); + fields.add(runtimeField(Integer.toString(i), leafLookup -> fieldValue)); + expected.append(i); } - assertEquals(List.of(expectedFirstDoc.toString(), expectedSecondDoc.toString()), collect("field", queryShardContext)); + assertEquals( + List.of(expected.toString(), expected.toString()), + collect("root", createQueryShardContext("uuid", null, fields, Map.of(), List.of())) + ); } public void testSearchRequestRuntimeFields() { @@ -332,7 +336,7 @@ public void testSearchRequestRuntimeFields() { QueryShardContext qsc = createQueryShardContext( "uuid", null, - Map.of("pig", new MockFieldMapper.FakeFieldType("pig"), "cat", new MockFieldMapper.FakeFieldType("cat")), + List.of(new MockFieldMapper.FakeFieldType("pig"), new MockFieldMapper.FakeFieldType("cat")), runtimeMappings, Collections.singletonList(new TestRuntimeField.Plugin())); assertTrue(qsc.isFieldMapped("cat")); @@ -348,28 +352,17 @@ public void testSearchRequestRuntimeFields() { } public static QueryShardContext createQueryShardContext(String indexUuid, String clusterAlias) { - return createQueryShardContext(indexUuid, clusterAlias, name -> { - throw new UnsupportedOperationException(); - }); + return createQueryShardContext(indexUuid, clusterAlias, List.of(), Map.of(), List.of()); } - private static QueryShardContext createQueryShardContext( - String indexUuid, - String clusterAlias, - Function fieldTypeLookup - ) { - return createQueryShardContext(indexUuid, clusterAlias, new HashMap<>() { - @Override - public MappedFieldType get(Object key) { - return fieldTypeLookup.apply(key.toString()); - } - }, Collections.emptyMap(), Collections.emptyList()); + private static QueryShardContext createQueryShardContext(MappedFieldType... fieldTypeLookup) { + return createQueryShardContext("uuid", null, Arrays.asList(fieldTypeLookup), Collections.emptyMap(), Collections.emptyList()); } private static QueryShardContext createQueryShardContext( String indexUuid, String clusterAlias, - Map fieldTypeLookup, + List fieldTypes, Map runtimeMappings, List mapperPlugins ) { @@ -381,18 +374,34 @@ private static QueryShardContext createQueryShardContext( ); IndexMetadata indexMetadata = indexMetadataBuilder.build(); IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); - MapperService.Snapshot mapperSnapshot = createMapperSnapshot(indexSettings, fieldTypeLookup, mapperPlugins); + MapperService mapperService = createMapperService(indexSettings, mapperPlugins); + MappingLookup mappingLookup = MappingLookupUtils.fromTypes(fieldTypes.stream()); final long nowInMillis = randomNonNegativeLong(); return new QueryShardContext( - 0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, - (mappedFieldType, idxName, searchLookup) -> mappedFieldType.fielddataBuilder(idxName, searchLookup).build(null, null), - mapperSnapshot, null, null, NamedXContentRegistry.EMPTY, new NamedWriteableRegistry(Collections.emptyList()), - null, null, () -> nowInMillis, clusterAlias, null, () -> true, null, runtimeMappings); + 0, + indexSettings, + BigArrays.NON_RECYCLING_INSTANCE, + null, + (mappedFieldType, idxName, searchLookup) -> mappedFieldType.fielddataBuilder(idxName, searchLookup).build(null, null), + mapperService, + mappingLookup, + null, + null, + NamedXContentRegistry.EMPTY, + new NamedWriteableRegistry(Collections.emptyList()), + null, + null, + () -> nowInMillis, + clusterAlias, + null, + () -> true, + null, + runtimeMappings + ); } - private static MapperService.Snapshot createMapperSnapshot( + private static MapperService createMapperService( IndexSettings indexSettings, - Map fieldTypeLookup, List mapperPlugins ) { IndexAnalyzers indexAnalyzers = new IndexAnalyzers( @@ -403,7 +412,9 @@ private static MapperService.Snapshot createMapperSnapshot( IndicesModule indicesModule = new IndicesModule(mapperPlugins); MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); Supplier queryShardContextSupplier = () -> { throw new UnsupportedOperationException(); }; - Mapper.TypeParser.ParserContext parseContext = new Mapper.TypeParser.ParserContext( + MapperService mapperService = mock(MapperService.class); + when(mapperService.getIndexAnalyzers()).thenReturn(indexAnalyzers); + when(mapperService.parserContext()).thenReturn(new Mapper.TypeParser.ParserContext( null, mapperRegistry.getMapperParsers()::get, mapperRegistry.getRuntimeFieldTypeParsers()::get, @@ -415,23 +426,16 @@ private static MapperService.Snapshot createMapperSnapshot( indexSettings, () -> true, false - ); - return new StubSnapshot(fieldTypeLookup) { - @Override - public IndexAnalyzers getIndexAnalyzers() { - return indexAnalyzers; - } + )); + return mapperService; + } - @Override - public ParserContext parserContext() { - return parseContext; - } - }; + private static MappedFieldType runtimeField(String name, Function runtimeDocValues) { + return runtimeField(name, (leafLookup, docId) -> runtimeDocValues.apply(leafLookup)); } - private static Function fieldTypeLookup( - TriFunction runtimeDocValues) { - return name -> new TestRuntimeField(name, null) { + private static MappedFieldType runtimeField(String name, BiFunction runtimeDocValues) { + return new TestRuntimeField(name, null) { @Override public IndexFieldData.Builder fielddataBuilder(String fullyQualifiedIndexName, Supplier searchLookup) { @@ -471,7 +475,7 @@ public void setNextDocId(int docId) { LeafSearchLookup leafLookup = searchLookup.get() .getLeafSearchLookup(context); leafLookup.setDocument(docId); - value = runtimeDocValues.apply(name, leafLookup, docId); + value = runtimeDocValues.apply(leafLookup, docId); } }; } diff --git a/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java b/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java index 1b904cb93f3ef..3e0b9a0f909b6 100644 --- a/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java @@ -41,9 +41,27 @@ public class RangeQueryRewriteTests extends ESSingleNodeTestCase { public void testRewriteMissingField() throws Exception { IndexService indexService = createIndex("test"); IndexReader reader = new MultiReader(); - QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE, - null, null, indexService.mapperService().snapshot(), null, null, xContentRegistry(), writableRegistry(), - null, new IndexSearcher(reader), null, null, null, () -> true, null, emptyMap()); + QueryRewriteContext context = new QueryShardContext( + 0, + indexService.getIndexSettings(), + BigArrays.NON_RECYCLING_INSTANCE, + null, + null, + indexService.mapperService(), + indexService.mapperService().lookup(), + null, + null, + xContentRegistry(), + writableRegistry(), + null, + new IndexSearcher(reader), + null, + null, + null, + () -> true, + null, + emptyMap() + ); RangeQueryBuilder range = new RangeQueryBuilder("foo"); assertEquals(Relation.DISJOINT, range.getRelation(context)); } @@ -60,7 +78,7 @@ public void testRewriteMissingReader() throws Exception { indexService.mapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, null, - indexService.mapperService().snapshot(), null, null, xContentRegistry(), writableRegistry(), + indexService.mapperService(), indexService.mapperService().lookup(), null, null, xContentRegistry(), writableRegistry(), null, null, null, null, null, () -> true, null, emptyMap()); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // can't make assumptions on a missing reader, so it must return INTERSECT @@ -79,9 +97,27 @@ public void testRewriteEmptyReader() throws Exception { indexService.mapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); IndexReader reader = new MultiReader(); - QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE, - null, null, indexService.mapperService().snapshot(), null, null, xContentRegistry(), writableRegistry(), - null, new IndexSearcher(reader), null, null, null, () -> true, null, emptyMap()); + QueryRewriteContext context = new QueryShardContext( + 0, + indexService.getIndexSettings(), + BigArrays.NON_RECYCLING_INSTANCE, + null, + null, + indexService.mapperService(), + indexService.mapperService().lookup(), + null, + null, + xContentRegistry(), + writableRegistry(), + null, + new IndexSearcher(reader), + null, + null, + null, + () -> true, + null, + emptyMap() + ); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // no values -> DISJOINT assertEquals(Relation.DISJOINT, range.getRelation(context)); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java index 078450445f546..dc6540f1cfee1 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java @@ -37,15 +37,18 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.NestedPathFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.mapper.ObjectMapper; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.query.MatchAllQueryBuilder; @@ -87,7 +90,6 @@ import java.util.Locale; import java.util.Map; import java.util.function.Function; -import java.util.stream.Collectors; import java.util.stream.DoubleStream; import java.util.stream.LongStream; @@ -108,16 +110,6 @@ public class NestedAggregatorTests extends AggregatorTestCase { private static final SeqNoFieldMapper.SequenceIDFields sequenceIDFields = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); - /** - * For each provided field type, we also register an alias with name {@code -alias}. - */ - @Override - protected Map getFieldAliases(MappedFieldType... fieldTypes) { - return Arrays.stream(fieldTypes).collect(Collectors.toMap( - ft -> ft.name() + "-alias", - Function.identity())); - } - /** * Nested aggregations need the {@linkplain DirectoryReader} wrapped. */ @@ -889,4 +881,20 @@ private List generateBook(String id, String[] authors, int[] numPages) return documents; } + @Override + protected List objectMappers() { + return MOCK_OBJECT_MAPPERS; + } + + static final List MOCK_OBJECT_MAPPERS = List.of( + nestedObject(NESTED_OBJECT), + nestedObject(NESTED_OBJECT + "." + NESTED_OBJECT2), + nestedObject("nested_reseller"), + nestedObject("nested_chapters"), + nestedObject("nested_field") + ); + + private static ObjectMapper nestedObject(String path) { + return new ObjectMapper.Builder(path, Version.CURRENT).nested(ObjectMapper.Nested.newNested()).build(new ContentPath()); + } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorTests.java index dbcfa73d895b8..dd0ffdcfcea60 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregatorTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.NestedPathFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.mapper.ObjectMapper; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.search.aggregations.AggregationBuilder; @@ -43,11 +44,7 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; import java.util.stream.LongStream; import static java.util.stream.Collectors.toList; @@ -64,16 +61,6 @@ public class ReverseNestedAggregatorTests extends AggregatorTestCase { private static final String REVERSE_AGG_NAME = "reverseNestedAgg"; private static final String MAX_AGG_NAME = "maxAgg"; - /** - * For each provided field type, we also register an alias with name {@code -alias}. - */ - @Override - protected Map getFieldAliases(MappedFieldType... fieldTypes) { - return Arrays.stream(fieldTypes).collect(Collectors.toMap( - ft -> ft.name() + "-alias", - Function.identity())); - } - /** * Nested aggregations need the {@linkplain DirectoryReader} wrapped. */ @@ -264,4 +251,8 @@ public void testNestedUnderTerms() throws IOException { }, NestedAggregatorTests.resellersMappedFields()); } + @Override + protected List objectMappers() { + return NestedAggregatorTests.MOCK_OBJECT_MAPPERS; + } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTermsAggregatorTests.java index 1c8b469b907f2..ef8f2dac47ed1 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTermsAggregatorTests.java @@ -55,12 +55,8 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceType; import java.io.IOException; -import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; import static org.elasticsearch.search.aggregations.AggregationBuilders.significantTerms; import static org.hamcrest.Matchers.equalTo; @@ -90,16 +86,6 @@ protected List unsupportedMappedFieldTypes() { ); } - /** - * For each provided field type, we also register an alias with name {@code -alias}. - */ - @Override - protected Map getFieldAliases(MappedFieldType... fieldTypes) { - return Arrays.stream(fieldTypes).collect(Collectors.toMap( - ft -> ft.name() + "-alias", - Function.identity())); - } - /** * Uses the significant terms aggregation to find the keywords in text fields */ diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java index fca4313740077..8f60825732757 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java @@ -34,14 +34,10 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.index.analysis.AnalyzerScope; -import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.mapper.BinaryFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType; -import org.elasticsearch.mock.orig.Mockito; import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.bucket.sampler.InternalSampler; @@ -53,27 +49,12 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; import static org.elasticsearch.search.aggregations.AggregationBuilders.sampler; import static org.elasticsearch.search.aggregations.AggregationBuilders.significantText; import static org.hamcrest.Matchers.equalTo; -import static org.mockito.Mockito.when; public class SignificantTextAggregatorTests extends AggregatorTestCase { - - /** - * For each provided field type, we also register an alias with name {@code -alias}. - */ - @Override - protected Map getFieldAliases(MappedFieldType... fieldTypes) { - return Arrays.stream(fieldTypes).collect(Collectors.toMap( - ft -> ft.name() + "-alias", - Function.identity())); - } - @Override protected AggregationBuilder createAggBuilderForTypeTest(MappedFieldType fieldType, String fieldName) { return new SignificantTextAggregationBuilder("foo", fieldName); @@ -365,13 +346,4 @@ public void testSignificanceOnTextArrays() throws IOException { } } } - - @Override - protected MapperService.Snapshot mapperSnapshotMock() { - MapperService.Snapshot mapperSnapshot = super.mapperSnapshotMock(); - when(mapperSnapshot.indexAnalyzer(Mockito.any(), Mockito.any())).thenReturn( - new NamedAnalyzer("typeTestFieldName", AnalyzerScope.GLOBAL, new StandardAnalyzer()) - ); - return mapperSnapshot; - } } diff --git a/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java b/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java index abde78980bebc..3ef267cdfb93f 100644 --- a/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java @@ -716,7 +716,8 @@ private static QueryShardContext newQueryShardContext(MapperService mapperServic null, null, null, - mapperService.snapshot(), + mapperService, + mapperService.lookup(), null, null, null, diff --git a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java index f30158f2fbad0..5fe79724c9ac5 100644 --- a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java @@ -279,7 +279,7 @@ public void testBuildSearchContextHighlight() throws IOException { IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings); // shard context will only need indicesQueriesRegistry for building Query objects nested in highlighter QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, null, null, null, null, xContentRegistry(), namedWriteableRegistry, + null, null, null, null, null, null, xContentRegistry(), namedWriteableRegistry, null, null, System::currentTimeMillis, null, null, () -> true, null, emptyMap()) { @Override public MappedFieldType getFieldType(String name) { diff --git a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java index 1aadff69e24c5..9e27f5a663346 100644 --- a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.DeprecationHandler; @@ -157,8 +158,9 @@ private static void assertEquals(ShardSearchRequest orig, ShardSearchRequest cop assertEquals(orig.searchType(), copy.searchType()); assertEquals(orig.shardId(), copy.shardId()); assertEquals(orig.numberOfShards(), copy.numberOfShards()); - long localMappingVersion = randomLong(); - assertEquals(orig.cacheKey(localMappingVersion), copy.cacheKey(localMappingVersion)); + String randomCacheKey = randomAlphaOfLength(5); + Writeable cacheKey = w -> w.writeString(randomCacheKey); + assertEquals(orig.cacheKey(cacheKey), copy.cacheKey(cacheKey)); assertNotSame(orig, copy); assertEquals(orig.getAliasFilter(), copy.getAliasFilter()); assertEquals(orig.indexBoost(), copy.indexBoost(), 0.0f); diff --git a/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java b/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java index 4c2749c5c0c7b..c6ff6ed32d8ec 100644 --- a/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java @@ -143,7 +143,7 @@ public void testBuildRescoreSearchContext() throws ElasticsearchParseException, IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAlphaOfLengthBetween(1, 10), indexSettings); // shard context will only need indicesQueriesRegistry for building Query objects nested in query rescorer QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, null, null, null, null, + null, null, null, null, null, null, xContentRegistry(), namedWriteableRegistry, null, null, () -> nowInMillis, null, null, () -> true, null, emptyMap()) { @Override public MappedFieldType getFieldType(String name) { @@ -187,7 +187,7 @@ public void testRewritingKeepsSettings() throws IOException { IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAlphaOfLengthBetween(1, 10), indexSettings); // shard context will only need indicesQueriesRegistry for building Query objects nested in query rescorer QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, null, null, null, null, + null, null, null, null, null, null, xContentRegistry(), namedWriteableRegistry, null, null, () -> nowInMillis, null, null, () -> true, null, emptyMap()) { @Override public MappedFieldType getFieldType(String name) { diff --git a/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java b/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java index 38a721fd117dc..894dba54efff7 100644 --- a/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java @@ -202,7 +202,7 @@ protected final QueryShardContext createMockShardContext(IndexSearcher searcher) return builder.build(new IndexFieldDataCache.None(), null); }; return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, bitsetFilterCache, indexFieldDataLookup, - null, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, searcher, + null, null, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, searcher, () -> randomNonNegativeLong(), null, null, () -> true, null, emptyMap()) { @Override diff --git a/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java b/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java index d365ed414c6d0..d939d661d0a5c 100644 --- a/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java @@ -39,7 +39,8 @@ import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.StubSnapshot; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MappingLookupUtils; import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.index.mapper.TextSearchInfo; import org.elasticsearch.index.query.QueryShardContext; @@ -56,7 +57,6 @@ import java.io.IOException; import java.util.Collections; import java.util.HashMap; -import java.util.Map; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; @@ -176,15 +176,13 @@ public NamedAnalyzer get(Object key) { }, Collections.emptyMap(), Collections.emptyMap()); - MapperService.Snapshot mapperSnapshot = new StubSnapshot(fieldName -> fieldType) { - public IndexAnalyzers getIndexAnalyzers() { - return indexAnalyzers; - } - }; + MapperService mapperService = mock(MapperService.class); + when(mapperService.getIndexAnalyzers()).thenReturn(indexAnalyzers); + MappingLookup lookup = MappingLookupUtils.fromTypes(fieldType); when(scriptService.compile(any(Script.class), any())).then(invocation -> new TestTemplateService.MockTemplateScript.Factory( ((Script) invocation.getArguments()[0]).getIdOrCode())); QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, - null, mapperSnapshot, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, + null, mapperService, lookup, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, System::currentTimeMillis, null, null, () -> true, null, emptyMap()); SuggestionContext suggestionContext = suggestionBuilder.build(mockShardContext); @@ -218,10 +216,9 @@ public void testBuildWithUnmappedField() { Settings indexSettings = builder.build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index(randomAlphaOfLengthBetween(1, 10), "_na_"), indexSettings); - MapperService.Snapshot mapperSnapshot = new StubSnapshot(Map.of()); QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, - null, mapperSnapshot, null, null, xContentRegistry(), namedWriteableRegistry, null, null, + null, mock(MapperService.class), MappingLookup.EMPTY, null, null, xContentRegistry(), namedWriteableRegistry, null, null, System::currentTimeMillis, null, null, () -> true, null, emptyMap()); if (randomBoolean()) { mockShardContext.setAllowUnmappedFields(randomBoolean()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java new file mode 100644 index 0000000000000..6715577e0177b --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java @@ -0,0 +1,47 @@ +package org.elasticsearch.index.mapper; + +import org.elasticsearch.index.analysis.NamedAnalyzer; +import org.elasticsearch.index.mapper.FieldMapper.CopyTo; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; + +import static java.util.stream.Collectors.toList; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class MappingLookupUtils { + public static MappingLookup fromTypes(MappedFieldType... types) { + return fromTypes(Arrays.stream(types)); + } + + public static MappingLookup fromTypes(Stream types) { + List mappers = types.map(MappingLookupUtils::mockFieldMapper).collect(toList()); + // Alias -alias to so we can test aliases + return new MappingLookup( + "_doc", + mappers, + List.of(), + List.of(), + List.of(), + 0, + souceToParse -> null, + w -> w.writeString("test" + mappers.hashCode()) + ); + } + + public static FieldMapper mockFieldMapper(MappedFieldType type) { + FieldMapper mapper = mock(FieldMapper.class); + when(mapper.fieldType()).thenReturn(type); + when(mapper.name()).thenReturn(type.name()); + when(mapper.copyTo()).thenReturn(CopyTo.empty()); + Map indexAnalyzers = Map.of(); + if (type.getTextSearchInfo() != TextSearchInfo.NONE) { + indexAnalyzers = Map.of(mapper.name(), type.getTextSearchInfo().getSearchAnalyzer()); + } + when(mapper.indexAnalyzers()).thenReturn(indexAnalyzers); + return mapper; + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/StubSnapshot.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/StubSnapshot.java deleted file mode 100644 index d8093839208c6..0000000000000 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/StubSnapshot.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.mapper; - -import org.apache.lucene.search.Query; -import org.apache.lucene.search.join.BitSetProducer; -import org.elasticsearch.common.regex.Regex; -import org.elasticsearch.index.analysis.IndexAnalyzers; -import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext; -import org.elasticsearch.search.NestedDocuments; - -import java.util.Map; -import java.util.Set; -import java.util.function.Function; -import java.util.function.Supplier; - -import static java.util.Collections.singleton; -import static java.util.Collections.unmodifiableSet; - -/** - * A {@link MapperService.Snapshot} with the "central" methods that are useful for testing. - */ -public class StubSnapshot implements MapperService.Snapshot { - private final Function lookup; - private final Supplier> fields; - - public StubSnapshot(Function lookup) { - this.lookup = lookup; - this.fields = () -> { throw new UnsupportedOperationException(); }; - } - - public StubSnapshot(Map lookup) { - this.lookup = lookup::get; - this.fields = lookup::keySet; - } - - @Override - public MappedFieldType fieldType(String fullName) { - return lookup.apply(fullName); - } - - @Override - public Set simpleMatchToFullName(String pattern) { - if (Regex.isSimpleMatchPattern(pattern) == false) { - return singleton(pattern); - } - if (Regex.isMatchAllPattern(pattern)) { - return unmodifiableSet(fields.get()); - } - throw new UnsupportedOperationException(); - } - - @Override - public boolean hasNested() { - return false; - } - - @Override - public boolean hasMappings() { - return true; - } - - @Override - public boolean sourceEnabled() { - return true; - } - - @Override - public ObjectMapper getObjectMapper(String name) { - throw new UnsupportedOperationException(); - } - - @Override - public Set sourcePath(String fullName) { - throw new UnsupportedOperationException(); - } - - @Override - public DocumentMapperForType documentMapperWithAutoCreate() { - throw new UnsupportedOperationException(); - } - - @Override - public Iterable getEagerGlobalOrdinalsFields() { - throw new UnsupportedOperationException(); - } - - @Override - public NamedAnalyzer indexAnalyzer(String field, Function unindexedFieldAnalyzer) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean containsBrokenAnalysis(String field) { - throw new UnsupportedOperationException(); - } - - @Override - public long version() { - throw new UnsupportedOperationException(); - } - - @Override - public ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { - throw new UnsupportedOperationException(); - } - - @Override - public ParserContext parserContext() { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isMetadataField(String field) { - throw new UnsupportedOperationException(); - } - - @Override - public IndexAnalyzers getIndexAnalyzers() { - throw new UnsupportedOperationException(); - } - - @Override - public NestedDocuments getNestedDocuments(Function filterProducer) { - throw new UnsupportedOperationException(); - } -} diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index a6c5511e71b38..ce3e0091c779b 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -56,7 +56,6 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.settings.Settings; @@ -86,10 +85,10 @@ import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; -import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MappingLookupUtils; import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.mapper.ObjectMapper; -import org.elasticsearch.index.mapper.ObjectMapper.Nested; import org.elasticsearch.index.mapper.RangeFieldMapper; import org.elasticsearch.index.mapper.RangeType; import org.elasticsearch.index.mapper.TextFieldMapper; @@ -104,7 +103,6 @@ import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.script.ScriptService; -import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.MultiBucketConsumerService.MultiBucketConsumer; @@ -137,19 +135,15 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.function.Consumer; -import java.util.function.Function; import java.util.function.Supplier; -import java.util.stream.Collectors; import static java.util.Collections.emptyMap; import static java.util.Collections.singletonList; +import static java.util.stream.Collectors.toList; import static org.elasticsearch.test.InternalAggregationTestCase.DEFAULT_MAX_BUCKETS; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -159,7 +153,6 @@ * {@link AggregationBuilder} instance. */ public abstract class AggregatorTestCase extends ESTestCase { - private static final String NESTEDFIELD_PREFIX = "nested_"; private List releasables = new ArrayList<>(); protected ValuesSourceRegistry valuesSourceRegistry; @@ -173,22 +166,6 @@ public abstract class AggregatorTestCase extends ESTestCase { FieldAliasMapper.CONTENT_TYPE // TODO support alias ); - /** - * Allows subclasses to provide alternate names for the provided field type, which - * can be useful when testing aggregations on field aliases. - */ - protected Map getFieldAliases(MappedFieldType... fieldTypes) { - return Collections.emptyMap(); - } - - private static void registerFieldTypes(MapperService.Snapshot mapperSnapshot, Map fieldNameToType) { - for (Map.Entry entry : fieldNameToType.entrySet()) { - String fieldName = entry.getKey(); - MappedFieldType fieldType = entry.getValue(); - when(mapperSnapshot.fieldType(fieldName)).thenReturn(fieldType); - } - } - @Before public void initValuesSourceRegistry() { List plugins = new ArrayList<>(getSearchPlugins()); @@ -246,23 +223,19 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche */ BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), breakerService).withCircuitBreaking(); - MapperService.Snapshot mapperSnapshot = mapperSnapshotMock(); - when(mapperSnapshot.hasNested()).thenReturn(false); - when(mapperSnapshot.getNestedDocuments(any())).thenReturn(new NestedDocuments(null, null)); - when(mapperSnapshot.indexAnalyzer(anyString(), any())).thenReturn(Lucene.STANDARD_ANALYZER); // for significant text - Map fieldNameToType = new HashMap<>(); - fieldNameToType.putAll(Arrays.stream(fieldTypes) - .filter(Objects::nonNull) - .collect(Collectors.toMap(MappedFieldType::name, Function.identity()))); - fieldNameToType.putAll(getFieldAliases(fieldTypes)); - registerFieldTypes(mapperSnapshot, fieldNameToType); - when(mapperSnapshot.getObjectMapper(anyString())).thenAnswer(invocation -> { - String fieldName = (String) invocation.getArguments()[0]; - if (fieldName.startsWith(NESTEDFIELD_PREFIX)) { - return new ObjectMapper.Builder(fieldName, Version.CURRENT).nested(Nested.newNested()).build(new ContentPath()); - } - return null; - }); + MappingLookup mappingLookup = new MappingLookup( + "_doc", + Arrays.stream(fieldTypes).map(MappingLookupUtils::mockFieldMapper).collect(toList()), + objectMappers(), + // Alias all fields to -alias to test aliases + Arrays.stream(fieldTypes) + .map(ft -> new FieldAliasMapper(ft.name() + "-alias", ft.name() + "-alias", ft.name())) + .collect(toList()), + List.of(), + 0, + souceToParse -> null, + w -> w.writeString("test") + ); TriFunction, IndexFieldData> fieldDataBuilder = ( fieldType, @@ -282,7 +255,8 @@ public void onCache(ShardId shardId, Accountable accountable) {} bigArrays, bitsetFilterCache, fieldDataBuilder, - mapperSnapshot, + null, + mappingLookup, null, getMockScriptService(), xContentRegistry(), @@ -303,7 +277,7 @@ public void onCache(ShardId shardId, Accountable accountable) {} query, null, consumer, - () -> buildSubSearchContext(indexSettings, mapperSnapshot, queryShardContext, bitsetFilterCache), + () -> buildSubSearchContext(indexSettings, queryShardContext, bitsetFilterCache), releasables::add, bitsetFilterCache, randomInt(), @@ -312,12 +286,19 @@ public void onCache(ShardId shardId, Accountable accountable) {} ); } + /** + * {@link ObjectMapper}s to add to the lookup. By default we don't need + * any {@link ObjectMapper}s but testing nested objects will require adding some. + */ + protected List objectMappers() { + return List.of(); + } + /** * Build a {@link SubSearchContext}s to power {@code top_hits}. */ private SubSearchContext buildSubSearchContext( IndexSettings indexSettings, - MapperService.Snapshot mapperSnapshot, QueryShardContext queryShardContext, BitsetFilterCache bitsetFilterCache ) { @@ -366,13 +347,6 @@ protected IndexSettings createIndexSettings() { ); } - /** - * sub-tests that need a more complex mock can overwrite this - */ - protected MapperService.Snapshot mapperSnapshotMock() { - return mock(MapperService.Snapshot.class); - } - /** * Sub-tests that need scripting can override this method to provide a script service and pre-baked scripts */ diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java index c2d84fec864f5..542ff3439d209 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java @@ -411,9 +411,27 @@ public void close() throws IOException { } QueryShardContext createShardContext(IndexSearcher searcher) { - return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, bitsetFilterCache, - indexFieldDataService::getForField, mapperService.snapshot(), similarityService, scriptService, xContentRegistry, - namedWriteableRegistry, this.client, searcher, () -> nowInMillis, null, indexNameMatcher(), () -> true, null, emptyMap()); + return new QueryShardContext( + 0, + idxSettings, + BigArrays.NON_RECYCLING_INSTANCE, + bitsetFilterCache, + indexFieldDataService::getForField, + mapperService, + mapperService.lookup(), + similarityService, + scriptService, + xContentRegistry, + namedWriteableRegistry, + this.client, + searcher, + () -> nowInMillis, + null, + indexNameMatcher(), + () -> true, + null, + emptyMap() + ); } ScriptModule createScriptModule(List scriptPlugins) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java index 1d7e57704d041..4065fdf2bb3a7 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java @@ -33,8 +33,9 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.KeywordFieldMapper; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.StubSnapshot; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MappingLookupUtils; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.TermQueryBuilder; @@ -75,7 +76,6 @@ public class DocumentSubsetBitsetCacheTests extends ESTestCase { - private static final String MISSING_FIELD_NAME = "does-not-exist"; private static final int FIELD_COUNT = 10; private ExecutorService singleThreadExecutor; @@ -107,7 +107,7 @@ public void testSameBitSetIsReturnedForIdenticalQuery() throws Exception { public void testNullBitSetIsReturnedForNonMatchingQuery() throws Exception { final DocumentSubsetBitsetCache cache = newCache(Settings.EMPTY); runTestOnIndex((shardContext, leafContext) -> { - final Query query = QueryBuilders.termQuery(MISSING_FIELD_NAME, "any-value").rewrite(shardContext).toQuery(shardContext); + final Query query = QueryBuilders.termQuery("not-mapped", "any-value").rewrite(shardContext).toQuery(shardContext); final BitSet bitSet = cache.getBitSet(query, leafContext); assertThat(bitSet, nullValue()); }); @@ -537,7 +537,7 @@ public void close() throws IOException { } } - private TestIndexContext testIndex(MapperService.Snapshot mapperSnapshot, Client client) throws IOException { + private TestIndexContext testIndex(MappingLookup mappingLookup, Client client) throws IOException { TestIndexContext context = null; final long nowInMillis = randomNonNegativeLong(); @@ -565,7 +565,7 @@ private TestIndexContext testIndex(MapperService.Snapshot mapperSnapshot, Client final LeafReaderContext leaf = directoryReader.leaves().get(0); final QueryShardContext shardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, null, mapperSnapshot, null, null, xContentRegistry(), writableRegistry(), + null, null, null, mappingLookup, null, null, xContentRegistry(), writableRegistry(), client, new IndexSearcher(directoryReader), () -> nowInMillis, null, null, () -> true, null, emptyMap()); context = new TestIndexContext(directory, iw, directoryReader, shardContext, leaf); @@ -586,13 +586,14 @@ null, null, mapperSnapshot, null, null, xContentRegistry(), writableRegistry(), } private void runTestOnIndices(int numberIndices, CheckedConsumer, Exception> body) throws Exception { - MapperService.Snapshot mapperSnapshot = new StubSnapshot(fieldName -> { - if (fieldName.equals(MISSING_FIELD_NAME)) { - return null; - } else { - return new KeywordFieldMapper.KeywordFieldType(fieldName); - } - }); + List types = new ArrayList<>(); + for (int i = 0; i < 7; i++) { // the tests use fields 1 to 6. + // This field has a value. + types.add(new KeywordFieldMapper.KeywordFieldType("field-" + i)); + // This field never has a value + types.add(new KeywordFieldMapper.KeywordFieldType("dne-" + i)); + } + MappingLookup mappingLookup = MappingLookupUtils.fromTypes(types.stream()); final Client client = mock(Client.class); when(client.settings()).thenReturn(Settings.EMPTY); @@ -600,7 +601,7 @@ private void runTestOnIndices(int numberIndices, CheckedConsumer context = new ArrayList<>(numberIndices); try { for (int i = 0; i < numberIndices; i++) { - context.add(testIndex(mapperSnapshot, client)); + context.add(testIndex(mappingLookup, client)); } body.accept(context); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java index 75d8c396cc560..f92fda1c6d119 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java @@ -29,9 +29,9 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.mapper.KeywordFieldMapper; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.StubSnapshot; +import org.elasticsearch.index.mapper.KeywordFieldMapper.KeywordFieldType; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MappingLookupUtils; import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.TermsQueryBuilder; @@ -66,7 +66,7 @@ public class SecurityIndexReaderWrapperIntegrationTests extends AbstractBuilderT public void testDLS() throws Exception { ShardId shardId = new ShardId("_index", "_na_", 0); - MapperService.Snapshot mapperSnapshot = new StubSnapshot(fieldName -> new KeywordFieldMapper.KeywordFieldType(fieldName)); + MappingLookup mappingLookup = MappingLookupUtils.fromTypes(new KeywordFieldType("field")); ScriptService scriptService = mock(ScriptService.class); final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); @@ -82,7 +82,7 @@ public void testDLS() throws Exception { when(client.settings()).thenReturn(Settings.EMPTY); final long nowInMillis = randomNonNegativeLong(); QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, null, mapperSnapshot, null, null, xContentRegistry(), writableRegistry(), + null, null, null, mappingLookup, null, null, xContentRegistry(), writableRegistry(), client, null, () -> nowInMillis, null, null, () -> true, null, emptyMap()); QueryShardContext queryShardContext = spy(realQueryShardContext); DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY, Executors.newSingleThreadExecutor()); @@ -173,7 +173,11 @@ protected IndicesAccessControl getIndicesAccessControl() { public void testDLSWithLimitedPermissions() throws Exception { ShardId shardId = new ShardId("_index", "_na_", 0); - MapperService.Snapshot mapperSnapshot = new StubSnapshot(fieldName -> new KeywordFieldMapper.KeywordFieldType(fieldName)); + MappingLookup mappingLookup = MappingLookupUtils.fromTypes( + new KeywordFieldType("field"), + new KeywordFieldType("f1"), + new KeywordFieldType("f2") + ); ScriptService scriptService = mock(ScriptService.class); final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); @@ -207,7 +211,7 @@ public void testDLSWithLimitedPermissions() throws Exception { when(client.settings()).thenReturn(Settings.EMPTY); final long nowInMillis = randomNonNegativeLong(); QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE, - null, null, mapperSnapshot, null, null, xContentRegistry(), writableRegistry(), + null, null, null, mappingLookup, null, null, xContentRegistry(), writableRegistry(), client, null, () -> nowInMillis, null, null, () -> true, null, emptyMap()); QueryShardContext queryShardContext = spy(realQueryShardContext); DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY, Executors.newSingleThreadExecutor()); diff --git a/x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/index/mapper/FlattenedFieldLookupTests.java b/x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/index/mapper/FlattenedFieldLookupTests.java index e31ea7868f791..bd7fa9fcb9968 100644 --- a/x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/index/mapper/FlattenedFieldLookupTests.java +++ b/x-pack/plugin/mapper-flattened/src/test/java/org/elasticsearch/index/mapper/FlattenedFieldLookupTests.java @@ -39,7 +39,7 @@ public void testFieldTypeLookup() { String fieldName = "object1.object2.field"; FlattenedFieldMapper mapper = createFlattenedMapper(fieldName); - FieldTypeLookup lookup = new FieldTypeLookup(singletonList(mapper), emptyList(), emptyList()); + FieldTypeLookup lookup = new FieldTypeLookup("_doc", singletonList(mapper), emptyList(), emptyList()); assertEquals(mapper.fieldType(), lookup.get(fieldName)); String objectKey = "key1.key2"; @@ -60,7 +60,7 @@ public void testFieldTypeLookupWithAlias() { String aliasName = "alias"; FieldAliasMapper alias = new FieldAliasMapper(aliasName, aliasName, fieldName); - FieldTypeLookup lookup = new FieldTypeLookup(singletonList(mapper), singletonList(alias), emptyList()); + FieldTypeLookup lookup = new FieldTypeLookup("_doc", singletonList(mapper), singletonList(alias), emptyList()); assertEquals(mapper.fieldType(), lookup.get(aliasName)); String objectKey = "key1.key2"; @@ -83,11 +83,11 @@ public void testFieldTypeLookupWithMultipleFields() { FlattenedFieldMapper mapper2 = createFlattenedMapper(field2); FlattenedFieldMapper mapper3 = createFlattenedMapper(field3); - FieldTypeLookup lookup = new FieldTypeLookup(Arrays.asList(mapper1, mapper2), emptyList(), emptyList()); + FieldTypeLookup lookup = new FieldTypeLookup("_doc", Arrays.asList(mapper1, mapper2), emptyList(), emptyList()); assertNotNull(lookup.get(field1 + ".some.key")); assertNotNull(lookup.get(field2 + ".some.key")); - lookup = new FieldTypeLookup(Arrays.asList(mapper1, mapper2, mapper3), emptyList(), emptyList()); + lookup = new FieldTypeLookup("_doc", Arrays.asList(mapper1, mapper2, mapper3), emptyList(), emptyList()); assertNotNull(lookup.get(field1 + ".some.key")); assertNotNull(lookup.get(field2 + ".some.key")); assertNotNull(lookup.get(field3 + ".some.key")); @@ -124,7 +124,7 @@ public void testFieldLookupIterator() { MockFieldMapper mapper = new MockFieldMapper("foo"); FlattenedFieldMapper flattenedMapper = createFlattenedMapper("object1.object2.field"); - FieldTypeLookup lookup = new FieldTypeLookup(Arrays.asList(mapper, flattenedMapper), emptyList(), emptyList()); + FieldTypeLookup lookup = new FieldTypeLookup("_doc", Arrays.asList(mapper, flattenedMapper), emptyList(), emptyList()); Set fieldNames = new HashSet<>(); lookup.filter(ft -> true).forEach(ft -> fieldNames.add(ft.name())); diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java index ab0a57d6d3e10..c1996a6734d83 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java @@ -89,7 +89,7 @@ public class RollupIndexerIndexingTests extends AggregatorTestCase { private void setup() { settings = createIndexSettings(); queryShardContext = new QueryShardContext(0, settings, - BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, + BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, null, null, null, null, null, () -> 0L, null, null, () -> true, null, emptyMap()); } diff --git a/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapperTests.java b/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapperTests.java index eb5b9b06bfb13..2eedcd680b97c 100644 --- a/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapperTests.java +++ b/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapperTests.java @@ -898,7 +898,7 @@ protected final QueryShardContext createMockShardContext() { return builder.build(new IndexFieldDataCache.None(), null); }; return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, bitsetFilterCache, indexFieldDataLookup, - null, null, null, xContentRegistry(), null, null, null, + null, null, null, null, xContentRegistry(), null, null, null, () -> randomNonNegativeLong(), null, null, () -> true, null, emptyMap()) { @Override From dde49961be732ae6bcbf8d4606ff906fbe99cb12 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 14 Dec 2020 12:59:50 -0500 Subject: [PATCH 14/47] Checkstyle --- .../index/mapper/MappingLookupUtils.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java index 6715577e0177b..795d489f1aed8 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java @@ -1,3 +1,22 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.elasticsearch.index.mapper; import org.elasticsearch.index.analysis.NamedAnalyzer; From 7519d7787b2f0b8b49b02f60e138e5d28a0ed73d Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 14 Dec 2020 13:14:51 -0500 Subject: [PATCH 15/47] Implement sourceEnabled --- .../index/mapper/MappingLookup.java | 22 ++++++++++----- .../MetadataRolloverServiceTests.java | 3 ++- .../FieldAliasMapperValidationTests.java | 27 ++++++++++++------- .../index/mapper/MappingLookupTests.java | 10 ++++--- .../index/mapper/MappingLookupUtils.java | 3 ++- .../aggregations/AggregatorTestCase.java | 3 ++- 6 files changed, 45 insertions(+), 23 deletions(-) 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 8897d3e7f1660..da572e757cdc7 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -46,7 +46,8 @@ public final class MappingLookup { List.of(), 0, souceToParse -> null, - writer -> writer.writeString("empty") + writer -> writer.writeString("empty"), + false ); /** Full field name to mapper */ @@ -55,11 +56,16 @@ public final class MappingLookup { private final boolean hasNested; private final FieldTypeLookup fieldTypeLookup; private final int metadataFieldCount; - private final Function documentParser; private final Map indexAnalyzers = new HashMap<>(); + private final Function documentParser; private final Writeable cacheKey; + private final boolean sourceEnabled; - public static MappingLookup fromMapping(Mapping mapping, Function documentParser, Writeable cacheKey) { + public static MappingLookup fromMapping( + Mapping mapping, + Function documentParser, + Writeable cacheKey + ) { List newObjectMappers = new ArrayList<>(); List newFieldMappers = new ArrayList<>(); List newFieldAliasMappers = new ArrayList<>(); @@ -79,7 +85,8 @@ public static MappingLookup fromMapping(Mapping mapping, Function runtimeFieldTypes, int metadataFieldCount, Function documentParser, - Writeable cacheKey) { + Writeable cacheKey, + boolean sourceEnabled) { this.documentParser = documentParser; this.cacheKey = cacheKey; + this.sourceEnabled = sourceEnabled; Map fieldMappers = new HashMap<>(); Map objects = new HashMap<>(); @@ -284,8 +293,7 @@ public boolean isEmpty() { } public boolean isSourceEnabled() { - // NOCOMMIT implement me - return false; + return sourceEnabled; } /** diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java index 158cd144472ba..86a68bb5fafa2 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java @@ -570,7 +570,8 @@ protected String contentType() { List.of(), 0, null, - null + null, + false ); ClusterService clusterService = ClusterServiceUtils.createClusterService(testThreadPool); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java index 280b73b491fbd..b625924915df4 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java @@ -46,7 +46,8 @@ public void testDuplicateFieldAliasAndObject() { emptyList(), 0, null, - null + null, + false ) ); assertEquals("Alias [some.path] is defined both as an object and an alias", e.getMessage()); @@ -67,7 +68,8 @@ public void testDuplicateFieldAliasAndConcreteField() { emptyList(), 0, null, - null + null, + false ) ); assertEquals("Alias [invalid] is defined both as an alias and a concrete field", e.getMessage()); @@ -86,7 +88,8 @@ public void testAliasThatRefersToAlias() { emptyList(), 0, null, - null + null, + false ); alias.validate(mappers); @@ -110,7 +113,8 @@ public void testAliasThatRefersToItself() { emptyList(), 0, null, - null + null, + false ); invalidAlias.validate(mappers); }); @@ -131,7 +135,8 @@ public void testAliasWithNonExistentPath() { emptyList(), 0, null, - null + null, + false ); invalidAlias.validate(mappers); }); @@ -152,7 +157,8 @@ public void testFieldAliasWithNestedScope() { emptyList(), 0, null, - null + null, + false ); aliasMapper.validate(mappers); } @@ -169,7 +175,8 @@ public void testFieldAliasWithDifferentObjectScopes() { emptyList(), 0, null, - null + null, + false ); aliasMapper.validate(mappers); } @@ -187,7 +194,8 @@ public void testFieldAliasWithNestedTarget() { emptyList(), 0, null, - null + null, + false ); aliasMapper.validate(mappers); }); @@ -210,7 +218,8 @@ public void testFieldAliasWithDifferentNestedScopes() { emptyList(), 0, null, - null + null, + false ); aliasMapper.validate(mappers); }); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java b/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java index bd05916d47fdc..48eff03347c2b 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java @@ -41,7 +41,7 @@ public class MappingLookupTests extends ESTestCase { public void testOnlyRuntimeField() { MappingLookup mappingLookup = new MappingLookup("_doc", Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), - Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, null); + Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, null, false); assertEquals(0, size(mappingLookup.fieldMappers())); assertEquals(0, mappingLookup.objectMappers().size()); assertNull(mappingLookup.getMapper("test")); @@ -51,7 +51,7 @@ public void testOnlyRuntimeField() { public void testRuntimeFieldLeafOverride() { MockFieldMapper fieldMapper = new MockFieldMapper("test"); MappingLookup mappingLookup = new MappingLookup("_doc", Collections.singletonList(fieldMapper), Collections.emptyList(), - Collections.emptyList(), Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, null); + Collections.emptyList(), Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, null, false); assertThat(mappingLookup.getMapper("test"), instanceOf(MockFieldMapper.class)); assertEquals(1, size(mappingLookup.fieldMappers())); assertEquals(0, mappingLookup.objectMappers().size()); @@ -71,7 +71,8 @@ public void testSubfieldOverride() { Collections.singletonList(new TestRuntimeField("object.subfield", "type")), 0, null, - null + null, + false ); assertThat(mappingLookup.getMapper("object.subfield"), instanceOf(MockFieldMapper.class)); assertEquals(1, size(mappingLookup.fieldMappers())); @@ -96,7 +97,8 @@ public void testAnalyzers() throws IOException { Collections.emptyList(), 0, null, - null + null, + false ); assertAnalyzes(mappingLookup.indexAnalyzer("field1", f -> null), "field1", "index1"); diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java index 795d489f1aed8..5b32a7e338283 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java @@ -47,7 +47,8 @@ public static MappingLookup fromTypes(Stream types) { List.of(), 0, souceToParse -> null, - w -> w.writeString("test" + mappers.hashCode()) + w -> w.writeString("test" + mappers.hashCode()), + true ); } diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index ce3e0091c779b..c514ffb447938 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -234,7 +234,8 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche List.of(), 0, souceToParse -> null, - w -> w.writeString("test") + w -> w.writeString("test"), + true ); TriFunction, IndexFieldData> fieldDataBuilder = ( From 56cc6472187f4fdc7376640ddbd1fecceebd021c Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 14 Dec 2020 13:23:20 -0500 Subject: [PATCH 16/47] Fixup test --- .../index/mapper/MappingLookup.java | 2 +- .../NumericHistogramAggregatorTests.java | 6 ++---- .../bucket/nested/NestedAggregatorTests.java | 2 +- .../bucket/range/IpRangeAggregatorTests.java | 4 ++-- .../bucket/terms/RareTermsAggregatorTests.java | 8 ++++++++ .../bucket/terms/TermsAggregatorTests.java | 6 ++++++ .../aggregations/metrics/AvgAggregatorTests.java | 7 ++++--- .../metrics/CardinalityAggregatorTests.java | 16 ++++++++++------ .../aggregations/metrics/MaxAggregatorTests.java | 14 +++++++++----- .../MedianAbsoluteDeviationAggregatorTests.java | 8 ++++---- .../metrics/ValueCountAggregatorTests.java | 9 +++++---- .../index/mapper/MappingLookupUtils.java | 3 ++- .../search/aggregations/AggregatorTestCase.java | 11 ++++++++++- 13 files changed, 64 insertions(+), 32 deletions(-) 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 da572e757cdc7..dc4f595c692e2 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -34,7 +34,7 @@ import java.util.function.Function; import java.util.stream.Stream; -public final class MappingLookup { +public class MappingLookup { /** * A lookup representing an empty mapping. */ diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregatorTests.java index da94c3a61db9f..6b5333a907a81 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/NumericHistogramAggregatorTests.java @@ -262,10 +262,9 @@ public void testMissingUnmappedField() throws Exception { .field("field") .interval(5) .missing(2d); - MappedFieldType type = null; try (IndexReader reader = w.getReader()) { IndexSearcher searcher = new IndexSearcher(reader); - InternalHistogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, type); + InternalHistogram histogram = searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder); assertEquals(1, histogram.getBuckets().size()); @@ -289,11 +288,10 @@ public void testMissingUnmappedFieldBadType() throws Exception { .field("field") .interval(5) .missing(missingValue); - MappedFieldType type = null; try (IndexReader reader = w.getReader()) { IndexSearcher searcher = new IndexSearcher(reader); Throwable t = expectThrows(IllegalArgumentException.class, () -> { - searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder, type); + searchAndReduce(searcher, new MatchAllDocsQuery(), aggBuilder); }); // This throws a number format exception (which is a subclass of IllegalArgumentException) and might be ok? assertThat(t.getMessage(), containsString(missingValue)); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java index dc6540f1cfee1..864e056d8b784 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTests.java @@ -894,7 +894,7 @@ protected List objectMappers() { nestedObject("nested_field") ); - private static ObjectMapper nestedObject(String path) { + public static ObjectMapper nestedObject(String path) { return new ObjectMapper.Builder(path, Version.CURRENT).nested(ObjectMapper.Nested.newNested()).build(new ContentPath()); } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/range/IpRangeAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/range/IpRangeAggregatorTests.java index b811e150fe407..20eb8810eb970 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/range/IpRangeAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/range/IpRangeAggregatorTests.java @@ -148,7 +148,7 @@ public void testMissingUnmapped() throws Exception { .missing("192.168.100.42"); // Apparently we expect a string here try (IndexReader reader = w.getReader()) { IndexSearcher searcher = new IndexSearcher(reader); - InternalBinaryRange range = searchAndReduce(searcher, new MatchAllDocsQuery(), builder, (MappedFieldType) null); + InternalBinaryRange range = searchAndReduce(searcher, new MatchAllDocsQuery(), builder); assertEquals(1, range.getBuckets().size()); } } @@ -169,7 +169,7 @@ public void testMissingUnmappedBadType() throws Exception { try (IndexReader reader = w.getReader()) { IndexSearcher searcher = new IndexSearcher(reader); expectThrows(IllegalArgumentException.class, () -> { - searchAndReduce(searcher, new MatchAllDocsQuery(), builder, (MappedFieldType) null); + searchAndReduce(searcher, new MatchAllDocsQuery(), builder); }); } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java index c11926c6b969d..fb37c4947a01a 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/RareTermsAggregatorTests.java @@ -46,6 +46,7 @@ import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.NestedPathFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.mapper.ObjectMapper; import org.elasticsearch.index.mapper.RangeFieldMapper; import org.elasticsearch.index.mapper.RangeType; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -64,6 +65,7 @@ import org.elasticsearch.search.aggregations.bucket.global.InternalGlobal; import org.elasticsearch.search.aggregations.bucket.nested.InternalNested; import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregatorTests; import org.elasticsearch.search.aggregations.metrics.InternalTopHits; import org.elasticsearch.search.aggregations.metrics.Max; import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; @@ -601,4 +603,10 @@ public void doAssertReducedMultiBucketConsumer(Aggregation agg, MultiBucketConsu * buckets we should have left after each reduction. */ } + + @Override + protected List objectMappers() { + return List.of(NestedAggregatorTests.nestedObject("nested_object")); + } + } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java index c119cc1e48458..755327ad2af06 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorTests.java @@ -53,6 +53,7 @@ import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.NestedPathFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.mapper.ObjectMapper; import org.elasticsearch.index.mapper.RangeFieldMapper; import org.elasticsearch.index.mapper.RangeType; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -84,6 +85,7 @@ import org.elasticsearch.search.aggregations.bucket.global.InternalGlobal; import org.elasticsearch.search.aggregations.bucket.nested.InternalNested; import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregatorTests; import org.elasticsearch.search.aggregations.metrics.InternalTopHits; import org.elasticsearch.search.aggregations.metrics.TopHitsAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.BucketScriptPipelineAggregationBuilder; @@ -1500,4 +1502,8 @@ private T reduce(Aggregator agg, BigArrays bigAr return result; } + @Override + protected List objectMappers() { + return List.of(NestedAggregatorTests.nestedObject("nested_object")); + } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorTests.java index b74bb25cc600e..e4bbcdeeda87c 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/AvgAggregatorTests.java @@ -235,7 +235,7 @@ public void testUnmappedField() throws IOException { }, avg -> { assertEquals(Double.NaN, avg.getValue(), 0); assertFalse(AggregationInspectionHelper.hasValue(avg)); - }, (MappedFieldType) null); + }); } public void testUnmappedWithMissingField() throws IOException { @@ -246,7 +246,7 @@ public void testUnmappedWithMissingField() throws IOException { }, avg -> { assertEquals(0.0, avg.getValue(), 0); assertTrue(AggregationInspectionHelper.hasValue(avg)); - }, (MappedFieldType) null); + }); } private void verifyAvgOfDoubles(double[] values, double expected, double delta) throws IOException { @@ -570,7 +570,8 @@ private void testAggregation( Query query, CheckedConsumer buildIndex, Consumer verify, - MappedFieldType fieldType) throws IOException { + MappedFieldType... fieldType + ) throws IOException { testCase(aggregationBuilder, query, buildIndex, verify, fieldType); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorTests.java index 8b8c9f868d773..766e7e0cbdb2b 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorTests.java @@ -138,7 +138,7 @@ public void testUnmappedMissingString() throws IOException { }, card -> { assertEquals(1, card.getValue(), 0); assertTrue(AggregationInspectionHelper.hasValue(card)); - }, null); + }); } public void testUnmappedMissingNumber() throws IOException { @@ -152,7 +152,7 @@ public void testUnmappedMissingNumber() throws IOException { }, card -> { assertEquals(1, card.getValue(), 0); assertTrue(AggregationInspectionHelper.hasValue(card)); - }, null); + }); } public void testUnmappedMissingGeoPoint() throws IOException { @@ -166,7 +166,7 @@ public void testUnmappedMissingGeoPoint() throws IOException { }, card -> { assertEquals(1, card.getValue(), 0); assertTrue(AggregationInspectionHelper.hasValue(card)); - }, null); + }); } private void testAggregation(Query query, CheckedConsumer buildIndex, @@ -176,9 +176,13 @@ private void testAggregation(Query query, CheckedConsumer buildIndex, Consumer verify, - MappedFieldType fieldType) throws IOException { + private void testAggregation( + AggregationBuilder aggregationBuilder, + Query query, + CheckedConsumer buildIndex, + Consumer verify, + MappedFieldType... fieldType + ) throws IOException { testCase(aggregationBuilder, query, buildIndex, verify, fieldType); } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java index 49952e61c2a47..39f6e7ce3cdaf 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MaxAggregatorTests.java @@ -241,7 +241,7 @@ public void testUnmappedField() throws IOException { }, max -> { assertEquals(max.getValue(), Double.NEGATIVE_INFINITY, 0); assertFalse(AggregationInspectionHelper.hasValue(max)); - }, (MappedFieldType) null); + }); } public void testUnmappedWithMissingField() throws IOException { @@ -253,7 +253,7 @@ public void testUnmappedWithMissingField() throws IOException { }, max -> { assertEquals(max.getValue(), 19.0, 0); assertTrue(AggregationInspectionHelper.hasValue(max)); - }, (MappedFieldType) null); + }); } public void testMissingFieldOptimization() throws IOException { @@ -294,9 +294,13 @@ private void testAggregation(Query query, testAggregation(aggregationBuilder, query, buildIndex, verify, fieldType); } - private void testAggregation(AggregationBuilder aggregationBuilder, Query query, - CheckedConsumer buildIndex, Consumer verify, - MappedFieldType fieldType) throws IOException { + private void testAggregation( + AggregationBuilder aggregationBuilder, + Query query, + CheckedConsumer buildIndex, + Consumer verify, + MappedFieldType... fieldType + ) throws IOException { testCase(aggregationBuilder, query, buildIndex, verify, fieldType); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregatorTests.java index 0a14ebfdd7102..e68f4a64c5884 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregatorTests.java @@ -182,7 +182,7 @@ public void testUnmapped() throws IOException { }, agg -> { assertEquals(Double.NaN, agg.getMedianAbsoluteDeviation(),0); assertFalse(AggregationInspectionHelper.hasValue(agg)); - }, null); + }); } public void testUnmappedMissing() throws IOException { @@ -197,7 +197,7 @@ public void testUnmappedMissing() throws IOException { }, agg -> { assertEquals(0, agg.getMedianAbsoluteDeviation(), 0); assertTrue(AggregationInspectionHelper.hasValue(agg)); - }, null); + }); } public void testValueScript() throws IOException { @@ -260,8 +260,8 @@ private void testAggregation(Query query, private void testAggregation(AggregationBuilder aggregationBuilder, Query query, CheckedConsumer indexer, - Consumer verify, MappedFieldType fieldType) throws IOException { - testCase(aggregationBuilder, query, indexer, verify, fieldType); + Consumer verify, MappedFieldType... fieldTypes) throws IOException { + testCase(aggregationBuilder, query, indexer, verify, fieldTypes); } public static class IsCloseToRelative extends TypeSafeMatcher { diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregatorTests.java index 60782469d8486..69474ab9c8281 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregatorTests.java @@ -220,7 +220,7 @@ public void testUnmappedMissingString() throws IOException { }, valueCount -> { assertEquals(3, valueCount.getValue(), 0); assertTrue(AggregationInspectionHelper.hasValue(valueCount)); - }, null); + }); } public void testUnmappedMissingNumber() throws IOException { @@ -234,7 +234,7 @@ public void testUnmappedMissingNumber() throws IOException { }, valueCount -> { assertEquals(3, valueCount.getValue(), 0); assertTrue(AggregationInspectionHelper.hasValue(valueCount)); - }, null); + }); } public void testUnmappedMissingGeoPoint() throws IOException { @@ -248,7 +248,7 @@ public void testUnmappedMissingGeoPoint() throws IOException { }, valueCount -> { assertEquals(3, valueCount.getValue(), 0); assertTrue(AggregationInspectionHelper.hasValue(valueCount)); - }, null); + }); } public void testRangeFieldValues() throws IOException { @@ -391,7 +391,8 @@ private void testAggregation( Query query, CheckedConsumer buildIndex, Consumer verify, - MappedFieldType fieldType) throws IOException { + MappedFieldType... fieldType + ) throws IOException { testCase(aggregationBuilder, query, buildIndex, verify, fieldType); } diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java index 5b32a7e338283..ae85393d9d945 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java @@ -55,7 +55,8 @@ public static MappingLookup fromTypes(Stream types) { public static FieldMapper mockFieldMapper(MappedFieldType type) { FieldMapper mapper = mock(FieldMapper.class); when(mapper.fieldType()).thenReturn(type); - when(mapper.name()).thenReturn(type.name()); + String name = type.name(); + when(mapper.name()).thenReturn(name); when(mapper.copyTo()).thenReturn(CopyTo.empty()); Map indexAnalyzers = Map.of(); if (type.getTextSearchInfo() != TextSearchInfo.NONE) { diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index c514ffb447938..5aa5719eb8055 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -236,7 +236,16 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche souceToParse -> null, w -> w.writeString("test"), true - ); + ) { + @Override + public boolean hasNested() { + /* + * Disable fetching nested documents. Luckily if confusingly, this does + * not disable nested aggregations which we need to test. + */ + return false; + } + }; TriFunction, IndexFieldData> fieldDataBuilder = ( fieldType, From 76a512cd98f5079892593302cf785b7691212525 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 15 Dec 2020 09:31:31 -0500 Subject: [PATCH 17/47] Fix test --- .../stringstats/StringStatsAggregatorTests.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregatorTests.java b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregatorTests.java index a0a536a7e98b4..cf142195dbdc3 100644 --- a/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregatorTests.java +++ b/x-pack/plugin/analytics/src/test/java/org/elasticsearch/xpack/analytics/stringstats/StringStatsAggregatorTests.java @@ -92,7 +92,7 @@ public void testUnmappedField() throws IOException { assertTrue(stats.getDistribution().isEmpty()); assertEquals(0.0, stats.getEntropy(), 0); - }, null); + }); } public void testUnmappedWithMissingField() throws IOException { @@ -113,7 +113,7 @@ public void testUnmappedWithMissingField() throws IOException { assertEquals(0.25, stats.getDistribution().get("b"), 0); assertEquals(0.25, stats.getDistribution().get("c"), 0); assertEquals(1.5, stats.getEntropy(), 0); - }, null); + }); } public void testMissing() throws IOException { @@ -395,8 +395,9 @@ private void testAggregation( Query query, CheckedConsumer buildIndex, Consumer verify, - MappedFieldType fieldType) throws IOException { - testCase(aggregationBuilder, query, buildIndex, verify, fieldType); + MappedFieldType... fieldTypes + ) throws IOException { + testCase(aggregationBuilder, query, buildIndex, verify, fieldTypes); } @Override From a359630f2ad2f0d2cef50b91b3a233832876e430 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 16 Dec 2020 13:21:17 -0500 Subject: [PATCH 18/47] Fixup --- .../ChildrenToParentAggregatorTests.java | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java index 4e8b515767181..104fa8c40cd12 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java @@ -301,26 +301,11 @@ private void testCaseTermsParentTerms(Query query, IndexSearcher indexSearcher, verify.accept(result); } - static MappedFieldType[] withJoinFields(MappedFieldType... fieldTypes) { - MappedFieldType[] result = new MappedFieldType[fieldTypes.length + 3]; - System.arraycopy(fieldTypes, 0, result, 0, fieldTypes.length); - - int i = fieldTypes.length; - result[i++] = new MetaJoinFieldMapper.MetaJoinFieldType("join_field"); - result[i++] = new ParentJoinFieldMapper.Builder("join_field").addRelation(PARENT_TYPE, Collections.singleton(CHILD_TYPE)) - .build(new ContentPath(0)) - .fieldType(); - result[i++] = new ParentIdFieldMapper.ParentIdFieldType("join_field#" + PARENT_TYPE, false); - assert i == result.length; - return result; - } - @Override protected List getSearchPlugins() { return Collections.singletonList(new ParentJoinPlugin()); } - static MappedFieldType[] withJoinFields(MappedFieldType... fieldTypes) { MappedFieldType[] result = new MappedFieldType[fieldTypes.length + 3]; System.arraycopy(fieldTypes, 0, result, 0, fieldTypes.length); From 1335aab3556145ba98e7215fa26ca6003ed3a2f7 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 16 Dec 2020 13:25:52 -0500 Subject: [PATCH 19/47] Remove spacing change --- .../join/aggregations/ChildrenToParentAggregatorTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java index 104fa8c40cd12..a410405c09feb 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/aggregations/ChildrenToParentAggregatorTests.java @@ -306,6 +306,7 @@ protected List getSearchPlugins() { return Collections.singletonList(new ParentJoinPlugin()); } + static MappedFieldType[] withJoinFields(MappedFieldType... fieldTypes) { MappedFieldType[] result = new MappedFieldType[fieldTypes.length + 3]; System.arraycopy(fieldTypes, 0, result, 0, fieldTypes.length); From 9900fcbfafdc5649fddda931bf45401908df5eaf Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 16 Dec 2020 13:27:34 -0500 Subject: [PATCH 20/47] Javadoc --- .../java/org/elasticsearch/index/mapper/MapperService.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 8e27623099619..24e9c01fea484 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -409,6 +409,9 @@ public Set simpleMatchToFullName(String pattern) { return lookup().simpleMatchToFullName(pattern); } + /** + * {@code volatile} read a (mostly) immutable snapshot current mapping. + */ public MappingLookup lookup() { DocumentMapper mapper = this.mapper; return mapper == null ? MappingLookup.EMPTY : mapper.mappers(); From ed582f4e3df951b065bf415116ccdf1270484279 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 16 Dec 2020 15:27:31 -0500 Subject: [PATCH 21/47] Switch cache around --- .../index/mapper/DocumentMapper.java | 3 +- .../index/mapper/MapperService.java | 1 + .../index/mapper/MappingLookup.java | 26 +-- .../index/query/QueryShardContext.java | 3 +- .../indices/IndicesRequestCache.java | 25 ++- .../elasticsearch/indices/IndicesService.java | 36 +++- .../search/internal/ShardSearchRequest.java | 4 +- .../MetadataRolloverServiceTests.java | 1 - .../FieldAliasMapperValidationTests.java | 9 - .../index/mapper/MappingLookupTests.java | 6 +- .../indices/IndicesRequestCacheTests.java | 195 ++++++++++++++---- .../indices/IndicesServiceCloseTests.java | 4 +- .../internal/ShardSearchRequestTests.java | 5 +- .../index/mapper/MappingLookupUtils.java | 1 - .../aggregations/AggregatorTestCase.java | 1 - 15 files changed, 220 insertions(+), 100 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index 80937ed100726..536e19cf35163 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -119,7 +119,8 @@ protected DocumentMapper(IndexSettings indexSettings, } catch (Exception e) { throw new ElasticsearchGenerationException("failed to serialize source for type [" + type + "]", e); } - this.fieldMappers = MappingLookup.fromMapping(mapping, this::parse, w -> w.write(mappingSource.compressed())); + // TODO this::parse performs a volatile read on mapping from MapperService. Yikes! + this.fieldMappers = MappingLookup.fromMapping(mapping, this::parse); final Collection deleteTombstoneMetadataFields = Arrays.asList(VersionFieldMapper.NAME, IdFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 24e9c01fea484..cfc1d5874f988 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -485,6 +485,7 @@ public synchronized List reloadSearchAnalyzers(AnalysisRegistry registry reloadedAnalyzers.add(analyzerName); } } + // TODO this should make a new MappingLookup to bust the cache return reloadedAnalyzers; } } 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 dc4f595c692e2..b4d2f6a08ca13 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.mapper; -import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.NamedAnalyzer; @@ -35,6 +34,13 @@ import java.util.stream.Stream; public class MappingLookup { + /** + * Key for the lookup to be used in caches. + */ + public class CacheKey { + private CacheKey() {} + } + /** * A lookup representing an empty mapping. */ @@ -46,10 +52,11 @@ public class MappingLookup { List.of(), 0, souceToParse -> null, - writer -> writer.writeString("empty"), false ); + private final CacheKey cacheKey = new CacheKey(); + /** Full field name to mapper */ private final Map fieldMappers; private final Map objectMappers; @@ -58,14 +65,9 @@ public class MappingLookup { private final int metadataFieldCount; private final Map indexAnalyzers = new HashMap<>(); private final Function documentParser; - private final Writeable cacheKey; private final boolean sourceEnabled; - public static MappingLookup fromMapping( - Mapping mapping, - Function documentParser, - Writeable cacheKey - ) { + public static MappingLookup fromMapping(Mapping mapping, Function documentParser) { List newObjectMappers = new ArrayList<>(); List newFieldMappers = new ArrayList<>(); List newFieldAliasMappers = new ArrayList<>(); @@ -85,7 +87,6 @@ public static MappingLookup fromMapping( mapping.root.runtimeFieldTypes(), mapping.metadataMappers.length, documentParser, - cacheKey, mapping.metadataMapper(SourceFieldMapper.class).enabled() ); } @@ -115,10 +116,8 @@ public MappingLookup(String type, Collection runtimeFieldTypes, int metadataFieldCount, Function documentParser, - Writeable cacheKey, boolean sourceEnabled) { this.documentParser = documentParser; - this.cacheKey = cacheKey; this.sourceEnabled = sourceEnabled; Map fieldMappers = new HashMap<>(); Map objects = new HashMap<>(); @@ -381,7 +380,10 @@ public boolean hasNonNestedParent(String path) { return false; } - public Writeable cacheKey() { + /** + * Key for the lookup to be used in caches. + */ + public CacheKey cacheKey() { return cacheKey; } } diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index 1cbc8efa54b8d..6fdfa2c99c4ea 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.util.BigArrays; @@ -604,7 +603,7 @@ public NestedDocuments getNestedDocuments() { /** * Cache key for current mapping. */ - public Writeable mappingKey() { + public MappingLookup.CacheKey mappingCacheKey() { return mappingLookup.cacheKey(); } } diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java b/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java index 973595b084b7e..088f4e729e0cc 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java @@ -41,6 +41,7 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.index.mapper.MappingLookup; import java.io.Closeable; import java.io.IOException; @@ -113,9 +114,9 @@ public void onRemoval(RemovalNotification notification) { } BytesReference getOrCompute(CacheEntity cacheEntity, CheckedSupplier loader, - DirectoryReader reader, BytesReference cacheKey) throws Exception { + MappingLookup.CacheKey mappingCacheKey, DirectoryReader reader, BytesReference cacheKey) throws Exception { assert reader.getReaderCacheHelper() != null; - final Key key = new Key(cacheEntity, reader.getReaderCacheHelper().getKey(), cacheKey); + final Key key = new Key(cacheEntity, mappingCacheKey, reader.getReaderCacheHelper().getKey(), cacheKey); Loader cacheLoader = new Loader(cacheEntity, loader); BytesReference value = cache.computeIfAbsent(key, cacheLoader); if (cacheLoader.isLoaded()) { @@ -128,6 +129,15 @@ BytesReference getOrCompute(CacheEntity cacheEntity, CheckedSupplier { @@ -211,11 +221,13 @@ static class Key implements Accountable { private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Key.class); public final CacheEntity entity; // use as identity equality + public final MappingLookup.CacheKey mappingCacheKey; public final IndexReader.CacheKey readerCacheKey; public final BytesReference value; - Key(CacheEntity entity, IndexReader.CacheKey readerCacheKey, BytesReference value) { + Key(CacheEntity entity, MappingLookup.CacheKey mappingCacheKey, IndexReader.CacheKey readerCacheKey, BytesReference value) { this.entity = entity; + this.mappingCacheKey = Objects.requireNonNull(mappingCacheKey); this.readerCacheKey = Objects.requireNonNull(readerCacheKey); this.value = value; } @@ -236,7 +248,8 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Key key = (Key) o; - if (Objects.equals(readerCacheKey, key.readerCacheKey) == false) return false; + if (mappingCacheKey.equals(key.mappingCacheKey) == false) return false; + if (readerCacheKey.equals(key.readerCacheKey) == false) return false; if (!entity.getCacheIdentity().equals(key.entity.getCacheIdentity())) return false; if (!value.equals(key.value)) return false; return true; diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index 83524a1a321e9..dbc1f0eb1edad 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -101,6 +101,7 @@ import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.query.CoordinatorRewriteContextProvider; import org.elasticsearch.index.query.QueryBuilder; @@ -1408,13 +1409,18 @@ public void loadIntoContext(ShardSearchRequest request, SearchContext context, Q final DirectoryReader directoryReader = context.searcher().getDirectoryReader(); boolean[] loadedFromCache = new boolean[] { true }; - BytesReference cacheKey = request.cacheKey(context.getQueryShardContext().mappingKey()); - BytesReference bytesReference = cacheShardLevelResult(context.indexShard(), directoryReader, cacheKey, + BytesReference cacheKey = request.cacheKey(); + BytesReference bytesReference = cacheShardLevelResult( + context.indexShard(), + context.getQueryShardContext().mappingCacheKey(), + directoryReader, + cacheKey, out -> { - queryPhase.execute(context); - context.queryResult().writeToNoId(out); - loadedFromCache[0] = false; - }); + queryPhase.execute(context); + context.queryResult().writeToNoId(out); + loadedFromCache[0] = false; + } + ); if (loadedFromCache[0]) { // restore the cached query result into the context @@ -1430,7 +1436,12 @@ public void loadIntoContext(ShardSearchRequest request, SearchContext context, Q // key invalidate the result in the thread that caused the timeout. This will end up to be simpler and eventually correct since // running a search that times out concurrently will likely timeout again if it's run while we have this `stale` result in the // cache. One other option is to not cache requests with a timeout at all... - indicesRequestCache.invalidate(new IndexShardCacheEntity(context.indexShard()), directoryReader, cacheKey); + indicesRequestCache.invalidate( + new IndexShardCacheEntity(context.indexShard()), + context.getQueryShardContext().mappingCacheKey(), + directoryReader, + cacheKey + ); if (logger.isTraceEnabled()) { logger.trace("Query timed out, invalidating cache entry for request on shard [{}]:\n {}", request.shardId(), request.source()); @@ -1450,8 +1461,13 @@ public ByteSizeValue getTotalIndexingBufferBytes() { * @param loader loads the data into the cache if needed * @return the contents of the cache or the result of calling the loader */ - private BytesReference cacheShardLevelResult(IndexShard shard, DirectoryReader reader, BytesReference cacheKey, - CheckedConsumer loader) throws Exception { + private BytesReference cacheShardLevelResult( + IndexShard shard, + MappingLookup.CacheKey mappingCacheKey, + DirectoryReader reader, + BytesReference cacheKey, + CheckedConsumer loader + ) throws Exception { IndexShardCacheEntity cacheEntity = new IndexShardCacheEntity(shard); CheckedSupplier supplier = () -> { /* BytesStreamOutput allows to pass the expected size but by default uses @@ -1469,7 +1485,7 @@ private BytesReference cacheShardLevelResult(IndexShard shard, DirectoryReader r return out.bytes(); } }; - return indicesRequestCache.getOrCompute(cacheEntity, supplier, reader, cacheKey); + return indicesRequestCache.getOrCompute(cacheEntity, supplier, mappingCacheKey, reader, cacheKey); } static final class IndexShardCacheEntity extends AbstractIndexShardCacheEntity { diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 88db0344ba5a1..1d768ab8c7b63 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -36,7 +36,6 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.index.query.BoolQueryBuilder; @@ -397,10 +396,9 @@ public TimeValue keepAlive() { /** * Returns the cache key for this shard search request, based on its content */ - public BytesReference cacheKey(Writeable mappingKey) throws IOException { + public BytesReference cacheKey() throws IOException { BytesStreamOutput out = scratch.get(); try { - mappingKey.writeTo(out); this.innerWriteTo(out, true); // copy it over since we don't want to share the thread-local bytes in #scratch return out.copyBytes(); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java index 86a68bb5fafa2..82aa2ea4449e8 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java @@ -570,7 +570,6 @@ protected String contentType() { List.of(), 0, null, - null, false ); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java index b625924915df4..487860282749a 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldAliasMapperValidationTests.java @@ -46,7 +46,6 @@ public void testDuplicateFieldAliasAndObject() { emptyList(), 0, null, - null, false ) ); @@ -68,7 +67,6 @@ public void testDuplicateFieldAliasAndConcreteField() { emptyList(), 0, null, - null, false ) ); @@ -88,7 +86,6 @@ public void testAliasThatRefersToAlias() { emptyList(), 0, null, - null, false ); alias.validate(mappers); @@ -113,7 +110,6 @@ public void testAliasThatRefersToItself() { emptyList(), 0, null, - null, false ); invalidAlias.validate(mappers); @@ -135,7 +131,6 @@ public void testAliasWithNonExistentPath() { emptyList(), 0, null, - null, false ); invalidAlias.validate(mappers); @@ -157,7 +152,6 @@ public void testFieldAliasWithNestedScope() { emptyList(), 0, null, - null, false ); aliasMapper.validate(mappers); @@ -175,7 +169,6 @@ public void testFieldAliasWithDifferentObjectScopes() { emptyList(), 0, null, - null, false ); aliasMapper.validate(mappers); @@ -194,7 +187,6 @@ public void testFieldAliasWithNestedTarget() { emptyList(), 0, null, - null, false ); aliasMapper.validate(mappers); @@ -218,7 +210,6 @@ public void testFieldAliasWithDifferentNestedScopes() { emptyList(), 0, null, - null, false ); aliasMapper.validate(mappers); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java b/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java index 48eff03347c2b..e709dbf842dd7 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MappingLookupTests.java @@ -41,7 +41,7 @@ public class MappingLookupTests extends ESTestCase { public void testOnlyRuntimeField() { MappingLookup mappingLookup = new MappingLookup("_doc", Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), - Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, null, false); + Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, false); assertEquals(0, size(mappingLookup.fieldMappers())); assertEquals(0, mappingLookup.objectMappers().size()); assertNull(mappingLookup.getMapper("test")); @@ -51,7 +51,7 @@ public void testOnlyRuntimeField() { public void testRuntimeFieldLeafOverride() { MockFieldMapper fieldMapper = new MockFieldMapper("test"); MappingLookup mappingLookup = new MappingLookup("_doc", Collections.singletonList(fieldMapper), Collections.emptyList(), - Collections.emptyList(), Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, null, false); + Collections.emptyList(), Collections.singletonList(new TestRuntimeField("test", "type")), 0, null, false); assertThat(mappingLookup.getMapper("test"), instanceOf(MockFieldMapper.class)); assertEquals(1, size(mappingLookup.fieldMappers())); assertEquals(0, mappingLookup.objectMappers().size()); @@ -71,7 +71,6 @@ public void testSubfieldOverride() { Collections.singletonList(new TestRuntimeField("object.subfield", "type")), 0, null, - null, false ); assertThat(mappingLookup.getMapper("object.subfield"), instanceOf(MockFieldMapper.class)); @@ -97,7 +96,6 @@ public void testAnalyzers() throws IOException { Collections.emptyList(), 0, null, - null, false ); diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java index e4eb70c3df03a..38f28f984345a 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java @@ -43,12 +43,16 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.cache.request.ShardRequestCache; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MappingLookupUtils; import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; public class IndicesRequestCacheTests extends ESTestCase { @@ -62,6 +66,7 @@ public void testBasicOperationsCache() throws Exception { writer.addDocument(newDoc(0, "foo")); DirectoryReader reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); + MappingLookup.CacheKey mappingKey = MappingLookupUtils.fromTypes().cacheKey(); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); BytesReference termBytes = XContentHelper.toXContent(termQuery, XContentType.JSON, false); AtomicBoolean indexShard = new AtomicBoolean(true); @@ -69,7 +74,7 @@ public void testBasicOperationsCache() throws Exception { // initial cache TestEntity entity = new TestEntity(requestCacheStats, indexShard); Loader loader = new Loader(reader, 0); - BytesReference value = cache.getOrCompute(entity, loader, reader, termBytes); + BytesReference value = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value.streamInput().readString()); assertEquals(0, requestCacheStats.stats().getHitCount()); assertEquals(1, requestCacheStats.stats().getMissCount()); @@ -80,14 +85,14 @@ public void testBasicOperationsCache() throws Exception { // cache hit entity = new TestEntity(requestCacheStats, indexShard); loader = new Loader(reader, 0); - value = cache.getOrCompute(entity, loader, reader, termBytes); + value = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value.streamInput().readString()); assertEquals(1, requestCacheStats.stats().getHitCount()); assertEquals(1, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); assertTrue(loader.loadedFromCache); assertEquals(1, cache.count()); - assertTrue(requestCacheStats.stats().getMemorySize().bytesAsInt() > value.length()); + assertTrue(requestCacheStats.stats().getMemorySize().getBytes() > value.length()); assertEquals(1, cache.numRegisteredCloseListeners()); // Closing the cache doesn't modify an already returned CacheEntity @@ -103,7 +108,7 @@ public void testBasicOperationsCache() throws Exception { assertEquals(0, requestCacheStats.stats().getEvictions()); assertTrue(loader.loadedFromCache); assertEquals(0, cache.count()); - assertEquals(0, requestCacheStats.stats().getMemorySize().bytesAsInt()); + assertEquals(0L, requestCacheStats.stats().getMemorySize().getBytes()); IOUtils.close(reader, writer, dir, cache); assertEquals(0, cache.numRegisteredCloseListeners()); @@ -111,6 +116,7 @@ public void testBasicOperationsCache() throws Exception { public void testCacheDifferentReaders() throws Exception { IndicesRequestCache cache = new IndicesRequestCache(Settings.EMPTY); + MappingLookup.CacheKey mappingKey = MappingLookupUtils.fromTypes().cacheKey(); AtomicBoolean indexShard = new AtomicBoolean(true); ShardRequestCache requestCacheStats = new ShardRequestCache(); Directory dir = newDirectory(); @@ -131,33 +137,33 @@ public void testCacheDifferentReaders() throws Exception { // initial cache TestEntity entity = new TestEntity(requestCacheStats, indexShard); Loader loader = new Loader(reader, 0); - BytesReference value = cache.getOrCompute(entity, loader, reader, termBytes); + BytesReference value = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value.streamInput().readString()); assertEquals(0, requestCacheStats.stats().getHitCount()); assertEquals(1, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); assertFalse(loader.loadedFromCache); assertEquals(1, cache.count()); - assertTrue(requestCacheStats.stats().getMemorySize().bytesAsInt() > value.length()); - final int cacheSize = requestCacheStats.stats().getMemorySize().bytesAsInt(); + assertTrue(requestCacheStats.stats().getMemorySize().getBytes() > value.length()); + final long cacheSize = requestCacheStats.stats().getMemorySize().getBytes(); assertEquals(1, cache.numRegisteredCloseListeners()); // cache the second TestEntity secondEntity = new TestEntity(requestCacheStats, indexShard); loader = new Loader(secondReader, 0); - value = cache.getOrCompute(entity, loader, secondReader, termBytes); + value = cache.getOrCompute(entity, loader, mappingKey, secondReader, termBytes); assertEquals("bar", value.streamInput().readString()); assertEquals(0, requestCacheStats.stats().getHitCount()); assertEquals(2, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); assertFalse(loader.loadedFromCache); assertEquals(2, cache.count()); - assertTrue(requestCacheStats.stats().getMemorySize().bytesAsInt() > cacheSize + value.length()); + assertTrue(requestCacheStats.stats().getMemorySize().getBytes() > cacheSize + value.length()); assertEquals(2, cache.numRegisteredCloseListeners()); secondEntity = new TestEntity(requestCacheStats, indexShard); loader = new Loader(secondReader, 0); - value = cache.getOrCompute(secondEntity, loader, secondReader, termBytes); + value = cache.getOrCompute(secondEntity, loader, mappingKey, secondReader, termBytes); assertEquals("bar", value.streamInput().readString()); assertEquals(1, requestCacheStats.stats().getHitCount()); assertEquals(2, requestCacheStats.stats().getMissCount()); @@ -167,7 +173,7 @@ public void testCacheDifferentReaders() throws Exception { entity = new TestEntity(requestCacheStats, indexShard); loader = new Loader(reader, 0); - value = cache.getOrCompute(entity, loader, reader, termBytes); + value = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value.streamInput().readString()); assertEquals(2, requestCacheStats.stats().getHitCount()); assertEquals(2, requestCacheStats.stats().getMissCount()); @@ -182,10 +188,9 @@ public void testCacheDifferentReaders() throws Exception { assertEquals(0, requestCacheStats.stats().getEvictions()); assertTrue(loader.loadedFromCache); assertEquals(1, cache.count()); - assertEquals(cacheSize, requestCacheStats.stats().getMemorySize().bytesAsInt()); + assertEquals(cacheSize, requestCacheStats.stats().getMemorySize().getBytes()); assertEquals(1, cache.numRegisteredCloseListeners()); - // release if (randomBoolean()) { secondReader.close(); @@ -198,13 +203,93 @@ public void testCacheDifferentReaders() throws Exception { assertEquals(0, requestCacheStats.stats().getEvictions()); assertTrue(loader.loadedFromCache); assertEquals(0, cache.count()); - assertEquals(0, requestCacheStats.stats().getMemorySize().bytesAsInt()); + assertEquals(0L, requestCacheStats.stats().getMemorySize().getBytes()); IOUtils.close(secondReader, writer, dir, cache); assertEquals(0, cache.numRegisteredCloseListeners()); } + public void testCacheDifferentMapping() throws Exception { + IndicesRequestCache cache = new IndicesRequestCache(Settings.EMPTY); + MappingLookup.CacheKey mappingKey1 = MappingLookupUtils.fromTypes().cacheKey(); + MappingLookup.CacheKey mappingKey2 = MappingLookupUtils.fromTypes().cacheKey(); + AtomicBoolean indexShard = new AtomicBoolean(true); + ShardRequestCache requestCacheStats = new ShardRequestCache(); + Directory dir = newDirectory(); + IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); + writer.addDocument(newDoc(0, "foo")); + writer.addDocument(newDoc(1, "bar")); + DirectoryReader reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); + TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); + BytesReference termBytes = XContentHelper.toXContent(termQuery, XContentType.JSON, false); + + // initial cache + TestEntity entity = new TestEntity(requestCacheStats, indexShard); + Loader loader = new Loader(reader, 0); + BytesReference value = cache.getOrCompute(entity, loader, mappingKey1, reader, termBytes); + assertEquals("foo", value.streamInput().readString()); + assertEquals(0, requestCacheStats.stats().getHitCount()); + assertEquals(1, requestCacheStats.stats().getMissCount()); + assertEquals(0, requestCacheStats.stats().getEvictions()); + assertFalse(loader.loadedFromCache); + assertEquals(1, cache.count()); + assertTrue(requestCacheStats.stats().getMemorySize().getBytes() > value.length()); + final long cacheSize = requestCacheStats.stats().getMemorySize().getBytes(); + assertEquals(1, cache.numRegisteredCloseListeners()); + + // cache the second + TestEntity secondEntity = new TestEntity(requestCacheStats, indexShard); + loader = new Loader(reader, 1); + value = cache.getOrCompute(entity, loader, mappingKey2, reader, termBytes); + assertEquals("bar", value.streamInput().readString()); + assertEquals(0, requestCacheStats.stats().getHitCount()); + assertEquals(2, requestCacheStats.stats().getMissCount()); + assertEquals(0, requestCacheStats.stats().getEvictions()); + assertFalse(loader.loadedFromCache); + assertEquals(2, cache.count()); + assertTrue(requestCacheStats.stats().getMemorySize().getBytes() > cacheSize + value.length()); + assertEquals(1, cache.numRegisteredCloseListeners()); + + secondEntity = new TestEntity(requestCacheStats, indexShard); + loader = new Loader(reader, 1); + value = cache.getOrCompute(secondEntity, loader, mappingKey2, reader, termBytes); + assertEquals("bar", value.streamInput().readString()); + assertEquals(1, requestCacheStats.stats().getHitCount()); + assertEquals(2, requestCacheStats.stats().getMissCount()); + assertEquals(0, requestCacheStats.stats().getEvictions()); + assertTrue(loader.loadedFromCache); + assertEquals(2, cache.count()); + + entity = new TestEntity(requestCacheStats, indexShard); + loader = new Loader(reader, 0); + value = cache.getOrCompute(entity, loader, mappingKey1, reader, termBytes); + assertEquals("foo", value.streamInput().readString()); + assertEquals(2, requestCacheStats.stats().getHitCount()); + assertEquals(2, requestCacheStats.stats().getMissCount()); + assertEquals(0, requestCacheStats.stats().getEvictions()); + assertTrue(loader.loadedFromCache); + assertEquals(2, cache.count()); + + // Closing the cache doesn't change returned entities + if (randomBoolean()) { + reader.close(); + } else { + indexShard.set(false); // closed shard but reader is still open + cache.clear(secondEntity); + } + cache.cleanCache(); + assertEquals(2, requestCacheStats.stats().getMissCount()); + assertEquals(0, requestCacheStats.stats().getEvictions()); + assertTrue(loader.loadedFromCache); + assertEquals(0, cache.count()); + assertEquals(0L, requestCacheStats.stats().getMemorySize().getBytes()); + + IOUtils.close(reader, writer, dir, cache); + assertEquals(0, cache.numRegisteredCloseListeners()); + } + public void testEviction() throws Exception { + MappingLookup.CacheKey mappingKey = MappingLookupUtils.fromTypes().cacheKey(); final ByteSizeValue size; { IndicesRequestCache cache = new IndicesRequestCache(Settings.EMPTY); @@ -227,9 +312,9 @@ public void testEviction() throws Exception { TestEntity secondEntity = new TestEntity(requestCacheStats, indexShard); Loader secondLoader = new Loader(secondReader, 0); - BytesReference value1 = cache.getOrCompute(entity, loader, reader, termBytes); + BytesReference value1 = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value1.streamInput().readString()); - BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, secondReader, termBytes); + BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, mappingKey, secondReader, termBytes); assertEquals("bar", value2.streamInput().readString()); size = requestCacheStats.stats().getMemorySize(); IOUtils.close(reader, secondReader, writer, dir, cache); @@ -262,12 +347,12 @@ public void testEviction() throws Exception { TestEntity thirddEntity = new TestEntity(requestCacheStats, indexShard); Loader thirdLoader = new Loader(thirdReader, 0); - BytesReference value1 = cache.getOrCompute(entity, loader, reader, termBytes); + BytesReference value1 = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value1.streamInput().readString()); - BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, secondReader, termBytes); + BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, mappingKey, secondReader, termBytes); assertEquals("bar", value2.streamInput().readString()); logger.info("Memory size: {}", requestCacheStats.stats().getMemorySize()); - BytesReference value3 = cache.getOrCompute(thirddEntity, thirdLoader, thirdReader, termBytes); + BytesReference value3 = cache.getOrCompute(thirddEntity, thirdLoader, mappingKey, thirdReader, termBytes); assertEquals("baz", value3.streamInput().readString()); assertEquals(2, cache.count()); assertEquals(1, requestCacheStats.stats().getEvictions()); @@ -285,6 +370,7 @@ public void testClearAllEntityIdentity() throws Exception { writer.addDocument(newDoc(0, "foo")); DirectoryReader reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); + MappingLookup.CacheKey mappingKey = MappingLookupUtils.fromTypes().cacheKey(); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); BytesReference termBytes = XContentHelper.toXContent(termQuery, XContentType.JSON, false); TestEntity entity = new TestEntity(requestCacheStats, indexShard); @@ -293,31 +379,33 @@ public void testClearAllEntityIdentity() throws Exception { writer.updateDocument(new Term("id", "0"), newDoc(0, "bar")); DirectoryReader secondReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); + MappingLookup.CacheKey secondMappingKey = MappingLookupUtils.fromTypes().cacheKey(); TestEntity secondEntity = new TestEntity(requestCacheStats, indexShard); Loader secondLoader = new Loader(secondReader, 0); writer.updateDocument(new Term("id", "0"), newDoc(0, "baz")); DirectoryReader thirdReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); + MappingLookup.CacheKey thirdMappingKey = MappingLookupUtils.fromTypes().cacheKey(); AtomicBoolean differentIdentity = new AtomicBoolean(true); - TestEntity thirddEntity = new TestEntity(requestCacheStats, differentIdentity); + TestEntity thirdEntity = new TestEntity(requestCacheStats, differentIdentity); Loader thirdLoader = new Loader(thirdReader, 0); - BytesReference value1 = cache.getOrCompute(entity, loader, reader, termBytes); + BytesReference value1 = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value1.streamInput().readString()); - BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, secondReader, termBytes); + BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, secondMappingKey, secondReader, termBytes); assertEquals("bar", value2.streamInput().readString()); logger.info("Memory size: {}", requestCacheStats.stats().getMemorySize()); - BytesReference value3 = cache.getOrCompute(thirddEntity, thirdLoader, thirdReader, termBytes); + BytesReference value3 = cache.getOrCompute(thirdEntity, thirdLoader, thirdMappingKey, thirdReader, termBytes); assertEquals("baz", value3.streamInput().readString()); assertEquals(3, cache.count()); final long hitCount = requestCacheStats.stats().getHitCount(); - // clear all for the indexShard Idendity even though is't still open + // clear all for the indexShard entity even though is't still open cache.clear(randomFrom(entity, secondEntity)); cache.cleanCache(); assertEquals(1, cache.count()); // third has not been validated since it's a different identity - value3 = cache.getOrCompute(thirddEntity, thirdLoader, thirdReader, termBytes); + value3 = cache.getOrCompute(thirdEntity, thirdLoader, thirdMappingKey, thirdReader, termBytes); assertEquals(hitCount + 1, requestCacheStats.stats().getHitCount()); assertEquals("baz", value3.streamInput().readString()); @@ -367,6 +455,7 @@ public void testInvalidate() throws Exception { IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); writer.addDocument(newDoc(0, "foo")); + MappingLookup.CacheKey mappingKey = MappingLookupUtils.fromTypes().cacheKey(); DirectoryReader reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); @@ -376,7 +465,7 @@ public void testInvalidate() throws Exception { // initial cache TestEntity entity = new TestEntity(requestCacheStats, indexShard); Loader loader = new Loader(reader, 0); - BytesReference value = cache.getOrCompute(entity, loader, reader, termBytes); + BytesReference value = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value.streamInput().readString()); assertEquals(0, requestCacheStats.stats().getHitCount()); assertEquals(1, requestCacheStats.stats().getMissCount()); @@ -387,28 +476,28 @@ public void testInvalidate() throws Exception { // cache hit entity = new TestEntity(requestCacheStats, indexShard); loader = new Loader(reader, 0); - value = cache.getOrCompute(entity, loader, reader, termBytes); + value = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value.streamInput().readString()); assertEquals(1, requestCacheStats.stats().getHitCount()); assertEquals(1, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); assertTrue(loader.loadedFromCache); assertEquals(1, cache.count()); - assertTrue(requestCacheStats.stats().getMemorySize().bytesAsInt() > value.length()); + assertTrue(requestCacheStats.stats().getMemorySize().getBytes() > value.length()); assertEquals(1, cache.numRegisteredCloseListeners()); // load again after invalidate entity = new TestEntity(requestCacheStats, indexShard); loader = new Loader(reader, 0); - cache.invalidate(entity, reader, termBytes); - value = cache.getOrCompute(entity, loader, reader, termBytes); + cache.invalidate(entity, mappingKey, reader, termBytes); + value = cache.getOrCompute(entity, loader, mappingKey, reader, termBytes); assertEquals("foo", value.streamInput().readString()); assertEquals(1, requestCacheStats.stats().getHitCount()); assertEquals(2, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); assertFalse(loader.loadedFromCache); assertEquals(1, cache.count()); - assertTrue(requestCacheStats.stats().getMemorySize().bytesAsInt() > value.length()); + assertTrue(requestCacheStats.stats().getMemorySize().getBytes() > value.length()); assertEquals(1, cache.numRegisteredCloseListeners()); // release @@ -423,7 +512,7 @@ public void testInvalidate() throws Exception { assertEquals(2, requestCacheStats.stats().getMissCount()); assertEquals(0, requestCacheStats.stats().getEvictions()); assertEquals(0, cache.count()); - assertEquals(0, requestCacheStats.stats().getMemorySize().bytesAsInt()); + assertEquals(0L, requestCacheStats.stats().getMemorySize().getBytes()); IOUtils.close(reader, writer, dir, cache); assertEquals(0, cache.numRegisteredCloseListeners()); @@ -432,6 +521,8 @@ public void testInvalidate() throws Exception { public void testEqualsKey() throws IOException { AtomicBoolean trueBoolean = new AtomicBoolean(true); AtomicBoolean falseBoolean = new AtomicBoolean(false); + MappingLookup.CacheKey mKey1 = MappingLookupUtils.fromTypes().cacheKey(); + MappingLookup.CacheKey mKey2 = MappingLookupUtils.fromTypes().cacheKey(); Directory dir = newDirectory(); IndexWriterConfig config = newIndexWriterConfig(); IndexWriter writer = new IndexWriter(dir, config); @@ -441,19 +532,33 @@ public void testEqualsKey() throws IOException { IndexReader reader2 = DirectoryReader.open(writer); IndexReader.CacheKey rKey2 = reader2.getReaderCacheHelper().getKey(); IOUtils.close(reader1, reader2, writer, dir); - IndicesRequestCache.Key key1 = new IndicesRequestCache.Key(new TestEntity(null, trueBoolean), rKey1, new TestBytesReference(1)); - IndicesRequestCache.Key key2 = new IndicesRequestCache.Key(new TestEntity(null, trueBoolean), rKey1, new TestBytesReference(1)); - IndicesRequestCache.Key key3 = new IndicesRequestCache.Key(new TestEntity(null, falseBoolean), rKey1, new TestBytesReference(1)); - IndicesRequestCache.Key key4 = new IndicesRequestCache.Key(new TestEntity(null, trueBoolean), rKey2, new TestBytesReference(1)); - IndicesRequestCache.Key key5 = new IndicesRequestCache.Key(new TestEntity(null, trueBoolean), rKey1, new TestBytesReference(2)); - String s = "Some other random object"; - assertEquals(key1, key1); - assertEquals(key1, key2); - assertNotEquals(key1, null); - assertNotEquals(key1, s); - assertNotEquals(key1, key3); - assertNotEquals(key1, key4); - assertNotEquals(key1, key5); + List keys = new ArrayList<>(); + for (AtomicBoolean bool : new AtomicBoolean[] { trueBoolean, falseBoolean }) { + for (MappingLookup.CacheKey mKey : new MappingLookup.CacheKey[] { mKey1, mKey2 }) { + for (IndexReader.CacheKey rKey : new IndexReader.CacheKey[] { rKey1, rKey2 }) { + for (BytesReference requestKey : new BytesReference[] { new TestBytesReference(1), new TestBytesReference(2) }) { + keys.add(new IndicesRequestCache.Key(new TestEntity(null, bool), mKey, rKey, requestKey)); + } + } + } + } + for (IndicesRequestCache.Key key : keys) { + assertNotEquals(key, null); + assertNotEquals(key, "Some other random object"); + } + for (IndicesRequestCache.Key key1 : keys) { + for (IndicesRequestCache.Key key2 : keys) { + if (key1 == key2) { + assertEquals(key1, key2); + } else { + assertNotEquals(key1, key2); + } + } + } + assertEquals( + new IndicesRequestCache.Key(new TestEntity(null, trueBoolean), mKey1, rKey1, new TestBytesReference(1)), + new IndicesRequestCache.Key(new TestEntity(null, trueBoolean), mKey1, rKey1, new TestBytesReference(1)) + ); } private class TestBytesReference extends AbstractBytesReference { diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java index ad8b3b0d54984..9d047e16614a4 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java @@ -33,6 +33,7 @@ import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.indices.IndicesRequestCache.Key; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; @@ -287,7 +288,8 @@ public void onMiss() {} @Override public void onRemoval(RemovalNotification notification) {} }; - cache.getOrCompute(cacheEntity, () -> new BytesArray("bar"), searcher.getDirectoryReader(), new BytesArray("foo")); + MappingLookup.CacheKey mappingCacheKey = indexService.mapperService().lookup().cacheKey(); + cache.getOrCompute(cacheEntity, () -> new BytesArray("bar"), mappingCacheKey, searcher.getDirectoryReader(), new BytesArray("foo")); assertEquals(1L, cache.count()); searcher.close(); diff --git a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java index 9e27f5a663346..630be3c44050f 100644 --- a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.DeprecationHandler; @@ -158,9 +157,7 @@ private static void assertEquals(ShardSearchRequest orig, ShardSearchRequest cop assertEquals(orig.searchType(), copy.searchType()); assertEquals(orig.shardId(), copy.shardId()); assertEquals(orig.numberOfShards(), copy.numberOfShards()); - String randomCacheKey = randomAlphaOfLength(5); - Writeable cacheKey = w -> w.writeString(randomCacheKey); - assertEquals(orig.cacheKey(cacheKey), copy.cacheKey(cacheKey)); + assertEquals(orig.cacheKey(), copy.cacheKey()); assertNotSame(orig, copy); assertEquals(orig.getAliasFilter(), copy.getAliasFilter()); assertEquals(orig.indexBoost(), copy.indexBoost(), 0.0f); diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java index ae85393d9d945..b11e26b7b4e95 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java @@ -47,7 +47,6 @@ public static MappingLookup fromTypes(Stream types) { List.of(), 0, souceToParse -> null, - w -> w.writeString("test" + mappers.hashCode()), true ); } diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 5aa5719eb8055..c2e3cc86615dd 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -234,7 +234,6 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche List.of(), 0, souceToParse -> null, - w -> w.writeString("test"), true ) { @Override From 2b6fbce312c8fb54b5255a6a0ec053ce0f775f41 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 16 Dec 2020 16:45:47 -0500 Subject: [PATCH 22/47] We use up to field 10! --- .../authz/accesscontrol/DocumentSubsetBitsetCacheTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java index 4065fdf2bb3a7..29ba0237d031f 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java @@ -587,7 +587,7 @@ null, null, null, mappingLookup, null, null, xContentRegistry(), writableRegistr private void runTestOnIndices(int numberIndices, CheckedConsumer, Exception> body) throws Exception { List types = new ArrayList<>(); - for (int i = 0; i < 7; i++) { // the tests use fields 1 to 6. + for (int i = 0; i < 11; i++) { // the tests use fields 1 to 10. // This field has a value. types.add(new KeywordFieldMapper.KeywordFieldType("field-" + i)); // This field never has a value From d9c19afaf77ffd5582eac54a745f83c184d887f3 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 11:37:49 -0500 Subject: [PATCH 23/47] Fixup --- .../org/elasticsearch/test/AbstractBuilderTestCase.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java index d85f433fd698f..fb848824afe3f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java @@ -411,8 +411,8 @@ public void close() throws IOException { } QueryShardContext createShardContext(IndexSearcher searcher) { -<<<<<<< HEAD return new QueryShardContext( + 0, 0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, @@ -433,11 +433,6 @@ QueryShardContext createShardContext(IndexSearcher searcher) { null, emptyMap() ); -======= - return new QueryShardContext(0, 0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, bitsetFilterCache, - indexFieldDataService::getForField, mapperService, similarityService, scriptService, xContentRegistry, - namedWriteableRegistry, this.client, searcher, () -> nowInMillis, null, indexNameMatcher(), () -> true, null, emptyMap()); ->>>>>>> master } ScriptModule createScriptModule(List scriptPlugins) { From 000ea6871d63cbf9db7fd23e6b597209e2b9c7ed Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 11:57:10 -0500 Subject: [PATCH 24/47] Iter --- .../org/elasticsearch/index/IndexService.java | 2 +- .../index/mapper/DocumentMapper.java | 3 +- .../index/mapper/FieldTypeLookup.java | 17 +++---- .../index/mapper/MapperService.java | 10 ++--- .../index/mapper/MappingLookup.java | 44 ++++++++++++++----- .../index/query/QueryShardContext.java | 6 +-- .../index/mapper/MapperServiceTests.java | 4 +- .../index/mapper/NestedDocumentsTests.java | 6 +-- .../index/mapper/NestedObjectMapperTests.java | 4 +- .../index/query/RangeQueryRewriteTests.java | 6 +-- .../indices/IndicesServiceCloseTests.java | 2 +- .../fetch/subphase/FieldFetcherTests.java | 2 +- .../test/AbstractBuilderTestCase.java | 2 +- 13 files changed, 65 insertions(+), 43 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 89e3e5f8286e7..23c605f7c4409 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -612,7 +612,7 @@ public QueryShardContext newQueryShardContext( indexCache.bitsetFilterCache(), indexFieldData::getForField, mapperService(), - mapperService().lookup(), + mapperService().mappingLookup(), similarityService(), scriptService, xContentRegistry, diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index 536e19cf35163..704a1166f1ea1 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -103,7 +103,7 @@ public DocumentMapper build() { private final MetadataFieldMapper[] deleteTombstoneMetadataFieldMappers; private final MetadataFieldMapper[] noopTombstoneMetadataFieldMappers; - protected DocumentMapper(IndexSettings indexSettings, + private DocumentMapper(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, DocumentParser documentParser, Mapping mapping) { @@ -119,7 +119,6 @@ protected DocumentMapper(IndexSettings indexSettings, } catch (Exception e) { throw new ElasticsearchGenerationException("failed to serialize source for type [" + type + "]", e); } - // TODO this::parse performs a volatile read on mapping from MapperService. Yikes! this.fieldMappers = MappingLookup.fromMapping(mapping, this::parse); final Collection deleteTombstoneMetadataFields = Arrays.asList(VersionFieldMapper.NAME, IdFieldMapper.NAME, diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java index 487eb7a68ce4f..22a904daf12b5 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.regex.Regex; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -32,7 +33,7 @@ /** * An immutable container for looking up {@link MappedFieldType}s by their name. */ -public final class FieldTypeLookup { +final class FieldTypeLookup { private final Map fullNameToFieldType = new HashMap<>(); /** @@ -93,7 +94,7 @@ public final class FieldTypeLookup { /** * Returns the mapped field type for the given field name. */ - public MappedFieldType get(String field) { + MappedFieldType get(String field) { if (field.equals(TypeFieldType.NAME)) { return new TypeFieldType(type); } @@ -112,6 +113,10 @@ public MappedFieldType get(String field) { * Returns a list of the full names of a simple match regex like pattern against full name and index name. */ Set simpleMatchToFullName(String pattern) { + if (Regex.isSimpleMatchPattern(pattern) == false) { + // no wildcards + return Collections.singleton(pattern); + } Set fields = new HashSet<>(); for (String field : fullNameToFieldType.keySet()) { if (Regex.simpleMatch(pattern, field)) { @@ -133,8 +138,8 @@ Set simpleMatchToFullName(String pattern) { * should be a concrete field and *not* an alias. * @return A set of paths in the _source that contain the field's values. */ - public Set sourcePaths(String field) { - if (isEmpty()) { + Set sourcePaths(String field) { + if (fullNameToFieldType.isEmpty()) { return Set.of(); } String resolvedField = field; @@ -161,8 +166,4 @@ Iterable filter(Predicate predicate) { return () -> Stream.concat(fullNameToFieldType.values().stream(), dynamicKeyLookup.fieldTypes()) .distinct().filter(predicate).iterator(); } - - public boolean isEmpty() { - return fullNameToFieldType.isEmpty(); - } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index cfc1d5874f988..9212c978ed19b 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -131,7 +131,7 @@ public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, } public boolean hasNested() { - return lookup().hasNested(); + return mappingLookup().hasNested(); } public IndexAnalyzers getIndexAnalyzers() { @@ -398,7 +398,7 @@ public DocumentMapperForType documentMapperWithAutoCreate() { * Given the full name of a field, returns its {@link MappedFieldType}. */ public MappedFieldType fieldType(String fullName) { - return lookup().fieldTypes().get(fullName); + return mappingLookup().fieldTypes().get(fullName); } /** @@ -406,13 +406,13 @@ public MappedFieldType fieldType(String fullName) { * then the fields will be returned with a type prefix. */ public Set simpleMatchToFullName(String pattern) { - return lookup().simpleMatchToFullName(pattern); + return mappingLookup().simpleMatchToFullName(pattern); } /** * {@code volatile} read a (mostly) immutable snapshot current mapping. */ - public MappingLookup lookup() { + public MappingLookup mappingLookup() { DocumentMapper mapper = this.mapper; return mapper == null ? MappingLookup.EMPTY : mapper.mappers(); } @@ -436,7 +436,7 @@ public ObjectMapper getObjectMapper(String name) { * directly associated index-time analyzer */ public NamedAnalyzer indexAnalyzer(String field, Function unindexedFieldAnalyzer) { - return lookup().indexAnalyzer(field, unindexedFieldAnalyzer); + return mappingLookup().indexAnalyzer(field, unindexedFieldAnalyzer); } @Override 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 b4d2f6a08ca13..048569a743bcf 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.mapper; -import org.elasticsearch.common.regex.Regex; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.NamedAnalyzer; @@ -33,11 +32,15 @@ import java.util.function.Function; import java.util.stream.Stream; +/** + * A (mostly) immutable snapshot current mapping of an index with access + * to everything we need for the search phase. + */ public class MappingLookup { /** * Key for the lookup to be used in caches. */ - public class CacheKey { + public static class CacheKey { private CacheKey() {} } @@ -169,7 +172,7 @@ public Mapper getMapper(String field) { return fieldMappers.get(field); } - public FieldTypeLookup fieldTypes() { + FieldTypeLookup fieldTypes() { return fieldTypeLookup; } @@ -276,19 +279,38 @@ private static String parentObject(String field) { } public Set simpleMatchToFullName(String pattern) { - if (Regex.isSimpleMatchPattern(pattern) == false) { - // no wildcards - return Collections.singleton(pattern); - } return fieldTypes().simpleMatchToFullName(pattern); } + /** + * Returns the mapped field type for the given field name. + */ + public MappedFieldType getFieldType(String field) { + return fieldTypes().get(field); + } + + /** + * Given a concrete field name, return its paths in the _source. + * + * For most fields, the source path is the same as the field itself. However + * there are cases where a field's values are found elsewhere in the _source: + * - For a multi-field, the source path is the parent field. + * - One field's content could have been copied to another through copy_to. + * + * @param field The field for which to look up the _source path. Note that the field + * should be a concrete field and *not* an alias. + * @return A set of paths in the _source that contain the field's values. + */ + public Set sourcePaths(String field) { + return fieldTypes().sourcePaths(field); + } + public ParsedDocument parseDocument(SourceToParse source) { return documentParser.apply(source); } - public boolean isEmpty() { - return objectMappers.isEmpty() && fieldTypeLookup.isEmpty(); + public boolean hasMappings() { + return this != EMPTY; } public boolean isSourceEnabled() { @@ -296,9 +318,9 @@ public boolean isSourceEnabled() { } /** - * Returns all nested object mappers which contain further nested object mappers + * Returns all nested object mappers which contain further nested object mappers. * - * Used by BitSetProducerWarmer + * Used by {@link BitSetProducerWarmer}. */ public List getNestedParentMappers() { List parents = new ArrayList<>(); diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index a5761c197a114..63c6e32036341 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -294,7 +294,7 @@ public boolean hasNested() { } public boolean hasMappings() { - return false == mappingLookup.isEmpty(); + return mappingLookup.hasMappings(); } /** @@ -340,7 +340,7 @@ public boolean isFieldMapped(String name) { private MappedFieldType fieldType(String name) { MappedFieldType fieldType = runtimeMappings.get(name); - return fieldType == null ? mappingLookup.fieldTypes().get(name) : fieldType; + return fieldType == null ? mappingLookup.getFieldType(name) : fieldType; } public ObjectMapper getObjectMapper(String name) { @@ -352,7 +352,7 @@ public boolean isMetadataField(String field) { } public Set sourcePath(String fullName) { - return mappingLookup.fieldTypes().sourcePaths(fullName); + return mappingLookup.sourcePaths(fullName); } public boolean isSourceEnabled() { 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 a7a74363b9a7e..affd52bba4b16 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java @@ -49,11 +49,11 @@ public void testPreflightUpdateDoesNotChangeMapping() throws Throwable { public void testLookup() throws IOException { MapperService service = createMapperService(mapping(b -> {})); - MappingLookup oldLookup = service.lookup(); + MappingLookup oldLookup = service.mappingLookup(); assertThat(oldLookup.fieldTypes().get("cat"), nullValue()); merge(service, mapping(b -> b.startObject("cat").field("type", "keyword").endObject())); - MappingLookup newLookup = service.lookup(); + MappingLookup newLookup = service.mappingLookup(); assertThat(newLookup.fieldTypes().get("cat"), not(nullValue())); assertThat(oldLookup.fieldTypes().get("cat"), nullValue()); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java index 9e79f62308924..7627ed4beba75 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java @@ -56,7 +56,7 @@ public void testSimpleNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService.lookup(), QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -153,7 +153,7 @@ public void testMultiLevelNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService.lookup(), QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -264,7 +264,7 @@ public void testNestedObjectWithinNonNestedObject() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService.lookup(), QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java index 9acfffb62f25f..83434b7f0e926 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java @@ -617,7 +617,7 @@ public void testParentObjectMapperAreNested() throws Exception { } b.endObject(); })); - assertFalse(mapperService.lookup().hasNonNestedParent("comments.messages")); + assertFalse(mapperService.mappingLookup().hasNonNestedParent("comments.messages")); mapperService = createMapperService(mapping(b -> { b.startObject("comments"); @@ -631,7 +631,7 @@ public void testParentObjectMapperAreNested() throws Exception { } b.endObject(); })); - assertTrue(mapperService.lookup().hasNonNestedParent("comments.messages")); + assertTrue(mapperService.mappingLookup().hasNonNestedParent("comments.messages")); } public void testLimitNestedDocsDefaultSettings() throws Exception { diff --git a/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java b/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java index 87b56a2c6955d..82a626213bdc0 100644 --- a/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java @@ -49,7 +49,7 @@ public void testRewriteMissingField() throws Exception { null, null, indexService.mapperService(), - indexService.mapperService().lookup(), + indexService.mapperService().mappingLookup(), null, null, xContentRegistry(), @@ -79,7 +79,7 @@ public void testRewriteMissingReader() throws Exception { indexService.mapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); QueryRewriteContext context = new QueryShardContext(0, 0, indexService.getIndexSettings(), null, null, null, - indexService.mapperService(), indexService.mapperService().lookup(), null, null, xContentRegistry(), writableRegistry(), + indexService.mapperService(), indexService.mapperService().mappingLookup(), null, null, xContentRegistry(), writableRegistry(), null, null, null, null, null, () -> true, null, emptyMap()); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // can't make assumptions on a missing reader, so it must return INTERSECT @@ -106,7 +106,7 @@ public void testRewriteEmptyReader() throws Exception { null, null, indexService.mapperService(), - indexService.mapperService().lookup(), + indexService.mapperService().mappingLookup(), null, null, xContentRegistry(), diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java index 9d047e16614a4..23736d3c36c84 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesServiceCloseTests.java @@ -288,7 +288,7 @@ public void onMiss() {} @Override public void onRemoval(RemovalNotification notification) {} }; - MappingLookup.CacheKey mappingCacheKey = indexService.mapperService().lookup().cacheKey(); + MappingLookup.CacheKey mappingCacheKey = indexService.mapperService().mappingLookup().cacheKey(); cache.getOrCompute(cacheEntity, () -> new BytesArray("bar"), mappingCacheKey, searcher.getDirectoryReader(), new BytesArray("foo")); assertEquals(1L, cache.count()); diff --git a/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java b/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java index 60975a1a19f99..0c8107e665f37 100644 --- a/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/subphase/FieldFetcherTests.java @@ -745,7 +745,7 @@ private static QueryShardContext newQueryShardContext(MapperService mapperServic null, null, mapperService, - mapperService.lookup(), + mapperService.mappingLookup(), null, null, null, diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java index fb848824afe3f..8cba816a5e4ff 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java @@ -419,7 +419,7 @@ QueryShardContext createShardContext(IndexSearcher searcher) { bitsetFilterCache, indexFieldDataService::getForField, mapperService, - mapperService.lookup(), + mapperService.mappingLookup(), similarityService, scriptService, xContentRegistry, From e151e5d14d3061727d9326b57ab49dd00a0a99bb Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 11:58:33 -0500 Subject: [PATCH 25/47] link to public --- .../java/org/elasticsearch/index/mapper/MappingLookup.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 048569a743bcf..658482a5b9f12 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -21,6 +21,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.NamedAnalyzer; +import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import java.util.ArrayList; import java.util.Collection; @@ -320,7 +321,7 @@ public boolean isSourceEnabled() { /** * Returns all nested object mappers which contain further nested object mappers. * - * Used by {@link BitSetProducerWarmer}. + * Used by {@link BitsetFilterCache}. */ public List getNestedParentMappers() { List parents = new ArrayList<>(); From de00a13b7357baf58fe2a3ff110235cfaea521c9 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 12:05:04 -0500 Subject: [PATCH 26/47] Revert cleanup of nested --- .../index/mapper/DocumentMapper.java | 87 +++++++++++++++++++ .../index/query/QueryShardContext.java | 8 -- .../search/DefaultSearchContext.java | 5 ++ .../elasticsearch/search/NestedDocuments.java | 20 ++--- .../search/fetch/FetchPhase.java | 2 +- .../internal/FilteredSearchContext.java | 6 ++ .../search/internal/SearchContext.java | 3 + .../index/mapper/NestedDocumentsTests.java | 6 +- .../elasticsearch/test/TestSearchContext.java | 6 ++ 9 files changed, 121 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index 704a1166f1ea1..79cd2cda42407 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -34,8 +34,10 @@ import org.elasticsearch.index.mapper.MapperService.MergeReason; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.stream.Stream; @@ -206,6 +208,91 @@ public ParsedDocument createNoopTombstoneDoc(String index, String reason) throws return parsedDoc; } + /** + * Given an object path, checks to see if any of its parents are non-nested objects + */ + public boolean hasNonNestedParent(String path) { + ObjectMapper mapper = mappers().objectMappers().get(path); + if (mapper == null) { + return false; + } + while (mapper != null) { + if (mapper.nested().isNested() == false) { + return true; + } + if (path.contains(".") == false) { + return false; + } + path = path.substring(0, path.lastIndexOf(".")); + mapper = mappers().objectMappers().get(path); + } + return false; + } + + /** + * Returns all nested object mappers + */ + public List getNestedMappers() { + List childMappers = new ArrayList<>(); + for (ObjectMapper mapper : mappers().objectMappers().values()) { + if (mapper.nested().isNested() == false) { + continue; + } + childMappers.add(mapper); + } + return childMappers; + } + + /** + * Returns all nested object mappers which contain further nested object mappers + * + * Used by BitSetProducerWarmer + */ + public List getNestedParentMappers() { + List parents = new ArrayList<>(); + for (ObjectMapper mapper : mappers().objectMappers().values()) { + String nestedParentPath = getNestedParent(mapper.fullPath()); + if (nestedParentPath == null) { + continue; + } + ObjectMapper parent = mappers().objectMappers().get(nestedParentPath); + if (parent.nested().isNested()) { + parents.add(parent); + } + } + return parents; + } + + /** + * Given a nested object path, returns the path to its nested parent + * + * In particular, if a nested field `foo` contains an object field + * `bar.baz`, then calling this method with `foo.bar.baz` will return + * the path `foo`, skipping over the object-but-not-nested `foo.bar` + */ + public String getNestedParent(String path) { + ObjectMapper mapper = mappers().objectMappers().get(path); + if (mapper == null) { + return null; + } + if (path.contains(".") == false) { + return null; + } + do { + path = path.substring(0, path.lastIndexOf(".")); + mapper = mappers().objectMappers().get(path); + if (mapper == null) { + return null; + } + if (mapper.nested().isNested()) { + return path; + } + if (path.contains(".") == false) { + return null; + } + } while(true); + } + public DocumentMapper merge(Mapping mapping, MergeReason reason) { Mapping merged = this.mapping.merge(mapping, reason); return new DocumentMapper(this.indexSettings, this.indexAnalyzers, this.documentParser, merged); diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index 63c6e32036341..5932baf574177 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -64,7 +64,6 @@ import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptFactory; import org.elasticsearch.script.ScriptService; -import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.transport.RemoteClusterAware; @@ -623,13 +622,6 @@ private static Map parseRuntimeMappings(Map childObjectFilters = new HashMap<>(); private final Map childObjectMappers = new HashMap<>(); private final BitSetProducer parentDocumentFilter; - private final MappingLookup mappingLookup; + private final MapperService mapperService; /** * Create a new NestedDocuments object for an index - * @param mappingLookup a snapshot of the index's mapping + * @param mapperService the index's MapperService * @param filterProducer a function to build BitSetProducers from filter queries */ - public NestedDocuments(MappingLookup mappingLookup, Function filterProducer) { - this.mappingLookup = mappingLookup; - if (mappingLookup.hasNested() == false) { + public NestedDocuments(MapperService mapperService, Function filterProducer) { + this.mapperService = mapperService; + if (mapperService.hasNested() == false) { this.parentDocumentFilter = null; } else { this.parentDocumentFilter = filterProducer.apply(Queries.newNonNestedFilter()); - for (ObjectMapper mapper : mappingLookup.getNestedParentMappers()) { + for (ObjectMapper mapper : mapperService.documentMapper().getNestedParentMappers()) { parentObjectFilters.put(mapper.name(), filterProducer.apply(mapper.nestedTypeFilter())); } - for (ObjectMapper mapper : mappingLookup.getNestedMappers()) { + for (ObjectMapper mapper : mapperService.documentMapper().getNestedMappers()) { childObjectFilters.put(mapper.name(), null); childObjectMappers.put(mapper.name(), mapper); } @@ -98,7 +98,7 @@ private Weight getNestedChildWeight(LeafReaderContext ctx, String path) throws I * Given an object path, returns whether or not any of its parents are plain objects */ public boolean hasNonNestedParent(String path) { - return mappingLookup.hasNonNestedParent(path); + return mapperService.documentMapper().hasNonNestedParent(path); } private class HasNestedDocuments implements LeafNestedDocuments { @@ -185,7 +185,7 @@ private SearchHit.NestedIdentity loadNestedIdentity() throws IOException { int parentNameLength; String path = findObjectPath(doc); while (path != null) { - String parent = mappingLookup.getNestedParent(path); + String parent = mapperService.documentMapper().getNestedParent(path); // We have to pull a new scorer for each document here, because we advance from // the last parent which will be behind the doc Scorer childScorer = getNestedChildWeight(ctx, path).scorer(ctx); diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index f5a4809d319a1..1e24f7f243b1d 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -110,7 +110,7 @@ public void execute(SearchContext context) { SearchHit[] hits = new SearchHit[context.docIdsToLoadSize()]; List processors = getProcessors(context.shardTarget(), fetchContext); - NestedDocuments nestedDocuments = context.getQueryShardContext().getNestedDocuments(); + NestedDocuments nestedDocuments = context.getNestedDocuments(); int currentReaderIndex = -1; LeafReaderContext currentReaderContext = null; diff --git a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index 2c787f1149915..b4ae043bdb206 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -29,6 +29,7 @@ import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.SearchExtBuilder; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.SearchContextAggregations; @@ -389,6 +390,11 @@ public FetchSearchResult fetchResult() { return in.fetchResult(); } + @Override + public NestedDocuments getNestedDocuments() { + return in.getNestedDocuments(); + } + @Override public FetchPhase fetchPhase() { return in.fetchPhase(); diff --git a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java index a27b268dec376..59e9aabb64810 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -32,6 +32,7 @@ import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.RescoreDocIds; import org.elasticsearch.search.SearchExtBuilder; import org.elasticsearch.search.SearchShardTarget; @@ -310,6 +311,8 @@ public final void assignRescoreDocIds(RescoreDocIds rescoreDocIds) { public abstract QuerySearchResult queryResult(); + public abstract NestedDocuments getNestedDocuments(); + public abstract FetchPhase fetchPhase(); public abstract FetchSearchResult fetchResult(); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java index 7627ed4beba75..16280eea7ce9a 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NestedDocumentsTests.java @@ -56,7 +56,7 @@ public void testSimpleNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService, QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -153,7 +153,7 @@ public void testMultiLevelNestedHierarchy() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService, QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); @@ -264,7 +264,7 @@ public void testNestedObjectWithinNonNestedObject() throws IOException { })); withLuceneIndex(mapperService, iw -> iw.addDocuments(doc.docs()), reader -> { - NestedDocuments nested = new NestedDocuments(mapperService.mappingLookup(), QueryBitSetProducer::new); + NestedDocuments nested = new NestedDocuments(mapperService, QueryBitSetProducer::new); LeafNestedDocuments leaf = nested.getLeafNestedDocuments(reader.leaves().get(0)); assertNotNull(leaf.advance(0)); diff --git a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java index f7898132d294a..29b64da0122cf 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java @@ -31,6 +31,7 @@ import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.SearchExtBuilder; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.SearchContextAggregations; @@ -497,6 +498,11 @@ public FetchSearchResult fetchResult() { return null; } + @Override + public NestedDocuments getNestedDocuments() { + return new NestedDocuments(indexService.mapperService(), bitsetFilterCache()::getBitSetProducer); + } + @Override public FetchPhase fetchPhase() { return null; From fafa83812939474b5e443dc3512dc7fccd7bab98 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 12:08:14 -0500 Subject: [PATCH 27/47] Fix test --- .../org/elasticsearch/index/mapper/FieldTypeLookupTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java index ab302d0f83975..7f4e05f3a398b 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldTypeLookupTests.java @@ -39,7 +39,7 @@ public void testEmpty() { assertNull(lookup.get("foo")); Collection names = lookup.simpleMatchToFullName("foo"); assertNotNull(names); - assertTrue(names.isEmpty()); + assertThat(names, equalTo(Set.of("foo"))); assertEquals(0, size(lookup.filter(ft -> true))); } From 4ecd7813a3980996648ed445d0a9070c1f269fca Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 12:20:04 -0500 Subject: [PATCH 28/47] Finish reverting nested change --- .../index/cache/bitset/BitsetFilterCache.java | 2 +- .../index/mapper/DocumentMapper.java | 8 +- .../index/mapper/MappingLookup.java | 85 ----------------- .../index/mapper/NestedObjectMapperTests.java | 91 ++++++++++--------- .../aggregations/AggregatorTestCase.java | 8 +- 5 files changed, 61 insertions(+), 133 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java b/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java index 288c584200b46..511f64f947c6c 100644 --- a/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java +++ b/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java @@ -240,7 +240,7 @@ public IndexWarmer.TerminationHandle warmReader(final IndexShard indexShard, fin if (docMapper != null) { if (docMapper.mappers().hasNested()) { warmUp.add(Queries.newNonNestedFilter()); - docMapper.mappers().getNestedParentMappers().stream().map(ObjectMapper::nestedTypeFilter).forEach(warmUp::add); + docMapper.getNestedParentMappers().stream().map(ObjectMapper::nestedTypeFilter).forEach(warmUp::add); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index 79cd2cda42407..4d6c252b6044f 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -115,13 +115,13 @@ private DocumentMapper(IndexSettings indexSettings, this.documentParser = documentParser; this.indexSettings = indexSettings; this.indexAnalyzers = indexAnalyzers; + this.fieldMappers = MappingLookup.fromMapping(mapping, this::parse); try { mappingSource = new CompressedXContent(this, XContentType.JSON, ToXContent.EMPTY_PARAMS); } catch (Exception e) { throw new ElasticsearchGenerationException("failed to serialize source for type [" + type + "]", e); } - this.fieldMappers = MappingLookup.fromMapping(mapping, this::parse); final Collection deleteTombstoneMetadataFields = Arrays.asList(VersionFieldMapper.NAME, IdFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); @@ -185,6 +185,10 @@ public IndexFieldMapper IndexFieldMapper() { return metadataMapper(IndexFieldMapper.class); } + public boolean hasNestedObjects() { + return mappers().hasNested(); + } + public MappingLookup mappers() { return this.fieldMappers; } @@ -306,7 +310,7 @@ public void validate(IndexSettings settings, boolean checkLimits) { + "required for partitioned index [" + settings.getIndex().getName() + "]"); } } - if (settings.getIndexSortConfig().hasIndexSort() && fieldMappers.hasNested()) { + if (settings.getIndexSortConfig().hasIndexSort() && hasNestedObjects()) { throw new IllegalArgumentException("cannot have nested fields when index sort is activated"); } if (checkLimits) { 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 658482a5b9f12..3bf2733845500 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -318,91 +318,6 @@ public boolean isSourceEnabled() { return sourceEnabled; } - /** - * Returns all nested object mappers which contain further nested object mappers. - * - * Used by {@link BitsetFilterCache}. - */ - public List getNestedParentMappers() { - List parents = new ArrayList<>(); - for (ObjectMapper mapper : objectMappers.values()) { - String nestedParentPath = getNestedParent(mapper.fullPath()); - if (nestedParentPath == null) { - continue; - } - ObjectMapper parent = objectMappers.get(nestedParentPath); - if (parent.nested().isNested()) { - parents.add(parent); - } - } - return parents; - } - - /** - * Given a nested object path, returns the path to its nested parent - * - * In particular, if a nested field `foo` contains an object field - * `bar.baz`, then calling this method with `foo.bar.baz` will return - * the path `foo`, skipping over the object-but-not-nested `foo.bar` - */ - public String getNestedParent(String path) { - ObjectMapper mapper = objectMappers.get(path); - if (mapper == null) { - return null; - } - if (path.contains(".") == false) { - return null; - } - do { - path = path.substring(0, path.lastIndexOf(".")); - mapper = objectMappers.get(path); - if (mapper == null) { - return null; - } - if (mapper.nested().isNested()) { - return path; - } - if (path.contains(".") == false) { - return null; - } - } while(true); - } - - /** - * Returns all nested object mappers - */ - public List getNestedMappers() { - List childMappers = new ArrayList<>(); - for (ObjectMapper mapper : objectMappers.values()) { - if (mapper.nested().isNested() == false) { - continue; - } - childMappers.add(mapper); - } - return childMappers; - } - - /** - * Given an object path, checks to see if any of its parents are non-nested objects - */ - public boolean hasNonNestedParent(String path) { - ObjectMapper mapper = objectMappers.get(path); - if (mapper == null) { - return false; - } - while (mapper != null) { - if (mapper.nested().isNested() == false) { - return true; - } - if (path.contains(".") == false) { - return false; - } - path = path.substring(0, path.lastIndexOf(".")); - mapper = objectMappers.get(path); - } - return false; - } - /** * Key for the lookup to be used in caches. */ diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java index 83434b7f0e926..b236801a1f5c7 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NestedObjectMapperTests.java @@ -63,13 +63,14 @@ public void testEmptyNested() throws Exception { } public void testSingleNested() throws Exception { - MappingLookup lookup = createDocumentMapper(mapping(b -> b.startObject("nested1").field("type", "nested").endObject())).mappers(); - assertThat(lookup.hasNested(), equalTo(true)); - ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); + DocumentMapper docMapper = createDocumentMapper(mapping(b -> b.startObject("nested1").field("type", "nested").endObject())); + + assertThat(docMapper.hasNestedObjects(), equalTo(true)); + ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); - ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference + ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference .bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -85,7 +86,7 @@ public void testSingleNested() throws Exception { assertThat(doc.docs().get(1).get("field"), equalTo("value")); - doc = lookup.parseDocument(new SourceToParse("test", "1", BytesReference + doc = docMapper.parse(new SourceToParse("test", "1", BytesReference .bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -108,7 +109,7 @@ public void testSingleNested() throws Exception { } public void testMultiNested() throws Exception { - MappingLookup lookup = createDocumentMapper(mapping(b -> { + DocumentMapper docMapper = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -119,19 +120,19 @@ public void testMultiNested() throws Exception { b.endObject(); } b.endObject(); - })).mappers(); + })); - assertThat(lookup.hasNested(), equalTo(true)); - ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); + assertThat(docMapper.hasNestedObjects(), equalTo(true)); + ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested1Mapper.nested().isIncludeInRoot(), equalTo(false)); - ObjectMapper nested2Mapper = lookup.objectMappers().get("nested1.nested2"); + ObjectMapper nested2Mapper = docMapper.mappers().objectMappers().get("nested1.nested2"); assertThat(nested2Mapper.nested().isNested(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(false)); - ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", + ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -173,7 +174,7 @@ public void testMultiNested() throws Exception { } public void testMultiObjectAndNested1() throws Exception { - MappingLookup lookup = createDocumentMapper(mapping(b -> { + DocumentMapper docMapper = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -189,19 +190,19 @@ public void testMultiObjectAndNested1() throws Exception { b.endObject(); } b.endObject(); - })).mappers(); + })); - assertThat(lookup.hasNested(), equalTo(true)); - ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); + assertThat(docMapper.hasNestedObjects(), equalTo(true)); + ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested1Mapper.nested().isIncludeInRoot(), equalTo(false)); - ObjectMapper nested2Mapper = lookup.objectMappers().get("nested1.nested2"); + ObjectMapper nested2Mapper = docMapper.mappers().objectMappers().get("nested1.nested2"); assertThat(nested2Mapper.nested().isNested(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(false)); - ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", + ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -243,7 +244,7 @@ public void testMultiObjectAndNested1() throws Exception { } public void testMultiObjectAndNested2() throws Exception { - MappingLookup lookup = createDocumentMapper(mapping(b -> { + DocumentMapper docMapper = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -260,19 +261,19 @@ public void testMultiObjectAndNested2() throws Exception { b.endObject(); } b.endObject(); - })).mappers(); + })); - assertThat(lookup.hasNested(), equalTo(true)); - ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); + assertThat(docMapper.hasNestedObjects(), equalTo(true)); + ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInParent(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInRoot(), equalTo(false)); - ObjectMapper nested2Mapper = lookup.objectMappers().get("nested1.nested2"); + ObjectMapper nested2Mapper = docMapper.mappers().objectMappers().get("nested1.nested2"); assertThat(nested2Mapper.nested().isNested(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(false)); - ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", + ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -314,7 +315,7 @@ public void testMultiObjectAndNested2() throws Exception { } public void testMultiRootAndNested1() throws Exception { - MappingLookup lookup = createDocumentMapper(mapping(b -> { + DocumentMapper docMapper = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -330,23 +331,23 @@ public void testMultiRootAndNested1() throws Exception { b.endObject(); } b.endObject(); - })).mappers(); + })); - assertEquals("nested1", lookup.getNestedParent("nested1.nested2")); - assertNull(lookup.getNestedParent("nonexistent")); - assertNull(lookup.getNestedParent("nested1")); + assertEquals("nested1", docMapper.getNestedParent("nested1.nested2")); + assertNull(docMapper.getNestedParent("nonexistent")); + assertNull(docMapper.getNestedParent("nested1")); - assertThat(lookup.hasNested(), equalTo(true)); - ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); + assertThat(docMapper.hasNestedObjects(), equalTo(true)); + ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested1Mapper.nested().isIncludeInRoot(), equalTo(false)); - ObjectMapper nested2Mapper = lookup.objectMappers().get("nested1.nested2"); + ObjectMapper nested2Mapper = docMapper.mappers().objectMappers().get("nested1.nested2"); assertThat(nested2Mapper.nested().isNested(), equalTo(true)); assertThat(nested2Mapper.nested().isIncludeInParent(), equalTo(false)); assertThat(nested2Mapper.nested().isIncludeInRoot(), equalTo(true)); - ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", + ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -524,7 +525,7 @@ public void testMultipleLevelsIncludeRootWithMerge() throws Exception { } public void testNestedArrayStrict() throws Exception { - MappingLookup lookup = createDocumentMapper(mapping(b -> { + DocumentMapper docMapper = createDocumentMapper(mapping(b -> { b.startObject("nested1"); { b.field("type", "nested"); @@ -536,14 +537,14 @@ public void testNestedArrayStrict() throws Exception { b.endObject(); } b.endObject(); - })).mappers(); + })); - assertThat(lookup.hasNested(), equalTo(true)); - ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); + assertThat(docMapper.hasNestedObjects(), equalTo(true)); + ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); assertThat(nested1Mapper.dynamic(), equalTo(Dynamic.STRICT)); - ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", + ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") @@ -617,7 +618,7 @@ public void testParentObjectMapperAreNested() throws Exception { } b.endObject(); })); - assertFalse(mapperService.mappingLookup().hasNonNestedParent("comments.messages")); + assertFalse(mapperService.documentMapper().hasNonNestedParent("comments.messages")); mapperService = createMapperService(mapping(b -> { b.startObject("comments"); @@ -631,7 +632,7 @@ public void testParentObjectMapperAreNested() throws Exception { } b.endObject(); })); - assertTrue(mapperService.mappingLookup().hasNonNestedParent("comments.messages")); + assertTrue(mapperService.documentMapper().hasNonNestedParent("comments.messages")); } public void testLimitNestedDocsDefaultSettings() throws Exception { @@ -776,15 +777,17 @@ public void testLimitNestedDocsMultipleNestedFields() throws Exception { } public void testReorderParent() throws IOException { + Version version = VersionUtils.randomIndexCompatibleVersion(random()); - MappingLookup lookup = createDocumentMapper(version, mapping(b -> b.startObject("nested1").field("type", "nested").endObject())) - .mappers(); - assertThat(lookup.hasNested(), equalTo(true)); - ObjectMapper nested1Mapper = lookup.objectMappers().get("nested1"); + DocumentMapper docMapper + = createDocumentMapper(version, mapping(b -> b.startObject("nested1").field("type", "nested").endObject())); + + assertThat(docMapper.hasNestedObjects(), equalTo(true)); + ObjectMapper nested1Mapper = docMapper.mappers().objectMappers().get("nested1"); assertThat(nested1Mapper.nested().isNested(), equalTo(true)); - ParsedDocument doc = lookup.parseDocument(new SourceToParse("test", "1", + ParsedDocument doc = docMapper.parse(new SourceToParse("test", "1", BytesReference.bytes(XContentFactory.jsonBuilder() .startObject() .field("field", "value") diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 0477fb38104a8..f0343540ad7b0 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -85,6 +85,7 @@ import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.MappingLookupUtils; import org.elasticsearch.index.mapper.NumberFieldMapper; @@ -103,6 +104,7 @@ import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.NestedDocuments; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.MultiBucketConsumerService.MultiBucketConsumer; @@ -237,7 +239,7 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche true ) { @Override - public boolean hasNested() { + public boolean hasNested() { // NOCOMMIT still needed? /* * Disable fetching nested documents. Luckily if confusingly, this does * not disable nested aggregations which we need to test. @@ -343,6 +345,10 @@ public boolean shouldCache(Query query) { IndexShard indexShard = mock(IndexShard.class); when(indexShard.shardId()).thenReturn(new ShardId("test", "test", 0)); when(ctx.indexShard()).thenReturn(indexShard); + MapperService mapperService = mock(MapperService.class); + when(mapperService.hasNested()).thenReturn(false); + NestedDocuments nested = new NestedDocuments(mapperService, bitsetFilterCache::getBitSetProducer); + when(ctx.getNestedDocuments()).thenReturn(nested); return new SubSearchContext(ctx); } From 6ef65a7e048704580c48e20c0a03e996776e636b Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 12:21:43 -0500 Subject: [PATCH 29/47] Checkstyle, you loveable scamp --- .../main/java/org/elasticsearch/index/mapper/MappingLookup.java | 1 - 1 file changed, 1 deletion(-) 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 3bf2733845500..07850c8700839 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -21,7 +21,6 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import java.util.ArrayList; import java.util.Collection; From 4fc3ed96ac66294ce3d00c183c7dfdc94cd87224 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 12:23:54 -0500 Subject: [PATCH 30/47] Probably not neeeded --- .../search/aggregations/AggregatorTestCase.java | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index f0343540ad7b0..19667c4bc3cb8 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -237,16 +237,7 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche 0, souceToParse -> null, true - ) { - @Override - public boolean hasNested() { // NOCOMMIT still needed? - /* - * Disable fetching nested documents. Luckily if confusingly, this does - * not disable nested aggregations which we need to test. - */ - return false; - } - }; + ); TriFunction, IndexFieldData> fieldDataBuilder = ( fieldType, From 07c74b37ed7fbc137c5bef587a108289bb0fb7e2 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 12:32:22 -0500 Subject: [PATCH 31/47] Revert moar --- .../org/elasticsearch/index/cache/bitset/BitsetFilterCache.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java b/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java index 511f64f947c6c..a48f4a24fbbb4 100644 --- a/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java +++ b/server/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java @@ -238,7 +238,7 @@ public IndexWarmer.TerminationHandle warmReader(final IndexShard indexShard, fin final MapperService mapperService = indexShard.mapperService(); DocumentMapper docMapper = mapperService.documentMapper(); if (docMapper != null) { - if (docMapper.mappers().hasNested()) { + if (docMapper.hasNestedObjects()) { warmUp.add(Queries.newNonNestedFilter()); docMapper.getNestedParentMappers().stream().map(ObjectMapper::nestedTypeFilter).forEach(warmUp::add); } From dcfdf441b03dacaa0c9fea9bd49d7f6a4c833ac5 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 21 Dec 2020 17:50:53 -0500 Subject: [PATCH 32/47] Document not fixed issue --- docs/reference/indices/apis/reload-analyzers.asciidoc | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/reference/indices/apis/reload-analyzers.asciidoc b/docs/reference/indices/apis/reload-analyzers.asciidoc index eba55ff615bdc..b09ac06c2af95 100644 --- a/docs/reference/indices/apis/reload-analyzers.asciidoc +++ b/docs/reference/indices/apis/reload-analyzers.asciidoc @@ -13,9 +13,17 @@ stream's backing indices. [source,console] -------------------------------------------------- POST /my-index-000001/_reload_search_analyzers +POST /my-index-000001/_cache/clear?request=true -------------------------------------------------- // TEST[setup:my_index] +IMPORTANT: After reloading the search analyzers you should clear the request + to make sure it doesn't contain responses derrived from the + previous versions of the analyzer. +// the need for this is tracked in https://github.com/elastic/elasticsearch/issues/66722 + + + [discrete] [[indices-reload-analyzers-api-request]] === {api-request-title} From 67364828c4273a83df8ad074fdf6115bb8461cd0 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 07:51:23 -0500 Subject: [PATCH 33/47] words are hard --- .../java/org/elasticsearch/index/mapper/MappingLookup.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 07850c8700839..418ab83663dad 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -33,8 +33,8 @@ import java.util.stream.Stream; /** - * A (mostly) immutable snapshot current mapping of an index with access - * to everything we need for the search phase. + * A (mostly) immutable snapshot of the current mapping of an index with + * access to everything we need for the search phase. */ public class MappingLookup { /** From 65962e2fdd083fb54ac8ef53425216475cecc0dc Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 07:57:51 -0500 Subject: [PATCH 34/47] Switch to MockFieldMapper --- .../org/elasticsearch/index/mapper/FieldMapper.java | 2 +- .../index/mapper/MappingLookupUtils.java | 13 ++----------- .../elasticsearch/index/mapper/MockFieldMapper.java | 9 ++++++++- 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java index afc1eb2baf6c3..b90fb8ff8fe81 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java @@ -309,7 +309,7 @@ protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, protected abstract String contentType(); - public Map indexAnalyzers() { + public final Map indexAnalyzers() { return indexAnalyzers; } diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java index b11e26b7b4e95..6daf0d33aa987 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.mapper; import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.mapper.FieldMapper.CopyTo; import java.util.Arrays; import java.util.List; @@ -28,8 +27,6 @@ import java.util.stream.Stream; import static java.util.stream.Collectors.toList; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class MappingLookupUtils { public static MappingLookup fromTypes(MappedFieldType... types) { @@ -52,16 +49,10 @@ public static MappingLookup fromTypes(Stream types) { } public static FieldMapper mockFieldMapper(MappedFieldType type) { - FieldMapper mapper = mock(FieldMapper.class); - when(mapper.fieldType()).thenReturn(type); - String name = type.name(); - when(mapper.name()).thenReturn(name); - when(mapper.copyTo()).thenReturn(CopyTo.empty()); Map indexAnalyzers = Map.of(); if (type.getTextSearchInfo() != TextSearchInfo.NONE) { - indexAnalyzers = Map.of(mapper.name(), type.getTextSearchInfo().getSearchAnalyzer()); + indexAnalyzers = Map.of(type.name(), type.getTextSearchInfo().getSearchAnalyzer()); } - when(mapper.indexAnalyzers()).thenReturn(indexAnalyzers); - return mapper; + return new MockFieldMapper(type, indexAnalyzers); } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MockFieldMapper.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MockFieldMapper.java index 4072083ab59c5..454bc2ae4b2c9 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MockFieldMapper.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MockFieldMapper.java @@ -19,10 +19,13 @@ package org.elasticsearch.index.mapper; +import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.query.QueryShardContext; import java.util.Collections; import java.util.List; +import java.util.Map; // this sucks how much must be overridden just do get a dummy field mapper... public class MockFieldMapper extends FieldMapper { @@ -32,7 +35,11 @@ public MockFieldMapper(String fullName) { } public MockFieldMapper(MappedFieldType fieldType) { - super(findSimpleName(fieldType.name()), fieldType, + this(fieldType, Map.of()); + } + + public MockFieldMapper(MappedFieldType fieldType, Map indexAnalyzers) { + super(findSimpleName(fieldType.name()), fieldType, indexAnalyzers, MultiFields.empty(), new CopyTo.Builder().build()); } From 0f4adcca0e78c0889b172a9d91d6a4ee838157bb Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 07:59:15 -0500 Subject: [PATCH 35/47] Speeling --- .../elasticsearch/search/aggregations/AggregatorTestCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 19667c4bc3cb8..5139cafab3953 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -235,7 +235,7 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche .collect(toList()), List.of(), 0, - souceToParse -> null, + sourceToParse -> null, true ); From 068e67d02a1861698d63411824eb7053e513753f Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 08:00:24 -0500 Subject: [PATCH 36/47] Add link in todo --- .../main/java/org/elasticsearch/index/mapper/MapperService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 9212c978ed19b..96282c9ffad28 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -485,7 +485,7 @@ public synchronized List reloadSearchAnalyzers(AnalysisRegistry registry reloadedAnalyzers.add(analyzerName); } } - // TODO this should make a new MappingLookup to bust the cache + // TODO this should bust the cache somehow. Tracked in https://github.com/elastic/elasticsearch/issues/66722 return reloadedAnalyzers; } } From 7b7cc2786da39c1e5e81435cbec1b5ed4cb15b07 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 08:01:29 -0500 Subject: [PATCH 37/47] Finish rename --- .../java/org/elasticsearch/index/mapper/MapperServiceTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 affd52bba4b16..cc6fe07e6e92b 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/MapperServiceTests.java @@ -47,7 +47,7 @@ public void testPreflightUpdateDoesNotChangeMapping() throws Throwable { assertThat("field was not created by mapping update", mapperService.fieldType("field0"), notNullValue()); } - public void testLookup() throws IOException { + public void testMappingLookup() throws IOException { MapperService service = createMapperService(mapping(b -> {})); MappingLookup oldLookup = service.mappingLookup(); assertThat(oldLookup.fieldTypes().get("cat"), nullValue()); From 83f9f30acfe3557643603d00444fd05ff48a1c42 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 08:14:13 -0500 Subject: [PATCH 38/47] Special handling of runtime fields --- .../index/query/QueryShardContextTests.java | 2 +- .../index/mapper/MappingLookupUtils.java | 32 ++++++++----------- .../DocumentSubsetBitsetCacheTests.java | 2 +- 3 files changed, 16 insertions(+), 20 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java index 222d2d003109e..da2424e7a200e 100644 --- a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java @@ -376,7 +376,7 @@ private static QueryShardContext createQueryShardContext( IndexMetadata indexMetadata = indexMetadataBuilder.build(); IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); MapperService mapperService = createMapperService(indexSettings, mapperPlugins); - MappingLookup mappingLookup = MappingLookupUtils.fromTypes(fieldTypes.stream()); + MappingLookup mappingLookup = MappingLookupUtils.fromTypes(fieldTypes); final long nowInMillis = randomNonNegativeLong(); return new QueryShardContext( 0, diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java index 6daf0d33aa987..77df51f888033 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java @@ -21,31 +21,27 @@ import org.elasticsearch.index.analysis.NamedAnalyzer; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.stream.Stream; - -import static java.util.stream.Collectors.toList; public class MappingLookupUtils { public static MappingLookup fromTypes(MappedFieldType... types) { - return fromTypes(Arrays.stream(types)); + return fromTypes(Arrays.asList(types)); } - - public static MappingLookup fromTypes(Stream types) { - List mappers = types.map(MappingLookupUtils::mockFieldMapper).collect(toList()); - // Alias -alias to so we can test aliases - return new MappingLookup( - "_doc", - mappers, - List.of(), - List.of(), - List.of(), - 0, - souceToParse -> null, - true - ); + + public static MappingLookup fromTypes(List fields) { + List mappers = new ArrayList<>(); + List runtimeFields = new ArrayList<>(); + for (MappedFieldType type : fields) { + if (type instanceof RuntimeFieldType) { + runtimeFields.add((RuntimeFieldType) type); + } else { + mappers.add(mockFieldMapper(type)); + } + } + return new MappingLookup("_doc", mappers, List.of(), List.of(), runtimeFields, 0, souceToParse -> null, true); } public static FieldMapper mockFieldMapper(MappedFieldType type) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java index 5c2f996e6ac04..814bbf2af354a 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java @@ -593,7 +593,7 @@ private void runTestOnIndices(int numberIndices, CheckedConsumer Date: Tue, 22 Dec 2020 08:15:21 -0500 Subject: [PATCH 39/47] Extra casting --- .../elasticsearch/index/query/QueryShardContextTests.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java index da2424e7a200e..589899fd1828b 100644 --- a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java @@ -60,6 +60,7 @@ import org.elasticsearch.index.mapper.MappingLookupUtils; import org.elasticsearch.index.mapper.MockFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.mapper.RuntimeFieldType; import org.elasticsearch.index.mapper.TestRuntimeField; import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.indices.IndicesModule; @@ -432,11 +433,11 @@ private static MapperService createMapperService( return mapperService; } - private static MappedFieldType runtimeField(String name, Function runtimeDocValues) { + private static RuntimeFieldType runtimeField(String name, Function runtimeDocValues) { return runtimeField(name, (leafLookup, docId) -> runtimeDocValues.apply(leafLookup)); } - private static MappedFieldType runtimeField(String name, BiFunction runtimeDocValues) { + private static RuntimeFieldType runtimeField(String name, BiFunction runtimeDocValues) { return new TestRuntimeField(name, null) { @Override public IndexFieldData.Builder fielddataBuilder(String fullyQualifiedIndexName, From 78d5b334f2526363f58dc35fd343239bd3225c48 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 08:30:16 -0500 Subject: [PATCH 40/47] equals and hashcode for the key --- .../indices/IndicesRequestCache.java | 1 + .../indices/IndicesRequestCacheTests.java | 25 ++++++++++++++++--- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java b/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java index 088f4e729e0cc..5fb9f06050263 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java @@ -258,6 +258,7 @@ public boolean equals(Object o) { @Override public int hashCode() { int result = entity.getCacheIdentity().hashCode(); + result = 31 * result + mappingCacheKey.hashCode(); result = 31 * result + readerCacheKey.hashCode(); result = 31 * result + value.hashCode(); return result; diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java index 38f28f984345a..e57a41a56f09a 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java @@ -518,7 +518,7 @@ public void testInvalidate() throws Exception { assertEquals(0, cache.numRegisteredCloseListeners()); } - public void testEqualsKey() throws IOException { + public void testKeyEqualsAndHashCode() throws IOException { AtomicBoolean trueBoolean = new AtomicBoolean(true); AtomicBoolean falseBoolean = new AtomicBoolean(false); MappingLookup.CacheKey mKey1 = MappingLookupUtils.fromTypes().cacheKey(); @@ -550,15 +550,32 @@ public void testEqualsKey() throws IOException { for (IndicesRequestCache.Key key2 : keys) { if (key1 == key2) { assertEquals(key1, key2); + assertEquals(key1.hashCode(), key2.hashCode()); } else { assertNotEquals(key1, key2); + assertNotEquals(key1.hashCode(), key2.hashCode()); + /* + * If we made random keys it'd be possible for us to have + * hash collisions and for the assertion above to fail. + * But we don't use random keys for this test. + */ } } } - assertEquals( - new IndicesRequestCache.Key(new TestEntity(null, trueBoolean), mKey1, rKey1, new TestBytesReference(1)), - new IndicesRequestCache.Key(new TestEntity(null, trueBoolean), mKey1, rKey1, new TestBytesReference(1)) + IndicesRequestCache.Key key1 = new IndicesRequestCache.Key( + new TestEntity(null, trueBoolean), + mKey1, + rKey1, + new TestBytesReference(1) ); + IndicesRequestCache.Key key2 = new IndicesRequestCache.Key( + new TestEntity(null, trueBoolean), + mKey1, + rKey1, + new TestBytesReference(1) + ); + assertEquals(key1, key2); + assertEquals(key1.hashCode(), key2.hashCode()); } private class TestBytesReference extends AbstractBytesReference { From d4795914b6dc5f888466e583ae22c0c85771c203 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 08:35:24 -0500 Subject: [PATCH 41/47] Speeling --- .../main/java/org/elasticsearch/index/mapper/MappingLookup.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 418ab83663dad..83b44963a7516 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -54,7 +54,7 @@ private CacheKey() {} List.of(), List.of(), 0, - souceToParse -> null, + soucreToParse -> null, false ); From 86575731d7c8c3e7b4732c593f8455c336cd9e88 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 08:39:26 -0500 Subject: [PATCH 42/47] Start your paragraph with a capital letter --- .../java/org/elasticsearch/indices/IndicesRequestCache.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java b/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java index 5fb9f06050263..0e0a2fcd0e6dd 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesRequestCache.java @@ -130,7 +130,7 @@ BytesReference getOrCompute(CacheEntity cacheEntity, CheckedSupplier Date: Tue, 22 Dec 2020 08:43:04 -0500 Subject: [PATCH 43/47] Not null --- .../java/org/elasticsearch/indices/IndicesRequestCacheTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java index e57a41a56f09a..3262b925966be 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java @@ -547,6 +547,7 @@ public void testKeyEqualsAndHashCode() throws IOException { assertNotEquals(key, "Some other random object"); } for (IndicesRequestCache.Key key1 : keys) { + assertNotEquals(key1, null); for (IndicesRequestCache.Key key2 : keys) { if (key1 == key2) { assertEquals(key1, key2); From 465e6a649b51ddd58e6bca20cc317bff92bc0963 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Tue, 22 Dec 2020 12:25:40 -0500 Subject: [PATCH 44/47] Try this --- .../index/query/QueryShardContextTests.java | 22 +++++++++------ .../terms/SignificantTextAggregatorTests.java | 8 ++++++ .../index/mapper/MappingLookupUtils.java | 28 ++++--------------- .../index/mapper/MockFieldMapper.java | 1 - .../aggregations/AggregatorTestCase.java | 12 ++++++-- .../DocumentSubsetBitsetCacheTests.java | 2 +- 6 files changed, 37 insertions(+), 36 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java index 589899fd1828b..58b034bb802a2 100644 --- a/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/QueryShardContextTests.java @@ -78,7 +78,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -305,7 +304,7 @@ public void testFielddataLookupReferencesBelowMaxDepth() throws IOException { public void testFielddataLookupOneFieldManyReferences() throws IOException { int numFields = randomIntBetween(5, 20); - List fields = new ArrayList<>(numFields + 1); + List fields = new ArrayList<>(numFields + 1); fields.add(runtimeField("root", leafLookup -> { StringBuilder value = new StringBuilder(); for (int i = 0; i < numFields; i++) { @@ -321,7 +320,7 @@ public void testFielddataLookupOneFieldManyReferences() throws IOException { } assertEquals( List.of(expected.toString(), expected.toString()), - collect("root", createQueryShardContext("uuid", null, fields, Map.of(), List.of())) + collect("root", createQueryShardContext("uuid", null, MappingLookupUtils.fromTypes(List.of(), fields), Map.of(), List.of())) ); } @@ -338,7 +337,7 @@ public void testSearchRequestRuntimeFields() { QueryShardContext qsc = createQueryShardContext( "uuid", null, - List.of(new MockFieldMapper.FakeFieldType("pig"), new MockFieldMapper.FakeFieldType("cat")), + MappingLookupUtils.fromTypes(new MockFieldMapper.FakeFieldType("pig"), new MockFieldMapper.FakeFieldType("cat")), runtimeMappings, Collections.singletonList(new TestRuntimeField.Plugin())); assertTrue(qsc.isFieldMapped("cat")); @@ -354,17 +353,23 @@ public void testSearchRequestRuntimeFields() { } public static QueryShardContext createQueryShardContext(String indexUuid, String clusterAlias) { - return createQueryShardContext(indexUuid, clusterAlias, List.of(), Map.of(), List.of()); + return createQueryShardContext(indexUuid, clusterAlias, MappingLookup.EMPTY, Map.of(), List.of()); } - private static QueryShardContext createQueryShardContext(MappedFieldType... fieldTypeLookup) { - return createQueryShardContext("uuid", null, Arrays.asList(fieldTypeLookup), Collections.emptyMap(), Collections.emptyList()); + private static QueryShardContext createQueryShardContext(RuntimeFieldType... fieldTypes) { + return createQueryShardContext( + "uuid", + null, + MappingLookupUtils.fromTypes(List.of(), List.of(fieldTypes)), + Collections.emptyMap(), + Collections.emptyList() + ); } private static QueryShardContext createQueryShardContext( String indexUuid, String clusterAlias, - List fieldTypes, + MappingLookup mappingLookup, Map runtimeMappings, List mapperPlugins ) { @@ -377,7 +382,6 @@ private static QueryShardContext createQueryShardContext( IndexMetadata indexMetadata = indexMetadataBuilder.build(); IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); MapperService mapperService = createMapperService(indexSettings, mapperPlugins); - MappingLookup mappingLookup = MappingLookupUtils.fromTypes(fieldTypes); final long nowInMillis = randomNonNegativeLong(); return new QueryShardContext( 0, diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java index 8f60825732757..305aa1ec3d06c 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTextAggregatorTests.java @@ -35,7 +35,9 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; import org.elasticsearch.index.mapper.BinaryFieldMapper; +import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MockFieldMapper; import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType; import org.elasticsearch.search.aggregations.AggregationBuilder; @@ -49,6 +51,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Map; import static org.elasticsearch.search.aggregations.AggregationBuilders.sampler; import static org.elasticsearch.search.aggregations.AggregationBuilders.significantText; @@ -346,4 +349,9 @@ public void testSignificanceOnTextArrays() throws IOException { } } } + + @Override + protected FieldMapper buildMockFieldMapper(MappedFieldType ft) { + return new MockFieldMapper(ft, Map.of(ft.name(), ft.getTextSearchInfo().getSearchAnalyzer())); + } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java index 77df51f888033..56d866f849805 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MappingLookupUtils.java @@ -19,36 +19,18 @@ package org.elasticsearch.index.mapper; -import org.elasticsearch.index.analysis.NamedAnalyzer; - -import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Map; + +import static java.util.stream.Collectors.toList; public class MappingLookupUtils { public static MappingLookup fromTypes(MappedFieldType... types) { - return fromTypes(Arrays.asList(types)); + return fromTypes(Arrays.asList(types), List.of()); } - public static MappingLookup fromTypes(List fields) { - List mappers = new ArrayList<>(); - List runtimeFields = new ArrayList<>(); - for (MappedFieldType type : fields) { - if (type instanceof RuntimeFieldType) { - runtimeFields.add((RuntimeFieldType) type); - } else { - mappers.add(mockFieldMapper(type)); - } - } + public static MappingLookup fromTypes(List concreteFields, List runtimeFields) { + List mappers = concreteFields.stream().map(MockFieldMapper::new).collect(toList()); return new MappingLookup("_doc", mappers, List.of(), List.of(), runtimeFields, 0, souceToParse -> null, true); } - - public static FieldMapper mockFieldMapper(MappedFieldType type) { - Map indexAnalyzers = Map.of(); - if (type.getTextSearchInfo() != TextSearchInfo.NONE) { - indexAnalyzers = Map.of(type.name(), type.getTextSearchInfo().getSearchAnalyzer()); - } - return new MockFieldMapper(type, indexAnalyzers); - } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MockFieldMapper.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MockFieldMapper.java index 454bc2ae4b2c9..f46cac153f925 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MockFieldMapper.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MockFieldMapper.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.mapper; -import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.query.QueryShardContext; diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 5139cafab3953..9a0ab844cead8 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -87,7 +87,7 @@ import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MappingLookup; -import org.elasticsearch.index.mapper.MappingLookupUtils; +import org.elasticsearch.index.mapper.MockFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.mapper.ObjectMapper; import org.elasticsearch.index.mapper.RangeFieldMapper; @@ -227,7 +227,7 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche MappingLookup mappingLookup = new MappingLookup( "_doc", - Arrays.stream(fieldTypes).map(MappingLookupUtils::mockFieldMapper).collect(toList()), + Arrays.stream(fieldTypes).map(this::buildMockFieldMapper).collect(toList()), objectMappers(), // Alias all fields to -alias to test aliases Arrays.stream(fieldTypes) @@ -289,6 +289,14 @@ public void onCache(ShardId shardId, Accountable accountable) {} ); } + /** + * Build a {@link FieldMapper} to create the {@link MappingLookup} used for the aggs. + * {@code protected} so subclasses can have it. + */ + protected FieldMapper buildMockFieldMapper(MappedFieldType ft) { + return new MockFieldMapper(ft); + } + /** * {@link ObjectMapper}s to add to the lookup. By default we don't need * any {@link ObjectMapper}s but testing nested objects will require adding some. diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java index 814bbf2af354a..181b5cec3bd08 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java @@ -593,7 +593,7 @@ private void runTestOnIndices(int numberIndices, CheckedConsumer Date: Wed, 23 Dec 2020 09:17:03 -0500 Subject: [PATCH 45/47] Update cache invalidation --- .../reference/modules/indices/request_cache.asciidoc | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/reference/modules/indices/request_cache.asciidoc b/docs/reference/modules/indices/request_cache.asciidoc index 6208f09bf0832..17bacdae69ace 100644 --- a/docs/reference/modules/indices/request_cache.asciidoc +++ b/docs/reference/modules/indices/request_cache.asciidoc @@ -32,14 +32,14 @@ Scripted queries that use the API calls which are non-deterministic, such as The cache is smart -- it keeps the same _near real-time_ promise as uncached search. -Cached results are invalidated automatically whenever the shard refreshes, but -only if the data in the shard has actually changed. In other words, you will -always get the same results from the cache as you would for an uncached search -request. +Cached results are invalidated automatically whenever the shard refreshes to +pick up changes to the documents or when you update the mapping. In other +words you will always get the same results from the cache as you would for an +uncached search request. The longer the refresh interval, the longer that cached entries will remain -valid. If the cache is full, the least recently used cache keys will be -evicted. +valid even if ther eare changes to the documents. If the cache is full, the +least recently used cache keys will be evicted. The cache can be expired manually with the <>: From 629f510ea9db1212df07531509b098766703bc27 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 23 Dec 2020 09:17:56 -0500 Subject: [PATCH 46/47] Speeling --- docs/reference/indices/apis/reload-analyzers.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/indices/apis/reload-analyzers.asciidoc b/docs/reference/indices/apis/reload-analyzers.asciidoc index b09ac06c2af95..63af37ae32a9e 100644 --- a/docs/reference/indices/apis/reload-analyzers.asciidoc +++ b/docs/reference/indices/apis/reload-analyzers.asciidoc @@ -18,7 +18,7 @@ POST /my-index-000001/_cache/clear?request=true // TEST[setup:my_index] IMPORTANT: After reloading the search analyzers you should clear the request - to make sure it doesn't contain responses derrived from the + cache to make sure it doesn't contain responses derived from the previous versions of the analyzer. // the need for this is tracked in https://github.com/elastic/elasticsearch/issues/66722 From 4f4efd64699b89dae240a2e3b1a929f8105942d5 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 23 Dec 2020 12:24:02 -0500 Subject: [PATCH 47/47] Typo --- docs/reference/modules/indices/request_cache.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/modules/indices/request_cache.asciidoc b/docs/reference/modules/indices/request_cache.asciidoc index 17bacdae69ace..4504f1f4f368b 100644 --- a/docs/reference/modules/indices/request_cache.asciidoc +++ b/docs/reference/modules/indices/request_cache.asciidoc @@ -38,7 +38,7 @@ words you will always get the same results from the cache as you would for an uncached search request. The longer the refresh interval, the longer that cached entries will remain -valid even if ther eare changes to the documents. If the cache is full, the +valid even if there are changes to the documents. If the cache is full, the least recently used cache keys will be evicted. The cache can be expired manually with the <>: