diff --git a/core/src/main/java/org/elasticsearch/action/bulk/MappingUpdatePerformer.java b/core/src/main/java/org/elasticsearch/action/bulk/MappingUpdatePerformer.java index a5238273ccbb8..812653d58266b 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/MappingUpdatePerformer.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/MappingUpdatePerformer.java @@ -19,14 +19,9 @@ package org.elasticsearch.action.bulk; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.Mapping; -import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; -import java.util.Objects; - public interface MappingUpdatePerformer { /** @@ -39,6 +34,6 @@ public interface MappingUpdatePerformer { * retried on the primary due to the mappings not being present yet, or a different exception if * updating the mappings on the master failed. */ - void verifyMappings(Engine.Index operation, ShardId shardId) throws Exception; + void verifyMappings(Mapping update, ShardId shardId) throws Exception; } diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 21dd799122e39..4e1419099b0bf 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -56,6 +56,7 @@ import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.mapper.MapperParsingException; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.seqno.SequenceNumbersService; @@ -150,8 +151,9 @@ private static BulkItemResultHolder executeIndexRequest(final IndexRequest index private static BulkItemResultHolder executeDeleteRequest(final DeleteRequest deleteRequest, final BulkItemRequest bulkItemRequest, - final IndexShard primary) throws IOException { - Engine.DeleteResult deleteResult = executeDeleteRequestOnPrimary(deleteRequest, primary); + final IndexShard primary, + final MappingUpdatePerformer mappingUpdater) throws Exception { + Engine.DeleteResult deleteResult = executeDeleteRequestOnPrimary(deleteRequest, primary, mappingUpdater); if (deleteResult.hasFailure()) { return new BulkItemResultHolder(null, deleteResult, bulkItemRequest); } else { @@ -241,7 +243,7 @@ static Translog.Location executeBulkItemRequest(IndexMetaData metaData, IndexSha requestIndex, updateHelper, nowInMillisSupplier, mappingUpdater); break; case DELETE: - responseHolder = executeDeleteRequest((DeleteRequest) itemRequest, request.items()[requestIndex], primary); + responseHolder = executeDeleteRequest((DeleteRequest) itemRequest, request.items()[requestIndex], primary, mappingUpdater); break; default: throw new IllegalStateException("unexpected opType [" + itemRequest.opType() + "] found"); } @@ -303,7 +305,7 @@ private static BulkItemResultHolder executeUpdateRequest(UpdateRequest updateReq break; case DELETED: DeleteRequest deleteRequest = translate.action(); - result = executeDeleteRequestOnPrimary(deleteRequest, primary); + result = executeDeleteRequestOnPrimary(deleteRequest, primary, mappingUpdater); break; case NOOP: primary.noopUpdate(updateRequest.type()); @@ -609,7 +611,7 @@ static Engine.IndexResult executeIndexRequestOnPrimary(IndexRequest request, Ind if (mappingUpdateNeeded) { try { operation = prepareIndexOperationOnPrimary(request, primary); - mappingUpdater.verifyMappings(operation, primary.shardId()); + mappingUpdater.verifyMappings(operation.parsedDoc().dynamicMappingsUpdate(), primary.shardId()); } catch (MapperParsingException | IllegalStateException e) { // there was an error in parsing the document that was not because // of pending mapping updates, so return a failure for the result @@ -623,12 +625,52 @@ static Engine.IndexResult executeIndexRequestOnPrimary(IndexRequest request, Ind return primary.index(operation); } - private static Engine.DeleteResult executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard primary) throws IOException { + private static Engine.DeleteResult executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard primary, + final MappingUpdatePerformer mappingUpdater) throws Exception { + boolean mappingUpdateNeeded = false; + if (primary.indexSettings().isSingleType()) { + // When there is a single type, the unique identifier is only composed of the _id, + // so there is no way to differenciate foo#1 from bar#1. This is especially an issue + // if a user first deletes foo#1 and then indexes bar#1: since we do not encode the + // _type in the uid it might look like we are reindexing the same document, which + // would fail if bar#1 is indexed with a lower version than foo#1 was deleted with. + // In order to work around this issue, we make deletions create types. This way, we + // fail if index and delete operations do not use the same type. + try { + Mapping update = primary.mapperService().documentMapperWithAutoCreate(request.type()).getMapping(); + if (update != null) { + mappingUpdateNeeded = true; + mappingUpdater.updateMappings(update, primary.shardId(), request.type()); + } + } catch (MapperParsingException | IllegalArgumentException e) { + return new Engine.DeleteResult(e, request.version(), SequenceNumbersService.UNASSIGNED_SEQ_NO, false); + } + } + if (mappingUpdateNeeded) { + Mapping update = primary.mapperService().documentMapperWithAutoCreate(request.type()).getMapping(); + mappingUpdater.verifyMappings(update, primary.shardId()); + } final Engine.Delete delete = primary.prepareDeleteOnPrimary(request.type(), request.id(), request.version(), request.versionType()); return primary.delete(delete); } - private static Engine.DeleteResult executeDeleteRequestOnReplica(DocWriteResponse primaryResponse, DeleteRequest request, IndexShard replica) throws IOException { + private static Engine.DeleteResult executeDeleteRequestOnReplica(DocWriteResponse primaryResponse, DeleteRequest request, + IndexShard replica) throws Exception { + if (replica.indexSettings().isSingleType()) { + // We need to wait for the replica to have the mappings + Mapping update; + try { + update = replica.mapperService().documentMapperWithAutoCreate(request.type()).getMapping(); + } catch (MapperParsingException | IllegalArgumentException e) { + return new Engine.DeleteResult(e, request.version(), primaryResponse.getSeqNo(), false); + } + if (update != null) { + final ShardId shardId = replica.shardId(); + throw new RetryOnReplicaException(shardId, + "Mappings are not available on the replica yet, triggered update: " + update); + } + } + final VersionType versionType = request.versionType().versionTypeForReplicationAndRecovery(); final long version = primaryResponse.getVersion(); assert versionType.validateVersionForWrites(version); @@ -654,9 +696,9 @@ public void updateMappings(final Mapping update, final ShardId shardId, } } - public void verifyMappings(final Engine.Index operation, + public void verifyMappings(Mapping update, final ShardId shardId) throws Exception { - if (operation.parsedDoc().dynamicMappingsUpdate() != null) { + if (update != null) { throw new ReplicationOperation.RetryOnPrimaryException(shardId, "Dynamic mappings are not available on the node that holds the primary yet"); } diff --git a/core/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java b/core/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java index 65176c1df392c..72aaeb9eb371a 100644 --- a/core/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java +++ b/core/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java @@ -35,8 +35,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.get.GetResult; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.internal.AliasFilter; @@ -93,10 +91,13 @@ protected ExplainResponse shardOperation(ExplainRequest request, ShardId shardId ShardSearchLocalRequest shardSearchLocalRequest = new ShardSearchLocalRequest(shardId, new String[]{request.type()}, request.nowInMillis, request.filteringAlias()); SearchContext context = searchService.createSearchContext(shardSearchLocalRequest, SearchService.NO_TIMEOUT, null); - Term uidTerm = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(request.type(), request.id())); Engine.GetResult result = null; try { - result = context.indexShard().get(new Engine.Get(false, uidTerm)); + Term uidTerm = context.mapperService().createUidTerm(request.type(), request.id()); + if (uidTerm == null) { + return new ExplainResponse(shardId.getIndexName(), request.type(), request.id(), false); + } + result = context.indexShard().get(new Engine.Get(false, request.type(), request.id(), uidTerm)); if (!result.exists()) { return new ExplainResponse(shardId.getIndexName(), request.type(), request.id(), false); } diff --git a/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java b/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java index 96c9f30a9546f..e8b47783afb1d 100644 --- a/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java +++ b/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.SequenceNumbersService; @@ -51,6 +50,7 @@ final class PerThreadIDVersionAndSeqNoLookup { // we keep it around for now, to reduce the amount of e.g. hash lookups by field and stuff /** terms enum for uid field */ + final String uidField; private final TermsEnum termsEnum; /** Reused for iteration (when the term exists) */ @@ -62,13 +62,14 @@ final class PerThreadIDVersionAndSeqNoLookup { /** * Initialize lookup for the provided segment */ - PerThreadIDVersionAndSeqNoLookup(LeafReader reader) throws IOException { + PerThreadIDVersionAndSeqNoLookup(LeafReader reader, String uidField) throws IOException { + this.uidField = uidField; Fields fields = reader.fields(); - Terms terms = fields.terms(UidFieldMapper.NAME); - termsEnum = terms.iterator(); - if (termsEnum == null) { - throw new IllegalArgumentException("reader misses the [" + UidFieldMapper.NAME + "] field"); + Terms terms = fields.terms(uidField); + if (terms == null) { + throw new IllegalArgumentException("reader misses the [" + uidField + "] field"); } + termsEnum = terms.iterator(); if (reader.getNumericDocValues(VersionFieldMapper.NAME) == null) { throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field"); } diff --git a/core/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java b/core/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java index 409ce8dec29be..3cdbfa38b62e9 100644 --- a/core/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java +++ b/core/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java @@ -25,10 +25,10 @@ import org.apache.lucene.index.Term; import org.apache.lucene.util.CloseableThreadLocal; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.index.mapper.UidFieldMapper; import java.io.IOException; import java.util.List; +import java.util.Objects; import java.util.concurrent.ConcurrentMap; import static org.elasticsearch.common.lucene.uid.Versions.NOT_FOUND; @@ -47,7 +47,7 @@ public final class VersionsAndSeqNoResolver { } }; - private static PerThreadIDVersionAndSeqNoLookup getLookupState(LeafReader reader) throws IOException { + private static PerThreadIDVersionAndSeqNoLookup getLookupState(LeafReader reader, String uidField) throws IOException { IndexReader.CacheHelper cacheHelper = reader.getCoreCacheHelper(); CloseableThreadLocal ctl = lookupStates.get(cacheHelper.getKey()); if (ctl == null) { @@ -65,8 +65,11 @@ private static PerThreadIDVersionAndSeqNoLookup getLookupState(LeafReader reader PerThreadIDVersionAndSeqNoLookup lookupState = ctl.get(); if (lookupState == null) { - lookupState = new PerThreadIDVersionAndSeqNoLookup(reader); + lookupState = new PerThreadIDVersionAndSeqNoLookup(reader, uidField); ctl.set(lookupState); + } else if (Objects.equals(lookupState.uidField, uidField) == false) { + throw new AssertionError("Index does not consistently use the same uid field: [" + + uidField + "] != [" + lookupState.uidField + "]"); } return lookupState; @@ -109,7 +112,6 @@ public static class DocIdAndSeqNo { * */ public static DocIdAndVersion loadDocIdAndVersion(IndexReader reader, Term term) throws IOException { - assert term.field().equals(UidFieldMapper.NAME) : "unexpected term field " + term.field(); List leaves = reader.leaves(); if (leaves.isEmpty()) { return null; @@ -119,7 +121,7 @@ public static DocIdAndVersion loadDocIdAndVersion(IndexReader reader, Term term) for (int i = leaves.size() - 1; i >= 0; i--) { LeafReaderContext context = leaves.get(i); LeafReader leaf = context.reader(); - PerThreadIDVersionAndSeqNoLookup lookup = getLookupState(leaf); + PerThreadIDVersionAndSeqNoLookup lookup = getLookupState(leaf, term.field()); DocIdAndVersion result = lookup.lookupVersion(term.bytes(), leaf.getLiveDocs(), context); if (result != null) { return result; @@ -135,7 +137,6 @@ public static DocIdAndVersion loadDocIdAndVersion(IndexReader reader, Term term) * */ public static DocIdAndSeqNo loadDocIdAndSeqNo(IndexReader reader, Term term) throws IOException { - assert term.field().equals(UidFieldMapper.NAME) : "unexpected term field " + term.field(); List leaves = reader.leaves(); if (leaves.isEmpty()) { return null; @@ -145,7 +146,7 @@ public static DocIdAndSeqNo loadDocIdAndSeqNo(IndexReader reader, Term term) thr for (int i = leaves.size() - 1; i >= 0; i--) { LeafReaderContext context = leaves.get(i); LeafReader leaf = context.reader(); - PerThreadIDVersionAndSeqNoLookup lookup = getLookupState(leaf); + PerThreadIDVersionAndSeqNoLookup lookup = getLookupState(leaf, term.field()); DocIdAndSeqNo result = lookup.lookupSeqNo(term.bytes(), leaf.getLiveDocs(), context); if (result != null) { return result; @@ -157,9 +158,9 @@ public static DocIdAndSeqNo loadDocIdAndSeqNo(IndexReader reader, Term term) thr /** * Load the primaryTerm associated with the given {@link DocIdAndSeqNo} */ - public static long loadPrimaryTerm(DocIdAndSeqNo docIdAndSeqNo) throws IOException { + public static long loadPrimaryTerm(DocIdAndSeqNo docIdAndSeqNo, String uidField) throws IOException { LeafReader leaf = docIdAndSeqNo.context.reader(); - PerThreadIDVersionAndSeqNoLookup lookup = getLookupState(leaf); + PerThreadIDVersionAndSeqNoLookup lookup = getLookupState(leaf, uidField); long result = lookup.lookUpPrimaryTerm(docIdAndSeqNo.docId, leaf); assert result > 0 : "should always resolve a primary term for a resolved sequence number. primary_term [" + result + "]" + " docId [" + docIdAndSeqNo.docId + "] seqNo [" + docIdAndSeqNo.seqNo + "]"; diff --git a/core/src/main/java/org/elasticsearch/index/IndexSettings.java b/core/src/main/java/org/elasticsearch/index/IndexSettings.java index 8acdf7d1360cb..05b8f509eae25 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/core/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -31,12 +31,12 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.mapper.AllFieldMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.node.Node; import java.util.Locale; import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; import java.util.function.Function; /** @@ -192,7 +192,10 @@ public final class IndexSettings { * The maximum number of slices allowed in a scroll request. */ private volatile int maxSlicesPerScroll; - + /** + * Whether the index is required to have at most one type. + */ + private final boolean singleType; /** * Returns the default search field for this index. @@ -280,6 +283,7 @@ public IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSetti maxSlicesPerScroll = scopedSettings.get(MAX_SLICES_PER_SCROLL); this.mergePolicyConfig = new MergePolicyConfig(logger, this); this.indexSortConfig = new IndexSortConfig(this); + singleType = scopedSettings.get(MapperService.INDEX_MAPPING_SINGLE_TYPE_SETTING); scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_COMPOUND_FORMAT_SETTING, mergePolicyConfig::setNoCFSRatio); scopedSettings.addSettingsUpdateConsumer(MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED_SETTING, mergePolicyConfig::setExpungeDeletesAllowed); @@ -391,6 +395,11 @@ public IndexMetaData getIndexMetaData() { */ public int getNumberOfReplicas() { return settings.getAsInt(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, null); } + /** + * Returns whether the index enforces at most one type. + */ + public boolean isSingleType() { return singleType; } + /** * Returns the node settings. The settings returned from {@link #getSettings()} are a merged version of the * index settings and the node settings where node settings are overwritten by index settings. diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 31e0d1e422acf..295f559da1698 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -473,8 +473,7 @@ protected final GetResult getFromSearcher(Get get, Function se if (docIdAndVersion != null) { if (get.versionType().isVersionConflictForReads(docIdAndVersion.version, get.version())) { Releasables.close(searcher); - Uid uid = Uid.createUid(get.uid().text()); - throw new VersionConflictEngineException(shardId, uid.type(), uid.id(), + throw new VersionConflictEngineException(shardId, get.type(), get.id(), get.versionType().explainConflictForReads(docIdAndVersion.version, get.version())); } } @@ -1028,7 +1027,6 @@ public static class Index extends Operation { public Index(Term uid, ParsedDocument doc, long seqNo, long primaryTerm, long version, VersionType versionType, Origin origin, long startTime, long autoGeneratedIdTimestamp, boolean isRetry) { super(uid, seqNo, primaryTerm, version, versionType, origin, startTime); - assert uid.bytes().equals(doc.uid()) : "term uid " + uid + " doesn't match doc uid " + doc.uid(); this.doc = doc; this.isRetry = isRetry; this.autoGeneratedIdTimestamp = autoGeneratedIdTimestamp; @@ -1199,11 +1197,14 @@ public int estimatedSizeInBytes() { public static class Get { private final boolean realtime; private final Term uid; + private final String type, id; private long version = Versions.MATCH_ANY; private VersionType versionType = VersionType.INTERNAL; - public Get(boolean realtime, Term uid) { + public Get(boolean realtime, String type, String id, Term uid) { this.realtime = realtime; + this.type = type; + this.id = id; this.uid = uid; } @@ -1211,6 +1212,14 @@ public boolean realtime() { return this.realtime; } + public String type() { + return type; + } + + public String id() { + return id; + } + public Term uid() { return uid; } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 9bfbb1467a20f..03007d96b9f23 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -60,8 +60,9 @@ import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.SeqNoStats; @@ -80,6 +81,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -122,6 +124,8 @@ public class InternalEngine extends Engine { private final SequenceNumbersService seqNoService; + private final String uidField; + // How many callers are currently requesting index throttling. Currently there are only two situations where we do this: when merges // are falling behind and when writing indexing buffer to disk is too slow. When this is 0, there is no throttling, else we throttling // incoming indexing ops to a single thread: @@ -139,6 +143,7 @@ public InternalEngine(EngineConfig engineConfig) throws EngineException { if (engineConfig.isAutoGeneratedIDsOptimizationEnabled() == false) { maxUnsafeAutoIdTimestamp.set(Long.MAX_VALUE); } + this.uidField = engineConfig.getIndexSettings().isSingleType() ? IdFieldMapper.NAME : UidFieldMapper.NAME; this.versionMap = new LiveVersionMap(); store.incRef(); IndexWriter writer = null; @@ -404,6 +409,7 @@ private SearcherManager createSearcherManager() throws EngineException { @Override public GetResult get(Get get, Function searcherFactory) throws EngineException { + assert Objects.equals(get.uid().field(), uidField) : get.uid().field(); try (ReleasableLock ignored = readLock.acquire()) { ensureOpen(); if (get.realtime()) { @@ -413,8 +419,7 @@ public GetResult get(Get get, Function searcherFactory) throws return GetResult.NOT_EXISTS; } if (get.versionType().isVersionConflictForReads(versionValue.version, get.version())) { - Uid uid = Uid.createUid(get.uid().text()); - throw new VersionConflictEngineException(shardId, uid.type(), uid.id(), + throw new VersionConflictEngineException(shardId, get.type(), get.id(), get.versionType().explainConflictForReads(versionValue.version, get.version())); } refresh("realtime_get"); @@ -462,7 +467,7 @@ private OpVsLuceneDocStatus compareOpToLuceneDocBasedOnSeqNo(final Operation op) status = OpVsLuceneDocStatus.OP_NEWER; } else if (op.seqNo() == docAndSeqNo.seqNo) { // load term to tie break - final long existingTerm = VersionsAndSeqNoResolver.loadPrimaryTerm(docAndSeqNo); + final long existingTerm = VersionsAndSeqNoResolver.loadPrimaryTerm(docAndSeqNo, op.uid().field()); if (op.primaryTerm() > existingTerm) { status = OpVsLuceneDocStatus.OP_NEWER; } else { @@ -571,6 +576,7 @@ private boolean assertSequenceNumberBeforeIndexing(final Engine.Operation.Origin @Override public IndexResult index(Index index) throws IOException { + assert Objects.equals(index.uid().field(), uidField) : index.uid().field(); final boolean doThrottle = index.origin().isRecovery() == false; try (ReleasableLock releasableLock = readLock.acquire()) { ensureOpen(); @@ -891,6 +897,7 @@ private static void update(final Term uid, final List doc @Override public DeleteResult delete(Delete delete) throws IOException { + assert Objects.equals(delete.uid().field(), uidField) : delete.uid().field(); assert assertVersionType(delete); assert assertIncomingSequenceNumber(delete.origin(), delete.seqNo()); final DeleteResult deleteResult; diff --git a/core/src/main/java/org/elasticsearch/index/fielddata/UidIndexFieldData.java b/core/src/main/java/org/elasticsearch/index/fielddata/UidIndexFieldData.java new file mode 100644 index 0000000000000..e8dea836e322c --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/fielddata/UidIndexFieldData.java @@ -0,0 +1,186 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.fielddata; + +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.search.SortField; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested; +import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource; +import org.elasticsearch.index.fielddata.plain.AbstractAtomicOrdinalsFieldData; +import org.elasticsearch.index.mapper.UidFieldMapper; +import org.elasticsearch.search.MultiValueMode; + +import java.io.IOException; + +/** Fielddata view of the _uid field on indices that do not index _uid but _id. + * It gets fielddata on the {@code _id field}, which is in-memory since the _id + * field does not have doc values, and prepends {@code ${type}#} to all values. + * Note that it does not add memory compared to what fielddata on the _id is + * already using: this is just a view. + * TODO: Remove fielddata access on _uid and _id, or add doc values to _id. + */ +public final class UidIndexFieldData implements IndexOrdinalsFieldData { + + private final Index index; + private final String type; + private final BytesRef prefix; + private final IndexOrdinalsFieldData idFieldData; + + public UidIndexFieldData(Index index, String type, IndexOrdinalsFieldData idFieldData) { + this.index = index; + this.type = type; + BytesRefBuilder prefix = new BytesRefBuilder(); + prefix.append(new BytesRef(type)); + prefix.append((byte) '#'); + this.prefix = prefix.toBytesRef(); + this.idFieldData = idFieldData; + } + + @Override + public Index index() { + return index; + } + + @Override + public String getFieldName() { + return UidFieldMapper.NAME; + } + + @Override + public SortField sortField(Object missingValue, MultiValueMode sortMode, Nested nested, boolean reverse) { + XFieldComparatorSource source = new BytesRefFieldComparatorSource(this, missingValue, sortMode, nested); + return new SortField(getFieldName(), source, reverse); + } + + @Override + public AtomicOrdinalsFieldData load(LeafReaderContext context) { + return new UidAtomicFieldData(prefix, idFieldData.load(context)); + } + + @Override + public AtomicOrdinalsFieldData loadDirect(LeafReaderContext context) throws Exception { + return new UidAtomicFieldData(prefix, idFieldData.loadDirect(context)); + } + + @Override + public void clear() { + idFieldData.clear(); + } + + @Override + public IndexOrdinalsFieldData loadGlobal(DirectoryReader indexReader) { + return new UidIndexFieldData(index, type, idFieldData.loadGlobal(indexReader)); + } + + @Override + public IndexOrdinalsFieldData localGlobalDirect(DirectoryReader indexReader) throws Exception { + return new UidIndexFieldData(index, type, idFieldData.localGlobalDirect(indexReader)); + } + + static final class UidAtomicFieldData implements AtomicOrdinalsFieldData { + + private final BytesRef prefix; + private final AtomicOrdinalsFieldData idFieldData; + + UidAtomicFieldData(BytesRef prefix, AtomicOrdinalsFieldData idFieldData) { + this.prefix = prefix; + this.idFieldData = idFieldData; + } + + @Override + public ScriptDocValues getScriptValues() { + return AbstractAtomicOrdinalsFieldData.DEFAULT_SCRIPT_FUNCTION.apply(getOrdinalsValues()); + } + + @Override + public SortedBinaryDocValues getBytesValues() { + return FieldData.toString(getOrdinalsValues()); + } + + @Override + public long ramBytesUsed() { + return 0; // simple wrapper + } + + @Override + public void close() { + idFieldData.close(); + } + + @Override + public SortedSetDocValues getOrdinalsValues() { + SortedSetDocValues idValues = idFieldData.getOrdinalsValues(); + return new SortedSetDocValues() { + + private final BytesRefBuilder scratch = new BytesRefBuilder(); + + @Override + public int nextDoc() throws IOException { + return idValues.nextDoc(); + } + + @Override + public int docID() { + return idValues.docID(); + } + + @Override + public long cost() { + return idValues.cost(); + } + + @Override + public int advance(int target) throws IOException { + return idValues.advance(target); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return idValues.advanceExact(target); + } + + @Override + public long nextOrd() throws IOException { + return idValues.nextOrd(); + } + + @Override + public BytesRef lookupOrd(long ord) throws IOException { + scratch.setLength(0); + scratch.append(prefix); + scratch.append(idValues.lookupOrd(ord)); + return scratch.get(); + } + + @Override + public long getValueCount() { + return idValues.getValueCount(); + } + }; + } + + } + +} diff --git a/core/src/main/java/org/elasticsearch/index/fielddata/plain/AbstractIndexOrdinalsFieldData.java b/core/src/main/java/org/elasticsearch/index/fielddata/plain/AbstractIndexOrdinalsFieldData.java index fef4b3b0a7d70..ce05e226b7e13 100644 --- a/core/src/main/java/org/elasticsearch/index/fielddata/plain/AbstractIndexOrdinalsFieldData.java +++ b/core/src/main/java/org/elasticsearch/index/fielddata/plain/AbstractIndexOrdinalsFieldData.java @@ -26,16 +26,12 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.Nullable; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.fielddata.AtomicOrdinalsFieldData; -import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested; import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData; -import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource; import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsBuilder; import org.elasticsearch.indices.breaker.CircuitBreakerService; -import org.elasticsearch.search.MultiValueMode; import java.io.IOException; diff --git a/core/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java b/core/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java index 372e7caf921d5..8c49185197f24 100644 --- a/core/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java +++ b/core/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java @@ -23,6 +23,7 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParentFieldMapper; @@ -34,6 +35,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -50,13 +52,14 @@ public class FieldsVisitor extends StoredFieldVisitor { private static final Set BASE_REQUIRED_FIELDS = unmodifiableSet(newHashSet( UidFieldMapper.NAME, + IdFieldMapper.NAME, RoutingFieldMapper.NAME, ParentFieldMapper.NAME)); private final boolean loadSource; private final Set requiredFields; protected BytesReference source; - protected Uid uid; + protected String type, id; protected Map> fieldsValues; public FieldsVisitor(boolean loadSource) { @@ -78,6 +81,13 @@ public Status needsField(FieldInfo fieldInfo) throws IOException { } public void postProcess(MapperService mapperService) { + if (mapperService.getIndexSettings().isSingleType()) { + final Collection types = mapperService.types(); + assert types.size() <= 1 : types; + if (types.isEmpty() == false) { + type = types.iterator().next(); + } + } for (Map.Entry> entry : fields().entrySet()) { MappedFieldType fieldType = mapperService.fullName(entry.getKey()); if (fieldType == null) { @@ -104,7 +114,11 @@ public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { public void stringField(FieldInfo fieldInfo, byte[] bytes) throws IOException { final String value = new String(bytes, StandardCharsets.UTF_8); if (UidFieldMapper.NAME.equals(fieldInfo.name)) { - uid = Uid.createUid(value); + Uid uid = Uid.createUid(value); + type = uid.type(); + id = uid.id(); + } else if (IdFieldMapper.NAME.equals(fieldInfo.name)) { + id = value; } else { addValue(fieldInfo.name, value); } @@ -135,7 +149,12 @@ public BytesReference source() { } public Uid uid() { - return uid; + if (id == null) { + return null; + } else if (type == null) { + throw new IllegalStateException("Call postProcess before getting the uid"); + } + return new Uid(type, id); } public String routing() { @@ -157,7 +176,8 @@ public Map> fields() { public void reset() { if (fieldsValues != null) fieldsValues.clear(); source = null; - uid = null; + type = null; + id = null; requiredFields.addAll(BASE_REQUIRED_FIELDS); if (loadSource) { diff --git a/core/src/main/java/org/elasticsearch/index/fieldvisitor/JustUidFieldsVisitor.java b/core/src/main/java/org/elasticsearch/index/fieldvisitor/JustUidFieldsVisitor.java deleted file mode 100644 index 661d729b55f79..0000000000000 --- a/core/src/main/java/org/elasticsearch/index/fieldvisitor/JustUidFieldsVisitor.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.index.fieldvisitor; - -import org.apache.lucene.index.FieldInfo; -import org.elasticsearch.index.mapper.UidFieldMapper; - -import java.io.IOException; - -public class JustUidFieldsVisitor extends FieldsVisitor { - - public JustUidFieldsVisitor() { - super(false); - } - - @Override - public Status needsField(FieldInfo fieldInfo) throws IOException { - if (UidFieldMapper.NAME.equals(fieldInfo.name)) { - return Status.YES; - } - return uid != null ? Status.STOP : Status.NO; - } -} diff --git a/core/src/main/java/org/elasticsearch/index/fieldvisitor/SingleFieldsVisitor.java b/core/src/main/java/org/elasticsearch/index/fieldvisitor/SingleFieldsVisitor.java index 556f43d06a1c6..5d5040c637f04 100644 --- a/core/src/main/java/org/elasticsearch/index/fieldvisitor/SingleFieldsVisitor.java +++ b/core/src/main/java/org/elasticsearch/index/fieldvisitor/SingleFieldsVisitor.java @@ -20,12 +20,12 @@ import org.apache.lucene.index.FieldInfo; import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.TypeFieldMapper; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.UidFieldMapper; import java.io.IOException; -import java.util.List; public class SingleFieldsVisitor extends FieldsVisitor { @@ -54,30 +54,17 @@ public void reset(String field) { super.reset(); } - public void postProcess(MappedFieldType fieldType) { - if (uid != null) { - switch (field) { - case UidFieldMapper.NAME: - addValue(field, uid.toString()); - break; - case IdFieldMapper.NAME: - addValue(field, uid.id()); - break; - case TypeFieldMapper.NAME: - addValue(field, uid.type()); - break; - } - } - - if (fieldsValues == null) { - return; + @Override + public void postProcess(MapperService mapperService) { + super.postProcess(mapperService); + if (id != null) { + addValue(IdFieldMapper.NAME, id); } - List fieldValues = fieldsValues.get(fieldType.name()); - if (fieldValues == null) { - return; + if (type != null) { + addValue(TypeFieldMapper.NAME, type); } - for (int i = 0; i < fieldValues.size(); i++) { - fieldValues.set(i, fieldType.valueForDisplay(fieldValues.get(i))); + if (type != null && id != null) { + addValue(UidFieldMapper.NAME, Uid.createUid(type, id)); } } } diff --git a/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java b/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java index 6d3e1e3ab6a56..15e6e23428465 100644 --- a/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java +++ b/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java @@ -42,14 +42,13 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParentFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.search.fetch.subphase.ParentFieldSubFetchPhase; import java.io.IOException; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -139,11 +138,18 @@ private FetchSourceContext normalizeFetchSourceContent(@Nullable FetchSourceCont private GetResult innerGet(String type, String id, String[] gFields, boolean realtime, long version, VersionType versionType, FetchSourceContext fetchSourceContext) { fetchSourceContext = normalizeFetchSourceContent(fetchSourceContext, gFields); + final Collection types; + if (type == null || type.equals("_all")) { + types = mapperService.types(); + } else { + types = Collections.singleton(type); + } Engine.GetResult get = null; - if (type == null || type.equals("_all")) { - for (String typeX : mapperService.types()) { - get = indexShard.get(new Engine.Get(realtime, new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(typeX, id))) + for (String typeX : types) { + Term uidTerm = mapperService.createUidTerm(typeX, id); + if (uidTerm != null) { + get = indexShard.get(new Engine.Get(realtime, typeX, id, uidTerm) .version(version).versionType(versionType)); if (get.exists()) { type = typeX; @@ -152,20 +158,10 @@ private GetResult innerGet(String type, String id, String[] gFields, boolean rea get.release(); } } - if (get == null) { - return new GetResult(shardId.getIndexName(), type, id, -1, false, null, null); - } - if (!get.exists()) { - // no need to release here as well..., we release in the for loop for non exists - return new GetResult(shardId.getIndexName(), type, id, -1, false, null, null); - } - } else { - get = indexShard.get(new Engine.Get(realtime, new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(type, id))) - .version(version).versionType(versionType)); - if (!get.exists()) { - get.release(); - return new GetResult(shardId.getIndexName(), type, id, -1, false, null, null); - } + } + + if (get == null || get.exists() == false) { + return new GetResult(shardId.getIndexName(), type, id, -1, false, null, null); } try { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/IdFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/IdFieldMapper.java index 8e18c820b7965..a9a765f1c3a0e 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/IdFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/IdFieldMapper.java @@ -22,26 +22,18 @@ import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.TermInSetQuery; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.MultiTermQuery; -import org.apache.lucene.search.PrefixQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.RegexpQuery; -import org.apache.lucene.util.BytesRef; +import org.apache.lucene.search.TermInSetQuery; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData; import org.elasticsearch.index.query.QueryShardContext; import java.io.IOException; -import java.util.Collection; +import java.util.Arrays; import java.util.List; import java.util.Map; @@ -63,8 +55,8 @@ public static class Defaults { static { FIELD_TYPE.setTokenized(false); - FIELD_TYPE.setIndexOptions(IndexOptions.NONE); - FIELD_TYPE.setStored(false); + FIELD_TYPE.setIndexOptions(IndexOptions.DOCS); + FIELD_TYPE.setStored(true); FIELD_TYPE.setOmitNorms(true); FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER); FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER); @@ -81,7 +73,7 @@ public MetadataFieldMapper.Builder parse(String name, Map node, @Override public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) { - final Settings indexSettings = context.mapperService().getIndexSettings().getSettings(); + final IndexSettings indexSettings = context.mapperService().getIndexSettings(); return new IdFieldMapper(indexSettings, fieldType); } } @@ -113,32 +105,66 @@ public boolean isSearchable() { @Override public Query termQuery(Object value, @Nullable QueryShardContext context) { - final BytesRef[] uids = Uid.createUidsForTypesAndId(context.queryTypes(), value); - return new TermInSetQuery(UidFieldMapper.NAME, uids); + return termsQuery(Arrays.asList(value), context); } @Override - public Query termsQuery(List values, @Nullable QueryShardContext context) { + public Query termsQuery(List values, @Nullable QueryShardContext context) { + if (indexOptions() != IndexOptions.NONE) { + // 6.x index, _id is indexed + return super.termsQuery(values, context); + } + // 5.x index, _uid is indexed return new TermInSetQuery(UidFieldMapper.NAME, Uid.createUidsForTypesAndIds(context.queryTypes(), values)); } + + @Override + public IndexFieldData.Builder fielddataBuilder() { + if (indexOptions() == IndexOptions.NONE) { + throw new IllegalArgumentException("Fielddata access on the _uid field is disallowed"); + } + return new PagedBytesIndexFieldData.Builder( + TextFieldMapper.Defaults.FIELDDATA_MIN_FREQUENCY, + TextFieldMapper.Defaults.FIELDDATA_MAX_FREQUENCY, + TextFieldMapper.Defaults.FIELDDATA_MIN_SEGMENT_SIZE); + } } - private IdFieldMapper(Settings indexSettings, MappedFieldType existing) { - this(existing != null ? existing : Defaults.FIELD_TYPE, indexSettings); + static MappedFieldType defaultFieldType(IndexSettings indexSettings) { + MappedFieldType defaultFieldType = Defaults.FIELD_TYPE.clone(); + if (indexSettings.isSingleType()) { + defaultFieldType.setIndexOptions(IndexOptions.DOCS); + defaultFieldType.setStored(true); + } else { + defaultFieldType.setIndexOptions(IndexOptions.NONE); + defaultFieldType.setStored(false); + } + return defaultFieldType; } - private IdFieldMapper(MappedFieldType fieldType, Settings indexSettings) { - super(NAME, fieldType, Defaults.FIELD_TYPE, indexSettings); + private IdFieldMapper(IndexSettings indexSettings, MappedFieldType existing) { + this(existing == null ? defaultFieldType(indexSettings) : existing, indexSettings); + } + + private IdFieldMapper(MappedFieldType fieldType, IndexSettings indexSettings) { + super(NAME, fieldType, defaultFieldType(indexSettings), indexSettings.getSettings()); } @Override - public void preParse(ParseContext context) throws IOException {} + public void preParse(ParseContext context) throws IOException { + super.parse(context); + } @Override public void postParse(ParseContext context) throws IOException {} @Override - protected void parseCreateField(ParseContext context, List fields) throws IOException {} + protected void parseCreateField(ParseContext context, List fields) throws IOException { + if (fieldType.indexOptions() != IndexOptions.NONE || fieldType.stored()) { + Field id = new Field(NAME, context.sourceToParse().id(), fieldType); + fields.add(id); + } + } @Override protected String contentType() { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java b/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java index 55c2e4cb3c698..9e8b2d9018f25 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java @@ -24,15 +24,19 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.PrefixCodedTerms; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; +import org.apache.lucene.index.PrefixCodedTerms.TermIterator; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.MultiTermQuery; import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermInSetQuery; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.joda.DateMathParser; @@ -473,6 +477,15 @@ public static Term extractTerm(Query termQuery) { assert ((TypeFieldMapper.TypesQuery) termQuery).getTerms().length == 1; return new Term(TypeFieldMapper.NAME, ((TypeFieldMapper.TypesQuery) termQuery).getTerms()[0]); } + if (termQuery instanceof TermInSetQuery) { + TermInSetQuery tisQuery = (TermInSetQuery) termQuery; + PrefixCodedTerms terms = tisQuery.getTermData(); + if (terms.size() == 1) { + TermIterator it = terms.iterator(); + BytesRef term = it.next(); + return new Term(it.field(), term); + } + } if (termQuery instanceof TermQuery == false) { throw new IllegalArgumentException("Cannot extract a term from a query of type " + termQuery.getClass() + ": " + termQuery); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java index bd65d123f81d6..4d68deef3b7c0 100755 --- a/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -24,6 +24,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.DelegatingAnalyzerWrapper; +import org.apache.lucene.index.Term; import org.elasticsearch.ElasticsearchGenerationException; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -470,7 +471,7 @@ private synchronized Map internalMerge(@Nullable Documen } } - if (indexSettings.getValue(INDEX_MAPPING_SINGLE_TYPE_SETTING)) { + if (indexSettings.isSingleType()) { Set actualTypes = new HashSet<>(mappers.keySet()); actualTypes.remove(DEFAULT_MAPPING); if (actualTypes.size() > 1) { @@ -803,4 +804,15 @@ protected Analyzer getWrappedAnalyzer(String fieldName) { } } + /** Return a term that uniquely identifies the document, or {@code null} if the type is not allowed. */ + public Term createUidTerm(String type, String id) { + if (hasMapping(type) == false) { + return null; + } + if (indexSettings.isSingleType()) { + return new Term(IdFieldMapper.NAME, id); + } else { + return new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(type, id)); + } + } } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java b/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java index 91cf2aa4fa4a8..14b3291f441a9 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -70,10 +70,6 @@ public ParsedDocument(Field version, this.xContentType = xContentType; } - public BytesRef uid() { - return uid; - } - public String id() { return this.id; } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/TypeFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/TypeFieldMapper.java index 9d4a4a6987ba3..2092e2521d38e 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/TypeFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/TypeFieldMapper.java @@ -149,7 +149,7 @@ public Query termQuery(Object value, QueryShardContext context) { @Override public Query termsQuery(List values, QueryShardContext context) { - if (context.getIndexSettings().getValue(MapperService.INDEX_MAPPING_SINGLE_TYPE_SETTING)) { + if (context.getIndexSettings().isSingleType()) { Collection indexTypes = context.getMapperService().types(); if (indexTypes.isEmpty()) { return new MatchNoDocsQuery("No types"); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/UidFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/UidFieldMapper.java index a2d76dcf2c091..ba06361502353 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/UidFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/UidFieldMapper.java @@ -19,18 +19,33 @@ package org.elasticsearch.index.mapper; -import org.apache.lucene.document.BinaryDocValuesField; import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermInSetQuery; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.StringHelper; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.logging.DeprecationLogger; +import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; +import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData; +import org.elasticsearch.index.fielddata.UidIndexFieldData; import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.indices.breaker.CircuitBreakerService; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; @@ -62,6 +77,8 @@ public static class Defaults { } } + private static final DeprecationLogger DEPRECATION_LOGGER = new DeprecationLogger(Loggers.getLogger(UidFieldMapper.class)); + public static class TypeParser implements MetadataFieldMapper.TypeParser { @Override public MetadataFieldMapper.Builder parse(String name, Map node, ParserContext parserContext) throws MapperParsingException { @@ -70,7 +87,7 @@ public static class TypeParser implements MetadataFieldMapper.TypeParser { @Override public MetadataFieldMapper getDefault(MappedFieldType fieldType, ParserContext context) { - final Settings indexSettings = context.mapperService().getIndexSettings().getSettings(); + final IndexSettings indexSettings = context.mapperService().getIndexSettings(); return new UidFieldMapper(indexSettings, fieldType); } } @@ -96,20 +113,81 @@ public String typeName() { @Override public IndexFieldData.Builder fielddataBuilder() { - // TODO: add doc values support? - return new PagedBytesIndexFieldData.Builder( - TextFieldMapper.Defaults.FIELDDATA_MIN_FREQUENCY, - TextFieldMapper.Defaults.FIELDDATA_MAX_FREQUENCY, - TextFieldMapper.Defaults.FIELDDATA_MIN_SEGMENT_SIZE); + if (indexOptions() == IndexOptions.NONE) { + DEPRECATION_LOGGER.deprecated("Fielddata access on the _uid field is deprecated, use _id instead"); + return new IndexFieldData.Builder() { + @Override + public IndexFieldData build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache, + CircuitBreakerService breakerService, MapperService mapperService) { + MappedFieldType idFieldType = mapperService.fullName(IdFieldMapper.NAME); + IndexOrdinalsFieldData idFieldData = (IndexOrdinalsFieldData) idFieldType.fielddataBuilder() + .build(indexSettings, idFieldType, cache, breakerService, mapperService); + final String type = mapperService.types().iterator().next(); + return new UidIndexFieldData(indexSettings.getIndex(), type, idFieldData); + } + }; + } else { + // Old index, _uid was indexed + return new PagedBytesIndexFieldData.Builder( + TextFieldMapper.Defaults.FIELDDATA_MIN_FREQUENCY, + TextFieldMapper.Defaults.FIELDDATA_MAX_FREQUENCY, + TextFieldMapper.Defaults.FIELDDATA_MIN_SEGMENT_SIZE); + } + } + + @Override + public Query termQuery(Object value, @Nullable QueryShardContext context) { + return termsQuery(Arrays.asList(value), context); + } + + @Override + public Query termsQuery(List values, @Nullable QueryShardContext context) { + if (indexOptions() != IndexOptions.NONE) { + return super.termsQuery(values, context); + } + Collection indexTypes = context.getMapperService().types(); + if (indexTypes.isEmpty()) { + return new MatchNoDocsQuery("No types"); + } + assert indexTypes.size() == 1; + BytesRef indexType = indexedValueForSearch(indexTypes.iterator().next()); + BytesRefBuilder prefixBuilder = new BytesRefBuilder(); + prefixBuilder.append(indexType); + prefixBuilder.append((byte) '#'); + BytesRef expectedPrefix = prefixBuilder.get(); + List ids = new ArrayList<>(); + for (Object uid : values) { + BytesRef uidBytes = indexedValueForSearch(uid); + if (StringHelper.startsWith(uidBytes, expectedPrefix)) { + BytesRef id = new BytesRef(); + id.bytes = uidBytes.bytes; + id.offset = uidBytes.offset + expectedPrefix.length; + id.length = uidBytes.length - expectedPrefix.length; + ids.add(id); + } + } + return new TermInSetQuery(IdFieldMapper.NAME, ids); + } + } + + static MappedFieldType defaultFieldType(IndexSettings indexSettings) { + MappedFieldType defaultFieldType = Defaults.FIELD_TYPE.clone(); + if (indexSettings.isSingleType()) { + defaultFieldType.setIndexOptions(IndexOptions.NONE); + defaultFieldType.setStored(false); + } else { + defaultFieldType.setIndexOptions(IndexOptions.DOCS); + defaultFieldType.setStored(true); } + return defaultFieldType; } - private UidFieldMapper(Settings indexSettings, MappedFieldType existing) { - this(existing == null ? Defaults.FIELD_TYPE.clone() : existing, Defaults.FIELD_TYPE, indexSettings); + private UidFieldMapper(IndexSettings indexSettings, MappedFieldType existing) { + this(existing == null ? defaultFieldType(indexSettings) : existing, indexSettings); } - private UidFieldMapper(MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings) { - super(NAME, fieldType, defaultFieldType, indexSettings); + private UidFieldMapper(MappedFieldType fieldType, IndexSettings indexSettings) { + super(NAME, fieldType, defaultFieldType(indexSettings), indexSettings.getSettings()); } @Override @@ -128,10 +206,9 @@ public Mapper parse(ParseContext context) throws IOException { @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { - Field uid = new Field(NAME, Uid.createUid(context.sourceToParse().type(), context.sourceToParse().id()), Defaults.FIELD_TYPE); - fields.add(uid); - if (fieldType().hasDocValues()) { - fields.add(new BinaryDocValuesField(NAME, new BytesRef(uid.stringValue()))); + if (fieldType.indexOptions() != IndexOptions.NONE || fieldType.stored()) { + Field uid = new Field(NAME, Uid.createUid(context.sourceToParse().type(), context.sourceToParse().id()), fieldType); + fields.add(uid); } } diff --git a/core/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java index 5857ef9abf373..1b756062dc9bd 100644 --- a/core/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java @@ -19,8 +19,8 @@ package org.elasticsearch.index.query; +import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.TermInSetQuery; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; @@ -30,6 +30,7 @@ import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.UidFieldMapper; @@ -162,6 +163,10 @@ public String getWriteableName() { @Override protected Query doToQuery(QueryShardContext context) throws IOException { Query query; + MappedFieldType uidField = context.fieldMapper(UidFieldMapper.NAME); + if (uidField == null) { + return new MatchNoDocsQuery("No mappings"); + } if (this.ids.isEmpty()) { query = Queries.newMatchNoDocsQuery("Missing ids in \"" + this.getName() + "\" query."); } else { @@ -175,7 +180,7 @@ protected Query doToQuery(QueryShardContext context) throws IOException { Collections.addAll(typesForQuery, types); } - query = new TermInSetQuery(UidFieldMapper.NAME, Uid.createUidsForTypesAndIds(typesForQuery, ids)); + query = uidField.termsQuery(Arrays.asList(Uid.createUidsForTypesAndIds(typesForQuery, ids)), context); } return query; } diff --git a/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java index 5df7ace69bb39..fd83465b7e2f7 100644 --- a/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java @@ -24,7 +24,6 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.TermInSetQuery; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ExceptionsHelper; @@ -1102,7 +1101,7 @@ private Query handleItems(QueryShardContext context, MoreLikeThisQuery mltQuery, // exclude the items from the search if (!include) { - handleExclude(boolQuery, likeItems); + handleExclude(boolQuery, likeItems, context); } return boolQuery.build(); } @@ -1155,7 +1154,12 @@ private static Fields[] getFieldsFor(MultiTermVectorsResponse responses) throws return likeFields.toArray(Fields.EMPTY_ARRAY); } - private static void handleExclude(BooleanQuery.Builder boolQuery, Item[] likeItems) { + private static void handleExclude(BooleanQuery.Builder boolQuery, Item[] likeItems, QueryShardContext context) { + MappedFieldType uidField = context.fieldMapper(UidFieldMapper.NAME); + if (uidField == null) { + // no mappings, nothing to exclude + return; + } // artificial docs get assigned a random id and should be disregarded List uids = new ArrayList<>(); for (Item item : likeItems) { @@ -1165,7 +1169,7 @@ private static void handleExclude(BooleanQuery.Builder boolQuery, Item[] likeIte uids.add(createUidAsBytes(item.type(), item.id())); } if (!uids.isEmpty()) { - TermInSetQuery query = new TermInSetQuery(UidFieldMapper.NAME, uids.toArray(new BytesRef[uids.size()])); + Query query = uidField.termsQuery(uids, context); boolQuery.add(query, BooleanClause.Occur.MUST_NOT); } } diff --git a/core/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java b/core/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java index ddf796503b705..f2c92cc7d1ded 100644 --- a/core/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java @@ -26,10 +26,12 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryShardContext; -import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.Objects; @@ -126,7 +128,12 @@ protected int doHashCode() { @Override protected ScoreFunction doToFunction(QueryShardContext context) { - final MappedFieldType fieldType = context.getMapperService().fullName("_uid"); + final MappedFieldType fieldType; + if (context.getIndexSettings().isSingleType()) { + fieldType = context.getMapperService().fullName(IdFieldMapper.NAME); + } else { + fieldType = context.getMapperService().fullName(UidFieldMapper.NAME); + } if (fieldType == null) { // mapper could be null if we are on a shard with no docs yet, so this won't actually be used return new RandomScoreFunction(); diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 45e7bd3d1e9ff..e08011d8f296f 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -21,16 +21,13 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CheckIndex; -import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexFormatTooNewException; -import org.apache.lucene.index.IndexFormatTooOldException; +import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SnapshotDeletionPolicy; import org.apache.lucene.index.Term; -import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.Sort; import org.apache.lucene.search.UsageTrackingQueryCachingPolicy; @@ -91,19 +88,18 @@ import org.elasticsearch.index.flush.FlushStats; import org.elasticsearch.index.get.GetStats; import org.elasticsearch.index.get.ShardGetService; -import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperForType; -import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.recovery.RecoveryStats; import org.elasticsearch.index.refresh.RefreshStats; import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.ShardSearchStats; -import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.similarity.SimilarityService; @@ -128,12 +124,10 @@ import org.elasticsearch.search.suggest.completion.CompletionStats; import org.elasticsearch.threadpool.ThreadPool; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintStream; import java.nio.channels.ClosedByInterruptException; import java.nio.charset.StandardCharsets; -import java.nio.file.NoSuchFileException; import java.util.ArrayList; import java.util.EnumSet; import java.util.List; @@ -550,9 +544,12 @@ static Engine.Index prepareIndex(DocumentMapperForType docMapper, SourceToParse if (docMapper.getMapping() != null) { doc.addDynamicMappingsUpdate(docMapper.getMapping()); } - MappedFieldType uidFieldType = docMapper.getDocumentMapper().uidMapper().fieldType(); - Query uidQuery = uidFieldType.termQuery(doc.uid(), null); - Term uid = MappedFieldType.extractTerm(uidQuery); + Term uid; + if (docMapper.getDocumentMapper().idFieldMapper().fieldType().indexOptions() != IndexOptions.NONE) { + uid = new Term(IdFieldMapper.NAME, doc.id()); + } else { + uid = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(doc.type(), doc.id())); + } return new Engine.Index(uid, doc, seqNo, primaryTerm, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry); } @@ -594,7 +591,7 @@ public Engine.NoOpResult markSeqNoAsNoOp(Engine.NoOp noOp) throws IOException { public Engine.Delete prepareDeleteOnPrimary(String type, String id, long version, VersionType versionType) { verifyPrimary(); - final Term uid = extractUid(type, id); + final Term uid = extractUidForDelete(type, id); return prepareDelete(type, id, uid, SequenceNumbersService.UNASSIGNED_SEQ_NO, primaryTerm, version, versionType, Engine.Operation.Origin.PRIMARY); } @@ -602,7 +599,7 @@ public Engine.Delete prepareDeleteOnPrimary(String type, String id, long version public Engine.Delete prepareDeleteOnReplica(String type, String id, long seqNo, long primaryTerm, long version, VersionType versionType) { verifyReplicationTarget(); - final Term uid = extractUid(type, id); + final Term uid = extractUidForDelete(type, id); return prepareDelete(type, id, uid, seqNo, primaryTerm, version, versionType, Engine.Operation.Origin.REPLICA); } @@ -618,11 +615,14 @@ public Engine.DeleteResult delete(Engine.Delete delete) throws IOException { return delete(engine, delete); } - private Term extractUid(String type, String id) { - final DocumentMapper documentMapper = docMapper(type).getDocumentMapper(); - final MappedFieldType uidFieldType = documentMapper.uidMapper().fieldType(); - final Query uidQuery = uidFieldType.termQuery(Uid.createUid(type, id), null); - return MappedFieldType.extractTerm(uidQuery); + private Term extractUidForDelete(String type, String id) { + if (indexSettings.isSingleType()) { + // This is only correct because we create types dynamically on delete operations + // otherwise this could match the same _id from a different type + return new Term(IdFieldMapper.NAME, id); + } else { + return new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(type, id)); + } } private Engine.DeleteResult delete(Engine engine, Engine.Delete delete) throws IOException { @@ -907,12 +907,13 @@ public void releaseIndexCommit(IndexCommit snapshot) throws IOException { * without having to worry about the current state of the engine and concurrent flushes. * * @throws org.apache.lucene.index.IndexNotFoundException if no index is found in the current directory - * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an - * unexpected exception when opening the index reading the segments file. - * @throws IndexFormatTooOldException if the lucene index is too old to be opened. - * @throws IndexFormatTooNewException if the lucene index is too new to be opened. - * @throws FileNotFoundException if one or more files referenced by a commit are not present. - * @throws NoSuchFileException if one or more files referenced by a commit are not present. + * @throws org.apache.lucene.index.CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum + * mismatch or an unexpected exception when opening the index reading the + * segments file. + * @throws org.apache.lucene.index.IndexFormatTooOldException if the lucene index is too old to be opened. + * @throws org.apache.lucene.index.IndexFormatTooNewException if the lucene index is too new to be opened. + * @throws java.io.FileNotFoundException if one or more files referenced by a commit are not present. + * @throws java.nio.file.NoSuchFileException if one or more files referenced by a commit are not present. */ public Store.MetadataSnapshot snapshotStoreMetadata() throws IOException { IndexCommit indexCommit = null; @@ -1453,7 +1454,8 @@ public void writeIndexingBuffer() { /** * Notifies the service to update the local checkpoint for the shard with the provided allocation ID. See - * {@link GlobalCheckpointTracker#updateLocalCheckpoint(String, long)} for details. + * {@link org.elasticsearch.index.seqno.GlobalCheckpointTracker#updateLocalCheckpoint(String, long)} for + * details. * * @param allocationId the allocation ID of the shard to update the local checkpoint for * @param checkpoint the local checkpoint for the shard @@ -1475,7 +1477,8 @@ public void waitForOpsToComplete(final long seqNo) throws InterruptedException { /** * Marks the shard with the provided allocation ID as in-sync with the primary shard. See - * {@link GlobalCheckpointTracker#markAllocationIdAsInSync(String, long)} for additional details. + * {@link org.elasticsearch.index.seqno.GlobalCheckpointTracker#markAllocationIdAsInSync(String, long)} + * for additional details. * * @param allocationId the allocation ID of the shard to mark as in-sync * @param localCheckpoint the current local checkpoint on the shard @@ -1543,7 +1546,8 @@ public void updateGlobalCheckpointOnReplica(final long globalCheckpoint) { /** * Notifies the service of the current allocation IDs in the cluster state. See - * {@link GlobalCheckpointTracker#updateAllocationIdsFromMaster(Set, Set)} for details. + * {@link org.elasticsearch.index.seqno.GlobalCheckpointTracker#updateAllocationIdsFromMaster(Set, Set)} + * for details. * * @param activeAllocationIds the allocation IDs of the currently active shard copies * @param initializingAllocationIds the allocation IDs of the currently initializing shard copies diff --git a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java index 8842cbf3c0bd4..f4f611b733b5a 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java +++ b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java @@ -166,9 +166,8 @@ private void performRecoveryOperation(Engine engine, Translog.Operation operatio break; case DELETE: Translog.Delete delete = (Translog.Delete) operation; - Uid uid = Uid.createUid(delete.uid().text()); - logger.trace("[translog] recover [delete] op [({}, {})] of [{}][{}]", delete.seqNo(), delete.primaryTerm(), uid.type(), uid.id()); - final Engine.Delete engineDelete = new Engine.Delete(uid.type(), uid.id(), delete.uid(), delete.seqNo(), + logger.trace("[translog] recover [delete] op [({}, {})] of [{}][{}]", delete.seqNo(), delete.primaryTerm(), delete.type(), delete.id()); + final Engine.Delete engineDelete = new Engine.Delete(delete.type(), delete.id(), delete.uid(), delete.seqNo(), delete.primaryTerm(), delete.version(), delete.versionType().versionTypeForReplicationAndRecovery(), origin, System.nanoTime()); delete(engine, engineDelete); diff --git a/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java b/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java index 6351282a38a18..8291e76c2acac 100644 --- a/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java +++ b/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java @@ -49,8 +49,6 @@ import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.TextFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.search.dfs.AggregatedDfs; @@ -82,9 +80,13 @@ public static TermVectorsResponse getTermVectors(IndexShard indexShard, TermVect static TermVectorsResponse getTermVectors(IndexShard indexShard, TermVectorsRequest request, LongSupplier nanoTimeSupplier) { final long startTime = nanoTimeSupplier.getAsLong(); final TermVectorsResponse termVectorsResponse = new TermVectorsResponse(indexShard.shardId().getIndex().getName(), request.type(), request.id()); - final Term uidTerm = new Term(UidFieldMapper.NAME, Uid.createUidAsBytes(request.type(), request.id())); - - Engine.GetResult get = indexShard.get(new Engine.Get(request.realtime(), uidTerm).version(request.version()).versionType(request.versionType())); + final Term uidTerm = indexShard.mapperService().createUidTerm(request.type(), request.id()); + if (uidTerm == null) { + termVectorsResponse.setExists(false); + return termVectorsResponse; + } + Engine.GetResult get = indexShard.get(new Engine.Get(request.realtime(), request.type(), request.id(), uidTerm) + .version(request.version()).versionType(request.versionType())); Fields termVectorsByField = null; AggregatedDfs dfs = null; diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index a5e6255ecb6dd..d6457e1d32af7 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -22,7 +22,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; import org.apache.lucene.index.Term; -import org.apache.lucene.index.TwoPhaseCommit; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.IOUtils; import org.elasticsearch.action.index.IndexRequest; @@ -1052,6 +1051,7 @@ public static class Delete implements Operation { private static final int FORMAT_SEQ_NO = FORMAT_5_X + 1; public static final int SERIALIZATION_FORMAT = FORMAT_SEQ_NO; + private String type, id; private Term uid; private long seqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; private long primaryTerm = 0; @@ -1072,15 +1072,17 @@ public Delete(StreamInput in) throws IOException { } public Delete(Engine.Delete delete, Engine.DeleteResult deleteResult) { - this(delete.uid(), deleteResult.getSeqNo(), delete.primaryTerm(), deleteResult.getVersion(), delete.versionType()); + this(delete.type(), delete.id(), delete.uid(), deleteResult.getSeqNo(), delete.primaryTerm(), deleteResult.getVersion(), delete.versionType()); } /** utility for testing */ - public Delete(Term uid) { - this(uid, 0, 0, Versions.MATCH_ANY, VersionType.INTERNAL); + public Delete(String type, String id, Term uid) { + this(type, id, uid, 0, 0, Versions.MATCH_ANY, VersionType.INTERNAL); } - public Delete(Term uid, long seqNo, long primaryTerm, long version, VersionType versionType) { + public Delete(String type, String id, Term uid, long seqNo, long primaryTerm, long version, VersionType versionType) { + this.type = type; + this.id = id; this.uid = uid; this.seqNo = seqNo; this.primaryTerm = primaryTerm; @@ -1098,6 +1100,14 @@ public long estimateSize() { return ((uid.field().length() + uid.text().length()) * 2) + 20; } + public String type() { + return type; + } + + public String id() { + return id; + } + public Term uid() { return this.uid; } diff --git a/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsContext.java b/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsContext.java index e84fcc1f51dc8..c1155bdfbd6bf 100644 --- a/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsContext.java +++ b/core/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsContext.java @@ -23,6 +23,7 @@ import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.DocValuesTermsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; @@ -31,15 +32,12 @@ import org.apache.lucene.search.TopScoreDocCollector; import org.apache.lucene.search.join.BitSetProducer; import org.apache.lucene.search.join.ParentChildrenBlockJoinQuery; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ObjectMapper; import org.elasticsearch.index.mapper.ParentFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.fetch.FetchSubPhase; @@ -170,7 +168,12 @@ public TopDocs topDocs(SearchContext context, FetchSubPhase.HitContext hitContex if (parentField == null) { throw new IllegalStateException("All children must have a _parent"); } - hitQuery = new TermQuery(new Term(UidFieldMapper.NAME, Uid.createUid(parentType, parentField.getValue()))); + Term uidTerm = context.mapperService().createUidTerm(parentType, parentField.getValue()); + if (uidTerm == null) { + hitQuery = new MatchNoDocsQuery("Missing type: " + parentType); + } else { + hitQuery = new TermQuery(uidTerm); + } } else { return Lucene.EMPTY_TOP_DOCS; } diff --git a/core/src/main/java/org/elasticsearch/search/lookup/LeafFieldsLookup.java b/core/src/main/java/org/elasticsearch/search/lookup/LeafFieldsLookup.java index 2d1c855b7c328..4a104797d7c1b 100644 --- a/core/src/main/java/org/elasticsearch/search/lookup/LeafFieldsLookup.java +++ b/core/src/main/java/org/elasticsearch/search/lookup/LeafFieldsLookup.java @@ -22,13 +22,16 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.Nullable; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.UidFieldMapper; import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; @@ -37,6 +40,7 @@ public class LeafFieldsLookup implements Map { private final MapperService mapperService; + private final boolean singleType; @Nullable private final String[] types; @@ -51,6 +55,7 @@ public class LeafFieldsLookup implements Map { LeafFieldsLookup(MapperService mapperService, @Nullable String[] types, LeafReader reader) { this.mapperService = mapperService; + this.singleType = mapperService.getIndexSettings().isSingleType(); this.types = types; this.reader = reader; this.fieldVisitor = new SingleFieldsVisitor(null); @@ -142,11 +147,15 @@ private FieldLookup loadFieldData(String name) { } if (data.fields() == null) { String fieldName = data.fieldType().name(); + if (singleType && UidFieldMapper.NAME.equals(fieldName)) { + fieldName = IdFieldMapper.NAME; + } fieldVisitor.reset(fieldName); try { reader.document(docId, fieldVisitor); - fieldVisitor.postProcess(data.fieldType()); - data.fields(singletonMap(name, fieldVisitor.fields().get(data.fieldType().name()))); + fieldVisitor.postProcess(mapperService); + List storedFields = fieldVisitor.fields().get(data.fieldType().name()); + data.fields(singletonMap(name, storedFields)); } catch (IOException e) { throw new ElasticsearchParseException("failed to load field [{}]", e, name); } diff --git a/core/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java b/core/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java index 4fcf3a94f98d2..522ecb44d09bd 100644 --- a/core/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java @@ -32,7 +32,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexNumericFieldData; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryShardContext; @@ -194,9 +196,14 @@ public Query toFilter(QueryShardContext context, int shardId, int numShards) { throw new IllegalArgumentException("field " + field + " not found"); } + String field = this.field; boolean useTermQuery = false; if (UidFieldMapper.NAME.equals(field)) { - useTermQuery = true; + if (context.getIndexSettings().isSingleType()) { + // on new indices, the _id acts as a _uid + field = IdFieldMapper.NAME; + } + useTermQuery = true; } else if (type.hasDocValues() == false) { throw new IllegalArgumentException("cannot load numeric doc values on " + field); } else { diff --git a/core/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java b/core/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java index 941cdbf995752..20406acc9b6e7 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java @@ -571,7 +571,7 @@ public void updateMappings(Mapping update, ShardId shardId, } @Override - public void verifyMappings(Engine.Index operation, + public void verifyMappings(Mapping update, ShardId shardId) throws Exception { // No-op, will be called logger.info("--> verifying mappings noop"); @@ -600,7 +600,7 @@ public void updateMappings(Mapping update, ShardId shardId, } @Override - public void verifyMappings(Engine.Index operation, + public void verifyMappings(Mapping update, ShardId shardId) throws Exception { fail("should not have had to update the mappings"); } @@ -668,7 +668,7 @@ public static class NoopMappingUpdatePerformer implements MappingUpdatePerformer public void updateMappings(Mapping update, ShardId shardId, String type) throws Exception { } - public void verifyMappings(Engine.Index operation, ShardId shardId) throws Exception { + public void verifyMappings(Mapping update, ShardId shardId) throws Exception { } } @@ -683,7 +683,7 @@ public void updateMappings(Mapping update, ShardId shardId, String type) throws throw e; } - public void verifyMappings(Engine.Index operation, ShardId shardId) throws Exception { + public void verifyMappings(Mapping update, ShardId shardId) throws Exception { fail("should not have gotten to this point"); } } @@ -698,7 +698,7 @@ private class ThrowingVerifyingMappingUpdatePerformer implements MappingUpdatePe public void updateMappings(Mapping update, ShardId shardId, String type) throws Exception { } - public void verifyMappings(Engine.Index operation, ShardId shardId) throws Exception { + public void verifyMappings(Mapping update, ShardId shardId) throws Exception { throw e; } } diff --git a/core/src/test/java/org/elasticsearch/action/bulk/byscroll/BulkByScrollParallelizationHelperTests.java b/core/src/test/java/org/elasticsearch/action/bulk/byscroll/BulkByScrollParallelizationHelperTests.java index 5a12d54de04ca..498c6bf5286f9 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/byscroll/BulkByScrollParallelizationHelperTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/byscroll/BulkByScrollParallelizationHelperTests.java @@ -20,7 +20,7 @@ package org.elasticsearch.action.bulk.byscroll; import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.index.mapper.UidFieldMapper; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.test.ESTestCase; @@ -44,7 +44,7 @@ public void testSliceIntoSubRequests() throws IOException { searchRequest.source().slice(null); } int times = between(2, 100); - String field = randomBoolean() ? UidFieldMapper.NAME : randomAlphaOfLength(5); + String field = randomBoolean() ? IdFieldMapper.NAME : randomAlphaOfLength(5); int currentSliceId = 0; for (SearchRequest slice : sliceIntoSubRequests(searchRequest, field, times)) { assertEquals(field, slice.source().slice().getField()); diff --git a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java index fcec3d6b59445..711804153cf1e 100644 --- a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java +++ b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java @@ -188,7 +188,7 @@ public void testSearchingFilteringAliasesSingleIndex() throws Exception { assertHits(searchResponse.getHits(), "1", "2", "3"); logger.info("--> checking single filtering alias search with sort"); - searchResponse = client().prepareSearch("tests").setQuery(QueryBuilders.matchAllQuery()).addSort("_uid", SortOrder.ASC).get(); + searchResponse = client().prepareSearch("tests").setQuery(QueryBuilders.matchAllQuery()).addSort("_index", SortOrder.ASC).get(); assertHits(searchResponse.getHits(), "1", "2", "3"); logger.info("--> checking single filtering alias search with global facets"); @@ -203,7 +203,7 @@ public void testSearchingFilteringAliasesSingleIndex() throws Exception { logger.info("--> checking single filtering alias search with global facets and sort"); searchResponse = client().prepareSearch("tests").setQuery(QueryBuilders.matchQuery("name", "bar")) .addAggregation(AggregationBuilders.global("global").subAggregation(AggregationBuilders.terms("test").field("name"))) - .addSort("_uid", SortOrder.ASC).get(); + .addSort("_index", SortOrder.ASC).get(); assertSearchResponse(searchResponse); global = searchResponse.getAggregations().get("global"); terms = global.getAggregations().get("test"); @@ -212,7 +212,7 @@ public void testSearchingFilteringAliasesSingleIndex() throws Exception { logger.info("--> checking single filtering alias search with non-global facets"); searchResponse = client().prepareSearch("tests").setQuery(QueryBuilders.matchQuery("name", "bar")) .addAggregation(AggregationBuilders.terms("test").field("name")) - .addSort("_uid", SortOrder.ASC).get(); + .addSort("_index", SortOrder.ASC).get(); assertSearchResponse(searchResponse); terms = searchResponse.getAggregations().get("test"); assertThat(terms.getBuckets().size(), equalTo(2)); diff --git a/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java b/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java index 8b68e76957058..e8b5220396daa 100644 --- a/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java +++ b/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java @@ -32,7 +32,7 @@ import org.apache.lucene.util.FixedBitSet; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; -import org.elasticsearch.index.mapper.UidFieldMapper; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.test.ESTestCase; @@ -48,12 +48,12 @@ public void testSimple() throws Exception { Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); Document doc = new Document(); - doc.add(new Field(UidFieldMapper.NAME, "6", UidFieldMapper.Defaults.FIELD_TYPE)); + doc.add(new Field(IdFieldMapper.NAME, "6", IdFieldMapper.Defaults.FIELD_TYPE)); doc.add(new NumericDocValuesField(VersionFieldMapper.NAME, 87)); writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); LeafReaderContext segment = reader.leaves().get(0); - PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader()); + PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME); // found doc DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), null, segment); assertNotNull(result); @@ -75,13 +75,13 @@ public void testTwoDocuments() throws Exception { Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); Document doc = new Document(); - doc.add(new Field(UidFieldMapper.NAME, "6", UidFieldMapper.Defaults.FIELD_TYPE)); + doc.add(new Field(IdFieldMapper.NAME, "6", IdFieldMapper.Defaults.FIELD_TYPE)); doc.add(new NumericDocValuesField(VersionFieldMapper.NAME, 87)); writer.addDocument(doc); writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); LeafReaderContext segment = reader.leaves().get(0); - PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader()); + PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME); // return the last doc when there are duplicates DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), null, segment); assertNotNull(result); diff --git a/core/src/test/java/org/elasticsearch/fieldstats/FieldStatsTests.java b/core/src/test/java/org/elasticsearch/fieldstats/FieldStatsTests.java index 379806a7166fd..ee1f654642ce7 100644 --- a/core/src/test/java/org/elasticsearch/fieldstats/FieldStatsTests.java +++ b/core/src/test/java/org/elasticsearch/fieldstats/FieldStatsTests.java @@ -596,7 +596,7 @@ public void testMetaFieldsNotIndexed() { client().admin().indices().prepareRefresh().get(); FieldStatsResponse response = client().prepareFieldStats() - .setFields("_id", "_type") + .setFields("_uid", "_type") .get(); assertEquals(response.getAllFieldStats().size(), 1); assertEquals(response.getAllFieldStats().get("_type").isSearchable(), true); diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 6f80da2d639c7..1ae125cecd19e 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -51,6 +51,7 @@ import org.elasticsearch.index.engine.InternalEngineTests; import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.IndexingOperationListener; @@ -249,8 +250,8 @@ public Engine.Index preIndex(ShardId shardId, Engine.Index operation) { assertEquals(IndexingSlowLog.class, indexService.getIndexOperationListeners().get(0).getClass()); assertSame(listener, indexService.getIndexOperationListeners().get(1)); - ParsedDocument doc = InternalEngineTests.createParsedDoc("1", "test", null); - Engine.Index index = new Engine.Index(new Term("_uid", doc.uid()), doc); + ParsedDocument doc = InternalEngineTests.createParsedDoc("1", null); + Engine.Index index = new Engine.Index(new Term("_uid", Uid.createUidAsBytes(doc.type(), doc.id())), doc); ShardId shardId = new ShardId(new Index("foo", "bar"), 0); for (IndexingOperationListener l : indexService.getIndexOperationListeners()) { l.preIndex(shardId, index); diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 2a611adb7635f..1b40316fca1f3 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -105,6 +105,7 @@ import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperForType; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.Mapper.BuilderContext; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; @@ -291,12 +292,12 @@ private static Document testDocument() { return new Document(); } - public static ParsedDocument createParsedDoc(String id, String type, String routing) { - return testParsedDocument(id, type, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null); + public static ParsedDocument createParsedDoc(String id, String routing) { + return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null); } - private static ParsedDocument testParsedDocument(String id, String type, String routing, Document document, BytesReference source, Mapping mappingUpdate) { - Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); + private static ParsedDocument testParsedDocument(String id, String routing, Document document, BytesReference source, Mapping mappingUpdate) { + Field uidField = new Field("_id", id, IdFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); document.add(uidField); @@ -306,7 +307,7 @@ private static ParsedDocument testParsedDocument(String id, String type, String document.add(seqID.primaryTerm); BytesRef ref = source.toBytesRef(); document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); - return new ParsedDocument(versionField, seqID, id, type, routing, Arrays.asList(document), source, XContentType.JSON, + return new ParsedDocument(versionField, seqID, id, "test", routing, Arrays.asList(document), source, XContentType.JSON, mappingUpdate); } @@ -441,8 +442,9 @@ public void onFailedEngine(String reason, @Nullable Exception e) { }; EngineConfig config = new EngineConfig(openMode, shardId, threadPool, indexSettings, null, store, deletionPolicy, mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener, - new TranslogHandler(xContentRegistry(), shardId.getIndexName(), logger), IndexSearcher.getDefaultQueryCache(), - IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), refreshListener, indexSort); + new TranslogHandler(xContentRegistry(), shardId.getIndexName(), indexSettings.getSettings(), logger), + IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, + TimeValue.timeValueMinutes(5), refreshListener, indexSort); return config; } @@ -465,10 +467,10 @@ public void testSegments() throws Exception { assertThat(engine.segmentsStats(false).getMemoryInBytes(), equalTo(0L)); // create two docs and refresh - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); Engine.Index first = indexForDoc(doc); Engine.IndexResult firstResult = engine.index(first); - ParsedDocument doc2 = testParsedDocument("2", "test", null, testDocumentWithTextField(), B_2, null); + ParsedDocument doc2 = testParsedDocument("2", null, testDocumentWithTextField(), B_2, null); Engine.Index second = indexForDoc(doc2); Engine.IndexResult secondResult = engine.index(second); assertThat(secondResult.getTranslogLocation(), greaterThan(firstResult.getTranslogLocation())); @@ -501,7 +503,7 @@ public void testSegments() throws Exception { assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); assertThat(segments.get(0).isCompound(), equalTo(true)); - ParsedDocument doc3 = testParsedDocument("3", "test", null, testDocumentWithTextField(), B_3, null); + ParsedDocument doc3 = testParsedDocument("3", null, testDocumentWithTextField(), B_3, null); engine.index(indexForDoc(doc3)); engine.refresh("test"); @@ -548,7 +550,7 @@ public void testSegments() throws Exception { assertThat(segments.get(1).isCompound(), equalTo(true)); engine.onSettingsChanged(); - ParsedDocument doc4 = testParsedDocument("4", "test", null, testDocumentWithTextField(), B_3, null); + ParsedDocument doc4 = testParsedDocument("4", null, testDocumentWithTextField(), B_3, null); engine.index(indexForDoc(doc4)); engine.refresh("test"); @@ -582,7 +584,7 @@ public void testVerboseSegments() throws Exception { List segments = engine.segments(true); assertThat(segments.isEmpty(), equalTo(true)); - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); engine.refresh("test"); @@ -590,10 +592,10 @@ public void testVerboseSegments() throws Exception { assertThat(segments.size(), equalTo(1)); assertThat(segments.get(0).ramTree, notNullValue()); - ParsedDocument doc2 = testParsedDocument("2", "test", null, testDocumentWithTextField(), B_2, null); + ParsedDocument doc2 = testParsedDocument("2", null, testDocumentWithTextField(), B_2, null); engine.index(indexForDoc(doc2)); engine.refresh("test"); - ParsedDocument doc3 = testParsedDocument("3", "test", null, testDocumentWithTextField(), B_3, null); + ParsedDocument doc3 = testParsedDocument("3", null, testDocumentWithTextField(), B_3, null); engine.index(indexForDoc(doc3)); engine.refresh("test"); @@ -608,12 +610,12 @@ public void testVerboseSegments() throws Exception { public void testSegmentsWithMergeFlag() throws Exception { try (Store store = createStore(); Engine engine = createEngine(defaultSettings, store, createTempDir(), new TieredMergePolicy())) { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocument(), B_1, null); Engine.Index index = indexForDoc(doc); engine.index(index); engine.flush(); assertThat(engine.segments(false).size(), equalTo(1)); - index = indexForDoc(testParsedDocument("2", "test", null, testDocument(), B_1, null)); + index = indexForDoc(testParsedDocument("2", null, testDocument(), B_1, null)); engine.index(index); engine.flush(); List segments = engine.segments(false); @@ -621,7 +623,7 @@ public void testSegmentsWithMergeFlag() throws Exception { for (Segment segment : segments) { assertThat(segment.getMergeId(), nullValue()); } - index = indexForDoc(testParsedDocument("3", "test", null, testDocument(), B_1, null)); + index = indexForDoc(testParsedDocument("3", null, testDocument(), B_1, null)); engine.index(index); engine.flush(); segments = engine.segments(false); @@ -665,7 +667,7 @@ public void testSegmentsWithIndexSort() throws Exception { List segments = engine.segments(true); assertThat(segments.isEmpty(), equalTo(true)); - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); engine.refresh("test"); @@ -673,10 +675,10 @@ public void testSegmentsWithIndexSort() throws Exception { assertThat(segments.size(), equalTo(1)); assertThat(segments.get(0).getSegmentSort(), equalTo(indexSort)); - ParsedDocument doc2 = testParsedDocument("2", "test", null, testDocumentWithTextField(), B_2, null); + ParsedDocument doc2 = testParsedDocument("2", null, testDocumentWithTextField(), B_2, null); engine.index(indexForDoc(doc2)); engine.refresh("test"); - ParsedDocument doc3 = testParsedDocument("3", "test", null, testDocumentWithTextField(), B_3, null); + ParsedDocument doc3 = testParsedDocument("3", null, testDocumentWithTextField(), B_3, null); engine.index(indexForDoc(doc3)); engine.refresh("test"); @@ -693,7 +695,7 @@ public void testSegmentsStatsIncludingFileSizes() throws Exception { Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { assertThat(engine.segmentsStats(true).getFileSizes().size(), equalTo(0)); - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); engine.refresh("test"); @@ -703,7 +705,7 @@ public void testSegmentsStatsIncludingFileSizes() throws Exception { ObjectObjectCursor firstEntry = stats.getFileSizes().iterator().next(); - ParsedDocument doc2 = testParsedDocument("2", "test", null, testDocumentWithTextField(), B_2, null); + ParsedDocument doc2 = testParsedDocument("2", null, testDocumentWithTextField(), B_2, null); engine.index(indexForDoc(doc2)); engine.refresh("test"); @@ -804,7 +806,7 @@ public IndexSearcher wrap(IndexSearcher searcher) throws EngineException { public void testFlushIsDisabledDuringTranslogRecovery() throws IOException { assertFalse(engine.isRecovering()); - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), SOURCE, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null); engine.index(indexForDoc(doc)); engine.close(); @@ -813,7 +815,7 @@ public void testFlushIsDisabledDuringTranslogRecovery() throws IOException { assertTrue(engine.isRecovering()); engine.recoverFromTranslog(); assertFalse(engine.isRecovering()); - doc = testParsedDocument("2", "test", null, testDocumentWithTextField(), SOURCE, null); + doc = testParsedDocument("2", null, testDocumentWithTextField(), SOURCE, null); engine.index(indexForDoc(doc)); engine.flush(); } @@ -825,7 +827,7 @@ public void testTranslogMultipleOperationsSameDocument() throws IOException { try { initialEngine = engine; for (int i = 0; i < ops; i++) { - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), SOURCE, null); + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null); if (randomBoolean()) { final Engine.Index operation = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false); operations.add(operation); @@ -861,7 +863,7 @@ public void testTranslogRecoveryDoesNotReplayIntoTranslog() throws IOException { initialEngine = engine; for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); - final ParsedDocument doc = testParsedDocument(id, "test", null, testDocumentWithTextField(), SOURCE, null); + final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null); initialEngine.index(indexForDoc(doc)); } } finally { @@ -915,7 +917,7 @@ public long generateSeqNo() { }; for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); - final ParsedDocument doc = testParsedDocument(id, "test", null, testDocumentWithTextField(), SOURCE, null); + final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null); initialEngine.index(indexForDoc(doc)); if (rarely()) { initialEngine.getTranslog().rollGeneration(); @@ -942,11 +944,11 @@ public long generateSeqNo() { } public void testConcurrentGetAndFlush() throws Exception { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); final AtomicReference latestGetResult = new AtomicReference<>(); - latestGetResult.set(engine.get(new Engine.Get(true, newUid(doc)))); + latestGetResult.set(engine.get(newGet(true, doc))); final AtomicBoolean flushFinished = new AtomicBoolean(false); final CyclicBarrier barrier = new CyclicBarrier(2); Thread getThread = new Thread(() -> { @@ -960,7 +962,7 @@ public void testConcurrentGetAndFlush() throws Exception { if (previousGetResult != null) { previousGetResult.release(); } - latestGetResult.set(engine.get(new Engine.Get(true, newUid(doc)))); + latestGetResult.set(engine.get(newGet(true, doc))); if (latestGetResult.get().exists() == false) { break; } @@ -983,7 +985,7 @@ public void testSimpleOperations() throws Exception { // create a document Document document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - ParsedDocument doc = testParsedDocument("1", "test", null, document, B_1, null); + ParsedDocument doc = testParsedDocument("1", null, document, B_1, null); engine.index(indexForDoc(doc)); // its not there... @@ -993,12 +995,12 @@ public void testSimpleOperations() throws Exception { searchResult.close(); // but, not there non realtime - Engine.GetResult getResult = engine.get(new Engine.Get(false, newUid(doc))); + Engine.GetResult getResult = engine.get(newGet(false, doc)); assertThat(getResult.exists(), equalTo(false)); getResult.release(); // but, we can still get it (in realtime) - getResult = engine.get(new Engine.Get(true, newUid(doc))); + getResult = engine.get(newGet(true, doc)); assertThat(getResult.exists(), equalTo(true)); assertThat(getResult.docIdAndVersion(), notNullValue()); getResult.release(); @@ -1013,7 +1015,7 @@ public void testSimpleOperations() throws Exception { searchResult.close(); // also in non realtime - getResult = engine.get(new Engine.Get(false, newUid(doc))); + getResult = engine.get(newGet(false, doc)); assertThat(getResult.exists(), equalTo(true)); assertThat(getResult.docIdAndVersion(), notNullValue()); getResult.release(); @@ -1022,7 +1024,7 @@ public void testSimpleOperations() throws Exception { document = testDocument(); document.add(new TextField("value", "test1", Field.Store.YES)); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_2), SourceFieldMapper.Defaults.FIELD_TYPE)); - doc = testParsedDocument("1", "test", null, document, B_2, null); + doc = testParsedDocument("1", null, document, B_2, null); engine.index(indexForDoc(doc)); // its not updated yet... @@ -1033,7 +1035,7 @@ public void testSimpleOperations() throws Exception { searchResult.close(); // but, we can still get it (in realtime) - getResult = engine.get(new Engine.Get(true, newUid(doc))); + getResult = engine.get(newGet(true, doc)); assertThat(getResult.exists(), equalTo(true)); assertThat(getResult.docIdAndVersion(), notNullValue()); getResult.release(); @@ -1058,7 +1060,7 @@ public void testSimpleOperations() throws Exception { searchResult.close(); // but, get should not see it (in realtime) - getResult = engine.get(new Engine.Get(true, newUid(doc))); + getResult = engine.get(newGet(true, doc)); assertThat(getResult.exists(), equalTo(false)); getResult.release(); @@ -1074,7 +1076,7 @@ public void testSimpleOperations() throws Exception { // add it back document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - doc = testParsedDocument("1", "test", null, document, B_1, null); + doc = testParsedDocument("1", null, document, B_1, null); engine.index(new Engine.Index(newUid(doc), doc, Versions.MATCH_DELETED)); // its not there... @@ -1098,7 +1100,7 @@ public void testSimpleOperations() throws Exception { engine.flush(); // and, verify get (in real time) - getResult = engine.get(new Engine.Get(true, newUid(doc))); + getResult = engine.get(newGet(true, doc)); assertThat(getResult.exists(), equalTo(true)); assertThat(getResult.docIdAndVersion(), notNullValue()); getResult.release(); @@ -1107,7 +1109,7 @@ public void testSimpleOperations() throws Exception { // now do an update document = testDocument(); document.add(new TextField("value", "test1", Field.Store.YES)); - doc = testParsedDocument("1", "test", null, document, B_1, null); + doc = testParsedDocument("1", null, document, B_1, null); engine.index(indexForDoc(doc)); // its not updated yet... @@ -1133,7 +1135,7 @@ public void testSearchResultRelease() throws Exception { searchResult.close(); // create a document - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); // its not there... @@ -1168,7 +1170,7 @@ public void testSyncedFlush() throws IOException { try (Store store = createStore(); Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new LogByteSizeMergePolicy(), null))) { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); Engine.CommitId commitID = engine.flush(); assertThat(commitID, equalTo(new Engine.CommitId(store.readLastCommittedSegmentsInfo().getId()))); @@ -1195,16 +1197,16 @@ public void testRenewSyncFlush() throws Exception { InternalEngine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new LogDocMergePolicy(), null))) { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); - Engine.Index doc1 = indexForDoc(testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null)); + Engine.Index doc1 = indexForDoc(testParsedDocument("1", null, testDocumentWithTextField(), B_1, null)); engine.index(doc1); assertEquals(engine.getLastWriteNanos(), doc1.startTime()); engine.flush(); - Engine.Index doc2 = indexForDoc(testParsedDocument("2", "test", null, testDocumentWithTextField(), B_1, null)); + Engine.Index doc2 = indexForDoc(testParsedDocument("2", null, testDocumentWithTextField(), B_1, null)); engine.index(doc2); assertEquals(engine.getLastWriteNanos(), doc2.startTime()); engine.flush(); final boolean forceMergeFlushes = randomBoolean(); - final ParsedDocument parsedDoc3 = testParsedDocument("3", "test", null, testDocumentWithTextField(), B_1, null); + final ParsedDocument parsedDoc3 = testParsedDocument("3", null, testDocumentWithTextField(), B_1, null); if (forceMergeFlushes) { engine.index(new Engine.Index(newUid(parsedDoc3), parsedDoc3, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos(), -1, false)); } else { @@ -1230,7 +1232,7 @@ public void testRenewSyncFlush() throws Exception { assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); if (randomBoolean()) { - Engine.Index doc4 = indexForDoc(testParsedDocument("4", "test", null, testDocumentWithTextField(), B_1, null)); + Engine.Index doc4 = indexForDoc(testParsedDocument("4", null, testDocumentWithTextField(), B_1, null)); engine.index(doc4); assertEquals(engine.getLastWriteNanos(), doc4.startTime()); } else { @@ -1248,7 +1250,7 @@ public void testRenewSyncFlush() throws Exception { public void testSyncedFlushSurvivesEngineRestart() throws IOException { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); final Engine.CommitId commitID = engine.flush(); assertEquals("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), @@ -1271,14 +1273,14 @@ public void testSyncedFlushSurvivesEngineRestart() throws IOException { public void testSyncedFlushVanishesOnReplay() throws IOException { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); final Engine.CommitId commitID = engine.flush(); assertEquals("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), Engine.SyncedFlushResult.SUCCESS); assertEquals(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); - doc = testParsedDocument("2", "test", null, testDocumentWithTextField(), new BytesArray("{}"), null); + doc = testParsedDocument("2", null, testDocumentWithTextField(), new BytesArray("{}"), null); engine.index(indexForDoc(doc)); EngineConfig config = engine.config(); engine.close(); @@ -1288,7 +1290,7 @@ public void testSyncedFlushVanishesOnReplay() throws IOException { } public void testVersioningNewCreate() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocument(), B_1, null); Engine.Index create = new Engine.Index(newUid(doc), doc, Versions.MATCH_DELETED); Engine.IndexResult indexResult = engine.index(create); assertThat(indexResult.getVersion(), equalTo(1L)); @@ -1299,7 +1301,7 @@ public void testVersioningNewCreate() throws IOException { } public void testVersioningNewIndex() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocument(), B_1, null); Engine.Index index = indexForDoc(doc); Engine.IndexResult indexResult = engine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); @@ -1315,7 +1317,7 @@ public void testForceMerge() throws IOException { new LogByteSizeMergePolicy(), null))) { // use log MP here we test some behavior in ESMP int numDocs = randomIntBetween(10, 100); for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), "test", null, testDocument(), B_1, null); + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); Engine.Index index = indexForDoc(doc); engine.index(index); engine.refresh("test"); @@ -1326,7 +1328,7 @@ public void testForceMerge() throws IOException { engine.forceMerge(true, 1, false, false, false); assertEquals(engine.segments(true).size(), 1); - ParsedDocument doc = testParsedDocument(Integer.toString(0), "test", null, testDocument(), B_1, null); + ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), B_1, null); Engine.Index index = indexForDoc(doc); engine.delete(new Engine.Delete(index.type(), index.id(), index.uid())); engine.forceMerge(true, 10, true, false, false); //expunge deletes @@ -1337,7 +1339,7 @@ public void testForceMerge() throws IOException { assertEquals(engine.config().getMergePolicy().toString(), numDocs - 1, test.reader().maxDoc()); } - doc = testParsedDocument(Integer.toString(1), "test", null, testDocument(), B_1, null); + doc = testParsedDocument(Integer.toString(1), null, testDocument(), B_1, null); index = indexForDoc(doc); engine.delete(new Engine.Delete(index.type(), index.id(), index.uid())); engine.forceMerge(true, 10, false, false, false); //expunge deletes @@ -1372,7 +1374,7 @@ public void run() { int numDocs = randomIntBetween(1, 20); for (int j = 0; j < numDocs; j++) { i++; - ParsedDocument doc = testParsedDocument(Integer.toString(i), "test", null, testDocument(), B_1, null); + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); Engine.Index index = indexForDoc(doc); engine.index(index); } @@ -1407,7 +1409,7 @@ public void run() { } public void testVersioningCreateExistsException() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocument(), B_1, null); Engine.Index create = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); Engine.IndexResult indexResult = engine.index(create); assertThat(indexResult.getVersion(), equalTo(1L)); @@ -1423,7 +1425,7 @@ protected List generateSingleDocHistory(boolean forReplica, Ve int minOpCount, int maxOpCount) { final int numOfOps = randomIntBetween(minOpCount, maxOpCount); final List ops = new ArrayList<>(); - final Term id = newUid(Uid.createUid("test", "1")); + final Term id = newUid("1"); final int startWithSeqNo; if (partialOldPrimary) { startWithSeqNo = randomBoolean() ? numOfOps - 1 : randomIntBetween(0, numOfOps - 1); @@ -1452,7 +1454,7 @@ protected List generateSingleDocHistory(boolean forReplica, Ve throw new UnsupportedOperationException("unknown version type: " + versionType); } if (randomBoolean()) { - op = new Engine.Index(id, testParsedDocument("1", "test", null, testDocumentWithTextField(valuePrefix + i), B_1, null), + op = new Engine.Index(id, testParsedDocument("1", null, testDocumentWithTextField(valuePrefix + i), B_1, null), forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbersService.UNASSIGNED_SEQ_NO, forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, version, @@ -1485,6 +1487,7 @@ public void testOutOfOrderDocsOnReplicaOldPrimary() throws IOException { .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_5_4_0_UNRELEASED) + .put(MapperService.INDEX_MAPPING_SINGLE_TYPE_SETTING.getKey(), true) .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) .build()); @@ -1813,7 +1816,8 @@ public void testVersioningPromotedReplica() throws IOException { final long finalReplicaSeqNo = lastReplicaOp.seqNo(); assertOpsOnReplica(replicaOps, replicaEngine, true); final int opsOnPrimary = assertOpsOnPrimary(primaryOps, finalReplicaVersion, deletedOnReplica, replicaEngine); - final long currentSeqNo = getSequenceID(replicaEngine, new Engine.Get(false, lastReplicaOp.uid())).v1(); + final long currentSeqNo = getSequenceID(replicaEngine, + new Engine.Get(false, "type", lastReplicaOp.uid().text(), lastReplicaOp.uid())).v1(); try (Searcher searcher = engine.acquireSearcher("test")) { final TotalHitCountCollector collector = new TotalHitCountCollector(); searcher.searcher().search(new MatchAllDocsQuery(), collector); @@ -1865,7 +1869,7 @@ class OpAndVersion { } final AtomicInteger idGenerator = new AtomicInteger(); final Queue history = ConcurrentCollections.newQueue(); - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), bytesArray(""), null); + ParsedDocument doc = testParsedDocument("1", null, testDocument(), bytesArray(""), null); final Term uidTerm = newUid(doc); engine.index(indexForDoc(doc)); for (int i = 0; i < thread.length; i++) { @@ -1877,7 +1881,7 @@ class OpAndVersion { throw new AssertionError(e); } for (int op = 0; op < opsPerThread; op++) { - try (Engine.GetResult get = engine.get(new Engine.Get(true, uidTerm))) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, doc.type(), doc.id(), uidTerm))) { FieldsVisitor visitor = new FieldsVisitor(true); get.docIdAndVersion().context.reader().document(get.docIdAndVersion().docId, visitor); List values = new ArrayList<>(Strings.commaDelimitedListToSet(visitor.source().utf8ToString())); @@ -1885,7 +1889,7 @@ class OpAndVersion { String added = "v_" + idGenerator.incrementAndGet(); values.add(added); Engine.Index index = new Engine.Index(uidTerm, - testParsedDocument("1", "test", null, testDocument(), + testParsedDocument("1", null, testDocument(), bytesArray(Strings.collectionToCommaDelimitedString(values)), null), SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, get.version(), VersionType.INTERNAL, @@ -1919,7 +1923,7 @@ class OpAndVersion { assertTrue(op.added + " should not exist", exists); } - try (Engine.GetResult get = engine.get(new Engine.Get(true, uidTerm))) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, doc.type(), doc.id(), uidTerm))) { FieldsVisitor visitor = new FieldsVisitor(true); get.docIdAndVersion().context.reader().document(get.docIdAndVersion().docId, visitor); List values = Arrays.asList(Strings.commaDelimitedListToStringArray(visitor.source().utf8ToString())); @@ -1928,7 +1932,7 @@ class OpAndVersion { } public void testBasicCreatedFlag() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocument(), B_1, null); Engine.Index index = indexForDoc(doc); Engine.IndexResult indexResult = engine.index(index); assertTrue(indexResult.isCreated()); @@ -1984,7 +1988,7 @@ public void testIndexWriterInfoStream() throws IllegalAccessException, IOExcepti try { // First, with DEBUG, which should NOT log IndexWriter output: - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); engine.flush(); assertFalse(mockAppender.sawIndexWriterMessage); @@ -2025,7 +2029,7 @@ public void testSeqNoAndCheckpoints() throws IOException { // we have some docs indexed, so delete one of them id = randomFrom(indexedIds); final Engine.Delete delete = new Engine.Delete( - "test", id, newUid("test#" + id), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, + "test", id, newUid(id), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); final Engine.DeleteResult result = initialEngine.delete(delete); if (!result.hasFailure()) { @@ -2040,7 +2044,7 @@ public void testSeqNoAndCheckpoints() throws IOException { } else { // index a document id = randomFrom(ids); - ParsedDocument doc = testParsedDocument(id, "test", null, testDocumentWithTextField(), SOURCE, null); + ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null); final Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, @@ -2142,7 +2146,7 @@ public void testConcurrentWritesAndCommits() throws Exception { // index random number of docs for (int i = 0; i < numDocsPerThread; i++) { final String id = "thread" + threadIdx + "#" + i; - ParsedDocument doc = testParsedDocument(id, "test", null, testDocument(), B_1, null); + ParsedDocument doc = testParsedDocument(id, null, testDocument(), B_1, null); engine.index(indexForDoc(doc)); } } catch (Exception e) { @@ -2246,7 +2250,7 @@ public void testIndexWriterIFDInfoStream() throws IllegalAccessException, IOExce try { // First, with DEBUG, which should NOT log IndexWriter output: - ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); + ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); engine.flush(); assertFalse(mockAppender.sawIndexWriterMessage); @@ -2275,14 +2279,14 @@ public void testEnableGcDeletes() throws Exception { Document document = testDocument(); document.add(new TextField("value", "test1", Field.Store.YES)); - ParsedDocument doc = testParsedDocument("1", "test", null, document, B_2, null); + ParsedDocument doc = testParsedDocument("1", null, document, B_2, null); engine.index(new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false)); // Delete document we just added: engine.delete(new Engine.Delete("test", "1", newUid(doc), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); // Get should not find the document - Engine.GetResult getResult = engine.get(new Engine.Get(true, newUid(doc))); + Engine.GetResult getResult = engine.get(newGet(true, doc)); assertThat(getResult.exists(), equalTo(false)); // Give the gc pruning logic a chance to kick in @@ -2296,7 +2300,7 @@ public void testEnableGcDeletes() throws Exception { engine.delete(new Engine.Delete("test", "2", newUid("2"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); // Get should not find the document (we never indexed uid=2): - getResult = engine.get(new Engine.Get(true, newUid("2"))); + getResult = engine.get(new Engine.Get(true, "type", "2", newUid("2"))); assertThat(getResult.exists(), equalTo(false)); // Try to index uid=1 with a too-old version, should fail: @@ -2306,7 +2310,7 @@ public void testEnableGcDeletes() throws Exception { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // Get should still not find the document - getResult = engine.get(new Engine.Get(true, newUid(doc))); + getResult = engine.get(newGet(true, doc)); assertThat(getResult.exists(), equalTo(false)); // Try to index uid=2 with a too-old version, should fail: @@ -2316,17 +2320,21 @@ public void testEnableGcDeletes() throws Exception { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); // Get should not find the document - getResult = engine.get(new Engine.Get(true, newUid(doc))); + getResult = engine.get(newGet(true, doc)); assertThat(getResult.exists(), equalTo(false)); } } protected Term newUid(String id) { - return new Term("_uid", id); + return new Term("_id", id); } protected Term newUid(ParsedDocument doc) { - return new Term("_uid", doc.uid()); + return new Term("_id", doc.id()); + } + + protected Engine.Get newGet(boolean realtime, ParsedDocument doc) { + return new Engine.Get(realtime, doc.type(), doc.id(), newUid(doc)); } private Engine.Index indexForDoc(ParsedDocument doc) { @@ -2423,7 +2431,7 @@ public void testMissingTranslog() throws IOException { public void testTranslogReplayWithFailure() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null); + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); @@ -2481,7 +2489,7 @@ private static void assertVisibleCount(InternalEngine engine, int numDocs, boole public void testSkipTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null); + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); @@ -2570,8 +2578,14 @@ public void testUpgradeOldIndex() throws IOException { } final int numExtraDocs = randomIntBetween(1, 10); for (int i = 0; i < numExtraDocs; i++) { - ParsedDocument doc = testParsedDocument("extra" + Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + ParsedDocument doc = testParsedDocument("extra" + Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); + Term uid; + if (indexMetaData.getCreationVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { + uid = new Term(IdFieldMapper.NAME, doc.id()); + } else { + uid = new Term(UidFieldMapper.NAME, Uid.createUid(doc.type(), doc.id())); + } + Engine.Index firstIndexRequest = new Engine.Index(uid, doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -2599,7 +2613,7 @@ private Path[] filterExtraFSFiles(Path[] files) { public void testTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null); + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); @@ -2631,7 +2645,7 @@ public void testTranslogReplay() throws IOException { final boolean flush = randomBoolean(); int randomId = randomIntBetween(numDocs + 1, numDocs + 10); - ParsedDocument doc = testParsedDocument(Integer.toString(randomId), "test", null, testDocument(), new BytesArray("{}"), null); + ParsedDocument doc = testParsedDocument(Integer.toString(randomId), null, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); @@ -2639,7 +2653,7 @@ public void testTranslogReplay() throws IOException { engine.flush(); } - doc = testParsedDocument(Integer.toString(randomId), "test", null, testDocument(), new BytesArray("{}"), null); + doc = testParsedDocument(Integer.toString(randomId), null, testDocument(), new BytesArray("{}"), null); Engine.Index idxRequest = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false); Engine.IndexResult result = engine.index(idxRequest); engine.refresh("test"); @@ -2677,9 +2691,8 @@ public static class TranslogHandler extends TranslogRecoveryPerformer { public final AtomicInteger recoveredOps = new AtomicInteger(0); - public TranslogHandler(NamedXContentRegistry xContentRegistry, String indexName, Logger logger) { + public TranslogHandler(NamedXContentRegistry xContentRegistry, String indexName, Settings settings, Logger logger) { super(new ShardId("test", "_na_", 0), null, logger); - Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); Index index = new Index(indexName, "_na_"); IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings); NamedAnalyzer defaultAnalyzer = new NamedAnalyzer("default", AnalyzerScope.INDEX, new StandardAnalyzer()); @@ -2706,7 +2719,7 @@ protected void operationProcessed() { public void testRecoverFromForeignTranslog() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null); + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); Engine.IndexResult index = engine.index(firstIndexRequest); assertThat(index.getVersion(), equalTo(1L)); @@ -2787,7 +2800,7 @@ public void testCurrentTranslogIDisCommitted() throws IOException { // create { - ParsedDocument doc = testParsedDocument(Integer.toString(0), "test", null, testDocument(), new BytesArray("{}"), null); + ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG))){ @@ -2889,9 +2902,9 @@ public void clearFailure() { public void testHandleDocumentFailure() throws Exception { try (Store store = createStore()) { - final ParsedDocument doc1 = testParsedDocument("1", "test", null, testDocumentWithTextField(), B_1, null); - final ParsedDocument doc2 = testParsedDocument("2", "test", null, testDocumentWithTextField(), B_1, null); - final ParsedDocument doc3 = testParsedDocument("3", "test", null, testDocumentWithTextField(), B_1, null); + final ParsedDocument doc1 = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); + final ParsedDocument doc2 = testParsedDocument("2", null, testDocumentWithTextField(), B_1, null); + final ParsedDocument doc3 = testParsedDocument("3", null, testDocumentWithTextField(), B_1, null); AtomicReference throwingIndexWriter = new AtomicReference<>(); try (Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE, @@ -2974,7 +2987,7 @@ public BytesRef binaryValue() { } public void testDoubleDeliveryPrimary() throws IOException { - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); Engine.Index operation = appendOnlyPrimary(doc, false, 1); Engine.Index retry = appendOnlyPrimary(doc, true, 1); if (randomBoolean()) { @@ -3028,7 +3041,7 @@ public void testDoubleDeliveryPrimary() throws IOException { } public void testDoubleDeliveryReplicaAppendingOnly() throws IOException { - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); Engine.Index operation = appendOnlyReplica(doc, false, 1, randomIntBetween(0, 20)); Engine.Index retry = appendOnlyReplica(doc, true, 1, operation.seqNo()); @@ -3083,7 +3096,7 @@ public void testDoubleDeliveryReplicaAppendingOnly() throws IOException { } public void testDoubleDeliveryReplica() throws IOException { - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); Engine.Index operation = replicaIndexForDoc(doc, 1, 20, false); Engine.Index duplicate = replicaIndexForDoc(doc, 1, 20, true); @@ -3129,7 +3142,7 @@ public void testDoubleDeliveryReplica() throws IOException { public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOException { - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); boolean isRetry = false; long autoGeneratedIdTimestamp = 0; @@ -3163,7 +3176,7 @@ public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOExcep public void testRetryWithAutogeneratedIdsAndWrongOrderWorksAndNoDuplicateDocs() throws IOException { - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); boolean isRetry = true; long autoGeneratedIdTimestamp = 0; @@ -3218,7 +3231,7 @@ public void testRetryConcurrently() throws InterruptedException, IOException { List docs = new ArrayList<>(); final boolean primary = randomBoolean(); for (int i = 0; i < numDocs; i++) { - final ParsedDocument doc = testParsedDocument(Integer.toString(i), "test", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); + final ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); final Engine.Index originalIndex; final Engine.Index retryIndex; if (primary) { @@ -3288,7 +3301,7 @@ public void testEngineMaxTimestampIsInitialized() throws IOException { try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = new InternalEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) { assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); engine.index(appendOnlyPrimary(doc, true, timestamp1)); assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); @@ -3298,7 +3311,7 @@ public void testEngineMaxTimestampIsInitialized() throws IOException { assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); engine.recoverFromTranslog(); assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); engine.index(appendOnlyPrimary(doc, true, timestamp2)); assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); @@ -3319,7 +3332,7 @@ public void testAppendConcurrently() throws InterruptedException, IOException { assertEquals(0, engine.getNumIndexVersionsLookups()); List docs = new ArrayList<>(); for (int i = 0; i < numDocs; i++) { - final ParsedDocument doc = testParsedDocument(Integer.toString(i), "test", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); + final ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); Engine.Index index = randomAppendOnly(doc, false, i); docs.add(index); } @@ -3400,7 +3413,7 @@ public void afterRefresh(boolean didRefresh) throws IOException { }); InternalEngine internalEngine = new InternalEngine(config); int docId = 0; - final ParsedDocument doc = testParsedDocument(Integer.toString(docId), "test", null, + final ParsedDocument doc = testParsedDocument(Integer.toString(docId), null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); Engine.Index index = randomBoolean() ? indexForDoc(doc) : randomAppendOnly(doc, false, docId); @@ -3437,7 +3450,7 @@ public void afterRefresh(boolean didRefresh) throws IOException { } public void testSequenceIDs() throws Exception { - Tuple seqID = getSequenceID(engine, new Engine.Get(false, newUid("1"))); + Tuple seqID = getSequenceID(engine, new Engine.Get(false, "type", "2", newUid("1"))); // Non-existent doc returns no seqnum and no primary term assertThat(seqID.v1(), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO)); assertThat(seqID.v2(), equalTo(0L)); @@ -3445,11 +3458,11 @@ public void testSequenceIDs() throws Exception { // create a document Document document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - ParsedDocument doc = testParsedDocument("1", "test", null, document, B_1, null); + ParsedDocument doc = testParsedDocument("1", null, document, B_1, null); engine.index(indexForDoc(doc)); engine.refresh("test"); - seqID = getSequenceID(engine, new Engine.Get(false, newUid(doc))); + seqID = getSequenceID(engine, newGet(false, doc)); logger.info("--> got seqID: {}", seqID); assertThat(seqID.v1(), equalTo(0L)); assertThat(seqID.v2(), equalTo(2L)); @@ -3457,11 +3470,11 @@ public void testSequenceIDs() throws Exception { // Index the same document again document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - doc = testParsedDocument("1", "test", null, document, B_1, null); + doc = testParsedDocument("1", null, document, B_1, null); engine.index(indexForDoc(doc)); engine.refresh("test"); - seqID = getSequenceID(engine, new Engine.Get(false, newUid(doc))); + seqID = getSequenceID(engine, newGet(false, doc)); logger.info("--> got seqID: {}", seqID); assertThat(seqID.v1(), equalTo(1L)); assertThat(seqID.v2(), equalTo(2L)); @@ -3469,13 +3482,13 @@ public void testSequenceIDs() throws Exception { // Index the same document for the third time, this time changing the primary term document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - doc = testParsedDocument("1", "test", null, document, B_1, null); + doc = testParsedDocument("1", null, document, B_1, null); engine.index(new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 3, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false)); engine.refresh("test"); - seqID = getSequenceID(engine, new Engine.Get(false, newUid(doc))); + seqID = getSequenceID(engine, newGet(false, doc)); logger.info("--> got seqID: {}", seqID); assertThat(seqID.v1(), equalTo(2L)); assertThat(seqID.v2(), equalTo(3L)); @@ -3546,7 +3559,7 @@ public void testSequenceNumberAdvancesToMaxSeqOnEngineOpenOnPrimary() throws Bro final InternalEngine finalInitialEngine = initialEngine; for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); - final ParsedDocument doc = testParsedDocument(id, "test", null, testDocumentWithTextField(), SOURCE, null); + final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null); stall.set(randomBoolean()); final Thread thread = new Thread(() -> { @@ -3595,7 +3608,7 @@ public void testSequenceNumberAdvancesToMaxSeqNoOnEngineOpenOnReplica() throws I initialEngine = engine; for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); - final ParsedDocument doc = testParsedDocument(id, "test", null, testDocumentWithTextField(), SOURCE, null); + final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null); final Term uid = newUid(doc); // create a gap at sequence number 3 * i + 1 initialEngine.index(new Engine.Index(uid, doc, 3 * i, 1, v, t, REPLICA, System.nanoTime(), ts, false)); @@ -3609,7 +3622,7 @@ public void testSequenceNumberAdvancesToMaxSeqNoOnEngineOpenOnReplica() throws I for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); - final ParsedDocument doc = testParsedDocument(id, "test", null, testDocumentWithTextField(), SOURCE, null); + final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null); final Term uid = newUid(doc); initialEngine.index(new Engine.Index(uid, doc, 3 * i + 1, 1, v, t, REPLICA, System.nanoTime(), ts, false)); } @@ -3636,7 +3649,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio final LongSupplier sequenceNumberSupplier = origin == PRIMARY ? () -> SequenceNumbersService.UNASSIGNED_SEQ_NO : sequenceNumber::getAndIncrement; document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - final ParsedDocument doc = testParsedDocument("1", "test", null, document, B_1, null); + final ParsedDocument doc = testParsedDocument("1", null, document, B_1, null); final Term uid = newUid(doc); for (int i = 0; i < numberOfOperations; i++) { if (randomBoolean()) { @@ -3699,7 +3712,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio } assertThat(engine.seqNoService().getLocalCheckpoint(), equalTo(expectedLocalCheckpoint)); - try (Engine.GetResult result = engine.get(new Engine.Get(true, uid))) { + try (Engine.GetResult result = engine.get(new Engine.Get(true, "type", "2", uid))) { assertThat(result.exists(), equalTo(exists)); } } @@ -3827,7 +3840,7 @@ public void testMinGenerationForSeqNo() throws IOException, BrokenBarrierExcepti private void index(final InternalEngine engine, final int id) throws IOException { final String docId = Integer.toString(id); final ParsedDocument doc = - testParsedDocument(docId, "test", null, testDocumentWithTextField(), SOURCE, null); + testParsedDocument(docId, null, testDocumentWithTextField(), SOURCE, null); engine.index(indexForDoc(doc)); } @@ -3846,7 +3859,7 @@ private Tuple getSequenceID(Engine engine, Engine.Get get) throws En seqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; } else { seqNo = docIdAndSeqNo.seqNo; - primaryTerm = VersionsAndSeqNoResolver.loadPrimaryTerm(docIdAndSeqNo); + primaryTerm = VersionsAndSeqNoResolver.loadPrimaryTerm(docIdAndSeqNo, get.uid().field()); } return new Tuple<>(seqNo, primaryTerm); } catch (Exception e) { @@ -3863,7 +3876,7 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { for (int i = 0; i < docs; i++) { final String docId = Integer.toString(i); final ParsedDocument doc = - testParsedDocument(docId, "test", null, testDocumentWithTextField(), SOURCE, null); + testParsedDocument(docId, null, testDocumentWithTextField(), SOURCE, null); Engine.Index primaryResponse = indexForDoc(doc); Engine.IndexResult indexResult = engine.index(primaryResponse); if (randomBoolean()) { diff --git a/core/src/test/java/org/elasticsearch/index/fielddata/UidFieldDataTests.java b/core/src/test/java/org/elasticsearch/index/fielddata/UidFieldDataTests.java new file mode 100644 index 0000000000000..6fa5fa8775e51 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/index/fielddata/UidFieldDataTests.java @@ -0,0 +1,131 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.fielddata; + +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.Collections; +import java.util.Locale; + +public class UidFieldDataTests extends ESTestCase { + + private static class DummySortedDocValues extends SortedDocValues { + + private int doc = -1; + + @Override + public int ordValue() throws IOException { + return doc; + } + + @Override + public BytesRef lookupOrd(int ord) throws IOException { + return new BytesRef(String.format(Locale.ENGLISH, "%03d", ord)); + } + + @Override + public int getValueCount() { + return 100; + } + + @Override + public boolean advanceExact(int target) throws IOException { + doc = target; + return true; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + if (target >= getValueCount()) { + return doc = NO_MORE_DOCS; + } else { + return doc = target; + } + } + + @Override + public long cost() { + return getValueCount(); + } + + } + + private static class DummyAtomicOrdinalsFieldData implements AtomicOrdinalsFieldData { + + @Override + public ScriptDocValues getScriptValues() { + throw new UnsupportedOperationException(); + } + + @Override + public SortedBinaryDocValues getBytesValues() { + return FieldData.toString(getOrdinalsValues()); + } + + @Override + public long ramBytesUsed() { + return 0; + } + + @Override + public void close() { + } + + @Override + public SortedSetDocValues getOrdinalsValues() { + return DocValues.singleton(new DummySortedDocValues()); + } + + } + + public void testSortedSetValues() throws Exception { + AtomicOrdinalsFieldData fd = new UidIndexFieldData.UidAtomicFieldData(new BytesRef("type#"), new DummyAtomicOrdinalsFieldData()); + SortedSetDocValues dv = fd.getOrdinalsValues(); + assertTrue(dv.advanceExact(30)); + assertEquals(30, dv.nextOrd()); + assertEquals(SortedSetDocValues.NO_MORE_ORDS, dv.nextOrd()); + assertEquals(new BytesRef("type#030"), dv.lookupOrd(30)); + assertEquals(30, dv.lookupTerm(new BytesRef("type#030"))); + assertEquals(-1 - 31, dv.lookupTerm(new BytesRef("type#0305"))); + } + + public void testScriptValues() throws IOException { + AtomicOrdinalsFieldData fd = new UidIndexFieldData.UidAtomicFieldData(new BytesRef("type#"), new DummyAtomicOrdinalsFieldData()); + ScriptDocValues values = fd.getScriptValues(); + values.setNextDocId(30); + assertEquals(Collections.singletonList("type#030"), values); + } + +} diff --git a/core/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java b/core/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java index cd4d1d21c6cd0..49864768edf8b 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java @@ -19,23 +19,11 @@ package org.elasticsearch.index.mapper; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import org.apache.lucene.index.IndexableField; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressedXContent; -import org.elasticsearch.common.lucene.all.AllField; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; @@ -45,6 +33,13 @@ import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.InternalSettingsPlugin; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.StreamsUtils.copyToBytesFromClasspath; import static org.elasticsearch.test.StreamsUtils.copyToStringFromClasspath; @@ -61,21 +56,6 @@ protected Collection> getPlugins() { return pluginList(InternalSettingsPlugin.class); } - public void testTypeDisabled() throws Exception { - DocumentMapperParser mapperParser = createIndex("test").mapperService().documentMapperParser(); - String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") - .field("enabled", false).endObject().endObject().string(); - DocumentMapper mapper = mapperParser.parse("type", new CompressedXContent(mapping)); - - BytesReference bytes = XContentFactory.jsonBuilder() - .startObject().startObject("foo") - .field("field", "1234") - .endObject().endObject().bytes(); - ParsedDocument doc = mapper.parse(SourceToParse.source("test", "type", "1", bytes, XContentType.JSON)); - assertNull(doc.rootDoc().getField("field")); - assertNotNull(doc.rootDoc().getField(UidFieldMapper.NAME)); - } - public void testFieldDisabled() throws Exception { DocumentMapperParser mapperParser = createIndex("test").mapperService().documentMapperParser(); String mapping = XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") @@ -92,7 +72,7 @@ public void testFieldDisabled() throws Exception { ParsedDocument doc = mapper.parse(SourceToParse.source("test", "type", "1", bytes, XContentType.JSON)); assertNull(doc.rootDoc().getField("foo")); assertNotNull(doc.rootDoc().getField("bar")); - assertNotNull(doc.rootDoc().getField(UidFieldMapper.NAME)); + assertNotNull(doc.rootDoc().getField(IdFieldMapper.NAME)); } public void testDotsWithExistingMapper() throws Exception { @@ -973,7 +953,7 @@ public void testParseToJsonAndParse() throws Exception { DocumentMapper builtDocMapper = parser.parse("person", new CompressedXContent(builtMapping)); BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/simple/test1.json")); Document doc = builtDocMapper.parse(SourceToParse.source("test", "person", "1", json, XContentType.JSON)).rootDoc(); - assertThat(doc.get(docMapper.uidMapper().fieldType().name()), equalTo(Uid.createUid("person", "1"))); + assertThat(doc.get(docMapper.idFieldMapper().fieldType().name()), equalTo("1")); assertThat(doc.get(docMapper.mappers().getMapper("name.first").fieldType().name()), equalTo("shay")); } @@ -985,7 +965,7 @@ public void testSimpleParser() throws Exception { BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/simple/test1.json")); Document doc = docMapper.parse(SourceToParse.source("test", "person", "1", json, XContentType.JSON)).rootDoc(); - assertThat(doc.get(docMapper.uidMapper().fieldType().name()), equalTo(Uid.createUid("person", "1"))); + assertThat(doc.get(docMapper.idFieldMapper().fieldType().name()), equalTo("1")); assertThat(doc.get(docMapper.mappers().getMapper("name.first").fieldType().name()), equalTo("shay")); } @@ -994,7 +974,7 @@ public void testSimpleParserNoTypeNoId() throws Exception { DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse("person", new CompressedXContent(mapping)); BytesReference json = new BytesArray(copyToBytesFromClasspath("/org/elasticsearch/index/mapper/simple/test1-notype-noid.json")); Document doc = docMapper.parse(SourceToParse.source("test", "person", "1", json, XContentType.JSON)).rootDoc(); - assertThat(doc.get(docMapper.uidMapper().fieldType().name()), equalTo(Uid.createUid("person", "1"))); + assertThat(doc.get(docMapper.idFieldMapper().fieldType().name()), equalTo("1")); assertThat(doc.get(docMapper.mappers().getMapper("name.first").fieldType().name()), equalTo("shay")); } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java index aedd332471ae6..af5460241278a 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldMapperTests.java @@ -99,7 +99,7 @@ public void testInjectIntoDocDuringParsing() throws Exception { .bytes(), XContentType.JSON)); - assertFieldNames(set("a", "a.keyword", "b", "b.c", "_uid", "_version", "_seq_no", "_primary_term", "_source"), doc); + assertFieldNames(set("a", "a.keyword", "b", "b.c", "_id", "_version", "_seq_no", "_primary_term", "_source"), doc); } public void testExplicitEnabled() throws Exception { @@ -117,7 +117,7 @@ public void testExplicitEnabled() throws Exception { .bytes(), XContentType.JSON)); - assertFieldNames(set("field", "field.keyword", "_uid", "_version", "_seq_no", "_primary_term", "_source"), doc); + assertFieldNames(set("field", "field.keyword", "_id", "_version", "_seq_no", "_primary_term", "_source"), doc); } public void testDisabled() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/index/mapper/IdFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/IdFieldMapperTests.java index 3c12d18b12861..185f1c51d2e91 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/IdFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/IdFieldMapperTests.java @@ -19,37 +19,23 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.IndexableField; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.mapper.DocumentMapper; -import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.SourceToParse; -import org.elasticsearch.index.mapper.UidFieldMapper; +import org.elasticsearch.index.mapper.MapperService.MergeReason; import org.elasticsearch.test.ESSingleNodeTestCase; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; public class IdFieldMapperTests extends ESSingleNodeTestCase { - public void testId() throws Exception { - String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") - .endObject().endObject().string(); - DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse("type", new CompressedXContent(mapping)); - - ParsedDocument doc = docMapper.parse(SourceToParse.source("test", "type", "1", XContentFactory.jsonBuilder() - .startObject() - .endObject() - .bytes(), - XContentType.JSON)); - - assertThat(doc.rootDoc().get(UidFieldMapper.NAME), notNullValue()); - assertThat(doc.rootDoc().get(IdFieldMapper.NAME), nullValue()); - } - public void testIncludeInObjectNotAllowed() throws Exception { String mapping = XContentFactory.jsonBuilder().startObject().startObject("type").endObject().endObject().string(); DocumentMapper docMapper = createIndex("test").mapperService().documentMapperParser().parse("type", new CompressedXContent(mapping)); @@ -62,4 +48,28 @@ public void testIncludeInObjectNotAllowed() throws Exception { assertTrue(e.getMessage(), e.getMessage().contains("Field [_id] is a metadata field and cannot be added inside a document")); } } + + public void testDefaultsMultipleTypes() throws IOException { + Settings indexSettings = Settings.builder() + .put("index.mapping.single_type", false) + .build(); + MapperService mapperService = createIndex("test", indexSettings).mapperService(); + DocumentMapper mapper = mapperService.merge("type", new CompressedXContent("{\"type\":{}}"), MergeReason.MAPPING_UPDATE, false); + ParsedDocument document = mapper.parse(SourceToParse.source("index", "type", "id", new BytesArray("{}"), XContentType.JSON)); + assertEquals(Collections.emptyList(), Arrays.asList(document.rootDoc().getFields(IdFieldMapper.NAME))); + } + + public void testDefaultsSingleType() throws IOException { + Settings indexSettings = Settings.builder() + .put("index.mapping.single_type", true) + .build(); + MapperService mapperService = createIndex("test", indexSettings).mapperService(); + DocumentMapper mapper = mapperService.merge("type", new CompressedXContent("{\"type\":{}}"), MergeReason.MAPPING_UPDATE, false); + ParsedDocument document = mapper.parse(SourceToParse.source("index", "type", "id", new BytesArray("{}"), XContentType.JSON)); + IndexableField[] fields = document.rootDoc().getFields(IdFieldMapper.NAME); + assertEquals(1, fields.length); + assertEquals(IndexOptions.DOCS, fields[0].fieldType().indexOptions()); + assertTrue(fields[0].fieldType().stored()); + assertEquals("id", fields[0].stringValue()); + } } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/IdFieldTypeTests.java b/core/src/test/java/org/elasticsearch/index/mapper/IdFieldTypeTests.java index b17722eaa768d..2209027c12f37 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/IdFieldTypeTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/IdFieldTypeTests.java @@ -18,8 +18,19 @@ */ package org.elasticsearch.index.mapper; -import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MappedFieldType; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermInSetQuery; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.query.QueryShardContext; +import org.mockito.Mockito; + +import java.util.Collection; +import java.util.Collections; public class IdFieldTypeTests extends FieldTypeTestCase { @Override @@ -34,4 +45,60 @@ public void testRangeQuery() { () -> ft.rangeQuery(null, null, randomBoolean(), randomBoolean(), null)); assertEquals("Field [_id] of type [_id] does not support range queries", e.getMessage()); } + + public void testTermsQueryWhenTypesAreEnabled() throws Exception { + QueryShardContext context = Mockito.mock(QueryShardContext.class); + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) + .put("index.mapping.single_type", false).build(); + IndexMetaData indexMetaData = IndexMetaData.builder(IndexMetaData.INDEX_UUID_NA_VALUE).settings(indexSettings).build(); + IndexSettings mockSettings = new IndexSettings(indexMetaData, Settings.EMPTY); + Mockito.when(context.getIndexSettings()).thenReturn(mockSettings); + + MapperService mapperService = Mockito.mock(MapperService.class); + Collection types = Collections.emptySet(); + Mockito.when(context.queryTypes()).thenReturn(types); + Mockito.when(context.getMapperService()).thenReturn(mapperService); + + MappedFieldType ft = IdFieldMapper.defaultFieldType(mockSettings); + ft.setName(IdFieldMapper.NAME); + Query query = ft.termQuery("id", context); + assertEquals(new TermInSetQuery("_uid"), query); + + types = Collections.singleton("type"); + Mockito.when(context.queryTypes()).thenReturn(types); + query = ft.termQuery("id", context); + assertEquals(new TermInSetQuery("_uid", new BytesRef("type#id")), query); + } + + public void testTermsQueryWhenTypesAreDisabled() throws Exception { + QueryShardContext context = Mockito.mock(QueryShardContext.class); + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) + .put("index.mapping.single_type", true).build(); + IndexMetaData indexMetaData = IndexMetaData.builder(IndexMetaData.INDEX_UUID_NA_VALUE).settings(indexSettings).build(); + IndexSettings mockSettings = new IndexSettings(indexMetaData, Settings.EMPTY); + Mockito.when(context.getIndexSettings()).thenReturn(mockSettings); + + MapperService mapperService = Mockito.mock(MapperService.class); + Collection types = Collections.emptySet(); + Mockito.when(context.queryTypes()).thenReturn(types); + Mockito.when(context.getMapperService()).thenReturn(mapperService); + + MappedFieldType ft = IdFieldMapper.defaultFieldType(mockSettings); + ft.setName(IdFieldMapper.NAME); + Query query = ft.termQuery("id", context); + assertEquals(new TermInSetQuery("_id", new BytesRef("id")), query); + + types = Collections.singleton("type"); + Mockito.when(context.queryTypes()).thenReturn(types); + query = ft.termQuery("id", context); + assertEquals(new TermInSetQuery("_id", new BytesRef("id")), query); + } } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java index e2dc7bc7a094c..367f79e5980b7 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java @@ -221,7 +221,7 @@ public void testDefaultPositionIncrementGap() throws IOException { assertEquals("b", fields[1].stringValue()); IndexShard shard = indexService.getShard(0); - shard.index(new Engine.Index(new Term("_uid", doc.uid() ), doc)); + shard.index(new Engine.Index(new Term("_id", doc.id()), doc)); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); @@ -261,7 +261,7 @@ public void testPositionIncrementGap() throws IOException { assertEquals("b", fields[1].stringValue()); IndexShard shard = indexService.getShard(0); - shard.index(new Engine.Index(new Term("_uid", doc.uid()), doc)); + shard.index(new Engine.Index(new Term("_id", doc.id()), doc)); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/UidFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/UidFieldMapperTests.java new file mode 100644 index 0000000000000..e5503738f0686 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/index/mapper/UidFieldMapperTests.java @@ -0,0 +1,60 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.mapper; + +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.IndexableField; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.mapper.MapperService.MergeReason; +import org.elasticsearch.test.ESSingleNodeTestCase; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; + +public class UidFieldMapperTests extends ESSingleNodeTestCase { + + public void testDefaultsMultipleTypes() throws IOException { + Settings indexSettings = Settings.builder() + .put("index.mapping.single_type", false) + .build(); + MapperService mapperService = createIndex("test", indexSettings).mapperService(); + DocumentMapper mapper = mapperService.merge("type", new CompressedXContent("{\"type\":{}}"), MergeReason.MAPPING_UPDATE, false); + ParsedDocument document = mapper.parse(SourceToParse.source("index", "type", "id", new BytesArray("{}"), XContentType.JSON)); + IndexableField[] fields = document.rootDoc().getFields(UidFieldMapper.NAME); + assertEquals(1, fields.length); + assertEquals(IndexOptions.DOCS, fields[0].fieldType().indexOptions()); + assertTrue(fields[0].fieldType().stored()); + assertEquals("type#id", fields[0].stringValue()); + } + + public void testDefaultsSingleType() throws IOException { + Settings indexSettings = Settings.builder() + .put("index.mapping.single_type", true) + .build(); + MapperService mapperService = createIndex("test", indexSettings).mapperService(); + DocumentMapper mapper = mapperService.merge("type", new CompressedXContent("{\"type\":{}}"), MergeReason.MAPPING_UPDATE, false); + ParsedDocument document = mapper.parse(SourceToParse.source("index", "type", "id", new BytesArray("{}"), XContentType.JSON)); + assertEquals(Collections.emptyList(), Arrays.asList(document.rootDoc().getFields(UidFieldMapper.NAME))); + } +} diff --git a/core/src/test/java/org/elasticsearch/index/mapper/UidFieldTypeTests.java b/core/src/test/java/org/elasticsearch/index/mapper/UidFieldTypeTests.java index 803c7013d457a..1a9a78f51cf2e 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/UidFieldTypeTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/UidFieldTypeTests.java @@ -18,8 +18,22 @@ */ package org.elasticsearch.index.mapper; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermInSetQuery; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.UidFieldMapper; +import org.elasticsearch.index.query.QueryShardContext; +import org.mockito.Mockito; + +import java.util.Collection; +import java.util.Collections; public class UidFieldTypeTests extends FieldTypeTestCase { @Override @@ -34,4 +48,57 @@ public void testRangeQuery() { () -> ft.rangeQuery(null, null, randomBoolean(), randomBoolean(), null)); assertEquals("Field [_uid] of type [_uid] does not support range queries", e.getMessage()); } + + public void testTermsQueryWhenTypesAreEnabled() throws Exception { + QueryShardContext context = Mockito.mock(QueryShardContext.class); + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) + .put("index.mapping.single_type", false).build(); + IndexMetaData indexMetaData = IndexMetaData.builder(IndexMetaData.INDEX_UUID_NA_VALUE).settings(indexSettings).build(); + IndexSettings mockSettings = new IndexSettings(indexMetaData, Settings.EMPTY); + Mockito.when(context.getIndexSettings()).thenReturn(mockSettings); + + MapperService mapperService = Mockito.mock(MapperService.class); + Collection types = Collections.emptySet(); + Mockito.when(context.queryTypes()).thenReturn(types); + Mockito.when(context.getMapperService()).thenReturn(mapperService); + + MappedFieldType ft = UidFieldMapper.defaultFieldType(mockSettings); + ft.setName(UidFieldMapper.NAME); + Query query = ft.termQuery("type#id", context); + assertEquals(new TermInSetQuery("_uid", new BytesRef("type#id")), query); + } + + public void testTermsQueryWhenTypesAreDisabled() throws Exception { + QueryShardContext context = Mockito.mock(QueryShardContext.class); + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) + .put("index.mapping.single_type", true).build(); + IndexMetaData indexMetaData = IndexMetaData.builder(IndexMetaData.INDEX_UUID_NA_VALUE).settings(indexSettings).build(); + IndexSettings mockSettings = new IndexSettings(indexMetaData, Settings.EMPTY); + Mockito.when(context.getIndexSettings()).thenReturn(mockSettings); + + MapperService mapperService = Mockito.mock(MapperService.class); + Collection types = Collections.emptySet(); + Mockito.when(mapperService.types()).thenReturn(types); + Mockito.when(context.getMapperService()).thenReturn(mapperService); + + MappedFieldType ft = UidFieldMapper.defaultFieldType(mockSettings); + ft.setName(UidFieldMapper.NAME); + Query query = ft.termQuery("type#id", context); + assertEquals(new MatchNoDocsQuery(), query); + + types = Collections.singleton("type"); + Mockito.when(mapperService.types()).thenReturn(types); + query = ft.termQuery("type#id", context); + assertEquals(new TermInSetQuery("_id", new BytesRef("id")), query); + query = ft.termQuery("type2#id", context); + assertEquals(new TermInSetQuery("_id"), query); + } } diff --git a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 57955adbf3b49..dc518dba45238 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -51,7 +51,6 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; import org.elasticsearch.index.engine.EngineFactory; -import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -310,11 +309,11 @@ public Future asyncRecoverReplica( } public synchronized void assertAllEqual(int expectedCount) throws IOException { - Set primaryIds = getShardDocUIDs(primary); + Set primaryIds = getShardDocUIDs(primary); assertThat(primaryIds.size(), equalTo(expectedCount)); for (IndexShard replica : replicas) { - Set replicaIds = getShardDocUIDs(replica); - Set temp = new HashSet<>(primaryIds); + Set replicaIds = getShardDocUIDs(replica); + Set temp = new HashSet<>(primaryIds); temp.removeAll(replicaIds); assertThat(replica.routingEntry() + " is missing docs", temp, empty()); temp = new HashSet<>(replicaIds); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index e68ee0758fc24..bcaccabd37869 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -53,12 +53,11 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.flush.FlushStats; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesService; @@ -105,7 +104,7 @@ protected Collection> getPlugins() { private ParsedDocument testParsedDocument(String id, String type, String routing, long seqNo, ParseContext.Document document, BytesReference source, XContentType xContentType, Mapping mappingUpdate) { - Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); + Field uidField = new Field("_id", id, IdFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); document.add(uidField); @@ -335,7 +334,7 @@ public void testMaybeFlush() throws Exception { SequenceNumbersService.UNASSIGNED_SEQ_NO, new ParseContext.Document(), new BytesArray(new byte[]{1}), XContentType.JSON, null); - Engine.Index index = new Engine.Index(new Term("_uid", doc.uid()), doc); + Engine.Index index = new Engine.Index(new Term("_id", doc.id()), doc); shard.index(index); assertTrue(shard.shouldFlush()); assertEquals(2, shard.getEngine().getTranslog().totalOperations()); @@ -390,7 +389,7 @@ public void testMaybeRollTranslogGeneration() throws Exception { SequenceNumbersService.UNASSIGNED_SEQ_NO, new ParseContext.Document(), new BytesArray(new byte[]{1}), XContentType.JSON, null); - final Engine.Index index = new Engine.Index(new Term("_uid", doc.uid()), doc); + final Engine.Index index = new Engine.Index(new Term("_id", doc.id()), doc); final Engine.IndexResult result = shard.index(index); final Translog.Location location = result.getTranslogLocation(); shard.afterWriteOperation(); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index a0d356c03e79c..b8e621df8872d 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -73,14 +73,13 @@ import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.fielddata.FieldDataStats; import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceToParse; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; @@ -551,10 +550,10 @@ public void testShardStats() throws IOException { private ParsedDocument testParsedDocument(String id, String type, String routing, ParseContext.Document document, BytesReference source, Mapping mappingUpdate) { - Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); + Field idField = new Field("_id", id, IdFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); - document.add(uidField); + document.add(idField); document.add(versionField); document.add(seqID.seqNo); document.add(seqID.seqNoDocValue); @@ -624,7 +623,7 @@ public void postDelete(ShardId shardId, Engine.Delete delete, Exception ex) { ParsedDocument doc = testParsedDocument("1", "test", null, new ParseContext.Document(), new BytesArray(new byte[]{1}), null); - Engine.Index index = new Engine.Index(new Term("_uid", doc.uid()), doc); + Engine.Index index = new Engine.Index(new Term("_id", doc.id()), doc); shard.index(index); assertEquals(1, preIndex.get()); assertEquals(1, postIndexCreate.get()); @@ -643,7 +642,7 @@ public void postDelete(ShardId shardId, Engine.Delete delete, Exception ex) { assertEquals(0, postDelete.get()); assertEquals(0, postDeleteException.get()); - Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", doc.uid())); + Engine.Delete delete = new Engine.Delete("test", "1", new Term("_id", doc.id())); shard.delete(delete); assertEquals(2, preIndex.get()); @@ -1037,7 +1036,7 @@ public void testRestoreShard() throws IOException { } indexDoc(target, "test", "1"); target.refresh("test"); - assertDocs(target, new Uid("test", "1")); + assertDocs(target, "1"); flushShard(source); // only flush source final ShardRouting origRouting = target.routingEntry(); ShardRouting routing = ShardRoutingHelper.reinitPrimary(origRouting); @@ -1069,7 +1068,7 @@ public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version versio })); target.updateRoutingEntry(routing.moveToStarted()); - assertDocs(target, new Uid("test", "0")); + assertDocs(target, "0"); closeShards(source, target); } @@ -1080,7 +1079,7 @@ public void testSearcherWrapperIsUsed() throws IOException { indexDoc(shard, "test", "1", "{\"foobar\" : \"bar\"}"); shard.refresh("test"); - Engine.GetResult getResult = shard.get(new Engine.Get(false, new Term(UidFieldMapper.NAME, Uid.createUid("test", "1")))); + Engine.GetResult getResult = shard.get(new Engine.Get(false, "test", "1", new Term(IdFieldMapper.NAME, "1"))); assertTrue(getResult.exists()); assertNotNull(getResult.searcher()); getResult.release(); @@ -1113,7 +1112,7 @@ public IndexSearcher wrap(IndexSearcher searcher) throws EngineException { search = searcher.searcher().search(new TermQuery(new Term("foobar", "bar")), 10); assertEquals(search.totalHits, 1); } - getResult = newShard.get(new Engine.Get(false, new Term(UidFieldMapper.NAME, Uid.createUid("test", "1")))); + getResult = newShard.get(new Engine.Get(false, "test", "1", new Term(IdFieldMapper.NAME, "1"))); assertTrue(getResult.exists()); assertNotNull(getResult.searcher()); // make sure get uses the wrapped reader assertTrue(getResult.searcher().reader() instanceof FieldMaskingReader); @@ -1424,7 +1423,7 @@ public void testDocStats() throws IOException { testParsedDocument(id, "test", null, new ParseContext.Document(), new BytesArray("{}"), null); final Engine.Index index = new Engine.Index( - new Term("_uid", doc.uid()), + new Term("_id", doc.id()), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, @@ -1454,7 +1453,7 @@ public void testDocStats() throws IOException { testParsedDocument(id, "test", null, new ParseContext.Document(), new BytesArray("{}"), null); final Engine.Index index = new Engine.Index( - new Term("_uid", doc.uid()), + new Term("_id", doc.id()), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index 88d8a075e1b3e..91ea9c6073a4e 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.InternalEngineTests; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.test.ESTestCase; @@ -134,9 +135,9 @@ public void postDelete(ShardId shardId, Engine.Delete delete, Exception ex) { Collections.shuffle(indexingOperationListeners, random()); IndexingOperationListener.CompositeListener compositeListener = new IndexingOperationListener.CompositeListener(indexingOperationListeners, logger); - ParsedDocument doc = InternalEngineTests.createParsedDoc("1", "test", null); - Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", doc.uid())); - Engine.Index index = new Engine.Index(new Term("_uid", doc.uid()), doc); + ParsedDocument doc = InternalEngineTests.createParsedDoc("1", null); + Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", Uid.createUidAsBytes(doc.type(), doc.id()))); + Engine.Index index = new Engine.Index(new Term("_uid", Uid.createUidAsBytes(doc.type(), doc.id())), doc); compositeListener.postDelete(randomShardId, delete, new Engine.DeleteResult(1, SequenceNumbersService.UNASSIGNED_SEQ_NO, true)); assertEquals(0, preIndex.get()); assertEquals(0, postIndex.get()); diff --git a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index 5ad430ca59e24..7ddd229a1172d 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -45,11 +45,10 @@ import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.engine.InternalEngineTests.TranslogHandler; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.TranslogConfig; @@ -118,7 +117,7 @@ public void onFailedEngine(String reason, @Nullable Exception e) { // we don't need to notify anybody in this test } }; - TranslogHandler translogHandler = new TranslogHandler(xContentRegistry(), shardId.getIndexName(), logger); + TranslogHandler translogHandler = new TranslogHandler(xContentRegistry(), shardId.getIndexName(), Settings.EMPTY, logger); EngineConfig config = new EngineConfig(EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG, shardId, threadPool, indexSettings, null, store, new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()), newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, translogHandler, @@ -300,7 +299,7 @@ public void testLotsOfThreads() throws Exception { } listener.assertNoError(); - Engine.Get get = new Engine.Get(false, new Term("_uid", Uid.createUid("test", threadId))); + Engine.Get get = new Engine.Get(false, "test", threadId, new Term(IdFieldMapper.NAME, threadId)); try (Engine.GetResult getResult = engine.get(get)) { assertTrue("document not found", getResult.exists()); assertEquals(iteration, getResult.version()); @@ -327,22 +326,20 @@ private Engine.IndexResult index(String id) throws IOException { } private Engine.IndexResult index(String id, String testFieldValue) throws IOException { - String type = "test"; - String uid = type + ":" + id; Document document = new Document(); document.add(new TextField("test", testFieldValue, Field.Store.YES)); - Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); + Field idField = new Field("_id", id, IdFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", Versions.MATCH_ANY); SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); - document.add(uidField); + document.add(idField); document.add(versionField); document.add(seqID.seqNo); document.add(seqID.seqNoDocValue); document.add(seqID.primaryTerm); BytesReference source = new BytesArray(new byte[] { 1 }); - ParsedDocument doc = new ParsedDocument(versionField, seqID, id, type, null, Arrays.asList(document), source, XContentType.JSON, + ParsedDocument doc = new ParsedDocument(versionField, seqID, id, "test", null, Arrays.asList(document), source, XContentType.JSON, null); - Engine.Index index = new Engine.Index(new Term("_uid", doc.uid()), doc); + Engine.Index index = new Engine.Index(new Term("_id", doc.id()), doc); return engine.index(index); } diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index dfb8efb9ab943..8a684e2f79cde 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -255,7 +255,7 @@ public void testSimpleOperations() throws IOException { assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); assertThat(snapshot.totalOperations(), equalTo(ops.size())); - addToTranslogAndList(translog, ops, new Translog.Delete(newUid("2"))); + addToTranslogAndList(translog, ops, new Translog.Delete("test", "2", newUid("2"))); snapshot = translog.newSnapshot(); assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); assertThat(snapshot.totalOperations(), equalTo(ops.size())); @@ -326,14 +326,14 @@ public void testStats() throws IOException { assertThat(stats.getTranslogSizeInBytes(), equalTo(97L)); } - translog.add(new Translog.Delete(newUid("2"))); + translog.add(new Translog.Delete("test", "2", newUid("2"))); { final TranslogStats stats = stats(); assertThat(stats.estimatedNumberOfOperations(), equalTo(2L)); assertThat(stats.getTranslogSizeInBytes(), equalTo(139L)); } - translog.add(new Translog.Delete(newUid("3"))); + translog.add(new Translog.Delete("test", "3", newUid("3"))); { final TranslogStats stats = stats(); assertThat(stats.estimatedNumberOfOperations(), equalTo(3L)); @@ -648,7 +648,7 @@ private void corruptTranslogs(Path directory) throws Exception { } private Term newUid(ParsedDocument doc) { - return new Term("_uid", doc.uid()); + return new Term("_uid", Uid.createUidAsBytes(doc.type(), doc.id())); } private Term newUid(String uid) { @@ -707,7 +707,7 @@ public void doRun() throws BrokenBarrierException, InterruptedException, IOExcep op = new Translog.Index("type", "" + id, new byte[]{(byte) id}); break; case DELETE: - op = new Translog.Delete(newUid("" + id)); + op = new Translog.Delete("test", Long.toString(id), newUid(Long.toString(id))); break; case NO_OP: op = new Translog.NoOp(id, id, Long.toString(id)); @@ -1403,6 +1403,7 @@ public void run() { break; case DELETE: op = new Translog.Delete( + "test", threadId + "_" + opCount, new Term("_uid", threadId + "_" + opCount), opCount, 0, diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 17497af8838bc..6a059b7484a04 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -215,7 +215,7 @@ private Engine.Index getIndex(final String id) { final BytesReference source = new BytesArray(new byte[] { 1 }); final ParsedDocument doc = new ParsedDocument(versionField, seqID, id, type, null, Arrays.asList(document), source, XContentType.JSON, null); - return new Engine.Index(new Term("_uid", doc.uid()), doc); + return new Engine.Index(new Term("_uid", Uid.createUidAsBytes(doc.type(), doc.id())), doc); } public void testHandleCorruptedIndexOnSendSendFiles() throws Throwable { diff --git a/core/src/test/java/org/elasticsearch/search/SearchCancellationIT.java b/core/src/test/java/org/elasticsearch/search/SearchCancellationIT.java index 02607f0c1fd06..1a55b38f390f6 100644 --- a/core/src/test/java/org/elasticsearch/search/SearchCancellationIT.java +++ b/core/src/test/java/org/elasticsearch/search/SearchCancellationIT.java @@ -301,7 +301,7 @@ public String getName() { public class NativeTestScriptedBlock extends AbstractSearchScript { @Override public Object run() { - Loggers.getLogger(SearchCancellationIT.class).info("Blocking on the document {}", doc().get("_uid")); + Loggers.getLogger(SearchCancellationIT.class).info("Blocking on the document {}", fields().get("_uid")); hits.incrementAndGet(); try { awaitBusy(() -> shouldBlock.get() == false); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index fd36a03ddf6ce..99cd626a7d7e1 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -96,6 +96,7 @@ protected AggregatorFactory createAggregatorFactory(AggregationBuilder aggreg when(searchContext.bigArrays()).thenReturn(new MockBigArrays(Settings.EMPTY, circuitBreakerService)); // TODO: now just needed for top_hits, this will need to be revised for other agg unit tests: MapperService mapperService = mapperServiceMock(); + when(mapperService.getIndexSettings()).thenReturn(indexSettings); when(mapperService.hasNested()).thenReturn(false); when(searchContext.mapperService()).thenReturn(mapperService); IndexFieldDataService ifds = new IndexFieldDataService(indexSettings, @@ -106,7 +107,7 @@ protected AggregatorFactory createAggregatorFactory(AggregationBuilder aggreg SearchLookup searchLookup = new SearchLookup(mapperService, ifds, new String[]{"type"}); when(searchContext.lookup()).thenReturn(searchLookup); - QueryShardContext queryShardContext = queryShardContextMock(fieldTypes, indexSettings, circuitBreakerService); + QueryShardContext queryShardContext = queryShardContextMock(mapperService, fieldTypes, circuitBreakerService); when(searchContext.getQueryShardContext()).thenReturn(queryShardContext); return aggregationBuilder.build(searchContext, null); @@ -180,13 +181,15 @@ protected MapperService mapperServiceMock() { /** * sub-tests that need a more complex mock can overwrite this */ - protected QueryShardContext queryShardContextMock(MappedFieldType[] fieldTypes, IndexSettings indexSettings, + protected QueryShardContext queryShardContextMock(MapperService mapperService, MappedFieldType[] fieldTypes, CircuitBreakerService circuitBreakerService) { QueryShardContext queryShardContext = mock(QueryShardContext.class); + when(queryShardContext.getMapperService()).thenReturn(mapperService); for (MappedFieldType fieldType : fieldTypes) { when(queryShardContext.fieldMapper(fieldType.name())).thenReturn(fieldType); - when(queryShardContext.getForField(fieldType)).then(invocation -> fieldType.fielddataBuilder().build(indexSettings, fieldType, - new IndexFieldDataCache.None(), circuitBreakerService, mock(MapperService.class))); + when(queryShardContext.getForField(fieldType)).then(invocation -> fieldType.fielddataBuilder().build( + mapperService.getIndexSettings(), fieldType, new IndexFieldDataCache.None(), circuitBreakerService, + mapperService)); } NestedScope nestedScope = new NestedScope(); when(queryShardContext.isFilter()).thenCallRealMethod(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java index e9b9ae20407da..6819fddf3e3e5 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java @@ -667,7 +667,7 @@ public void testTrackScores() throws Exception { topHits("hits") .trackScores(trackScore) .size(1) - .sort("_uid", SortOrder.DESC) + .sort("_index", SortOrder.DESC) ) ) .get(); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java index aa48c64c3d788..7c34ef0549694 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregatorTests.java @@ -28,8 +28,8 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.script.MockScriptEngine; @@ -196,7 +196,7 @@ public void testScriptedMetricWithCombineAccessesScores() throws IOException { * is final and cannot be mocked */ @Override - protected QueryShardContext queryShardContextMock(final MappedFieldType[] fieldTypes, IndexSettings idxSettings, + protected QueryShardContext queryShardContextMock(MapperService mapperService, final MappedFieldType[] fieldTypes, CircuitBreakerService circuitBreakerService) { Settings settings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) // no file watching, so we don't need a ResourceWatcherService @@ -212,7 +212,7 @@ protected QueryShardContext queryShardContextMock(final MappedFieldType[] fieldT } catch (IOException e) { throw new ElasticsearchException(e); } - return new QueryShardContext(0, idxSettings, null, null, null, null, scriptService, xContentRegistry(), - null, null, System::currentTimeMillis); + return new QueryShardContext(0, mapperService.getIndexSettings(), null, null, mapperService, null, scriptService, + xContentRegistry(), null, null, System::currentTimeMillis); } } diff --git a/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java b/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java index f93fbabdf2674..6fbda92ba2699 100644 --- a/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java +++ b/core/src/test/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java @@ -64,6 +64,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllSuccessful; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHit; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.hasId; @@ -191,7 +192,9 @@ public void testRandomNested() throws Exception { int[] field2InnerObjects = new int[numDocs]; for (int i = 0; i < numDocs; i++) { int numInnerObjects = field1InnerObjects[i] = scaledRandomIntBetween(1, numDocs); - XContentBuilder source = jsonBuilder().startObject().startArray("field1"); + XContentBuilder source = jsonBuilder().startObject() + .field("foo", i) + .startArray("field1"); for (int j = 0; j < numInnerObjects; j++) { source.startObject().field("x", "y").endObject(); } @@ -201,7 +204,7 @@ public void testRandomNested() throws Exception { source.startObject().field("x", "y").endObject(); } source.endArray().endObject(); - requestBuilders.add(client().prepareIndex("idx", "type", String.format(Locale.ENGLISH, "%03d", i)).setSource(source)); + requestBuilders.add(client().prepareIndex("idx", "type", Integer.toString(i)).setSource(source)); } indexRandom(true, requestBuilders); @@ -214,7 +217,7 @@ public void testRandomNested() throws Exception { SearchResponse searchResponse = client().prepareSearch("idx") .setQuery(boolQuery) .setSize(numDocs) - .addSort("_uid", SortOrder.ASC) + .addSort("foo", SortOrder.ASC) .get(); assertNoFailures(searchResponse); @@ -965,7 +968,6 @@ public void testDontExplode() throws Exception { .innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1), false); SearchResponse response = client().prepareSearch("index1") .setQuery(query) - .addSort("_uid", SortOrder.ASC) .get(); assertNoFailures(response); assertHitCount(response, 1); @@ -985,7 +987,6 @@ public void testDontExplode() throws Exception { .innerHit(new InnerHitBuilder().setSize(ArrayUtil.MAX_ARRAY_LENGTH - 1), false); response = client().prepareSearch("index2") .setQuery(query) - .addSort("_uid", SortOrder.ASC) .get(); assertNoFailures(response); assertHitCount(response, 1); @@ -1057,11 +1058,10 @@ public void testInnerHitsWithIgnoreUnmapped() throws Exception { .innerHit(new InnerHitBuilder(), true)) .should(termQuery("key", "value")) ) - .addSort("_uid", SortOrder.ASC) .get(); assertNoFailures(response); assertHitCount(response, 2); - assertThat(response.getHits().getAt(0).getId(), equalTo("1")); + assertSearchHits(response, "1", "3"); response = client().prepareSearch("index1", "index2") .setQuery(boolQuery() @@ -1069,11 +1069,10 @@ public void testInnerHitsWithIgnoreUnmapped() throws Exception { .innerHit(new InnerHitBuilder(), true)) .should(termQuery("key", "value")) ) - .addSort("_uid", SortOrder.ASC) .get(); assertNoFailures(response); assertHitCount(response, 2); - assertThat(response.getHits().getAt(0).getId(), equalTo("1")); + assertSearchHits(response, "1", "3"); } } diff --git a/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java index f20c575f0be8e..d5198485351b1 100644 --- a/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java +++ b/core/src/test/java/org/elasticsearch/search/profile/query/QueryProfilerIT.java @@ -133,14 +133,14 @@ public void testProfileMatchesRegular() throws Exception { SearchRequestBuilder vanilla = client().prepareSearch("test") .setQuery(q) .setProfile(false) - .addSort("_uid", SortOrder.ASC) + .addSort("_id", SortOrder.ASC) .setPreference("_primary") .setSearchType(SearchType.QUERY_THEN_FETCH); SearchRequestBuilder profile = client().prepareSearch("test") .setQuery(q) .setProfile(true) - .addSort("_uid", SortOrder.ASC) + .addSort("_id", SortOrder.ASC) .setPreference("_primary") .setSearchType(SearchType.QUERY_THEN_FETCH); diff --git a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java index 7ae22485df3d4..ba7a13cf0102e 100644 --- a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java @@ -264,19 +264,19 @@ public void testSingleField() throws NoSuchFieldException, IllegalAccessExceptio } MultiMatchQueryBuilder multiMatchQueryBuilder = randomizeType(multiMatchQuery(builder.toString(), field)); SearchResponse multiMatchResp = client().prepareSearch("test") - // _uid sort field is a tie, in case hits have the same score, + // _id sort field is a tie, in case hits have the same score, // the hits will be sorted the same consistently .addSort("_score", SortOrder.DESC) - .addSort("_uid", SortOrder.ASC) + .addSort("_id", SortOrder.ASC) .setQuery(multiMatchQueryBuilder).get(); MatchQueryBuilder matchQueryBuilder = QueryBuilders.matchQuery(field, builder.toString()); if (multiMatchQueryBuilder.getType() != null) { matchQueryBuilder.type(MatchQuery.Type.valueOf(multiMatchQueryBuilder.getType().matchQueryType().toString())); } SearchResponse matchResp = client().prepareSearch("test") - // _uid tie sort + // _id tie sort .addSort("_score", SortOrder.DESC) - .addSort("_uid", SortOrder.ASC) + .addSort("_id", SortOrder.ASC) .setQuery(matchQueryBuilder).get(); assertThat("field: " + field + " query: " + builder.toString(), multiMatchResp.getHits().getTotalHits(), equalTo(matchResp.getHits().getTotalHits())); SearchHits hits = multiMatchResp.getHits(); @@ -361,12 +361,12 @@ public void testEquivalence() { MultiMatchQueryBuilder multiMatchQueryBuilder = randomBoolean() ? multiMatchQuery("marvel hero captain america", "full_name", "first_name", "last_name", "category") : multiMatchQuery("marvel hero captain america", "*_name", randomBoolean() ? "category" : "categ*"); SearchResponse left = client().prepareSearch("test").setSize(numDocs) - .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) + .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_id")) .setQuery(randomizeType(multiMatchQueryBuilder .operator(Operator.OR).type(type))).get(); SearchResponse right = client().prepareSearch("test").setSize(numDocs) - .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) + .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_id")) .setQuery(disMaxQuery(). add(matchQuery("full_name", "marvel hero captain america")) .add(matchQuery("first_name", "marvel hero captain america")) @@ -383,12 +383,12 @@ public void testEquivalence() { MultiMatchQueryBuilder multiMatchQueryBuilder = randomBoolean() ? multiMatchQuery("captain america", "full_name", "first_name", "last_name", "category") : multiMatchQuery("captain america", "*_name", randomBoolean() ? "category" : "categ*"); SearchResponse left = client().prepareSearch("test").setSize(numDocs) - .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) + .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_id")) .setQuery(randomizeType(multiMatchQueryBuilder .operator(op).useDisMax(false).minimumShouldMatch(minShouldMatch).type(type))).get(); SearchResponse right = client().prepareSearch("test").setSize(numDocs) - .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) + .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_id")) .setQuery(boolQuery().minimumShouldMatch(minShouldMatch) .should(randomBoolean() ? termQuery("full_name", "captain america") : matchQuery("full_name", "captain america").operator(op)) .should(matchQuery("first_name", "captain america").operator(op)) @@ -401,12 +401,12 @@ public void testEquivalence() { { String minShouldMatch = randomBoolean() ? null : "" + between(0, 1); SearchResponse left = client().prepareSearch("test").setSize(numDocs) - .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) + .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_id")) .setQuery(randomizeType(multiMatchQuery("capta", "full_name", "first_name", "last_name", "category") .type(MatchQuery.Type.PHRASE_PREFIX).useDisMax(false).minimumShouldMatch(minShouldMatch))).get(); SearchResponse right = client().prepareSearch("test").setSize(numDocs) - .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) + .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_id")) .setQuery(boolQuery().minimumShouldMatch(minShouldMatch) .should(matchPhrasePrefixQuery("full_name", "capta")) .should(matchPhrasePrefixQuery("first_name", "capta")) @@ -420,17 +420,17 @@ public void testEquivalence() { SearchResponse left; if (randomBoolean()) { left = client().prepareSearch("test").setSize(numDocs) - .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) + .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_id")) .setQuery(randomizeType(multiMatchQuery("captain america", "full_name", "first_name", "last_name", "category") .type(MatchQuery.Type.PHRASE).useDisMax(false).minimumShouldMatch(minShouldMatch))).get(); } else { left = client().prepareSearch("test").setSize(numDocs) - .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) + .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_id")) .setQuery(randomizeType(multiMatchQuery("captain america", "full_name", "first_name", "last_name", "category") .type(MatchQuery.Type.PHRASE).tieBreaker(1.0f).minimumShouldMatch(minShouldMatch))).get(); } SearchResponse right = client().prepareSearch("test").setSize(numDocs) - .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) + .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_id")) .setQuery(boolQuery().minimumShouldMatch(minShouldMatch) .should(matchPhraseQuery("full_name", "captain america")) .should(matchPhraseQuery("first_name", "captain america")) diff --git a/core/src/test/java/org/elasticsearch/search/slice/SliceBuilderTests.java b/core/src/test/java/org/elasticsearch/search/slice/SliceBuilderTests.java index f7ba4286c2b3a..33c4f502b7033 100644 --- a/core/src/test/java/org/elasticsearch/search/slice/SliceBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/search/slice/SliceBuilderTests.java @@ -29,12 +29,16 @@ import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; import org.apache.lucene.store.RAMDirectory; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.fielddata.IndexNumericFieldData; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.UidFieldMapper; @@ -156,6 +160,14 @@ public Query termQuery(Object value, @Nullable QueryShardContext context) { fieldType.setHasDocValues(false); when(context.fieldMapper(UidFieldMapper.NAME)).thenReturn(fieldType); when(context.getIndexReader()).thenReturn(reader); + Settings settings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .build(); + IndexMetaData indexState = IndexMetaData.builder("index").settings(settings).build(); + IndexSettings indexSettings = new IndexSettings(indexState, Settings.EMPTY); + when(context.getIndexSettings()).thenReturn(indexSettings); SliceBuilder builder = new SliceBuilder(5, 10); Query query = builder.toFilter(context, 0, 1); assertThat(query, instanceOf(TermsSliceQuery.class)); diff --git a/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java b/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java index 4bddc1c556adc..91d6686714fb6 100644 --- a/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java +++ b/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java @@ -1356,7 +1356,9 @@ public void testSortMetaField() throws Exception { SearchHit[] hits = searchResponse.getHits().getHits(); BytesRef previous = order == SortOrder.ASC ? new BytesRef() : UnicodeUtil.BIG_TERM; for (int i = 0; i < hits.length; ++i) { - final BytesRef uid = new BytesRef(Uid.createUid(hits[i].getType(), hits[i].getId())); + String uidString = Uid.createUid(hits[i].getType(), hits[i].getId()); + final BytesRef uid = new BytesRef(uidString); + assertEquals(uidString, hits[i].getSortValues()[0]); assertThat(previous, order == SortOrder.ASC ? lessThan(uid) : greaterThan(uid)); previous = uid; } diff --git a/docs/reference/mapping/fields/id-field.asciidoc b/docs/reference/mapping/fields/id-field.asciidoc index 94c95a8bbeba5..82447fc44297d 100644 --- a/docs/reference/mapping/fields/id-field.asciidoc +++ b/docs/reference/mapping/fields/id-field.asciidoc @@ -7,9 +7,7 @@ indexed as its value can be derived automatically from the <> field. The value of the `_id` field is accessible in certain queries (`term`, -`terms`, `match`, `query_string`, `simple_query_string`), but -_not_ in aggregations, scripts or when sorting, where the <> -field should be used instead: +`terms`, `match`, `query_string`, `simple_query_string`). [source,js] -------------------------- @@ -36,3 +34,9 @@ GET my_index/_search // CONSOLE <1> Querying on the `_id` field (also see the <>) + +The value of the `_id` field is also accessible in aggregations or for sorting, +but doing so is discouraged as it requires to load a lot of data in memory. In +case sorting or aggregating on the `_id` field is required, it is advised to +duplicate the content of the `_id` field in another field that has `doc_values` +enabled. diff --git a/docs/reference/mapping/fields/uid-field.asciidoc b/docs/reference/mapping/fields/uid-field.asciidoc index bad9eb4768b47..a1a3e8d14bf2a 100644 --- a/docs/reference/mapping/fields/uid-field.asciidoc +++ b/docs/reference/mapping/fields/uid-field.asciidoc @@ -1,6 +1,10 @@ [[mapping-uid-field]] === `_uid` field +deprecated[6.0.0, Now that types have been removed, documents are uniquely +identified by their `_id` and the `_uid` field has only been kept as a view +over the `_id` field for backward compatibility.] + Each document indexed is associated with a <> (see <>) and an <>. These values are combined as `{type}#{id}` and indexed as the `_uid` field. @@ -20,7 +24,11 @@ PUT my_index/my_type/2?refresh=true { "text": "Document with ID 2" } +-------------------------- +// CONSOLE +[source,js] +-------------------------- GET my_index/_search { "query": { @@ -54,6 +62,8 @@ GET my_index/_search } -------------------------- // CONSOLE +// TEST[continued] +// TEST[warning:Fielddata access on the _uid field is deprecated, use _id instead] <1> Querying on the `_uid` field (also see the <>) <2> Aggregating on the `_uid` field diff --git a/docs/reference/migration/migrate_6_0/rest.asciidoc b/docs/reference/migration/migrate_6_0/rest.asciidoc index cfd8a9511a2ee..d7e55f0942fe9 100644 --- a/docs/reference/migration/migrate_6_0/rest.asciidoc +++ b/docs/reference/migration/migrate_6_0/rest.asciidoc @@ -53,3 +53,8 @@ response in the event there is at least one failure. In previous versions of Elasticsearch, delete by query requests without an explicit query were accepted, match_all was used as the default query and all documents were deleted as a result. From version 6.0.0, delete by query requests require an explicit query. + +=== DELETE document calls now implicitly create the type + +Running `DELETE index/type/id` now implicitly creates `type` with a default +mapping if it did not exist yet. diff --git a/docs/reference/migration/migrate_6_0/search.asciidoc b/docs/reference/migration/migrate_6_0/search.asciidoc index 7b080eeb3be83..80d67eae72d47 100644 --- a/docs/reference/migration/migrate_6_0/search.asciidoc +++ b/docs/reference/migration/migrate_6_0/search.asciidoc @@ -88,3 +88,11 @@ produces. BM25 is recommended instead. See https://issues.apache.org/jira/browse/LUCENE-7347[`LUCENE-7347`] for more information. + +==== Fielddata on _uid + +Fielddata on `_uid` is deprecated. It is possible to switch to `_id` instead +but the only reason why it has not been deprecated too is because it is used +for the `random_score` function. If you really need access to the id of +documents for sorting, aggregations or search scripts, the recommandation is +to duplicate the id as a field in the document. diff --git a/docs/reference/search/request/search-after.asciidoc b/docs/reference/search/request/search-after.asciidoc index bc94af935c2e3..87f5abba8b2e3 100644 --- a/docs/reference/search/request/search-after.asciidoc +++ b/docs/reference/search/request/search-after.asciidoc @@ -21,7 +21,7 @@ GET twitter/tweet/_search }, "sort": [ {"date": "asc"}, - {"_uid": "desc"} + {"_id": "desc"} ] } -------------------------------------------------- @@ -30,7 +30,7 @@ GET twitter/tweet/_search NOTE: A field with one unique value per document should be used as the tiebreaker of the sort specification. Otherwise the sort order for documents that have the same sort values would be undefined. The recommended way is to use -the field `_uid` which is certain to contain one unique value for each document. +the field `_id` which is certain to contain one unique value for each document. The result from the above request includes an array of `sort values` for each document. These `sort values` can be used in conjunction with the `search_after` parameter to start returning results "after" any @@ -47,10 +47,10 @@ GET twitter/tweet/_search "title" : "elasticsearch" } }, - "search_after": [1463538857, "tweet#654323"], + "search_after": [1463538857, "654323"], "sort": [ {"date": "asc"}, - {"_uid": "desc"} + {"_id": "desc"} ] } -------------------------------------------------- diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yaml index 8135d25399c1b..920f55ae1be5d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yaml @@ -30,6 +30,9 @@ setup: --- "search with search_after parameter": + - skip: + version: " - 5.99.99" + reason: fielddata on _id is only available as of 6.0.0 - do: search: index: test @@ -39,14 +42,14 @@ setup: query: match: foo: bar - sort: [{ age: desc }, { _uid: desc }] + sort: [{ age: desc }, { _id: desc }] - match: {hits.total: 3 } - length: {hits.hits: 1 } - match: {hits.hits.0._index: test } - match: {hits.hits.0._type: test } - match: {hits.hits.0._id: "172" } - - match: {hits.hits.0.sort: [24, "test#172"] } + - match: {hits.hits.0.sort: [24, "172"] } - do: search: @@ -57,15 +60,15 @@ setup: query: match: foo: bar - sort: [{ age: desc }, { _uid: desc }] - search_after: [24, "test#172"] + sort: [{ age: desc }, { _id: desc }] + search_after: [24, "172"] - match: {hits.total: 3 } - length: {hits.hits: 1 } - match: {hits.hits.0._index: test } - match: {hits.hits.0._type: test } - match: {hits.hits.0._id: "42" } - - match: {hits.hits.0.sort: [18, "test#42"] } + - match: {hits.hits.0.sort: [18, "42"] } - do: search: @@ -76,15 +79,15 @@ setup: query: match: foo: bar - sort: [ { age: desc }, { _uid: desc } ] - search_after: [18, "test#42"] + sort: [ { age: desc }, { _id: desc } ] + search_after: [18, "42"] - match: {hits.total: 3} - length: {hits.hits: 1 } - match: {hits.hits.0._index: test } - match: {hits.hits.0._type: test } - match: {hits.hits.0._id: "1" } - - match: {hits.hits.0.sort: [18, "test#1"] } + - match: {hits.hits.0.sort: [18, "1"] } - do: search: @@ -95,8 +98,8 @@ setup: query: match: foo: bar - sort: [{ age: desc }, { _uid: desc } ] - search_after: [18, "test#1"] + sort: [{ age: desc }, { _id: desc } ] + search_after: [18, "1"] - match: {hits.total: 3} - length: {hits.hits: 0 } diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 80a0e9486f1ae..f0c4a9c44d0a5 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -52,10 +52,9 @@ import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.SourceToParse; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.DirectoryService; @@ -444,17 +443,17 @@ private Store.MetadataSnapshot getMetadataSnapshotOrEmpty(IndexShard replica) th return result; } - protected Set getShardDocUIDs(final IndexShard shard) throws IOException { + protected Set getShardDocUIDs(final IndexShard shard) throws IOException { shard.refresh("get_uids"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { - Set ids = new HashSet<>(); + Set ids = new HashSet<>(); for (LeafReaderContext leafContext : searcher.reader().leaves()) { LeafReader reader = leafContext.reader(); Bits liveDocs = reader.getLiveDocs(); for (int i = 0; i < reader.maxDoc(); i++) { if (liveDocs == null || liveDocs.get(i)) { - Document uuid = reader.document(i, Collections.singleton(UidFieldMapper.NAME)); - ids.add(Uid.createUid(uuid.get(UidFieldMapper.NAME))); + Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); + ids.add(uuid.get(IdFieldMapper.NAME)); } } } @@ -466,10 +465,10 @@ protected void assertDocCount(IndexShard shard, int docDount) throws IOException assertThat(getShardDocUIDs(shard), hasSize(docDount)); } - protected void assertDocs(IndexShard shard, Uid... uids) throws IOException { - final Set shardDocUIDs = getShardDocUIDs(shard); - assertThat(shardDocUIDs, contains(uids)); - assertThat(shardDocUIDs, hasSize(uids.length)); + protected void assertDocs(IndexShard shard, String... ids) throws IOException { + final Set shardDocUIDs = getShardDocUIDs(shard); + assertThat(shardDocUIDs, contains(ids)); + assertThat(shardDocUIDs, hasSize(ids.length)); }