diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index be5c57de7fcda..ec25b4530816d 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -46,9 +46,9 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.Mapping; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.Uid; @@ -581,7 +581,7 @@ protected final GetResult getFromSearcher(Get get, Engine.Searcher searcher) thr } } - public abstract GetResult get(Get get, DocumentMapper mapper, Function searcherWrapper); + public abstract GetResult get(Get get, MappingLookup mappingLookup, Function searcherWrapper); /** * Acquires a point-in-time reader that can be used to create {@link Engine.Searcher}s on demand. diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index ba6bc1820f434..7fa5ea768edea 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -61,8 +61,8 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -606,10 +606,10 @@ private ExternalReaderManager createReaderManager(RefreshWarmerListener external } } - private GetResult getFromTranslog(Get get, Translog.Index index, DocumentMapper mapper, + private GetResult getFromTranslog(Get get, Translog.Index index, MappingLookup mappingLookup, Function searcherWrapper) throws IOException { assert get.isReadFromTranslog(); - final SingleDocDirectoryReader inMemoryReader = new SingleDocDirectoryReader(shardId, index, mapper, config().getAnalyzer()); + final SingleDocDirectoryReader inMemoryReader = new SingleDocDirectoryReader(shardId, index, mappingLookup, config().getAnalyzer()); final Engine.Searcher searcher = new Engine.Searcher("realtime_get", ElasticsearchDirectoryReader.wrap(inMemoryReader, shardId), config().getSimilarity(), config().getQueryCache(), config().getQueryCachingPolicy(), inMemoryReader); final Searcher wrappedSearcher = searcherWrapper.apply(searcher); @@ -628,7 +628,7 @@ private GetResult getFromTranslog(Get get, Translog.Index index, DocumentMapper } @Override - public GetResult get(Get get, DocumentMapper mapper, Function searcherWrapper) { + public GetResult get(Get get, MappingLookup mappingLookup, Function searcherWrapper) { assert Objects.equals(get.uid().field(), IdFieldMapper.NAME) : get.uid().field(); try (ReleasableLock ignored = readLock.acquire()) { ensureOpen(); @@ -659,7 +659,7 @@ public GetResult get(Get get, DocumentMapper mapper, Function searcherWrapper) { + public GetResult get(Get get, MappingLookup mappingLookup, Function searcherWrapper) { return getFromSearcher(get, acquireSearcher("get", SearcherScope.EXTERNAL, searcherWrapper)); } diff --git a/server/src/main/java/org/elasticsearch/index/engine/SingleDocDirectoryReader.java b/server/src/main/java/org/elasticsearch/index/engine/SingleDocDirectoryReader.java index 02ef823d79dc7..413fb606ba2bd 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/SingleDocDirectoryReader.java +++ b/server/src/main/java/org/elasticsearch/index/engine/SingleDocDirectoryReader.java @@ -30,7 +30,7 @@ import org.apache.lucene.util.Bits; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.shard.ShardId; @@ -47,8 +47,8 @@ final class SingleDocDirectoryReader extends DirectoryReader { private final SingleDocLeafReader leafReader; - SingleDocDirectoryReader(ShardId shardId, Translog.Index operation, DocumentMapper mapper, Analyzer analyzer) throws IOException { - this(new SingleDocLeafReader(shardId, operation, mapper, analyzer)); + SingleDocDirectoryReader(ShardId shardId, Translog.Index operation, MappingLookup mappingLookup, Analyzer analyzer) throws IOException { + this(new SingleDocLeafReader(shardId, operation, mappingLookup, analyzer)); } private SingleDocDirectoryReader(SingleDocLeafReader leafReader) throws IOException { @@ -109,15 +109,15 @@ private static class SingleDocLeafReader extends LeafReader { private final ShardId shardId; private final Translog.Index operation; - private final DocumentMapper mapper; + private final MappingLookup mappingLookup; private final Analyzer analyzer; private final Directory directory; private final AtomicReference delegate = new AtomicReference<>(); - SingleDocLeafReader(ShardId shardId, Translog.Index operation, DocumentMapper mapper, Analyzer analyzer) { + SingleDocLeafReader(ShardId shardId, Translog.Index operation, MappingLookup mappingLookup, Analyzer analyzer) { this.shardId = shardId; this.operation = operation; - this.mapper = mapper; + this.mappingLookup = mappingLookup; this.analyzer = analyzer; this.directory = new ByteBuffersDirectory(); } @@ -140,7 +140,7 @@ private LeafReader getDelegate() { private LeafReader createInMemoryLeafReader() { assert Thread.holdsLock(this); - final ParsedDocument parsedDocs = mapper.parse(new SourceToParse(shardId.getIndexName(), operation.id(), + final ParsedDocument parsedDocs = mappingLookup.parseDocument(new SourceToParse(shardId.getIndexName(), operation.id(), operation.source(), XContentHelper.xContentType(operation.source()), operation.routing(), Map.of())); parsedDocs.updateSeqID(operation.seqNo(), operation.primaryTerm()); diff --git a/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java b/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java index c5b61cd0b8b6d..0bb45b1f260c1 100644 --- a/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java +++ b/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java @@ -34,9 +34,9 @@ import org.elasticsearch.index.engine.TranslogLeafReader; import org.elasticsearch.index.fieldvisitor.CustomFieldsVisitor; import org.elasticsearch.index.fieldvisitor.FieldsVisitor; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.RoutingFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; @@ -122,7 +122,7 @@ public GetResult get(Engine.GetResult engineGetResult, String id, try { long now = System.nanoTime(); fetchSourceContext = normalizeFetchSourceContent(fetchSourceContext, fields); - GetResult getResult = innerGetLoadFromStoredFields(id, fields, fetchSourceContext, engineGetResult, mapperService); + GetResult getResult = innerGetLoadFromStoredFields(id, fields, fetchSourceContext, engineGetResult); if (getResult.isExists()) { existsMetric.inc(System.nanoTime() - now); } else { @@ -169,23 +169,23 @@ private GetResult innerGet(String id, String[] gFields, boolean realtime, long v try { // break between having loaded it from translog (so we only have _source), and having a document to load - return innerGetLoadFromStoredFields(id, gFields, fetchSourceContext, get, mapperService); + return innerGetLoadFromStoredFields(id, gFields, fetchSourceContext, get); } finally { get.close(); } } private GetResult innerGetLoadFromStoredFields(String id, String[] storedFields, FetchSourceContext fetchSourceContext, - Engine.GetResult get, MapperService mapperService) { + Engine.GetResult get) { assert get.exists() : "method should only be called if document could be retrieved"; // check first if stored fields to be loaded don't contain an object field - DocumentMapper docMapper = mapperService.documentMapper(); + MappingLookup mappingLookup = mapperService.mappingLookup(); if (storedFields != null) { for (String field : storedFields) { - Mapper fieldMapper = docMapper.mappers().getMapper(field); + Mapper fieldMapper = mappingLookup.getMapper(field); if (fieldMapper == null) { - if (docMapper.mappers().objectMappers().get(field) != null) { + if (mappingLookup.objectMappers().get(field) != null) { // Only fail if we know it is a object field, missing paths / fields shouldn't fail. throw new IllegalArgumentException("field [" + field + "] isn't a leaf field"); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperForType.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperForType.java deleted file mode 100644 index 25df9c0c0cb45..0000000000000 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperForType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 and the Server Side Public License, v 1; you may not use this file except - * in compliance with, at your election, the Elastic License 2.0 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.index.mapper; - -public class DocumentMapperForType { - private final DocumentMapper documentMapper; - private final Mapping mapping; - - public DocumentMapperForType(DocumentMapper documentMapper, Mapping mapping) { - this.mapping = mapping; - this.documentMapper = documentMapper; - } - - public DocumentMapper getDocumentMapper() { - return documentMapper; - } - - public Mapping getMapping() { - return mapping; - } -} 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 33bb1d3df39d0..f6c1ba34c1fd0 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -96,6 +96,7 @@ public enum MergeReason { private final Version indexVersionCreated; private final MapperRegistry mapperRegistry; private final Supplier parserContextSupplier; + private final MappingLookup emptyMappingLookup; private volatile DocumentMapper mapper; @@ -117,6 +118,10 @@ public MapperService(IndexSettings indexSettings, IndexAnalyzers indexAnalyzers, this.parserContextSupplier = () -> parserContextFunction.apply(null); this.mappingParser = new MappingParser(parserContextSupplier, metadataMapperParsers, this::getMetadataMappers, this::resolveDocumentType); + //pre-compute the empty mapping lookup for this mapper service so it can be returned whenever document mapper is null + MetadataFieldMapper[] metadataMappers = getMetadataMappers().values().toArray(new MetadataFieldMapper[0]); + Mapping mapping = Mapping.empty(MapperService.SINGLE_MAPPING_NAME, indexSettings.getIndexVersionCreated(), metadataMappers); + this.emptyMappingLookup = MappingLookup.fromMapping(mapping, documentParser, indexSettings, indexAnalyzers); } public boolean hasNested() { @@ -149,7 +154,6 @@ Map, MetadataFieldMapper> getMetadataMapper MetadataFieldMapper metadataFieldMapper = parser.getDefault(parserContext()); metadataMappers.put(metadataFieldMapper.getClass(), metadataFieldMapper); } - } else { metadataMappers.putAll(existingMapper.mapping().getMetadataMappersMap()); } @@ -343,20 +347,6 @@ private String resolveDocumentType(String type) { return type; } - /** - * Returns the document mapper for this MapperService. If no mapper exists, - * creates one and returns that. - */ - public DocumentMapperForType documentMapperWithAutoCreate() { - DocumentMapper mapper = documentMapper(); - if (mapper != null) { - return new DocumentMapperForType(mapper, null); - } - Mapping mapping = mappingParser.parse(SINGLE_MAPPING_NAME, null); - mapper = new DocumentMapper(indexSettings, indexAnalyzers, documentParser, mapping); - return new DocumentMapperForType(mapper, mapper.mapping()); - } - /** * Given the full name of a field, returns its {@link MappedFieldType}. */ @@ -369,7 +359,7 @@ public MappedFieldType fieldType(String fullName) { */ public MappingLookup mappingLookup() { DocumentMapper mapper = this.mapper; - return mapper == null ? MappingLookup.EMPTY : mapper.mappers(); + return mapper == null ? emptyMappingLookup : mapper.mappers(); } /** diff --git a/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java b/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java index 950dd7ed93838..6610a31554fb1 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java @@ -36,10 +36,19 @@ */ public final class Mapping implements ToXContentFragment { - public static final Mapping EMPTY = new Mapping( - new RootObjectMapper.Builder("_doc", Version.CURRENT).build(new ContentPath()), - new MetadataFieldMapper[0], - Collections.emptyMap()); + /** + * Creates a new instance of empty mapping that have no fields explicitly mapped + * @param type the type + * @param indexVersionCreated the version which the current index has been created on + * @param metadataMappers the metadata mappers registered + * @return the empty mapping instance + */ + public static Mapping empty(String type, Version indexVersionCreated, MetadataFieldMapper[] metadataMappers) { + return new Mapping( + new RootObjectMapper.Builder(type, indexVersionCreated).build(new ContentPath()), + metadataMappers, + Collections.emptyMap()); + } private final RootObjectMapper root; private final Map meta; @@ -66,7 +75,6 @@ public int compare(Mapper o1, Mapper o2) { this.metadataMappersMap = unmodifiableMap(metadataMappersMap); this.metadataMappersByName = unmodifiableMap(metadataMappersByName); this.meta = meta; - } CompressedXContent toCompressedXContent() { @@ -77,6 +85,10 @@ CompressedXContent toCompressedXContent() { } } + boolean hasMappings() { + return root.iterator().hasNext(); + } + /** * Returns the root object for the current mapping */ 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 f201ecc6c546f..6de866d30e73e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -34,19 +34,6 @@ public static class CacheKey { private CacheKey() {} } - /** - * A lookup representing an empty mapping. - */ - public static final MappingLookup EMPTY = new MappingLookup( - Mapping.EMPTY, - List.of(), - List.of(), - List.of(), - null, - null, - null - ); - private final CacheKey cacheKey = new CacheKey(); /** Full field name to mapper */ @@ -324,7 +311,7 @@ public ParsedDocument parseDocument(SourceToParse source) { } public boolean hasMappings() { - return this != EMPTY; + return mapping.hasMappings(); } public boolean isSourceEnabled() { diff --git a/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java index 23354c6998872..2c12aa9c510c1 100644 --- a/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java @@ -138,10 +138,11 @@ protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws @Override protected Query doToQuery(SearchExecutionContext context) throws IOException { - MappedFieldType idField = context.getFieldType(IdFieldMapper.NAME); - if (idField == null || ids.isEmpty()) { + if (context.hasMappings() == false || ids.isEmpty()) { throw new IllegalStateException("Rewrite first"); } + MappedFieldType idField = context.getFieldType(IdFieldMapper.NAME); + assert idField != null; return idField.termsQuery(new ArrayList<>(ids), context); } diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 980cda98dbb6d..36232eb487a4f 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -85,7 +85,6 @@ import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.CommitStats; import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.engine.Engine.GetResult; import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineFactory; @@ -101,11 +100,11 @@ import org.elasticsearch.index.get.ShardGetService; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.mapper.DocumentMapper; -import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SourceToParse; @@ -826,7 +825,7 @@ private Engine.IndexResult applyIndexOperation(Engine engine, long seqNo, long o ensureWriteAllowed(origin); Engine.Index operation; try { - operation = prepareIndex(docMapper(), sourceToParse, + operation = prepareIndex(mapperService.mappingLookup(), sourceToParse, seqNo, opPrimaryTerm, version, versionType, origin, autoGeneratedTimeStamp, isRetry, ifSeqNo, ifPrimaryTerm); Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); if (update != null) { @@ -844,17 +843,14 @@ private Engine.IndexResult applyIndexOperation(Engine engine, long seqNo, long o return index(engine, operation); } - public static Engine.Index prepareIndex(DocumentMapperForType docMapper, SourceToParse source, long seqNo, + public static Engine.Index prepareIndex(MappingLookup mappingLookup, SourceToParse source, long seqNo, long primaryTerm, long version, VersionType versionType, Engine.Operation.Origin origin, long autoGeneratedIdTimestamp, boolean isRetry, long ifSeqNo, long ifPrimaryTerm) { long startTime = System.nanoTime(); assert source.dynamicTemplates().isEmpty() || origin == Engine.Operation.Origin.PRIMARY : "dynamic_templates parameter can only be associated with primary operations"; - ParsedDocument doc = docMapper.getDocumentMapper().parse(source); - if (docMapper.getMapping() != null) { - doc.addDynamicMappingsUpdate(docMapper.getMapping()); - } + ParsedDocument doc = mappingLookup.parseDocument(source); Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(doc.id())); return new Engine.Index(uid, doc, seqNo, primaryTerm, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry, ifSeqNo, ifPrimaryTerm); @@ -974,11 +970,7 @@ private Engine.DeleteResult delete(Engine engine, Engine.Delete delete) throws I public Engine.GetResult get(Engine.Get get) { readAllowed(); - DocumentMapper mapper = mapperService.documentMapper(); - if (mapper == null) { - return GetResult.NOT_EXISTS; - } - return getEngine().get(get, mapper, this::wrapSearcher); + return getEngine().get(get, mapperService.mappingLookup(), this::wrapSearcher); } /** @@ -2821,10 +2813,6 @@ protected Analyzer getWrappedAnalyzer(String fieldName) { }; } - private DocumentMapperForType docMapper() { - return mapperService.documentMapperWithAutoCreate(); - } - private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) { final Sort indexSort = indexSortSupplier.get(); final Engine.Warmer warmer = reader -> { @@ -3502,7 +3490,8 @@ private EngineConfig.TombstoneDocSupplier tombstoneDocSupplier() { return new EngineConfig.TombstoneDocSupplier() { @Override public ParsedDocument newDeleteTombstoneDoc(String id) { - return docMapper().getDocumentMapper().createDeleteTombstoneDoc(shardId.getIndexName(), id); + //TODO not sure what to do when it's null. we may want to make sure it never is. + return mapperService.documentMapper().createDeleteTombstoneDoc(shardId.getIndexName(), id); } @Override public ParsedDocument newNoopTombstoneDoc(String reason) { diff --git a/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java b/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java index 280cc896275c0..a0eb5798309b5 100644 --- a/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java +++ b/server/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java @@ -30,7 +30,6 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.get.GetResult; -import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParseContext; @@ -322,13 +321,8 @@ public static String[] getValues(IndexableField[] fields) { private static ParsedDocument parseDocument(IndexShard indexShard, String index, BytesReference doc, XContentType xContentType, String routing) { MapperService mapperService = indexShard.mapperService(); - DocumentMapperForType docMapper = mapperService.documentMapperWithAutoCreate(); - ParsedDocument parsedDocument = docMapper.getDocumentMapper().parse( + return mapperService.mappingLookup().parseDocument( new SourceToParse(index, "_id_for_tv_api", doc, xContentType, routing, Map.of())); - if (docMapper.getMapping() != null) { - parsedDocument.addDynamicMappingsUpdate(docMapper.getMapping()); - } - return parsedDocument; } private static Fields mergeFields(Fields fields1, Fields fields2) throws IOException { 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 e663762669e21..31eb7f4b037da 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 @@ -15,13 +15,13 @@ import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.DataStreamTestHelper; import org.elasticsearch.cluster.metadata.AliasAction; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.AliasValidator; import org.elasticsearch.cluster.metadata.ComponentTemplate; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.cluster.metadata.DataStreamTestHelper; import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -575,7 +575,7 @@ protected String contentType() { } }; MappingLookup mappingLookup = new MappingLookup( - Mapping.EMPTY, + Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]), List.of(mockedTimestampField, dateFieldMapper), List.of(), List.of(), @@ -677,7 +677,7 @@ protected String contentType() { } }; MappingLookup mappingLookup = new MappingLookup( - Mapping.EMPTY, + Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]), List.of(mockedTimestampField, dateFieldMapper), List.of(), List.of(), diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index f3d4a809e835f..a75c51471ebb1 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -100,8 +100,8 @@ import org.elasticsearch.index.VersionType; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.fieldvisitor.FieldsVisitor; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; @@ -697,7 +697,7 @@ public void testConcurrentGetAndFlush() throws Exception { engine.index(indexForDoc(doc)); final AtomicReference latestGetResult = new AtomicReference<>(); - latestGetResult.set(engine.get(newGet(true, doc), docMapper(), randomSearcherWrapper())); + latestGetResult.set(engine.get(newGet(true, doc), mappingLookup(), randomSearcherWrapper())); final AtomicBoolean flushFinished = new AtomicBoolean(false); final CyclicBarrier barrier = new CyclicBarrier(2); Thread getThread = new Thread(() -> { @@ -711,7 +711,7 @@ public void testConcurrentGetAndFlush() throws Exception { if (previousGetResult != null) { previousGetResult.close(); } - latestGetResult.set(engine.get(newGet(true, doc), docMapper(), randomSearcherWrapper())); + latestGetResult.set(engine.get(newGet(true, doc), mappingLookup(), randomSearcherWrapper())); if (latestGetResult.get().exists() == false) { break; } @@ -727,7 +727,7 @@ public void testConcurrentGetAndFlush() throws Exception { } public void testSimpleOperations() throws Exception { - final DocumentMapper mapper = docMapper(); + final MappingLookup mappingLookup = mappingLookup(); engine.refresh("warm_up"); Engine.Searcher searchResult = engine.acquireSearcher("test"); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); @@ -749,18 +749,18 @@ public void testSimpleOperations() throws Exception { searchResult.close(); // but, not there non realtime - try (Engine.GetResult getResult = engine.get(newGet(false, doc), mapper, randomSearcherWrapper())) { + try (Engine.GetResult getResult = engine.get(newGet(false, doc), mappingLookup, randomSearcherWrapper())) { assertThat(getResult.exists(), equalTo(false)); } // but, we can still get it (in realtime) - try (Engine.GetResult getResult = engine.get(newGet(true, doc), mapper, randomSearcherWrapper())) { + try (Engine.GetResult getResult = engine.get(newGet(true, doc), mappingLookup, randomSearcherWrapper())) { assertThat(getResult.exists(), equalTo(true)); assertThat(getResult.docIdAndVersion(), notNullValue()); } // but not real time is not yet visible - try (Engine.GetResult getResult = engine.get(newGet(false, doc), mapper, randomSearcherWrapper())) { + try (Engine.GetResult getResult = engine.get(newGet(false, doc), mappingLookup, randomSearcherWrapper())) { assertThat(getResult.exists(), equalTo(false)); } @@ -775,7 +775,7 @@ public void testSimpleOperations() throws Exception { searchResult.close(); // also in non realtime - try (Engine.GetResult getResult = engine.get(newGet(false, doc), mapper, randomSearcherWrapper())) { + try (Engine.GetResult getResult = engine.get(newGet(false, doc), mappingLookup, randomSearcherWrapper())) { assertThat(getResult.exists(), equalTo(true)); assertThat(getResult.docIdAndVersion(), notNullValue()); } @@ -797,7 +797,7 @@ public void testSimpleOperations() throws Exception { searchResult.close(); // but, we can still get it (in realtime) - try (Engine.GetResult getResult = engine.get(newGet(true, doc), mapper, randomSearcherWrapper())) { + try (Engine.GetResult getResult = engine.get(newGet(true, doc), mappingLookup, randomSearcherWrapper())) { assertThat(getResult.exists(), equalTo(true)); assertThat(getResult.docIdAndVersion(), notNullValue()); } @@ -826,7 +826,7 @@ public void testSimpleOperations() throws Exception { searchResult.close(); // but, get should not see it (in realtime) - try (Engine.GetResult getResult = engine.get(newGet(true, doc), mapper, randomSearcherWrapper())) { + try (Engine.GetResult getResult = engine.get(newGet(true, doc), mappingLookup, randomSearcherWrapper())) { assertThat(getResult.exists(), equalTo(false)); } @@ -872,7 +872,7 @@ public void testSimpleOperations() throws Exception { engine.flush(); // and, verify get (in real time) - try (Engine.GetResult getResult = engine.get(newGet(true, doc), mapper, randomSearcherWrapper())) { + try (Engine.GetResult getResult = engine.get(newGet(true, doc), mappingLookup, randomSearcherWrapper())) { assertThat(getResult.exists(), equalTo(true)); assertThat(getResult.docIdAndVersion(), notNullValue()); } @@ -909,7 +909,7 @@ public void testGetWithSearcherWrapper() throws Exception { engine.refresh("warm_up"); engine.index(indexForDoc(createParsedDoc("1", null))); assertThat(engine.lastRefreshedCheckpoint(), equalTo(NO_OPS_PERFORMED)); - try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), docMapper(), randomSearcherWrapper())) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), mappingLookup(), randomSearcherWrapper())) { // we do not track the translog location yet assertTrue(get.exists()); assertFalse(get.isFromTranslog()); @@ -918,7 +918,7 @@ public void testGetWithSearcherWrapper() throws Exception { assertThat(engine.lastRefreshedCheckpoint(), equalTo(0L)); engine.index(indexForDoc(createParsedDoc("1", null))); - try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), docMapper(), searcher -> searcher)) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), mappingLookup(), searcher -> searcher)) { assertTrue(get.exists()); assertTrue(get.isFromTranslog()); } @@ -926,25 +926,25 @@ public void testGetWithSearcherWrapper() throws Exception { if (randomBoolean()) { engine.index(indexForDoc(createParsedDoc("1", null))); } - try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), docMapper(), + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), mappingLookup(), searcher -> SearcherHelper.wrapSearcher(searcher, reader -> new MatchingDirectoryReader(reader, new MatchAllDocsQuery())))) { assertTrue(get.exists()); assertFalse(get.isFromTranslog()); } - try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), docMapper(), + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), mappingLookup(), searcher -> SearcherHelper.wrapSearcher(searcher, reader -> new MatchingDirectoryReader(reader, new MatchNoDocsQuery())))) { assertFalse(get.exists()); assertFalse(get.isFromTranslog()); } - try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), docMapper(), + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), mappingLookup(), searcher -> SearcherHelper.wrapSearcher(searcher, reader -> new MatchingDirectoryReader(reader, new TermQuery(newUid("1")))))) { assertTrue(get.exists()); assertFalse(get.isFromTranslog()); } - try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), docMapper(), + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, "1"), mappingLookup(), searcher -> SearcherHelper.wrapSearcher(searcher, reader -> new MatchingDirectoryReader(reader, new TermQuery(newUid("2")))))) { assertFalse(get.exists()); assertFalse(get.isFromTranslog()); @@ -1205,7 +1205,7 @@ public void testVersionedUpdate() throws IOException { Engine.Index create = new Engine.Index(newUid(doc), primaryTerm.get(), doc, Versions.MATCH_DELETED); Engine.IndexResult indexResult = engine.index(create); assertThat(indexResult.getVersion(), equalTo(1L)); - try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), docMapper(), randomSearcherWrapper())) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), mappingLookup(), randomSearcherWrapper())) { assertEquals(1, get.version()); } @@ -1213,7 +1213,7 @@ public void testVersionedUpdate() throws IOException { Engine.IndexResult update_1_result = engine.index(update_1); assertThat(update_1_result.getVersion(), equalTo(2L)); - try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), docMapper(), randomSearcherWrapper())) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), mappingLookup(), randomSearcherWrapper())) { assertEquals(2, get.version()); } @@ -1221,7 +1221,7 @@ public void testVersionedUpdate() throws IOException { Engine.IndexResult update_2_result = engine.index(update_2); assertThat(update_2_result.getVersion(), equalTo(3L)); - try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), docMapper(), randomSearcherWrapper())) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), mappingLookup(), randomSearcherWrapper())) { assertEquals(3, get.version()); } @@ -1241,22 +1241,22 @@ public void testGetIfSeqNoIfPrimaryTerm() throws IOException { try (Engine.GetResult get = engine.get( new Engine.Get(true, true, doc.id()) .setIfSeqNo(indexResult.getSeqNo()).setIfPrimaryTerm(primaryTerm.get()), - docMapper(), randomSearcherWrapper())) { + mappingLookup(), randomSearcherWrapper())) { assertEquals(indexResult.getSeqNo(), get.docIdAndVersion().seqNo); } expectThrows(VersionConflictEngineException.class, () -> engine.get(new Engine.Get(true, false, doc.id()) .setIfSeqNo(indexResult.getSeqNo() + 1).setIfPrimaryTerm(primaryTerm.get()), - docMapper(), randomSearcherWrapper())); + mappingLookup(), randomSearcherWrapper())); expectThrows(VersionConflictEngineException.class, () -> engine.get(new Engine.Get(true, false, doc.id()) .setIfSeqNo(indexResult.getSeqNo()).setIfPrimaryTerm(primaryTerm.get() + 1), - docMapper(), randomSearcherWrapper())); + mappingLookup(), randomSearcherWrapper())); final VersionConflictEngineException versionConflictEngineException = expectThrows(VersionConflictEngineException.class, () -> engine.get(new Engine.Get(true, false, doc.id()) .setIfSeqNo(indexResult.getSeqNo() + 1).setIfPrimaryTerm(primaryTerm.get() + 1), - docMapper(), randomSearcherWrapper())); + mappingLookup(), randomSearcherWrapper())); assertThat(versionConflictEngineException.getStackTrace(), emptyArray()); } @@ -2008,7 +2008,8 @@ class OpAndVersion { throw new AssertionError(e); } for (int op = 0; op < opsPerThread; op++) { - try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), docMapper(), randomSearcherWrapper())) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), mappingLookup(), + randomSearcherWrapper())) { FieldsVisitor visitor = new FieldsVisitor(true); get.docIdAndVersion().reader.document(get.docIdAndVersion().docId, visitor); List values = new ArrayList<>(Strings.commaDelimitedListToSet(visitor.source().utf8ToString())); @@ -2050,7 +2051,7 @@ class OpAndVersion { assertTrue(op.added + " should not exist", exists); } - try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), docMapper(), randomSearcherWrapper())) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.id()), mappingLookup(), randomSearcherWrapper())) { FieldsVisitor visitor = new FieldsVisitor(true); get.docIdAndVersion().reader.document(get.docIdAndVersion().docId, visitor); List values = Arrays.asList(Strings.commaDelimitedListToStringArray(visitor.source().utf8ToString())); @@ -2446,7 +2447,7 @@ public void testEnableGcDeletes() throws Exception { Engine engine = createEngine(config(defaultSettings, store, createTempDir(), newMergePolicy(), null))) { engine.config().setEnableGcDeletes(false); - final DocumentMapper mapper = docMapper(); + final MappingLookup mappingLookup = mappingLookup(); // Add document Document document = testDocument(); @@ -2462,7 +2463,7 @@ public void testEnableGcDeletes() throws Exception { 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), UNASSIGNED_SEQ_NO, 0)); // Get should not find the document - Engine.GetResult getResult = engine.get(newGet(true, doc), mapper, randomSearcherWrapper()); + Engine.GetResult getResult = engine.get(newGet(true, doc), mappingLookup, randomSearcherWrapper()); assertThat(getResult.exists(), equalTo(false)); // Give the gc pruning logic a chance to kick in @@ -2477,7 +2478,7 @@ public void testEnableGcDeletes() throws Exception { 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), UNASSIGNED_SEQ_NO, 0)); // Get should not find the document (we never indexed uid=2): - getResult = engine.get(new Engine.Get(true, false, "2"), mapper, randomSearcherWrapper()); + getResult = engine.get(new Engine.Get(true, false, "2"), mappingLookup, randomSearcherWrapper()); assertThat(getResult.exists(), equalTo(false)); // Try to index uid=1 with a too-old version, should fail: @@ -2488,7 +2489,7 @@ public void testEnableGcDeletes() throws Exception { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // Get should still not find the document - getResult = engine.get(newGet(true, doc), mapper, randomSearcherWrapper()); + getResult = engine.get(newGet(true, doc), mappingLookup, randomSearcherWrapper()); assertThat(getResult.exists(), equalTo(false)); // Try to index uid=2 with a too-old version, should fail: @@ -2499,7 +2500,7 @@ public void testEnableGcDeletes() throws Exception { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // Get should not find the document - getResult = engine.get(newGet(true, doc), mapper, randomSearcherWrapper()); + getResult = engine.get(newGet(true, doc), mappingLookup, randomSearcherWrapper()); assertThat(getResult.exists(), equalTo(false)); } } @@ -4013,7 +4014,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio } assertThat(engine.getProcessedLocalCheckpoint(), equalTo(expectedLocalCheckpoint)); - try (Engine.GetResult result = engine.get(new Engine.Get(true, false, "1"), docMapper(), randomSearcherWrapper())) { + try (Engine.GetResult result = engine.get(new Engine.Get(true, false, "1"), mappingLookup(), randomSearcherWrapper())) { assertThat(result.exists(), equalTo(exists)); } } @@ -4945,13 +4946,13 @@ public void testStressUpdateSameDocWhileGettingIt() throws IOException, Interrup Thread thread = new Thread(() -> { awaitStarted.countDown(); try (Engine.GetResult getResult = engine.get( - new Engine.Get(true, false, doc3.id()), docMapper(), searcher -> searcher)) { + new Engine.Get(true, false, doc3.id()), mappingLookup(), searcher -> searcher)) { assertTrue(getResult.exists()); } }); thread.start(); awaitStarted.await(); - try (Engine.GetResult getResult = engine.get(new Engine.Get(true, false, doc.id()), docMapper(), + try (Engine.GetResult getResult = engine.get(new Engine.Get(true, false, doc.id()), mappingLookup(), searcher -> SearcherHelper.wrapSearcher(searcher, r -> new MatchingDirectoryReader(r, new MatchAllDocsQuery())))) { assertFalse(getResult.exists()); } @@ -5899,7 +5900,7 @@ public void afterRefresh(boolean didRefresh) { int iters = randomIntBetween(1, 10); for (int i = 0; i < iters; i++) { ParsedDocument doc = createParsedDoc(randomFrom(ids), null); - try (Engine.GetResult getResult = engine.get(newGet(true, doc), docMapper(), randomSearcherWrapper())) { + try (Engine.GetResult getResult = engine.get(newGet(true, doc), mappingLookup(), randomSearcherWrapper())) { assertThat(getResult.exists(), equalTo(true)); assertThat(getResult.docIdAndVersion(), notNullValue()); } diff --git a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java index 38e3f30b05404..29bbe7499f7af 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java @@ -98,7 +98,7 @@ public void testReadOnlyEngine() throws Exception { assertThat(readOnlyEngine.getPersistedLocalCheckpoint(), equalTo(lastSeqNoStats.getLocalCheckpoint())); assertThat(readOnlyEngine.getSeqNoStats(globalCheckpoint.get()).getMaxSeqNo(), equalTo(lastSeqNoStats.getMaxSeqNo())); assertThat(getDocIds(readOnlyEngine, false), equalTo(lastDocIds)); - try (Engine.GetResult getResult = readOnlyEngine.get(get, docMapper(), randomSearcherWrapper())) { + try (Engine.GetResult getResult = readOnlyEngine.get(get, mappingLookup(), randomSearcherWrapper())) { assertTrue(getResult.exists()); } } 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 9a22de3ff01b2..c198190ae96c6 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java @@ -34,7 +34,8 @@ public void testTermQuery() { IndexSettings indexSettings = new IndexSettings( new IndexMetadata.Builder("foo").settings(settings).numberOfShards(1).numberOfReplicas(0).build(), settings); List mappers = Stream.of(fieldNamesFieldType, fieldType).map(MockFieldMapper::new).collect(Collectors.toList()); - MappingLookup mappingLookup = new MappingLookup(Mapping.EMPTY, mappers, emptyList(), emptyList(), null, null, null); + Mapping mapping = Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]); + MappingLookup mappingLookup = new MappingLookup(mapping, mappers, emptyList(), emptyList(), null, null, null); SearchExecutionContext searchExecutionContext = new SearchExecutionContext(0, 0, indexSettings, null, null, null, mappingLookup, null, null, null, null, null, null, () -> 0L, null, null, () -> true, null, emptyMap()); diff --git a/server/src/test/java/org/elasticsearch/index/query/SearchExecutionContextTests.java b/server/src/test/java/org/elasticsearch/index/query/SearchExecutionContextTests.java index 8d9c62c00c99f..7450ddf2e32f3 100644 --- a/server/src/test/java/org/elasticsearch/index/query/SearchExecutionContextTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/SearchExecutionContextTests.java @@ -381,7 +381,8 @@ public void testSearchRequestRuntimeFieldsRemoval() { } public static SearchExecutionContext createSearchExecutionContext(String indexUuid, String clusterAlias) { - return createSearchExecutionContext(indexUuid, clusterAlias, MappingLookup.EMPTY, Map.of()); + MappingLookup mappingLookup = createMappingLookup(Collections.emptyList(), Collections.emptyList()); + return createSearchExecutionContext(indexUuid, clusterAlias, mappingLookup, Map.of()); } private static SearchExecutionContext createSearchExecutionContext(RuntimeField... fieldTypes) { diff --git a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index 1773ada48b0f8..756747b22a81d 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -36,7 +36,6 @@ import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngine; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; @@ -335,8 +334,8 @@ public void testLotsOfThreads() throws Exception { listener.assertNoError(); Engine.Get get = new Engine.Get(false, false, threadId); - final DocumentMapper mapper = EngineTestCase.docMapper(); - try (Engine.GetResult getResult = engine.get(get, mapper, EngineTestCase.randomSearcherWrapper())) { + try (Engine.GetResult getResult = engine.get(get, EngineTestCase.mappingLookup(), + EngineTestCase.randomSearcherWrapper())) { assertTrue("document not found", getResult.exists()); assertEquals(iteration, getResult.version()); org.apache.lucene.document.Document document = diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java index c058a0f5cb924..ceacd123fbe1a 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesRequestCacheTests.java @@ -21,6 +21,7 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; import org.elasticsearch.common.CheckedSupplier; import org.elasticsearch.common.bytes.AbstractBytesReference; import org.elasticsearch.common.bytes.BytesReference; @@ -32,8 +33,12 @@ 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.ContentPath; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MetadataFieldMapper; +import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; @@ -41,6 +46,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; @@ -48,6 +54,12 @@ public class IndicesRequestCacheTests extends ESTestCase { + private static MappingLookup createMappingLookup() { + RootObjectMapper.Builder builder = new RootObjectMapper.Builder(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT); + Mapping mapping = new Mapping(builder.build(new ContentPath()), new MetadataFieldMapper[0], Collections.emptyMap()); + return new MappingLookup(mapping, Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), null, null, null); + } + public void testBasicOperationsCache() throws Exception { ShardRequestCache requestCacheStats = new ShardRequestCache(); IndicesRequestCache cache = new IndicesRequestCache(Settings.EMPTY); @@ -57,7 +69,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 = MappingLookup.EMPTY.cacheKey(); + MappingLookup.CacheKey mappingKey = createMappingLookup().cacheKey(); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); BytesReference termBytes = XContentHelper.toXContent(termQuery, XContentType.JSON, false); AtomicBoolean indexShard = new AtomicBoolean(true); @@ -107,7 +119,7 @@ public void testBasicOperationsCache() throws Exception { public void testCacheDifferentReaders() throws Exception { IndicesRequestCache cache = new IndicesRequestCache(Settings.EMPTY); - MappingLookup.CacheKey mappingKey = MappingLookup.EMPTY.cacheKey(); + MappingLookup.CacheKey mappingKey = createMappingLookup().cacheKey(); AtomicBoolean indexShard = new AtomicBoolean(true); ShardRequestCache requestCacheStats = new ShardRequestCache(); Directory dir = newDirectory(); @@ -202,8 +214,9 @@ public void testCacheDifferentReaders() throws Exception { public void testCacheDifferentMapping() throws Exception { IndicesRequestCache cache = new IndicesRequestCache(Settings.EMPTY); - MappingLookup.CacheKey mappingKey1 = MappingLookup.EMPTY.cacheKey(); - MappingLookup.CacheKey mappingKey2 = new MappingLookup(Mapping.EMPTY, emptyList(), emptyList(), emptyList(),null, null, null) + MappingLookup.CacheKey mappingKey1 = createMappingLookup().cacheKey(); + Mapping mapping = Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]); + MappingLookup.CacheKey mappingKey2 = new MappingLookup(mapping, emptyList(), emptyList(), emptyList(),null, null, null) .cacheKey(); AtomicBoolean indexShard = new AtomicBoolean(true); ShardRequestCache requestCacheStats = new ShardRequestCache(); @@ -281,7 +294,7 @@ public void testCacheDifferentMapping() throws Exception { } public void testEviction() throws Exception { - MappingLookup.CacheKey mappingKey = MappingLookup.EMPTY.cacheKey(); + MappingLookup.CacheKey mappingKey = createMappingLookup().cacheKey(); final ByteSizeValue size; { IndicesRequestCache cache = new IndicesRequestCache(Settings.EMPTY); @@ -362,7 +375,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 = MappingLookup.EMPTY.cacheKey(); + MappingLookup.CacheKey mappingKey = createMappingLookup().cacheKey(); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); BytesReference termBytes = XContentHelper.toXContent(termQuery, XContentType.JSON, false); TestEntity entity = new TestEntity(requestCacheStats, indexShard); @@ -371,7 +384,8 @@ 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 = new MappingLookup(Mapping.EMPTY, emptyList(), emptyList(), emptyList(), null, null, null) + Mapping mapping = Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]); + MappingLookup.CacheKey secondMappingKey = new MappingLookup(mapping, emptyList(), emptyList(), emptyList(), null, null, null) .cacheKey(); TestEntity secondEntity = new TestEntity(requestCacheStats, indexShard); Loader secondLoader = new Loader(secondReader, 0); @@ -379,7 +393,7 @@ public void testClearAllEntityIdentity() throws Exception { writer.updateDocument(new Term("id", "0"), newDoc(0, "baz")); DirectoryReader thirdReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); - MappingLookup.CacheKey thirdMappingKey = new MappingLookup(Mapping.EMPTY, emptyList(), emptyList(), emptyList(), null, null, null) + MappingLookup.CacheKey thirdMappingKey = new MappingLookup(mapping, emptyList(), emptyList(), emptyList(), null, null, null) .cacheKey(); AtomicBoolean differentIdentity = new AtomicBoolean(true); TestEntity thirdEntity = new TestEntity(requestCacheStats, differentIdentity); @@ -449,7 +463,7 @@ public void testInvalidate() throws Exception { IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); writer.addDocument(newDoc(0, "foo")); - MappingLookup.CacheKey mappingKey = MappingLookup.EMPTY.cacheKey(); + MappingLookup.CacheKey mappingKey = createMappingLookup().cacheKey(); DirectoryReader reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); @@ -515,8 +529,9 @@ public void testInvalidate() throws Exception { public void testKeyEqualsAndHashCode() throws IOException { AtomicBoolean trueBoolean = new AtomicBoolean(true); AtomicBoolean falseBoolean = new AtomicBoolean(false); - MappingLookup.CacheKey mKey1 = MappingLookup.EMPTY.cacheKey(); - MappingLookup.CacheKey mKey2 = new MappingLookup(Mapping.EMPTY, emptyList(), emptyList(), emptyList(), null, null, null).cacheKey(); + MappingLookup.CacheKey mKey1 = createMappingLookup().cacheKey(); + Mapping mapping = Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]); + MappingLookup.CacheKey mKey2 = new MappingLookup(mapping, emptyList(), emptyList(), emptyList(), null, null, null).cacheKey(); Directory dir = newDirectory(); IndexWriterConfig config = newIndexWriterConfig(); IndexWriter writer = new IndexWriter(dir, config); 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 80722d7b7f6a0..692317a980ab6 100644 --- a/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java @@ -30,6 +30,7 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.MockFieldMapper; import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.index.mapper.TextSearchInfo; @@ -172,7 +173,8 @@ public NamedAnalyzer get(Object key) { when(scriptService.compile(any(Script.class), any())).then(invocation -> new TestTemplateService.MockTemplateScript.Factory( ((Script) invocation.getArguments()[0]).getIdOrCode())); List mappers = Collections.singletonList(new MockFieldMapper(fieldType)); - MappingLookup lookup = new MappingLookup(Mapping.EMPTY, mappers, emptyList(), emptyList(), null, null, null); + Mapping mapping = Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]); + MappingLookup lookup = new MappingLookup(mapping, mappers, emptyList(), emptyList(), null, null, null); SearchExecutionContext mockContext = new SearchExecutionContext(0, 0, idxSettings, null, null, mapperService, lookup, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, System::currentTimeMillis, null, null, () -> true, null, emptyMap()); @@ -209,8 +211,10 @@ public void testBuildWithUnmappedField() { IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index(randomAlphaOfLengthBetween(1, 10), "_na_"), indexSettings); + Mapping empty = Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]); + MappingLookup mappingLookup = MappingLookup.fromMapping(empty, null, null, null); SearchExecutionContext mockContext = new SearchExecutionContext(0, 0, idxSettings, null, - null, mock(MapperService.class), MappingLookup.EMPTY, null, null, xContentRegistry(), namedWriteableRegistry, null, null, + null, mock(MapperService.class), mappingLookup, null, null, xContentRegistry(), namedWriteableRegistry, null, null, System::currentTimeMillis, null, null, () -> true, null, emptyMap()); if (randomBoolean()) { mockContext.setAllowUnmappedFields(randomBoolean()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 97dd6c5264ed3..d35362792ce5b 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -82,6 +82,7 @@ import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -1190,9 +1191,9 @@ public static MapperService createMapperService() throws IOException { return mapperService; } - public static DocumentMapper docMapper() { + public static MappingLookup mappingLookup() { try { - return createMapperService().documentMapper(); + return createMapperService().mappingLookup(); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java b/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java index 33f4db118ba4b..2b0fe14787867 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java @@ -9,7 +9,6 @@ package org.elasticsearch.index.engine; import org.apache.lucene.analysis.standard.StandardAnalyzer; -import org.elasticsearch.Version; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.index.IndexSettings; @@ -18,11 +17,8 @@ import org.elasticsearch.index.analysis.AnalyzerScope; import org.elasticsearch.index.analysis.IndexAnalyzers; import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.mapper.DocumentMapper; -import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.MapperRegistry; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -58,11 +54,6 @@ public TranslogHandler(NamedXContentRegistry xContentRegistry, IndexSettings ind () -> null, () -> false, null); } - private DocumentMapperForType docMapper(String type) { - RootObjectMapper.Builder rootBuilder = new RootObjectMapper.Builder(type, Version.CURRENT); - return new DocumentMapperForType(new DocumentMapper(rootBuilder, mapperService), null); - } - private void applyOperation(Engine engine, Engine.Operation operation) throws IOException { switch (operation.operationType()) { case INDEX: @@ -99,21 +90,18 @@ public Engine.Operation convertToEngineOp(Translog.Operation operation, Engine.O case INDEX: final Translog.Index index = (Translog.Index) operation; final String indexName = mapperService.index().getName(); - final Engine.Index engineIndex = IndexShard.prepareIndex(docMapper(MapperService.SINGLE_MAPPING_NAME), + return IndexShard.prepareIndex(mapperService.mappingLookup(), new SourceToParse(indexName, index.id(), index.source(), XContentHelper.xContentType(index.source()), index.routing(), Map.of()), index.seqNo(), index.primaryTerm(), index.version(), versionType, origin, index.getAutoGeneratedIdTimestamp(), true, SequenceNumbers.UNASSIGNED_SEQ_NO, SequenceNumbers.UNASSIGNED_PRIMARY_TERM); - return engineIndex; case DELETE: final Translog.Delete delete = (Translog.Delete) operation; return IndexShard.prepareDelete(delete.id(), delete.seqNo(), delete.primaryTerm(), delete.version(), versionType, origin, SequenceNumbers.UNASSIGNED_SEQ_NO, SequenceNumbers.UNASSIGNED_PRIMARY_TERM); case NO_OP: final Translog.NoOp noOp = (Translog.NoOp) operation; - final Engine.NoOp engineNoOp = - new Engine.NoOp(noOp.seqNo(), noOp.primaryTerm(), origin, System.nanoTime(), noOp.reason()); - return engineNoOp; + return new Engine.NoOp(noOp.seqNo(), noOp.primaryTerm(), origin, System.nanoTime(), noOp.reason()); default: throw new IllegalStateException("No operation defined for [" + operation + "]"); } 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 fa6db39327d3f..32db50c5e9d05 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 @@ -78,8 +78,10 @@ import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperRegistry; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.MockFieldMapper; import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.mapper.ObjectMapper; @@ -236,8 +238,9 @@ protected AggregationContext createAggregationContext(IndexSearcher indexSearche long bytesToPreallocate, int maxBucket, MappedFieldType... fieldTypes) throws IOException { + MappingLookup mappingLookup = new MappingLookup( - Mapping.EMPTY, + Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]), Arrays.stream(fieldTypes).map(this::buildMockFieldMapper).collect(toList()), objectMappers(), // Alias all fields to -alias to test aliases @@ -360,7 +363,8 @@ public boolean shouldCache(Query query) { * of stuff. */ SearchExecutionContext subContext = spy(searchExecutionContext); - MappingLookup disableNestedLookup = new MappingLookup(Mapping.EMPTY, Set.of(), Set.of(), Set.of(), null, null, null); + Mapping mapping = Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]); + MappingLookup disableNestedLookup = new MappingLookup(mapping, Set.of(), Set.of(), Set.of(), null, null, null); doReturn(new NestedDocuments(disableNestedLookup, bitsetFilterCache::getBitSetProducer)).when(subContext).getNestedDocuments(); when(ctx.getSearchExecutionContext()).thenReturn(subContext); 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 6a3a16a815b30..1ebcb94d4616f 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 @@ -26,6 +26,7 @@ import org.apache.lucene.util.BitSet; import org.apache.lucene.util.BitSetIterator; import org.apache.lucene.util.FixedBitSet; +import org.elasticsearch.Version; import org.elasticsearch.client.Client; import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.CheckedConsumer; @@ -34,8 +35,10 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.MockFieldMapper; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.SearchExecutionContext; @@ -598,7 +601,8 @@ private void runTestOnIndices(int numberIndices, CheckedConsumer concreteFields) { List mappers = concreteFields.stream().map(MockFieldMapper::new).collect(Collectors.toList()); - return new MappingLookup(Mapping.EMPTY, mappers, emptyList(), emptyList(), null, null, null); + Mapping mapping = Mapping.empty(MapperService.SINGLE_MAPPING_NAME, Version.CURRENT, new MetadataFieldMapper[0]); + return new MappingLookup(mapping, mappers, emptyList(), emptyList(), null, null, null); } }