From 0761875caed51edc3caf6f7a0dd42211b970ab06 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 3 May 2021 14:32:08 +0200 Subject: [PATCH] Replace some DocumentMapper usages with MappingLookup (#72400) We recently replaced some usages of DocumentMapper with MappingLookup in the search layer, as document mapper is mutable which can cause issues. In order to do that, MappingLookup grew and became quite similar to DocumentMapper in what it does and holds. In many cases it makes sense to use MappingLookup instead of DocumentMapper, and we may even be able to remove DocumentMapper entirely in favour of MappingLookup in the long run. This commit replaces some of its straight-forward usages. --- .../action/PainlessExecuteAction.java | 9 ++- .../TransportGetFieldMappingsIndexAction.java | 22 +++--- .../MetadataCreateDataStreamService.java | 10 +-- .../metadata/MetadataCreateIndexService.java | 2 +- .../MetadataIndexTemplateService.java | 3 +- .../elasticsearch/index/engine/Engine.java | 4 +- .../index/engine/InternalEngine.java | 10 +-- .../index/engine/LuceneChangesSnapshot.java | 4 +- .../index/engine/ReadOnlyEngine.java | 4 +- .../engine/SingleDocDirectoryReader.java | 14 ++-- .../index/get/ShardGetService.java | 21 ++--- .../elasticsearch/index/mapper/Mapping.java | 6 +- .../index/mapper/MappingLookup.java | 6 +- .../elasticsearch/index/shard/IndexShard.java | 13 +-- .../MetadataRolloverServiceTests.java | 54 ++++++------- .../index/engine/InternalEngineTests.java | 79 ++++++++++--------- .../index/engine/ReadOnlyEngineTests.java | 2 +- .../index/shard/RefreshListenersTests.java | 2 +- .../index/engine/EngineTestCase.java | 12 +-- .../MetadataCreateDataStreamServiceTests.java | 24 +++--- 20 files changed, 150 insertions(+), 151 deletions(-) diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java index 47cbc65164134..67b24e4b4a23d 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessExecuteAction.java @@ -59,6 +59,7 @@ import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.query.AbstractQueryBuilder; @@ -657,12 +658,14 @@ private static Response prepareRamIndex(Request request, try (Directory directory = new ByteBuffersDirectory()) { try (IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig(defaultAnalyzer))) { + MappingLookup mappingLookup = indexService.mapperService().mappingLookup(); String index = indexService.index().getName(); - String type = indexService.mapperService().documentMapper().type(); BytesReference document = request.contextSetup.document; XContentType xContentType = request.contextSetup.xContentType; - SourceToParse sourceToParse = new SourceToParse(index, type, "_id", document, xContentType); - ParsedDocument parsedDocument = indexService.mapperService().documentMapper().parse(sourceToParse); + SourceToParse sourceToParse = new SourceToParse(index, mappingLookup.getType(), "_id", document, xContentType); + //TODO this throws NPE when called against an empty index with no provided mappings: DocumentMapper is null + // and the corresponding empty MappingLookup does not have a DocumentParser set + ParsedDocument parsedDocument = mappingLookup.parseDocument(sourceToParse); indexWriter.addDocuments(parsedDocument.docs()); try (IndexReader indexReader = DirectoryReader.open(indexWriter)) { final IndexSearcher searcher = new IndexSearcher(indexReader); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java index ac970fdf957e3..23dbc648c5bab 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.shard.ShardId; @@ -84,15 +83,15 @@ protected GetFieldMappingsResponse shardOperation(final GetFieldMappingsIndexReq Predicate metadataFieldPredicate = (f) -> indexService.mapperService().isMetadataField(f); Predicate fieldPredicate = metadataFieldPredicate.or(indicesService.getFieldFilter().apply(shardId.getIndexName())); - DocumentMapper mapper = indexService.mapperService().documentMapper(); + MappingLookup mappingLookup = indexService.mapperService().mappingLookup(); Collection typeIntersection; if (request.types().length == 0) { - typeIntersection = mapper == null - ? Collections.emptySet() - : Collections.singleton(mapper.type()); + typeIntersection = mappingLookup.hasMappings() + ? Collections.singleton(mappingLookup.getType()) + : Collections.emptySet(); } else { - typeIntersection = mapper != null && Regex.simpleMatch(request.types(), mapper.type()) - ? Collections.singleton(mapper.type()) + typeIntersection = mappingLookup.hasMappings() && Regex.simpleMatch(request.types(), mappingLookup.getType()) + ? Collections.singleton(mappingLookup.getType()) : Collections.emptySet(); if (typeIntersection.isEmpty()) { throw new TypeMissingException(shardId.getIndex(), request.types()); @@ -101,8 +100,8 @@ protected GetFieldMappingsResponse shardOperation(final GetFieldMappingsIndexReq Map> typeMappings = new HashMap<>(); for (String type : typeIntersection) { - DocumentMapper documentMapper = indexService.mapperService().documentMapper(type); - Map fieldMapping = findFieldMappingsByType(fieldPredicate, documentMapper, request); + MappingLookup mappers = indexService.mapperService().documentMapper(type).mappers(); + Map fieldMapping = findFieldMappingsByType(fieldPredicate, mappers, request); if (fieldMapping.isEmpty() == false) { typeMappings.put(type, fieldMapping); } @@ -158,12 +157,11 @@ public Boolean paramAsBoolean(String key, Boolean defaultValue) { }; private static Map findFieldMappingsByType(Predicate fieldPredicate, - DocumentMapper documentMapper, - GetFieldMappingsIndexRequest request) { + MappingLookup mappingLookup, + GetFieldMappingsIndexRequest request) { //TODO the logic here needs to be reworked to also include runtime fields. Though matching is against mappers rather // than field types, and runtime fields are mixed with ordinary fields in FieldTypeLookup Map fieldMappings = new HashMap<>(); - final MappingLookup mappingLookup = documentMapper.mappers(); for (String field : request.fields()) { if (Regex.isMatchAllPattern(field)) { for (Mapper fieldMapper : mappingLookup.fieldMappers()) { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateDataStreamService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateDataStreamService.java index 07072ef544446..22f70dc5d691b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateDataStreamService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateDataStreamService.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.xcontent.ObjectPath; import org.elasticsearch.index.Index; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.indices.SystemDataStreamDescriptor; import org.elasticsearch.rest.RestStatus; @@ -244,13 +245,12 @@ public static ComposableIndexTemplate lookupTemplateForDataStream(String dataStr return composableIndexTemplate; } - public static void validateTimestampFieldMapping(String timestampFieldName, MapperService mapperService) throws IOException { - MetadataFieldMapper fieldMapper = - (MetadataFieldMapper) mapperService.documentMapper().mappers().getMapper("_data_stream_timestamp"); + public static void validateTimestampFieldMapping(MappingLookup mappingLookup) throws IOException { + MetadataFieldMapper fieldMapper = (MetadataFieldMapper) mappingLookup.getMapper("_data_stream_timestamp"); assert fieldMapper != null : "[_data_stream_timestamp] meta field mapper must exist"; Map parsedTemplateMapping = - MapperService.parseMapping(NamedXContentRegistry.EMPTY, mapperService.documentMapper().mappingSource().string()); + MapperService.parseMapping(NamedXContentRegistry.EMPTY, mappingLookup.getMapping().toCompressedXContent().string()); Boolean enabled = ObjectPath.eval("_doc._data_stream_timestamp.enabled", parsedTemplateMapping); // Sanity check: if this fails then somehow the mapping for _data_stream_timestamp has been overwritten and // that would be a bug. @@ -259,7 +259,7 @@ public static void validateTimestampFieldMapping(String timestampFieldName, Mapp } // Sanity check (this validation logic should already have been executed when merging mappings): - fieldMapper.validate(mapperService.documentMapper().mappers()); + fieldMapper.validate(mappingLookup); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java index d3d1406c3c153..90709d0871b8c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java @@ -1055,7 +1055,7 @@ private static void updateIndexMappingsAndBuildSortOrder(IndexService indexServi indexService.getIndexSortSupplier().get(); } if (request.dataStreamName() != null) { - validateTimestampFieldMapping("@timestamp", mapperService); + validateTimestampFieldMapping(mapperService.mappingLookup()); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java index 9ec01e07bc6f7..5f257e9b5055f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataIndexTemplateService.java @@ -1226,8 +1226,7 @@ private static void validateCompositeTemplate(final ClusterState state, } if (template.getDataStreamTemplate() != null) { - String tsFieldName = template.getDataStreamTemplate().getTimestampField(); - validateTimestampFieldMapping(tsFieldName, mapperService); + validateTimestampFieldMapping(mapperService.mappingLookup()); } } catch (Exception e) { throw new IllegalArgumentException("invalid composite mappings for [" + templateName + "]", e); 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 1294943fe52b6..43a8135497cef 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -51,8 +51,8 @@ import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.DocumentMapper; 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.merge.MergeStats; @@ -601,7 +601,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 2a8a96d6a0cf3..00217df2b28c0 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -63,9 +63,9 @@ 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.MapperService; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -653,10 +653,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); @@ -675,7 +675,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(); @@ -706,7 +706,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 1818c8b22dc7b..16c8d2ec6e2e8 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.type(), + final ParsedDocument parsedDocs = mappingLookup.parseDocument(new SourceToParse(shardId.getIndexName(), operation.type(), operation.id(), operation.source(), XContentHelper.xContentType(operation.source()), operation.routing(), Collections.emptyMap())); 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 450aeed7d0fdf..d46e4e8320f69 100644 --- a/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java +++ b/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java @@ -39,6 +39,7 @@ import org.elasticsearch.index.mapper.IdFieldMapper; 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; @@ -125,7 +126,7 @@ public GetResult get(Engine.GetResult engineGetResult, String id, String type, try { long now = System.nanoTime(); fetchSourceContext = normalizeFetchSourceContent(fetchSourceContext, fields); - GetResult getResult = innerGetLoadFromStoredFields(type, id, fields, fetchSourceContext, engineGetResult, mapperService); + GetResult getResult = innerGetLoadFromStoredFields(type, id, fields, fetchSourceContext, engineGetResult); if (getResult.isExists()) { existsMetric.inc(System.nanoTime() - now); } else { @@ -159,8 +160,8 @@ private GetResult innerGet(String type, String id, String[] gFields, boolean rea long ifSeqNo, long ifPrimaryTerm, FetchSourceContext fetchSourceContext) { fetchSourceContext = normalizeFetchSourceContent(fetchSourceContext, gFields); if (type == null || type.equals("_all")) { - DocumentMapper mapper = mapperService.documentMapper(); - type = mapper == null ? null : mapper.type(); + MappingLookup mappingLookup = mapperService.mappingLookup(); + type = mappingLookup.hasMappings() ? mappingLookup.getType() : null; } Engine.GetResult get = null; @@ -180,23 +181,24 @@ private GetResult innerGet(String type, String id, String[] gFields, boolean rea try { // break between having loaded it from translog (so we only have _source), and having a document to load - return innerGetLoadFromStoredFields(type, id, gFields, fetchSourceContext, get, mapperService); + return innerGetLoadFromStoredFields(type, id, gFields, fetchSourceContext, get); } finally { get.close(); } } private GetResult innerGetLoadFromStoredFields(String type, 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 = docMapper.mappers(); 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"); } @@ -236,7 +238,7 @@ private GetResult innerGetLoadFromStoredFields(String type, String id, String[] assert source != null : "original source in translog must exist"; SourceToParse sourceToParse = new SourceToParse(shardId.getIndexName(), type, id, source, XContentHelper.xContentType(source), fieldVisitor.routing(), Collections.emptyMap()); - ParsedDocument doc = indexShard.mapperService().documentMapper().parse(sourceToParse); + ParsedDocument doc = indexShard.mapperService().mappingLookup().parseDocument(sourceToParse); assert doc.dynamicMappingsUpdate() == null : "mapping updates should not be required on already-indexed doc"; // update special fields doc.updateSeqID(docIdAndVersion.seqNo, docIdAndVersion.primaryTerm); @@ -270,8 +272,7 @@ private GetResult innerGetLoadFromStoredFields(String type, String id, String[] // put stored fields into result objects if (fieldVisitor.fields().isEmpty() == false) { - fieldVisitor.postProcess(mapperService::fieldType, - mapperService.documentMapper() == null ? null : mapperService.documentMapper().type()); + fieldVisitor.postProcess(mapperService::fieldType, mappingLookup.hasMappings() ? mappingLookup.getType() : null); documentFields = new HashMap<>(); metadataFields = new HashMap<>(); for (Map.Entry> entry : fieldVisitor.fields().entrySet()) { 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 66df4b320832d..7e4acb2dd5a7a 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/Mapping.java @@ -69,7 +69,11 @@ public int compare(Mapper o1, Mapper o2) { } - CompressedXContent toCompressedXContent() { + /** + * Outputs this mapping instance and returns it in {@link CompressedXContent} format + * @return the {@link CompressedXContent} representation of this mapping instance + */ + public CompressedXContent toCompressedXContent() { try { return new CompressedXContent(this, XContentType.JSON, ToXContent.EMPTY_PARAMS); } catch (Exception e) { 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 88f7df66ac2d1..05dd23ef0732e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -380,7 +380,11 @@ public String getType() { return mapping.type(); } - Mapping getMapping() { + /** + * Returns the mapping source that this lookup originated from + * @return the mapping source + */ + public Mapping getMapping() { return mapping; } 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 37553619ba1a4..c9780c37ab1e9 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -99,13 +99,13 @@ import org.elasticsearch.index.get.GetStats; 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.MapperParsingException; 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.SourceToParse; import org.elasticsearch.index.mapper.Uid; @@ -963,13 +963,13 @@ private Engine.DeleteResult applyDeleteOperation(Engine engine, long seqNo, long } catch (MapperParsingException | IllegalArgumentException | TypeMissingException e) { return new Engine.DeleteResult(e, version, getOperationPrimaryTerm(), seqNo, false); } - if (mapperService.resolveDocumentType(type).equals(mapperService.documentMapper().type()) == false) { + if (mapperService.resolveDocumentType(type).equals(mapperService.mappingLookup().getType()) == false) { // We should never get there due to the fact that we generate mapping updates on deletes, // but we still prefer to have a hard exception here as we would otherwise delete a // document in the wrong type. throw new IllegalStateException("Deleting document from type [" + mapperService.resolveDocumentType(type) + "] while current type is [" + - mapperService.documentMapper().type() + "]"); + mapperService.mappingLookup().getType() + "]"); } final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); final Engine.Delete delete = prepareDelete(type, id, uid, seqNo, opPrimaryTerm, version, @@ -1004,11 +1004,12 @@ 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 || mapper.type().equals(mapperService.resolveDocumentType(get.type())) == false) { + MappingLookup mappingLookup = mapperService.mappingLookup(); + if (mappingLookup.hasMappings() == false || + mappingLookup.getType().equals(mapperService.resolveDocumentType(get.type())) == false) { return GetResult.NOT_EXISTS; } - return getEngine().get(get, mapper, this::wrapSearcher); + return getEngine().get(get, mappingLookup, this::wrapSearcher); } /** 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 cbda697b24783..1e4433d398a73 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 @@ -29,13 +29,13 @@ import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.MetadataCreateIndexService; import org.elasticsearch.cluster.metadata.MetadataIndexAliasesService; +import org.elasticsearch.cluster.metadata.MetadataIndexTemplateServiceTests; import org.elasticsearch.cluster.metadata.Template; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; @@ -45,13 +45,12 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.DateFieldMapper; -import org.elasticsearch.index.mapper.DocumentMapper; 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.MetadataFieldMapper; -import org.elasticsearch.index.mapper.RoutingFieldMapper; +import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.indices.EmptySystemIndices; import org.elasticsearch.indices.IndicesService; @@ -72,7 +71,6 @@ import java.util.Locale; import java.util.Map; -import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.generateMapping; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -576,26 +574,22 @@ protected String contentType() { return null; } }; - MappingLookup mappingLookup = MappingLookup.fromMappers( - Mapping.EMPTY, - org.elasticsearch.common.collect.List.of(mockedTimestampField, dateFieldMapper), - org.elasticsearch.common.collect.List.of(), - org.elasticsearch.common.collect.List.of()); ClusterService clusterService = ClusterServiceUtils.createClusterService(testThreadPool); Environment env = mock(Environment.class); when(env.sharedDataFile()).thenReturn(null); AllocationService allocationService = mock(AllocationService.class); when(allocationService.reroute(any(ClusterState.class), any(String.class))).then(i -> i.getArguments()[0]); - DocumentMapper documentMapper = mock(DocumentMapper.class); - when(documentMapper.mappers()).thenReturn(mappingLookup); - when(documentMapper.type()).thenReturn("_doc"); - CompressedXContent mapping = - new CompressedXContent("{\"_doc\":" + generateMapping(dataStream.getTimeStampField().getName(), "date") + "}"); - when(documentMapper.mappingSource()).thenReturn(mapping); - RoutingFieldMapper routingFieldMapper = mock(RoutingFieldMapper.class); - when(routingFieldMapper.required()).thenReturn(false); - when(documentMapper.routingFieldMapper()).thenReturn(routingFieldMapper); - IndicesService indicesService = mockIndicesServices(documentMapper); + MetadataFieldMapper[] metadataFieldMappers = {new MetadataIndexTemplateServiceTests.MetadataTimestampFieldMapper(true)}; + RootObjectMapper.Builder root = new RootObjectMapper.Builder("_doc", Version.CURRENT); + root.add(new DateFieldMapper.Builder(dataStream.getTimeStampField().getName(), DateFieldMapper.Resolution.MILLISECONDS, + DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER, ScriptCompiler.NONE, true, Version.CURRENT)); + Mapping mapping = new Mapping(root.build(new ContentPath("")), metadataFieldMappers, Collections.emptyMap()); + MappingLookup mappingLookup = MappingLookup.fromMappers( + mapping, + org.elasticsearch.common.collect.List.of(mockedTimestampField, dateFieldMapper), + org.elasticsearch.common.collect.List.of(), + org.elasticsearch.common.collect.List.of()); + IndicesService indicesService = mockIndicesServices(mappingLookup); IndexNameExpressionResolver mockIndexNameExpressionResolver = mock(IndexNameExpressionResolver.class); when(mockIndexNameExpressionResolver.resolveDateMathExpression(any())).then(returnsFirstArg()); @@ -682,8 +676,13 @@ protected String contentType() { return null; } }; + MetadataFieldMapper[] metadataFieldMappers = {new MetadataIndexTemplateServiceTests.MetadataTimestampFieldMapper(true)}; + RootObjectMapper.Builder root = new RootObjectMapper.Builder("_doc", Version.CURRENT); + root.add(new DateFieldMapper.Builder(dataStream.getTimeStampField().getName(), DateFieldMapper.Resolution.MILLISECONDS, + DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER, ScriptCompiler.NONE, true, Version.CURRENT)); + Mapping mapping = new Mapping(root.build(new ContentPath("")), metadataFieldMappers, Collections.emptyMap()); MappingLookup mappingLookup = MappingLookup.fromMappers( - Mapping.EMPTY, + mapping, org.elasticsearch.common.collect.List.of(mockedTimestampField, dateFieldMapper), org.elasticsearch.common.collect.List.of(), org.elasticsearch.common.collect.List.of()); @@ -692,16 +691,7 @@ protected String contentType() { when(env.sharedDataFile()).thenReturn(null); AllocationService allocationService = mock(AllocationService.class); when(allocationService.reroute(any(ClusterState.class), any(String.class))).then(i -> i.getArguments()[0]); - DocumentMapper documentMapper = mock(DocumentMapper.class); - when(documentMapper.mappers()).thenReturn(mappingLookup); - when(documentMapper.type()).thenReturn("_doc"); - CompressedXContent mapping = - new CompressedXContent("{\"_doc\":" + generateMapping(dataStream.getTimeStampField().getName(), "date") + "}"); - when(documentMapper.mappingSource()).thenReturn(mapping); - RoutingFieldMapper routingFieldMapper = mock(RoutingFieldMapper.class); - when(routingFieldMapper.required()).thenReturn(false); - when(documentMapper.routingFieldMapper()).thenReturn(routingFieldMapper); - IndicesService indicesService = mockIndicesServices(documentMapper); + IndicesService indicesService = mockIndicesServices(mappingLookup); IndexNameExpressionResolver mockIndexNameExpressionResolver = mock(IndexNameExpressionResolver.class); when(mockIndexNameExpressionResolver.resolveDateMathExpression(any())).then(returnsFirstArg()); @@ -840,7 +830,7 @@ private IndicesService mockIndicesServices() throws Exception { return mockIndicesServices(null); } - private IndicesService mockIndicesServices(DocumentMapper documentMapper) throws Exception { + private IndicesService mockIndicesServices(MappingLookup mappingLookup) throws Exception { /* * Throws Exception because Eclipse uses the lower bound for * CheckedFunction's exception type so it thinks the "when" call @@ -855,7 +845,7 @@ private IndicesService mockIndicesServices(DocumentMapper documentMapper) throws when(indexService.index()).thenReturn(indexMetadata.getIndex()); MapperService mapperService = mock(MapperService.class); when(indexService.mapperService()).thenReturn(mapperService); - when(mapperService.documentMapper()).thenReturn(documentMapper); + when(mapperService.mappingLookup()).thenReturn(mappingLookup); when(indexService.getIndexEventListener()).thenReturn(new IndexEventListener() {}); when(indexService.getIndexSortSupplier()).thenReturn(() -> null); when(indexService.dateMathExpressionResolverAt(anyLong())).thenReturn(s -> s); 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 879593d64e477..e918168a07cb2 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -101,11 +101,11 @@ import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.ContentPath; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.MetadataFieldMapper; +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; @@ -901,7 +901,7 @@ public void testConcurrentGetAndFlush() throws Exception { engine.index(indexForDoc(doc)); final AtomicReference latestGetResult = new AtomicReference<>(); - latestGetResult.set(engine.get(newGet(true, doc), docMapper(doc.type()), randomSearcherWrapper())); + latestGetResult.set(engine.get(newGet(true, doc), mappingLookup(doc.type()), randomSearcherWrapper())); final AtomicBoolean flushFinished = new AtomicBoolean(false); final CyclicBarrier barrier = new CyclicBarrier(2); Thread getThread = new Thread(() -> { @@ -915,7 +915,7 @@ public void testConcurrentGetAndFlush() throws Exception { if (previousGetResult != null) { previousGetResult.close(); } - latestGetResult.set(engine.get(newGet(true, doc), docMapper(doc.type()), randomSearcherWrapper())); + latestGetResult.set(engine.get(newGet(true, doc), mappingLookup(doc.type()), randomSearcherWrapper())); if (latestGetResult.get().exists() == false) { break; } @@ -931,7 +931,7 @@ public void testConcurrentGetAndFlush() throws Exception { } public void testSimpleOperations() throws Exception { - final DocumentMapper mapper = docMapper("test"); + final MappingLookup mappingLookup = mappingLookup("test"); engine.refresh("warm_up"); Engine.Searcher searchResult = engine.acquireSearcher("test"); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); @@ -951,18 +951,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)); } @@ -977,7 +977,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()); } @@ -999,7 +999,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()); } @@ -1028,7 +1028,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)); } @@ -1074,7 +1074,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()); } @@ -1112,9 +1112,10 @@ public void testGetWithSearcherWrapper() throws Exception { final ParsedDocument doc = createParsedDoc("1", null); engine.index(indexForDoc(doc)); final String type = doc.type(); - final DocumentMapper mapper = docMapper(type); + final MappingLookup mappingLookup = mappingLookup(type); assertThat(engine.lastRefreshedCheckpoint(), equalTo(NO_OPS_PERFORMED)); - try (Engine.GetResult get = engine.get(new Engine.Get(true, true, type, "1", newUid("1")), mapper, randomSearcherWrapper())) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, type, "1", newUid("1")), + mappingLookup, randomSearcherWrapper())) { // we do not track the translog location yet assertTrue(get.exists()); assertFalse(get.isFromTranslog()); @@ -1123,7 +1124,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, type, "1", newUid("1")), mapper, searcher -> searcher)) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, type, "1", newUid("1")), mappingLookup, searcher -> searcher)) { assertTrue(get.exists()); assertTrue(get.isFromTranslog()); } @@ -1131,25 +1132,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, type, "1", newUid("1")), mapper, + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, type, "1", newUid("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, type, "1", newUid("1")), mapper, + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, type, "1", newUid("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, type, "1", newUid("1")), mapper, + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, type, "1", newUid("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, type, "1", newUid("1")), mapper, + try (Engine.GetResult get = engine.get(new Engine.Get(true, true, type, "1", newUid("1")), mappingLookup, searcher -> SearcherHelper.wrapSearcher(searcher, reader -> new MatchingDirectoryReader(reader, new TermQuery(newUid("2")))))) { assertFalse(get.exists()); assertFalse(get.isFromTranslog()); @@ -1478,9 +1479,9 @@ 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)); - final DocumentMapper mapper = docMapper(doc.type()); + final MappingLookup mappingLookup = mappingLookup(doc.type()); try (Engine.GetResult get = - engine.get(new Engine.Get(true, false, doc.type(), doc.id(), create.uid()), mapper, randomSearcherWrapper())) { + engine.get(new Engine.Get(true, false, doc.type(), doc.id(), create.uid()), mappingLookup, randomSearcherWrapper())) { assertEquals(1, get.version()); } @@ -1489,7 +1490,7 @@ public void testVersionedUpdate() throws IOException { assertThat(update_1_result.getVersion(), equalTo(2L)); try (Engine.GetResult get = - engine.get(new Engine.Get(true, false, doc.type(), doc.id(), create.uid()), mapper, randomSearcherWrapper())) { + engine.get(new Engine.Get(true, false, doc.type(), doc.id(), create.uid()), mappingLookup, randomSearcherWrapper())) { assertEquals(2, get.version()); } @@ -1498,7 +1499,7 @@ public void testVersionedUpdate() throws IOException { assertThat(update_2_result.getVersion(), equalTo(3L)); try (Engine.GetResult get = - engine.get(new Engine.Get(true, false, doc.type(), doc.id(), create.uid()), mapper, randomSearcherWrapper())) { + engine.get(new Engine.Get(true, false, doc.type(), doc.id(), create.uid()), mappingLookup, randomSearcherWrapper())) { assertEquals(3, get.version()); } @@ -1518,22 +1519,22 @@ public void testGetIfSeqNoIfPrimaryTerm() throws IOException { try (Engine.GetResult get = engine.get( new Engine.Get(true, true, doc.type(), doc.id(), create.uid()) .setIfSeqNo(indexResult.getSeqNo()).setIfPrimaryTerm(primaryTerm.get()), - docMapper(doc.type()), randomSearcherWrapper())) { + mappingLookup(doc.type()), randomSearcherWrapper())) { assertEquals(indexResult.getSeqNo(), get.docIdAndVersion().seqNo); } expectThrows(VersionConflictEngineException.class, () -> engine.get(new Engine.Get(true, false, doc.type(), doc.id(), create.uid()) .setIfSeqNo(indexResult.getSeqNo() + 1).setIfPrimaryTerm(primaryTerm.get()), - docMapper(doc.type()), randomSearcherWrapper())); + mappingLookup(doc.type()), randomSearcherWrapper())); expectThrows(VersionConflictEngineException.class, () -> engine.get(new Engine.Get(true, false, doc.type(), doc.id(), create.uid()) .setIfSeqNo(indexResult.getSeqNo()).setIfPrimaryTerm(primaryTerm.get() + 1), - docMapper(doc.type()), randomSearcherWrapper())); + mappingLookup(doc.type()), randomSearcherWrapper())); final VersionConflictEngineException versionConflictEngineException = expectThrows(VersionConflictEngineException.class, () -> engine.get(new Engine.Get(true, false, doc.type(), doc.id(), create.uid()) .setIfSeqNo(indexResult.getSeqNo() + 1).setIfPrimaryTerm(primaryTerm.get() + 1), - docMapper(doc.type()), randomSearcherWrapper())); + mappingLookup(doc.type()), randomSearcherWrapper())); assertThat(versionConflictEngineException.getStackTrace(), emptyArray()); } @@ -2345,10 +2346,10 @@ class OpAndVersion { } catch (InterruptedException e) { throw new AssertionError(e); } - final DocumentMapper mapper = docMapper(doc.type()); + final MappingLookup mappingLookup = mappingLookup(doc.type()); for (int op = 0; op < opsPerThread; op++) { - try (Engine.GetResult get = - engine.get(new Engine.Get(true, false, doc.type(), doc.id(), uidTerm), mapper, randomSearcherWrapper())) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.type(), doc.id(), uidTerm), + mappingLookup, randomSearcherWrapper())) { FieldsVisitor visitor = new FieldsVisitor(true); get.docIdAndVersion().reader.document(get.docIdAndVersion().docId, visitor); List values = new ArrayList<>(Strings.commaDelimitedListToSet(visitor.source().utf8ToString())); @@ -2390,8 +2391,8 @@ class OpAndVersion { assertTrue(op.added + " should not exist", exists); } - try (Engine.GetResult get = - engine.get(new Engine.Get(true, false, doc.type(), doc.id(), uidTerm), docMapper(doc.type()), randomSearcherWrapper())) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, false, doc.type(), doc.id(), uidTerm), + mappingLookup(doc.type()), randomSearcherWrapper())) { FieldsVisitor visitor = new FieldsVisitor(true); get.docIdAndVersion().reader.document(get.docIdAndVersion().docId, visitor); List values = Arrays.asList(Strings.commaDelimitedListToStringArray(visitor.source().utf8ToString())); @@ -2801,7 +2802,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), docMapper(doc.type()), randomSearcherWrapper()); + Engine.GetResult getResult = engine.get(newGet(true, doc), mappingLookup(doc.type()), randomSearcherWrapper()); assertThat(getResult.exists(), equalTo(false)); // Give the gc pruning logic a chance to kick in @@ -2816,7 +2817,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, "type", "2", newUid("2")), docMapper("type"), randomSearcherWrapper()); + getResult = engine.get(new Engine.Get(true, false, "type", "2", newUid("2")), mappingLookup("type"), randomSearcherWrapper()); assertThat(getResult.exists(), equalTo(false)); // Try to index uid=1 with a too-old version, should fail: @@ -2827,7 +2828,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), docMapper(doc.type()), randomSearcherWrapper()); + getResult = engine.get(newGet(true, doc), mappingLookup(doc.type()), randomSearcherWrapper()); assertThat(getResult.exists(), equalTo(false)); // Try to index uid=2 with a too-old version, should fail: @@ -2838,7 +2839,7 @@ public void testEnableGcDeletes() throws Exception { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // Get should not find the document - getResult = engine.get(newGet(true, doc), docMapper(doc.type()), randomSearcherWrapper()); + getResult = engine.get(newGet(true, doc), mappingLookup(doc.type()), randomSearcherWrapper()); assertThat(getResult.exists(), equalTo(false)); } } @@ -4380,7 +4381,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio assertThat(engine.getProcessedLocalCheckpoint(), equalTo(expectedLocalCheckpoint)); try (Engine.GetResult result = - engine.get(new Engine.Get(true, false, "type", "2", uid), docMapper("type"), randomSearcherWrapper())) { + engine.get(new Engine.Get(true, false, "type", "2", uid), mappingLookup("type"), randomSearcherWrapper())) { assertThat(result.exists(), equalTo(exists)); } } @@ -5323,14 +5324,14 @@ public void testStressUpdateSameDocWhileGettingIt() throws IOException, Interrup Thread thread = new Thread(() -> { awaitStarted.countDown(); try (Engine.GetResult getResult = engine.get(new Engine.Get(true, false, doc3.type(), - doc3.id(), doc3.uid()), docMapper(doc3.type()), randomSearcherWrapper())) { + doc3.id(), doc3.uid()), mappingLookup(doc3.type()), randomSearcherWrapper())) { assertTrue(getResult.exists()); } }); thread.start(); awaitStarted.await(); try (Engine.GetResult getResult = engine.get(new Engine.Get(true, false, doc.type(), - doc.id(), doc.uid()), docMapper(doc.type()), randomSearcherWrapper())) { + doc.id(), doc.uid()), mappingLookup(doc.type()), randomSearcherWrapper())) { assertFalse(getResult.exists()); } thread.join(); @@ -6270,7 +6271,7 @@ public void afterRefresh(boolean didRefresh) { for (int i = 0; i < iters; i++) { ParsedDocument doc = createParsedDoc(randomFrom(ids), null); try (Engine.GetResult getResult = - engine.get(newGet(true, doc), docMapper(doc.type()), randomSearcherWrapper())) { + engine.get(newGet(true, doc), mappingLookup(doc.type()), 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 c331b5a64d044..4e49e208ab3df 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java @@ -104,7 +104,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(get.type()), randomSearcherWrapper())) { + try (Engine.GetResult getResult = readOnlyEngine.get(get, mappingLookup(get.type()), randomSearcherWrapper())) { assertTrue(getResult.exists()); } } 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 5c31cb1e98a40..f9f7047fefafd 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -334,7 +334,7 @@ public void testLotsOfThreads() throws Exception { Engine.Get get = new Engine.Get(false, false, "test", threadId, new Term(IdFieldMapper.NAME, threadId)); try (Engine.GetResult getResult = - engine.get(get, EngineTestCase.docMapper(get.type()), EngineTestCase.randomSearcherWrapper())) { + engine.get(get, EngineTestCase.mappingLookup(get.type()), EngineTestCase.randomSearcherWrapper())) { assertTrue("document not found", getResult.exists()); assertEquals(iteration, getResult.version()); org.apache.lucene.document.Document document = 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 efb066c8bd587..70ec83df71c62 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 @@ -83,6 +83,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; @@ -1056,8 +1057,9 @@ public static List readAllOperationsInLucene(Engine engine, /** * Asserts the provided engine has a consistent document history between translog and Lucene index. */ - public static void assertConsistentHistoryBetweenTranslogAndLuceneIndex(Engine engine, MapperService mapper) throws IOException { - if (mapper == null || mapper.documentMapper() == null || engine.config().getIndexSettings().isSoftDeleteEnabled() == false + public static void assertConsistentHistoryBetweenTranslogAndLuceneIndex(Engine engine, MapperService mapperService) throws IOException { + if (mapperService == null || mapperService.mappingLookup().hasMappings() == false + || engine.config().getIndexSettings().isSoftDeleteEnabled() == false || (engine instanceof InternalEngine) == false) { return; } @@ -1068,7 +1070,7 @@ public static void assertConsistentHistoryBetweenTranslogAndLuceneIndex(Engine e translogOps.add(op); } } - final Map luceneOps = readAllOperationsInLucene(engine, mapper).stream() + final Map luceneOps = readAllOperationsInLucene(engine, mapperService).stream() .collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); final long maxSeqNo = ((InternalEngine) engine).getLocalCheckpointTracker().getMaxSeqNo(); for (Translog.Operation op : translogOps) { @@ -1164,9 +1166,9 @@ public static MapperService createMapperService(String type) throws IOException return mapperService; } - public static DocumentMapper docMapper(String type) { + public static MappingLookup mappingLookup(String type) { try { - return createMapperService(type).documentMapper(); + return createMapperService(type).mappingLookup(); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/mapper/MetadataCreateDataStreamServiceTests.java b/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/mapper/MetadataCreateDataStreamServiceTests.java index 949b29b85a6ae..d5bc279170cc9 100644 --- a/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/mapper/MetadataCreateDataStreamServiceTests.java +++ b/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/mapper/MetadataCreateDataStreamServiceTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.MapperTestUtils; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MappingLookup; import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.datastreams.DataStreamsPlugin; @@ -27,16 +28,13 @@ public class MetadataCreateDataStreamServiceTests extends ESTestCase { public void testValidateTimestampFieldMapping() throws Exception { String mapping = generateMapping("@timestamp", "date"); - validateTimestampFieldMapping("@timestamp", createMapperService(mapping)); + validateTimestampFieldMapping(createMappingLookup(mapping)); mapping = generateMapping("@timestamp", "date_nanos"); - validateTimestampFieldMapping("@timestamp", createMapperService(mapping)); + validateTimestampFieldMapping(createMappingLookup(mapping)); } public void testValidateTimestampFieldMappingNoFieldMapping() { - Exception e = expectThrows( - IllegalStateException.class, - () -> validateTimestampFieldMapping("@timestamp", createMapperService("{}")) - ); + Exception e = expectThrows(IllegalStateException.class, () -> validateTimestampFieldMapping(createMappingLookup("{}"))); assertThat(e.getMessage(), equalTo("[_data_stream_timestamp] meta field has been disabled")); String mapping1 = "{\n" + " \"_data_stream_timestamp\": {\n" @@ -48,27 +46,24 @@ public void testValidateTimestampFieldMappingNoFieldMapping() { + " }\n" + " }\n" + " }"; - e = expectThrows(IllegalStateException.class, () -> validateTimestampFieldMapping("@timestamp", createMapperService(mapping1))); + e = expectThrows(IllegalStateException.class, () -> validateTimestampFieldMapping(createMappingLookup(mapping1))); assertThat(e.getMessage(), equalTo("[_data_stream_timestamp] meta field has been disabled")); String mapping2 = generateMapping("@timestamp2", "date"); - e = expectThrows(IllegalArgumentException.class, () -> validateTimestampFieldMapping("@timestamp", createMapperService(mapping2))); + e = expectThrows(IllegalArgumentException.class, () -> validateTimestampFieldMapping(createMappingLookup(mapping2))); assertThat(e.getMessage(), equalTo("data stream timestamp field [@timestamp] does not exist")); } public void testValidateTimestampFieldMappingInvalidFieldType() { String mapping = generateMapping("@timestamp", "keyword"); - Exception e = expectThrows( - IllegalArgumentException.class, - () -> validateTimestampFieldMapping("@timestamp", createMapperService(mapping)) - ); + Exception e = expectThrows(IllegalArgumentException.class, () -> validateTimestampFieldMapping(createMappingLookup(mapping))); assertThat( e.getMessage(), equalTo("data stream timestamp field [@timestamp] is of type [keyword], " + "but [date,date_nanos] is expected") ); } - MapperService createMapperService(String mapping) throws IOException { + MappingLookup createMappingLookup(String mapping) throws IOException { String indexName = "test"; IndexMetadata indexMetadata = IndexMetadata.builder(indexName) .settings( @@ -88,7 +83,6 @@ MapperService createMapperService(String mapping) throws IOException { indexName ); mapperService.merge(indexMetadata, MapperService.MergeReason.MAPPING_UPDATE); - return mapperService; + return mapperService.mappingLookup(); } - }