diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java index 9c8979601e8..e6d637aabb1 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java @@ -77,7 +77,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; @@ -88,7 +87,6 @@ import org.elasticsearch.common.geo.ShapeRelation; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperService; @@ -1111,11 +1109,7 @@ public class CandidateQueryTests extends ESSingleNodeTestCase { } private void addQuery(Query query, List docs) { - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(query, parseContext); ParseContext.Document queryDocument = parseContext.doc(); diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java index 80524a2f862..ecff48b344c 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java @@ -42,7 +42,6 @@ import org.apache.lucene.search.join.ScoreMode; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.support.PlainActionFuture; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -59,7 +58,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperParser; import org.elasticsearch.index.mapper.MapperParsingException; @@ -184,11 +182,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -210,7 +204,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { bq.add(termQuery1, Occur.MUST); bq.add(termQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), + parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -238,12 +232,8 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { bq.add(rangeQuery2, Occur.MUST); DocumentMapper documentMapper = mapperService.documentMapper("doc"); - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -269,7 +259,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { .rangeQuery(15, 20, true, true, null, null, null, null); bq.add(rangeQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(settings, + parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -293,11 +283,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { TermRangeQuery query = new TermRangeQuery("field1", new BytesRef("a"), new BytesRef("z"), true, true); DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(query, parseContext); ParseContext.Document document = parseContext.doc(); @@ -312,11 +298,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase { PhraseQuery phraseQuery = new PhraseQuery("field", "term"); DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(phraseQuery, parseContext); ParseContext.Document document = parseContext.doc(); diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index 1c1e5687893..a24a6aea07f 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -27,10 +27,8 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.document.LatLonDocValuesField; -import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.FilterLeafReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFileNames; @@ -98,8 +96,6 @@ public class Lucene { assert annotation == null : "DocValuesFormat " + LATEST_DOC_VALUES_FORMAT + " is deprecated" ; } - public static final String SOFT_DELETES_FIELD = "__soft_deletes"; - public static final NamedAnalyzer STANDARD_ANALYZER = new NamedAnalyzer("_standard", AnalyzerScope.GLOBAL, new StandardAnalyzer()); public static final NamedAnalyzer KEYWORD_ANALYZER = new NamedAnalyzer("_keyword", AnalyzerScope.GLOBAL, new KeywordAnalyzer()); @@ -144,7 +140,7 @@ public class Lucene { public static int getNumDocs(SegmentInfos info) { int numDocs = 0; for (SegmentCommitInfo si : info) { - numDocs += si.info.maxDoc() - si.getDelCount() - si.getSoftDelCount(); + numDocs += si.info.maxDoc() - si.getDelCount(); } return numDocs; } @@ -201,7 +197,6 @@ public class Lucene { } final CommitPoint cp = new CommitPoint(si, directory); try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setIndexCommit(cp) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) @@ -225,7 +220,6 @@ public class Lucene { } } try (IndexWriter writer = new IndexWriter(directory, new IndexWriterConfig(Lucene.STANDARD_ANALYZER) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setMergePolicy(NoMergePolicy.INSTANCE) // no merges .setCommitOnClose(false) // no commits .setOpenMode(IndexWriterConfig.OpenMode.CREATE))) // force creation - don't append... @@ -835,82 +829,4 @@ public class Lucene { } }; } - - /** - * Wraps a directory reader to make all documents live except those were rolled back - * or hard-deleted due to non-aborting exceptions during indexing. - * The wrapped reader can be used to query all documents. - * - * @param in the input directory reader - * @return the wrapped reader - */ - public static DirectoryReader wrapAllDocsLive(DirectoryReader in) throws IOException { - return new DirectoryReaderWithAllLiveDocs(in); - } - - private static final class DirectoryReaderWithAllLiveDocs extends FilterDirectoryReader { - static final class LeafReaderWithLiveDocs extends FilterLeafReader { - final Bits liveDocs; - final int numDocs; - LeafReaderWithLiveDocs(LeafReader in, Bits liveDocs, int numDocs) { - super(in); - this.liveDocs = liveDocs; - this.numDocs = numDocs; - } - @Override - public Bits getLiveDocs() { - return liveDocs; - } - @Override - public int numDocs() { - return numDocs; - } - @Override - public CacheHelper getCoreCacheHelper() { - return in.getCoreCacheHelper(); - } - @Override - public CacheHelper getReaderCacheHelper() { - return null; // Modifying liveDocs - } - } - - DirectoryReaderWithAllLiveDocs(DirectoryReader in) throws IOException { - super(in, new SubReaderWrapper() { - @Override - public LeafReader wrap(LeafReader leaf) { - SegmentReader segmentReader = segmentReader(leaf); - Bits hardLiveDocs = segmentReader.getHardLiveDocs(); - if (hardLiveDocs == null) { - return new LeafReaderWithLiveDocs(leaf, null, leaf.maxDoc()); - } - // TODO: Can we avoid calculate numDocs by using SegmentReader#getSegmentInfo with LUCENE-8458? - int numDocs = 0; - for (int i = 0; i < hardLiveDocs.length(); i++) { - if (hardLiveDocs.get(i)) { - numDocs++; - } - } - return new LeafReaderWithLiveDocs(segmentReader, hardLiveDocs, numDocs); - } - }); - } - - @Override - protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { - return wrapAllDocsLive(in); - } - - @Override - public CacheHelper getReaderCacheHelper() { - return null; // Modifying liveDocs - } - } - - /** - * Returns a numeric docvalues which can be used to soft-delete documents. - */ - public static NumericDocValuesField newSoftDeletesField() { - return new NumericDocValuesField(SOFT_DELETES_FIELD, 1); - } } diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java index 3a037bed62b..38fcdfe5f1b 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java @@ -28,7 +28,6 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -67,22 +66,15 @@ final class PerThreadIDVersionAndSeqNoLookup { */ PerThreadIDVersionAndSeqNoLookup(LeafReader reader, String uidField) throws IOException { this.uidField = uidField; - final Terms terms = reader.terms(uidField); + Terms terms = reader.terms(uidField); if (terms == null) { - // If a segment contains only no-ops, it does not have _uid but has both _soft_deletes and _tombstone fields. - final NumericDocValues softDeletesDV = reader.getNumericDocValues(Lucene.SOFT_DELETES_FIELD); - final NumericDocValues tombstoneDV = reader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); - if (softDeletesDV == null || tombstoneDV == null) { - throw new IllegalArgumentException("reader does not have _uid terms but not a no-op segment; " + - "_soft_deletes [" + softDeletesDV + "], _tombstone [" + tombstoneDV + "]"); - } - termsEnum = null; - } else { - termsEnum = terms.iterator(); + 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; _uid terms [" + terms + "]"); + throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field"); } + Object readerKey = null; assert (readerKey = reader.getCoreCacheHelper().getKey()) != null; this.readerKey = readerKey; @@ -119,8 +111,7 @@ final class PerThreadIDVersionAndSeqNoLookup { * {@link DocIdSetIterator#NO_MORE_DOCS} is returned if not found * */ private int getDocID(BytesRef id, Bits liveDocs) throws IOException { - // termsEnum can possibly be null here if this leaf contains only no-ops. - if (termsEnum != null && termsEnum.seekExact(id)) { + if (termsEnum.seekExact(id)) { int docID = DocIdSetIterator.NO_MORE_DOCS; // there may be more than one matching docID, in the case of nested docs, so we want the last one: docsEnum = termsEnum.postings(docsEnum, 0); diff --git a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index f3de294046c..46e3867f7ae 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -129,8 +129,6 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.MAX_REGEX_LENGTH_SETTING, ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, IndexSettings.INDEX_GC_DELETES_SETTING, - IndexSettings.INDEX_SOFT_DELETES_SETTING, - IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java index 3ea022bbebd..44cd743bbd4 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -237,21 +237,6 @@ public final class IndexSettings { public static final Setting INDEX_GC_DELETES_SETTING = Setting.timeSetting("index.gc_deletes", DEFAULT_GC_DELETES, new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic, Property.IndexScope); - - /** - * Specifies if the index should use soft-delete instead of hard-delete for update/delete operations. - */ - public static final Setting INDEX_SOFT_DELETES_SETTING = - Setting.boolSetting("index.soft_deletes.enabled", false, Property.IndexScope, Property.Final); - - /** - * Controls how many soft-deleted documents will be kept around before being merged away. Keeping more deleted - * documents increases the chance of operation-based recoveries and allows querying a longer history of documents. - * If soft-deletes is enabled, an engine by default will retain all operations up to the global checkpoint. - **/ - public static final Setting INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING = - Setting.longSetting("index.soft_deletes.retention.operations", 0, 0, Property.IndexScope, Property.Dynamic); - /** * The maximum number of refresh listeners allows on this shard. */ @@ -304,8 +289,6 @@ public final class IndexSettings { private final IndexSortConfig indexSortConfig; private final IndexScopedSettings scopedSettings; private long gcDeletesInMillis = DEFAULT_GC_DELETES.millis(); - private final boolean softDeleteEnabled; - private volatile long softDeleteRetentionOperations; private volatile boolean warmerEnabled; private volatile int maxResultWindow; private volatile int maxInnerResultWindow; @@ -417,8 +400,6 @@ public final class IndexSettings { generationThresholdSize = scopedSettings.get(INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING); mergeSchedulerConfig = new MergeSchedulerConfig(this); gcDeletesInMillis = scopedSettings.get(INDEX_GC_DELETES_SETTING).getMillis(); - softDeleteEnabled = version.onOrAfter(Version.V_7_0_0_alpha1) && scopedSettings.get(INDEX_SOFT_DELETES_SETTING); - softDeleteRetentionOperations = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); warmerEnabled = scopedSettings.get(INDEX_WARMER_ENABLED_SETTING); maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING); maxInnerResultWindow = scopedSettings.get(MAX_INNER_RESULT_WINDOW_SETTING); @@ -477,7 +458,6 @@ public final class IndexSettings { scopedSettings.addSettingsUpdateConsumer(INDEX_SEARCH_IDLE_AFTER, this::setSearchIdleAfter); scopedSettings.addSettingsUpdateConsumer(MAX_REGEX_LENGTH_SETTING, this::setMaxRegexLength); scopedSettings.addSettingsUpdateConsumer(DEFAULT_PIPELINE, this::setDefaultPipeline); - scopedSettings.addSettingsUpdateConsumer(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, this::setSoftDeleteRetentionOperations); } private void setSearchIdleAfter(TimeValue searchIdleAfter) { this.searchIdleAfter = searchIdleAfter; } @@ -861,22 +841,4 @@ public final class IndexSettings { public void setDefaultPipeline(String defaultPipeline) { this.defaultPipeline = defaultPipeline; } - - /** - * Returns true if soft-delete is enabled. - */ - public boolean isSoftDeleteEnabled() { - return softDeleteEnabled; - } - - private void setSoftDeleteRetentionOperations(long ops) { - this.softDeleteRetentionOperations = ops; - } - - /** - * Returns the number of extra operations (i.e. soft-deleted documents) to be kept for recoveries and history purpose. - */ - public long getSoftDeleteRetentionOperations() { - return this.softDeleteRetentionOperations; - } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index d10690379ed..d0575c8a8c9 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -46,17 +46,14 @@ import java.util.function.LongSupplier; public final class CombinedDeletionPolicy extends IndexDeletionPolicy { private final Logger logger; private final TranslogDeletionPolicy translogDeletionPolicy; - private final SoftDeletesPolicy softDeletesPolicy; private final LongSupplier globalCheckpointSupplier; private final ObjectIntHashMap snapshottedCommits; // Number of snapshots held against each commit point. private volatile IndexCommit safeCommit; // the most recent safe commit point - its max_seqno at most the persisted global checkpoint. private volatile IndexCommit lastCommit; // the most recent commit point - CombinedDeletionPolicy(Logger logger, TranslogDeletionPolicy translogDeletionPolicy, - SoftDeletesPolicy softDeletesPolicy, LongSupplier globalCheckpointSupplier) { + CombinedDeletionPolicy(Logger logger, TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier) { this.logger = logger; this.translogDeletionPolicy = translogDeletionPolicy; - this.softDeletesPolicy = softDeletesPolicy; this.globalCheckpointSupplier = globalCheckpointSupplier; this.snapshottedCommits = new ObjectIntHashMap<>(); } @@ -83,7 +80,7 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { deleteCommit(commits.get(i)); } } - updateRetentionPolicy(); + updateTranslogDeletionPolicy(); } private void deleteCommit(IndexCommit commit) throws IOException { @@ -93,7 +90,7 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { assert commit.isDeleted() : "Deletion commit [" + commitDescription(commit) + "] was suppressed"; } - private void updateRetentionPolicy() throws IOException { + private void updateTranslogDeletionPolicy() throws IOException { assert Thread.holdsLock(this); logger.debug("Safe commit [{}], last commit [{}]", commitDescription(safeCommit), commitDescription(lastCommit)); assert safeCommit.isDeleted() == false : "The safe commit must not be deleted"; @@ -104,9 +101,6 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { assert minRequiredGen <= lastGen : "minRequiredGen must not be greater than lastGen"; translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen); translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); - - softDeletesPolicy.setLocalCheckpointOfSafeCommit( - Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY))); } /** diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index 08724d6e794..4d95cf89ef0 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -58,7 +58,6 @@ import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; @@ -98,7 +97,6 @@ public abstract class Engine implements Closeable { public static final String SYNC_COMMIT_ID = "sync_id"; public static final String HISTORY_UUID_KEY = "history_uuid"; - public static final String MIN_RETAINED_SEQNO = "min_retained_seq_no"; protected final ShardId shardId; protected final String allocationId; @@ -587,32 +585,18 @@ public abstract class Engine implements Closeable { public abstract void syncTranslog() throws IOException; - /** - * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed - */ - public abstract Closeable acquireRetentionLockForPeerRecovery(); + public abstract Closeable acquireTranslogRetentionLock(); /** - * Creates a new history snapshot from Lucene for reading operations whose seqno in the requesting seqno range (both inclusive) + * Creates a new translog snapshot from this engine for reading translog operations whose seq# at least the provided seq#. + * The caller has to close the returned snapshot after finishing the reading. */ - public abstract Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, - long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException; + public abstract Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException; /** - * Creates a new history snapshot for reading operations since {@code startingSeqNo} (inclusive). - * The returned snapshot can be retrieved from either Lucene index or translog files. + * Returns the estimated number of translog operations in this engine whose seq# at least the provided seq#. */ - public abstract Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; - - /** - * Returns the estimated number of history operations whose seq# at least {@code startingSeqNo}(inclusive) in this engine. - */ - public abstract int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; - - /** - * Checks if this engine has every operations since {@code startingSeqNo}(inclusive) in its history (either Lucene or translog) - */ - public abstract boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException; + public abstract int estimateTranslogOperationsFromMinSeq(long minSeqNo); public abstract TranslogStats getTranslogStats(); diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 23a90553f60..2deae61bd52 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.unit.MemorySizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; -import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; @@ -81,7 +80,6 @@ public final class EngineConfig { private final CircuitBreakerService circuitBreakerService; private final LongSupplier globalCheckpointSupplier; private final LongSupplier primaryTermSupplier; - private final TombstoneDocSupplier tombstoneDocSupplier; /** * Index setting to change the low level lucene codec used for writing new segments. @@ -128,8 +126,7 @@ public final class EngineConfig { List externalRefreshListener, List internalRefreshListener, Sort indexSort, TranslogRecoveryRunner translogRecoveryRunner, CircuitBreakerService circuitBreakerService, - LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier, - TombstoneDocSupplier tombstoneDocSupplier) { + LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier) { this.shardId = shardId; this.allocationId = allocationId; this.indexSettings = indexSettings; @@ -167,7 +164,6 @@ public final class EngineConfig { this.circuitBreakerService = circuitBreakerService; this.globalCheckpointSupplier = globalCheckpointSupplier; this.primaryTermSupplier = primaryTermSupplier; - this.tombstoneDocSupplier = tombstoneDocSupplier; } /** @@ -377,25 +373,4 @@ public final class EngineConfig { public LongSupplier getPrimaryTermSupplier() { return primaryTermSupplier; } - - /** - * A supplier supplies tombstone documents which will be used in soft-update methods. - * The returned document consists only _uid, _seqno, _term and _version fields; other metadata fields are excluded. - */ - public interface TombstoneDocSupplier { - /** - * Creates a tombstone document for a delete operation. - */ - ParsedDocument newDeleteTombstoneDoc(String type, String id); - - /** - * Creates a tombstone document for a noop operation. - * @param reason the reason of an a noop - */ - ParsedDocument newNoopTombstoneDoc(String reason); - } - - public TombstoneDocSupplier getTombstoneDocSupplier() { - return tombstoneDocSupplier; - } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index da4decc93b1..023e659ffab 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -21,20 +21,16 @@ package org.elasticsearch.index.engine; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.document.Field; -import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ReferenceManager; @@ -46,7 +42,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.InfoStream; -import org.elasticsearch.Assertions; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.Nullable; @@ -66,11 +61,7 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.LocalCheckpointTracker; @@ -149,10 +140,6 @@ public class InternalEngine extends Engine { private final CounterMetric numDocDeletes = new CounterMetric(); private final CounterMetric numDocAppends = new CounterMetric(); private final CounterMetric numDocUpdates = new CounterMetric(); - private final NumericDocValuesField softDeletesField = Lucene.newSoftDeletesField(); - private final boolean softDeleteEnabled; - private final SoftDeletesPolicy softDeletesPolicy; - private final LastRefreshedCheckpointListener lastRefreshedCheckpointListener; /** * How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh. IndexingMemoryController polls this @@ -197,10 +184,8 @@ public class InternalEngine extends Engine { assert translog.getGeneration() != null; this.translog = translog; this.localCheckpointTracker = createLocalCheckpointTracker(localCheckpointTrackerSupplier); - this.softDeleteEnabled = engineConfig.getIndexSettings().isSoftDeleteEnabled(); - this.softDeletesPolicy = newSoftDeletesPolicy(); this.combinedDeletionPolicy = - new CombinedDeletionPolicy(logger, translogDeletionPolicy, softDeletesPolicy, translog::getLastSyncedGlobalCheckpoint); + new CombinedDeletionPolicy(logger, translogDeletionPolicy, translog::getLastSyncedGlobalCheckpoint); writer = createWriter(); bootstrapAppendOnlyInfoFromWriter(writer); historyUUID = loadHistoryUUID(writer); @@ -230,8 +215,6 @@ public class InternalEngine extends Engine { for (ReferenceManager.RefreshListener listener: engineConfig.getInternalRefreshListener()) { this.internalSearcherManager.addListener(listener); } - this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getCheckpoint()); - this.internalSearcherManager.addListener(lastRefreshedCheckpointListener); success = true; } finally { if (success == false) { @@ -257,18 +240,6 @@ public class InternalEngine extends Engine { return localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint); } - private SoftDeletesPolicy newSoftDeletesPolicy() throws IOException { - final Map commitUserData = store.readLastCommittedSegmentsInfo().userData; - final long lastMinRetainedSeqNo; - if (commitUserData.containsKey(Engine.MIN_RETAINED_SEQNO)) { - lastMinRetainedSeqNo = Long.parseLong(commitUserData.get(Engine.MIN_RETAINED_SEQNO)); - } else { - lastMinRetainedSeqNo = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)) + 1; - } - return new SoftDeletesPolicy(translog::getLastSyncedGlobalCheckpoint, lastMinRetainedSeqNo, - engineConfig.getIndexSettings().getSoftDeleteRetentionOperations()); - } - /** * This reference manager delegates all it's refresh calls to another (internal) SearcherManager * The main purpose for this is that if we have external refreshes happening we don't issue extra @@ -480,31 +451,19 @@ public class InternalEngine extends Engine { revisitIndexDeletionPolicyOnTranslogSynced(); } - /** - * Creates a new history snapshot for reading operations since the provided seqno. - * The returned snapshot can be retrieved from either Lucene index or translog files. - */ @Override - public Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { - if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - return newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false); - } else { - return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo); - } + public Closeable acquireTranslogRetentionLock() { + return getTranslog().acquireRetentionLock(); } - /** - * Returns the estimated number of history operations whose seq# at least the provided seq# in this engine. - */ @Override - public int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { - if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - try (Translog.Snapshot snapshot = newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false)) { - return snapshot.totalOperations(); - } - } else { - return getTranslog().estimateTotalOperationsFromMinSeq(startingSeqNo); - } + public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { + return getTranslog().newSnapshotFromMinSeqNo(minSeqNo); + } + + @Override + public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { + return getTranslog().estimateTotalOperationsFromMinSeq(minSeqNo); } @Override @@ -831,7 +790,7 @@ public class InternalEngine extends Engine { if (plan.earlyResultOnPreFlightError.isPresent()) { indexResult = plan.earlyResultOnPreFlightError.get(); assert indexResult.getResultType() == Result.Type.FAILURE : indexResult.getResultType(); - } else if (plan.indexIntoLucene || plan.addStaleOpToLucene) { + } else if (plan.indexIntoLucene) { indexResult = indexIntoLucene(index, plan); } else { indexResult = new IndexResult( @@ -842,10 +801,8 @@ public class InternalEngine extends Engine { if (indexResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Index(index, indexResult)); } else if (indexResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - // if we have document failure, record it as a no-op in the translog and Lucene with the generated seq_no - final NoOp noOp = new NoOp(indexResult.getSeqNo(), index.primaryTerm(), index.origin(), - index.startTime(), indexResult.getFailure().toString()); - location = innerNoOp(noOp).getTranslogLocation(); + // if we have document failure, record it as a no-op in the translog with the generated seq_no + location = translog.add(new Translog.NoOp(indexResult.getSeqNo(), index.primaryTerm(), indexResult.getFailure().toString())); } else { location = null; } @@ -897,6 +854,7 @@ public class InternalEngine extends Engine { // unlike the primary, replicas don't really care to about creation status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return false for the created flag in favor of code simplicity + final OpVsLuceneDocStatus opVsLucene; if (index.seqNo() <= localCheckpointTracker.getCheckpoint()){ // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already @@ -905,15 +863,16 @@ public class InternalEngine extends Engine { // question may have been deleted in an out of order op that is not replayed. // See testRecoverFromStoreWithOutOfOrderDelete for an example of local recovery // See testRecoveryWithOutOfOrderDelete for an example of peer recovery + opVsLucene = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; + } else { + opVsLucene = compareOpToLuceneDocBasedOnSeqNo(index); + } + if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { plan = IndexingStrategy.processButSkipLucene(false, index.seqNo(), index.version()); } else { - final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(index); - if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { - plan = IndexingStrategy.processAsStaleOp(softDeleteEnabled, index.seqNo(), index.version()); - } else { - plan = IndexingStrategy.processNormally(opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, - index.seqNo(), index.version()); - } + plan = IndexingStrategy.processNormally( + opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, index.seqNo(), index.version() + ); } } return plan; @@ -962,7 +921,7 @@ public class InternalEngine extends Engine { throws IOException { assert plan.seqNoForIndexing >= 0 : "ops should have an assigned seq no.; origin: " + index.origin(); assert plan.versionForIndexing >= 0 : "version must be set. got " + plan.versionForIndexing; - assert plan.indexIntoLucene || plan.addStaleOpToLucene; + assert plan.indexIntoLucene; /* Update the document's sequence number and primary term; the sequence number here is derived here from either the sequence * number service if this is on the primary, or the existing document's sequence number if this is on the replica. The * primary term here has already been set, see IndexShard#prepareIndex where the Engine$Index operation is created. @@ -970,9 +929,7 @@ public class InternalEngine extends Engine { index.parsedDoc().updateSeqID(plan.seqNoForIndexing, index.primaryTerm()); index.parsedDoc().version().setLongValue(plan.versionForIndexing); try { - if (plan.addStaleOpToLucene) { - addStaleDocs(index.docs(), indexWriter); - } else if (plan.useLuceneUpdateDocument) { + if (plan.useLuceneUpdateDocument) { updateDocs(index.uid(), index.docs(), indexWriter); } else { // document does not exists, we can optimize for create, but double check if assertions are running @@ -1036,29 +993,16 @@ public class InternalEngine extends Engine { numDocAppends.inc(docs.size()); } - private void addStaleDocs(final List docs, final IndexWriter indexWriter) throws IOException { - assert softDeleteEnabled : "Add history documents but soft-deletes is disabled"; - for (ParseContext.Document doc : docs) { - doc.add(softDeletesField); // soft-deleted every document before adding to Lucene - } - if (docs.size() > 1) { - indexWriter.addDocuments(docs); - } else { - indexWriter.addDocument(docs.get(0)); - } - } - - protected static final class IndexingStrategy { + private static final class IndexingStrategy { final boolean currentNotFoundOrDeleted; final boolean useLuceneUpdateDocument; final long seqNoForIndexing; final long versionForIndexing; final boolean indexIntoLucene; - final boolean addStaleOpToLucene; final Optional earlyResultOnPreFlightError; private IndexingStrategy(boolean currentNotFoundOrDeleted, boolean useLuceneUpdateDocument, - boolean indexIntoLucene, boolean addStaleOpToLucene, long seqNoForIndexing, + boolean indexIntoLucene, long seqNoForIndexing, long versionForIndexing, IndexResult earlyResultOnPreFlightError) { assert useLuceneUpdateDocument == false || indexIntoLucene : "use lucene update is set to true, but we're not indexing into lucene"; @@ -1071,40 +1015,37 @@ public class InternalEngine extends Engine { this.seqNoForIndexing = seqNoForIndexing; this.versionForIndexing = versionForIndexing; this.indexIntoLucene = indexIntoLucene; - this.addStaleOpToLucene = addStaleOpToLucene; this.earlyResultOnPreFlightError = earlyResultOnPreFlightError == null ? Optional.empty() : Optional.of(earlyResultOnPreFlightError); } static IndexingStrategy optimizedAppendOnly(long seqNoForIndexing) { - return new IndexingStrategy(true, false, true, false, seqNoForIndexing, 1, null); + return new IndexingStrategy(true, false, true, seqNoForIndexing, 1, null); } static IndexingStrategy skipDueToVersionConflict( VersionConflictEngineException e, boolean currentNotFoundOrDeleted, long currentVersion, long term) { final IndexResult result = new IndexResult(e, currentVersion, term); return new IndexingStrategy( - currentNotFoundOrDeleted, false, false, false, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); + currentNotFoundOrDeleted, false, false, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); } static IndexingStrategy processNormally(boolean currentNotFoundOrDeleted, long seqNoForIndexing, long versionForIndexing) { return new IndexingStrategy(currentNotFoundOrDeleted, currentNotFoundOrDeleted == false, - true, false, seqNoForIndexing, versionForIndexing, null); + true, seqNoForIndexing, versionForIndexing, null); } static IndexingStrategy overrideExistingAsIfNotThere( long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(true, true, true, false, seqNoForIndexing, versionForIndexing, null); + return new IndexingStrategy(true, true, true, seqNoForIndexing, versionForIndexing, null); } - static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(currentNotFoundOrDeleted, false, false, false, seqNoForIndexing, versionForIndexing, null); - } - - static IndexingStrategy processAsStaleOp(boolean addStaleOpToLucene, long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(false, false, false, addStaleOpToLucene, seqNoForIndexing, versionForIndexing, null); + static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, + long seqNoForIndexing, long versionForIndexing) { + return new IndexingStrategy(currentNotFoundOrDeleted, false, + false, seqNoForIndexing, versionForIndexing, null); } } @@ -1131,18 +1072,10 @@ public class InternalEngine extends Engine { } private void updateDocs(final Term uid, final List docs, final IndexWriter indexWriter) throws IOException { - if (softDeleteEnabled) { - if (docs.size() > 1) { - indexWriter.softUpdateDocuments(uid, docs, softDeletesField); - } else { - indexWriter.softUpdateDocument(uid, docs.get(0), softDeletesField); - } + if (docs.size() > 1) { + indexWriter.updateDocuments(uid, docs); } else { - if (docs.size() > 1) { - indexWriter.updateDocuments(uid, docs); - } else { - indexWriter.updateDocument(uid, docs.get(0)); - } + indexWriter.updateDocument(uid, docs.get(0)); } numDocUpdates.inc(docs.size()); } @@ -1166,7 +1099,7 @@ public class InternalEngine extends Engine { if (plan.earlyResultOnPreflightError.isPresent()) { deleteResult = plan.earlyResultOnPreflightError.get(); - } else if (plan.deleteFromLucene || plan.addStaleOpToLucene) { + } else if (plan.deleteFromLucene) { deleteResult = deleteInLucene(delete, plan); } else { deleteResult = new DeleteResult( @@ -1177,10 +1110,8 @@ public class InternalEngine extends Engine { if (deleteResult.getResultType() == Result.Type.SUCCESS) { location = translog.add(new Translog.Delete(delete, deleteResult)); } else if (deleteResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - // if we have document failure, record it as a no-op in the translog and Lucene with the generated seq_no - final NoOp noOp = new NoOp(deleteResult.getSeqNo(), delete.primaryTerm(), delete.origin(), - delete.startTime(), deleteResult.getFailure().toString()); - location = innerNoOp(noOp).getTranslogLocation(); + location = translog.add(new Translog.NoOp(deleteResult.getSeqNo(), + delete.primaryTerm(), deleteResult.getFailure().toString())); } else { location = null; } @@ -1211,7 +1142,7 @@ public class InternalEngine extends Engine { // unlike the primary, replicas don't really care to about found status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return true for the found flag in favor of code simplicity - final DeletionStrategy plan; + final OpVsLuceneDocStatus opVsLucene; if (delete.seqNo() <= localCheckpointTracker.getCheckpoint()) { // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already @@ -1220,15 +1151,18 @@ public class InternalEngine extends Engine { // question may have been deleted in an out of order op that is not replayed. // See testRecoverFromStoreWithOutOfOrderDelete for an example of local recovery // See testRecoveryWithOutOfOrderDelete for an example of peer recovery + opVsLucene = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL; + } else { + opVsLucene = compareOpToLuceneDocBasedOnSeqNo(delete); + } + + final DeletionStrategy plan; + if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { plan = DeletionStrategy.processButSkipLucene(false, delete.seqNo(), delete.version()); } else { - final OpVsLuceneDocStatus opVsLucene = compareOpToLuceneDocBasedOnSeqNo(delete); - if (opVsLucene == OpVsLuceneDocStatus.OP_STALE_OR_EQUAL) { - plan = DeletionStrategy.processAsStaleOp(softDeleteEnabled, false, delete.seqNo(), delete.version()); - } else { - plan = DeletionStrategy.processNormally(opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, - delete.seqNo(), delete.version()); - } + plan = DeletionStrategy.processNormally( + opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, + delete.seqNo(), delete.version()); } return plan; } @@ -1263,31 +1197,15 @@ public class InternalEngine extends Engine { private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) throws IOException { try { - if (softDeleteEnabled) { - final ParsedDocument tombstone = engineConfig.getTombstoneDocSupplier().newDeleteTombstoneDoc(delete.type(), delete.id()); - assert tombstone.docs().size() == 1 : "Tombstone doc should have single doc [" + tombstone + "]"; - tombstone.updateSeqID(plan.seqNoOfDeletion, delete.primaryTerm()); - tombstone.version().setLongValue(plan.versionOfDeletion); - final ParseContext.Document doc = tombstone.docs().get(0); - assert doc.getField(SeqNoFieldMapper.TOMBSTONE_NAME) != null : - "Delete tombstone document but _tombstone field is not set [" + doc + " ]"; - doc.add(softDeletesField); - if (plan.addStaleOpToLucene || plan.currentlyDeleted) { - indexWriter.addDocument(doc); - } else { - indexWriter.softUpdateDocument(delete.uid(), doc, softDeletesField); - } - } else if (plan.currentlyDeleted == false) { + if (plan.currentlyDeleted == false) { // any exception that comes from this is a either an ACE or a fatal exception there // can't be any document failures coming from this indexWriter.deleteDocuments(delete.uid()); - } - if (plan.deleteFromLucene) { numDocDeletes.inc(); - versionMap.putDeleteUnderLock(delete.uid().bytes(), - new DeleteVersionValue(plan.versionOfDeletion, plan.seqNoOfDeletion, delete.primaryTerm(), - engineConfig.getThreadPool().relativeTimeInMillis())); } + versionMap.putDeleteUnderLock(delete.uid().bytes(), + new DeleteVersionValue(plan.versionOfDeletion, plan.seqNoOfDeletion, delete.primaryTerm(), + engineConfig.getThreadPool().relativeTimeInMillis())); return new DeleteResult( plan.versionOfDeletion, getPrimaryTerm(), plan.seqNoOfDeletion, plan.currentlyDeleted == false); } catch (Exception ex) { @@ -1301,16 +1219,15 @@ public class InternalEngine extends Engine { } } - protected static final class DeletionStrategy { + private static final class DeletionStrategy { // of a rare double delete final boolean deleteFromLucene; - final boolean addStaleOpToLucene; final boolean currentlyDeleted; final long seqNoOfDeletion; final long versionOfDeletion; final Optional earlyResultOnPreflightError; - private DeletionStrategy(boolean deleteFromLucene, boolean addStaleOpToLucene, boolean currentlyDeleted, + private DeletionStrategy(boolean deleteFromLucene, boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion, DeleteResult earlyResultOnPreflightError) { assert (deleteFromLucene && earlyResultOnPreflightError != null) == false : @@ -1318,7 +1235,6 @@ public class InternalEngine extends Engine { "deleteFromLucene: " + deleteFromLucene + " earlyResultOnPreFlightError:" + earlyResultOnPreflightError; this.deleteFromLucene = deleteFromLucene; - this.addStaleOpToLucene = addStaleOpToLucene; this.currentlyDeleted = currentlyDeleted; this.seqNoOfDeletion = seqNoOfDeletion; this.versionOfDeletion = versionOfDeletion; @@ -1330,22 +1246,16 @@ public class InternalEngine extends Engine { VersionConflictEngineException e, long currentVersion, long term, boolean currentlyDeleted) { final long unassignedSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; final DeleteResult deleteResult = new DeleteResult(e, currentVersion, term, unassignedSeqNo, currentlyDeleted == false); - return new DeletionStrategy(false, false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); + return new DeletionStrategy(false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); } static DeletionStrategy processNormally(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(true, false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + return new DeletionStrategy(true, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); } - public static DeletionStrategy processButSkipLucene(boolean currentlyDeleted, - long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(false, false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); - } - - static DeletionStrategy processAsStaleOp(boolean addStaleOpToLucene, boolean currentlyDeleted, - long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(false, addStaleOpToLucene, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + public static DeletionStrategy processButSkipLucene(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { + return new DeletionStrategy(false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); } } @@ -1374,28 +1284,7 @@ public class InternalEngine extends Engine { assert noOp.seqNo() > SequenceNumbers.NO_OPS_PERFORMED; final long seqNo = noOp.seqNo(); try { - Exception failure = null; - if (softDeleteEnabled) { - try { - final ParsedDocument tombstone = engineConfig.getTombstoneDocSupplier().newNoopTombstoneDoc(noOp.reason()); - tombstone.updateSeqID(noOp.seqNo(), noOp.primaryTerm()); - // A noop tombstone does not require a _version but it's added to have a fully dense docvalues for the version field. - // 1L is selected to optimize the compression because it might probably be the most common value in version field. - tombstone.version().setLongValue(1L); - assert tombstone.docs().size() == 1 : "Tombstone should have a single doc [" + tombstone + "]"; - final ParseContext.Document doc = tombstone.docs().get(0); - assert doc.getField(SeqNoFieldMapper.TOMBSTONE_NAME) != null - : "Noop tombstone document but _tombstone field is not set [" + doc + " ]"; - doc.add(softDeletesField); - indexWriter.addDocument(doc); - } catch (Exception ex) { - if (maybeFailEngine("noop", ex)) { - throw ex; - } - failure = ex; - } - } - final NoOpResult noOpResult = failure != null ? new NoOpResult(getPrimaryTerm(), noOp.seqNo(), failure) : new NoOpResult(getPrimaryTerm(), noOp.seqNo()); + final NoOpResult noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo()); if (noOp.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { final Translog.Location location = translog.add(new Translog.NoOp(noOp.seqNo(), noOp.primaryTerm(), noOp.reason())); noOpResult.setTranslogLocation(location); @@ -1420,7 +1309,6 @@ public class InternalEngine extends Engine { // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) // both refresh types will result in an internal refresh but only the external will also // pass the new reader reference to the external reader manager. - final long localCheckpointBeforeRefresh = getLocalCheckpoint(); // this will also cause version map ram to be freed hence we always account for it. final long bytes = indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh(); @@ -1446,7 +1334,6 @@ public class InternalEngine extends Engine { } finally { store.decRef(); } - lastRefreshedCheckpointListener.updateRefreshedCheckpoint(localCheckpointBeforeRefresh); } } catch (AlreadyClosedException e) { failOnTragicEvent(e); @@ -1461,8 +1348,7 @@ public class InternalEngine extends Engine { } finally { writingBytes.addAndGet(-bytes); } - assert lastRefreshedCheckpoint() >= localCheckpointBeforeRefresh : "refresh checkpoint was not advanced; " + - "local_checkpoint=" + localCheckpointBeforeRefresh + " refresh_checkpoint=" + lastRefreshedCheckpoint(); + // TODO: maybe we should just put a scheduled job in threadPool? // We check for pruning in each delete request, but we also prune here e.g. in case a delete burst comes in and then no more deletes // for a long time: @@ -2044,11 +1930,7 @@ public class InternalEngine extends Engine { // pkg-private for testing IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException { - if (Assertions.ENABLED) { - return new AssertingIndexWriter(directory, iwc); - } else { - return new IndexWriter(directory, iwc); - } + return new IndexWriter(directory, iwc); } private IndexWriterConfig getIndexWriterConfig() { @@ -2064,15 +1946,11 @@ public class InternalEngine extends Engine { } iwc.setInfoStream(verbose ? InfoStream.getDefault() : new LoggerInfoStream(logger)); iwc.setMergeScheduler(mergeScheduler); + MergePolicy mergePolicy = config().getMergePolicy(); // Give us the opportunity to upgrade old segments while performing // background merges - MergePolicy mergePolicy = config().getMergePolicy(); - if (softDeleteEnabled) { - iwc.setSoftDeletesField(Lucene.SOFT_DELETES_FIELD); - mergePolicy = new RecoverySourcePruneMergePolicy(SourceFieldMapper.RECOVERY_SOURCE_NAME, softDeletesPolicy::getRetentionQuery, - new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, softDeletesPolicy::getRetentionQuery, mergePolicy)); - } - iwc.setMergePolicy(new ElasticsearchMergePolicy(mergePolicy)); + mergePolicy = new ElasticsearchMergePolicy(mergePolicy); + iwc.setMergePolicy(mergePolicy); iwc.setSimilarity(engineConfig.getSimilarity()); iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac()); iwc.setCodec(engineConfig.getCodec()); @@ -2269,9 +2147,6 @@ public class InternalEngine extends Engine { commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(localCheckpointTracker.getMaxSeqNo())); commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get())); commitData.put(HISTORY_UUID_KEY, historyUUID); - if (softDeleteEnabled) { - commitData.put(Engine.MIN_RETAINED_SEQNO, Long.toString(softDeletesPolicy.getMinRetainedSeqNo())); - } logger.trace("committing writer with commit data [{}]", commitData); return commitData.entrySet().iterator(); }); @@ -2327,7 +2202,6 @@ public class InternalEngine extends Engine { final IndexSettings indexSettings = engineConfig.getIndexSettings(); translogDeletionPolicy.setRetentionAgeInMillis(indexSettings.getTranslogRetentionAge().getMillis()); translogDeletionPolicy.setRetentionSizeInBytes(indexSettings.getTranslogRetentionSize().getBytes()); - softDeletesPolicy.setRetentionOperations(indexSettings.getSoftDeleteRetentionOperations()); } public MergeStats getMergeStats() { @@ -2422,69 +2296,6 @@ public class InternalEngine extends Engine { return numDocUpdates.count(); } - @Override - public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperService, - long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { - // TODO: Should we defer the refresh until we really need it? - ensureOpen(); - if (lastRefreshedCheckpoint() < toSeqNo) { - refresh(source, SearcherScope.INTERNAL); - } - Searcher searcher = acquireSearcher(source, SearcherScope.INTERNAL); - try { - LuceneChangesSnapshot snapshot = new LuceneChangesSnapshot( - searcher, mapperService, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE, fromSeqNo, toSeqNo, requiredFullRange); - searcher = null; - return snapshot; - } catch (Exception e) { - try { - maybeFailEngine("acquire changes snapshot", e); - } catch (Exception inner) { - e.addSuppressed(inner); - } - throw e; - } finally { - IOUtils.close(searcher); - } - } - - @Override - public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException { - if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - return getMinRetainedSeqNo() <= startingSeqNo; - } else { - final long currentLocalCheckpoint = getLocalCheckpointTracker().getCheckpoint(); - final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); - try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(startingSeqNo)) { - Translog.Operation operation; - while ((operation = snapshot.next()) != null) { - if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - tracker.markSeqNoAsCompleted(operation.seqNo()); - } - } - } - return tracker.getCheckpoint() >= currentLocalCheckpoint; - } - } - - /** - * Returns the minimum seqno that is retained in the Lucene index. - * Operations whose seq# are at least this value should exist in the Lucene index. - */ - final long getMinRetainedSeqNo() { - assert softDeleteEnabled : Thread.currentThread().getName(); - return softDeletesPolicy.getMinRetainedSeqNo(); - } - - @Override - public Closeable acquireRetentionLockForPeerRecovery() { - if (softDeleteEnabled) { - return softDeletesPolicy.acquireRetentionLock(); - } else { - return translog.acquireRetentionLock(); - } - } - @Override public boolean isRecovering() { return pendingTranslogRecovery.get(); @@ -2500,69 +2311,4 @@ public class InternalEngine extends Engine { } return commitData; } - - private final class AssertingIndexWriter extends IndexWriter { - AssertingIndexWriter(Directory d, IndexWriterConfig conf) throws IOException { - super(d, conf); - } - @Override - public long updateDocument(Term term, Iterable doc) throws IOException { - assert softDeleteEnabled == false : "Call #updateDocument but soft-deletes is enabled"; - return super.updateDocument(term, doc); - } - @Override - public long updateDocuments(Term delTerm, Iterable> docs) throws IOException { - assert softDeleteEnabled == false : "Call #updateDocuments but soft-deletes is enabled"; - return super.updateDocuments(delTerm, docs); - } - @Override - public long deleteDocuments(Term... terms) throws IOException { - assert softDeleteEnabled == false : "Call #deleteDocuments but soft-deletes is enabled"; - return super.deleteDocuments(terms); - } - @Override - public long softUpdateDocument(Term term, Iterable doc, Field... softDeletes) throws IOException { - assert softDeleteEnabled : "Call #softUpdateDocument but soft-deletes is disabled"; - return super.softUpdateDocument(term, doc, softDeletes); - } - @Override - public long softUpdateDocuments(Term term, Iterable> docs, Field... softDeletes) throws IOException { - assert softDeleteEnabled : "Call #softUpdateDocuments but soft-deletes is disabled"; - return super.softUpdateDocuments(term, docs, softDeletes); - } - } - - /** - * Returned the last local checkpoint value has been refreshed internally. - */ - final long lastRefreshedCheckpoint() { - return lastRefreshedCheckpointListener.refreshedCheckpoint.get(); - } - - private final class LastRefreshedCheckpointListener implements ReferenceManager.RefreshListener { - final AtomicLong refreshedCheckpoint; - private long pendingCheckpoint; - - LastRefreshedCheckpointListener(long initialLocalCheckpoint) { - this.refreshedCheckpoint = new AtomicLong(initialLocalCheckpoint); - } - - @Override - public void beforeRefresh() { - // all changes until this point should be visible after refresh - pendingCheckpoint = localCheckpointTracker.getCheckpoint(); - } - - @Override - public void afterRefresh(boolean didRefresh) { - if (didRefresh) { - updateRefreshedCheckpoint(pendingCheckpoint); - } - } - - void updateRefreshedCheckpoint(long checkpoint) { - refreshedCheckpoint.updateAndGet(curr -> Math.max(curr, checkpoint)); - assert refreshedCheckpoint.get() >= checkpoint : refreshedCheckpoint.get() + " < " + checkpoint; - } - } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java deleted file mode 100644 index deebfba9ed4..00000000000 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ /dev/null @@ -1,368 +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.engine; - -import org.apache.lucene.document.LongPoint; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.util.ArrayUtil; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.fieldvisitor.FieldsVisitor; -import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.SourceFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.VersionFieldMapper; -import org.elasticsearch.index.translog.Translog; - -import java.io.Closeable; -import java.io.IOException; -import java.util.Comparator; -import java.util.List; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * A {@link Translog.Snapshot} from changes in a Lucene index - */ -final class LuceneChangesSnapshot implements Translog.Snapshot { - static final int DEFAULT_BATCH_SIZE = 1024; - - private final int searchBatchSize; - private final long fromSeqNo, toSeqNo; - private long lastSeenSeqNo; - private int skippedOperations; - private final boolean requiredFullRange; - - private final IndexSearcher indexSearcher; - private final MapperService mapperService; - private int docIndex = 0; - private final int totalHits; - private ScoreDoc[] scoreDocs; - private final ParallelArray parallelArray; - private final Closeable onClose; - - /** - * Creates a new "translog" snapshot from Lucene for reading operations whose seq# in the specified range. - * - * @param engineSearcher the internal engine searcher which will be taken over if the snapshot is opened successfully - * @param mapperService the mapper service which will be mainly used to resolve the document's type and uid - * @param searchBatchSize the number of documents should be returned by each search - * @param fromSeqNo the min requesting seq# - inclusive - * @param toSeqNo the maximum requesting seq# - inclusive - * @param requiredFullRange if true, the snapshot will strictly check for the existence of operations between fromSeqNo and toSeqNo - */ - LuceneChangesSnapshot(Engine.Searcher engineSearcher, MapperService mapperService, int searchBatchSize, - long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { - if (fromSeqNo < 0 || toSeqNo < 0 || fromSeqNo > toSeqNo) { - throw new IllegalArgumentException("Invalid range; from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "]"); - } - if (searchBatchSize <= 0) { - throw new IllegalArgumentException("Search_batch_size must be positive [" + searchBatchSize + "]"); - } - final AtomicBoolean closed = new AtomicBoolean(); - this.onClose = () -> { - if (closed.compareAndSet(false, true)) { - IOUtils.close(engineSearcher); - } - }; - this.mapperService = mapperService; - this.searchBatchSize = searchBatchSize; - this.fromSeqNo = fromSeqNo; - this.toSeqNo = toSeqNo; - this.lastSeenSeqNo = fromSeqNo - 1; - this.requiredFullRange = requiredFullRange; - this.indexSearcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); - this.indexSearcher.setQueryCache(null); - this.parallelArray = new ParallelArray(searchBatchSize); - final TopDocs topDocs = searchOperations(null); - this.totalHits = Math.toIntExact(topDocs.totalHits); - this.scoreDocs = topDocs.scoreDocs; - fillParallelArray(scoreDocs, parallelArray); - } - - @Override - public void close() throws IOException { - onClose.close(); - } - - @Override - public int totalOperations() { - return totalHits; - } - - @Override - public int skippedOperations() { - return skippedOperations; - } - - @Override - public Translog.Operation next() throws IOException { - Translog.Operation op = null; - for (int idx = nextDocIndex(); idx != -1; idx = nextDocIndex()) { - op = readDocAsOp(idx); - if (op != null) { - break; - } - } - if (requiredFullRange) { - rangeCheck(op); - } - if (op != null) { - lastSeenSeqNo = op.seqNo(); - } - return op; - } - - private void rangeCheck(Translog.Operation op) { - if (op == null) { - if (lastSeenSeqNo < toSeqNo) { - throw new IllegalStateException("Not all operations between from_seqno [" + fromSeqNo + "] " + - "and to_seqno [" + toSeqNo + "] found; prematurely terminated last_seen_seqno [" + lastSeenSeqNo + "]"); - } - } else { - final long expectedSeqNo = lastSeenSeqNo + 1; - if (op.seqNo() != expectedSeqNo) { - throw new IllegalStateException("Not all operations between from_seqno [" + fromSeqNo + "] " + - "and to_seqno [" + toSeqNo + "] found; expected seqno [" + expectedSeqNo + "]; found [" + op + "]"); - } - } - } - - private int nextDocIndex() throws IOException { - // we have processed all docs in the current search - fetch the next batch - if (docIndex == scoreDocs.length && docIndex > 0) { - final ScoreDoc prev = scoreDocs[scoreDocs.length - 1]; - scoreDocs = searchOperations(prev).scoreDocs; - fillParallelArray(scoreDocs, parallelArray); - docIndex = 0; - } - if (docIndex < scoreDocs.length) { - int idx = docIndex; - docIndex++; - return idx; - } - return -1; - } - - private void fillParallelArray(ScoreDoc[] scoreDocs, ParallelArray parallelArray) throws IOException { - if (scoreDocs.length > 0) { - for (int i = 0; i < scoreDocs.length; i++) { - scoreDocs[i].shardIndex = i; - } - // for better loading performance we sort the array by docID and - // then visit all leaves in order. - ArrayUtil.introSort(scoreDocs, Comparator.comparingInt(i -> i.doc)); - int docBase = -1; - int maxDoc = 0; - List leaves = indexSearcher.getIndexReader().leaves(); - int readerIndex = 0; - CombinedDocValues combinedDocValues = null; - LeafReaderContext leaf = null; - for (int i = 0; i < scoreDocs.length; i++) { - ScoreDoc scoreDoc = scoreDocs[i]; - if (scoreDoc.doc >= docBase + maxDoc) { - do { - leaf = leaves.get(readerIndex++); - docBase = leaf.docBase; - maxDoc = leaf.reader().maxDoc(); - } while (scoreDoc.doc >= docBase + maxDoc); - combinedDocValues = new CombinedDocValues(leaf.reader()); - } - final int segmentDocID = scoreDoc.doc - docBase; - final int index = scoreDoc.shardIndex; - parallelArray.leafReaderContexts[index] = leaf; - parallelArray.seqNo[index] = combinedDocValues.docSeqNo(segmentDocID); - parallelArray.primaryTerm[index] = combinedDocValues.docPrimaryTerm(segmentDocID); - parallelArray.version[index] = combinedDocValues.docVersion(segmentDocID); - parallelArray.isTombStone[index] = combinedDocValues.isTombstone(segmentDocID); - parallelArray.hasRecoverySource[index] = combinedDocValues.hasRecoverySource(segmentDocID); - } - // now sort back based on the shardIndex. we use this to store the previous index - ArrayUtil.introSort(scoreDocs, Comparator.comparingInt(i -> i.shardIndex)); - } - } - - private TopDocs searchOperations(ScoreDoc after) throws IOException { - final Query rangeQuery = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, lastSeenSeqNo + 1, toSeqNo); - final Sort sortedBySeqNoThenByTerm = new Sort( - new SortField(SeqNoFieldMapper.NAME, SortField.Type.LONG), - new SortField(SeqNoFieldMapper.PRIMARY_TERM_NAME, SortField.Type.LONG, true) - ); - return indexSearcher.searchAfter(after, rangeQuery, searchBatchSize, sortedBySeqNoThenByTerm); - } - - private Translog.Operation readDocAsOp(int docIndex) throws IOException { - final LeafReaderContext leaf = parallelArray.leafReaderContexts[docIndex]; - final int segmentDocID = scoreDocs[docIndex].doc - leaf.docBase; - final long primaryTerm = parallelArray.primaryTerm[docIndex]; - // We don't have to read the nested child documents - those docs don't have primary terms. - if (primaryTerm == -1) { - skippedOperations++; - return null; - } - final long seqNo = parallelArray.seqNo[docIndex]; - // Only pick the first seen seq# - if (seqNo == lastSeenSeqNo) { - skippedOperations++; - return null; - } - final long version = parallelArray.version[docIndex]; - final String sourceField = parallelArray.hasRecoverySource[docIndex] ? SourceFieldMapper.RECOVERY_SOURCE_NAME : - SourceFieldMapper.NAME; - final FieldsVisitor fields = new FieldsVisitor(true, sourceField); - leaf.reader().document(segmentDocID, fields); - fields.postProcess(mapperService); - - final Translog.Operation op; - final boolean isTombstone = parallelArray.isTombStone[docIndex]; - if (isTombstone && fields.uid() == null) { - op = new Translog.NoOp(seqNo, primaryTerm, fields.source().utf8ToString()); - assert version == 1L : "Noop tombstone should have version 1L; actual version [" + version + "]"; - assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Noop but soft_deletes field is not set [" + op + "]"; - } else { - final String id = fields.uid().id(); - final String type = fields.uid().type(); - final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); - if (isTombstone) { - op = new Translog.Delete(type, id, uid, seqNo, primaryTerm, version); - assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Delete op but soft_deletes field is not set [" + op + "]"; - } else { - final BytesReference source = fields.source(); - if (source == null) { - // TODO: Callers should ask for the range that source should be retained. Thus we should always - // check for the existence source once we make peer-recovery to send ops after the local checkpoint. - if (requiredFullRange) { - throw new IllegalStateException("source not found for seqno=" + seqNo + - " from_seqno=" + fromSeqNo + " to_seqno=" + toSeqNo); - } else { - skippedOperations++; - return null; - } - } - // TODO: pass the latest timestamp from engine. - final long autoGeneratedIdTimestamp = -1; - op = new Translog.Index(type, id, seqNo, primaryTerm, version, - source.toBytesRef().bytes, fields.routing(), autoGeneratedIdTimestamp); - } - } - assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() : "Unexpected operation; " + - "last_seen_seqno [" + lastSeenSeqNo + "], from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "], op [" + op + "]"; - return op; - } - - private boolean assertDocSoftDeleted(LeafReader leafReader, int segmentDocId) throws IOException { - final NumericDocValues ndv = leafReader.getNumericDocValues(Lucene.SOFT_DELETES_FIELD); - if (ndv == null || ndv.advanceExact(segmentDocId) == false) { - throw new IllegalStateException("DocValues for field [" + Lucene.SOFT_DELETES_FIELD + "] is not found"); - } - return ndv.longValue() == 1; - } - - private static final class ParallelArray { - final LeafReaderContext[] leafReaderContexts; - final long[] version; - final long[] seqNo; - final long[] primaryTerm; - final boolean[] isTombStone; - final boolean[] hasRecoverySource; - - ParallelArray(int size) { - version = new long[size]; - seqNo = new long[size]; - primaryTerm = new long[size]; - isTombStone = new boolean[size]; - hasRecoverySource = new boolean[size]; - leafReaderContexts = new LeafReaderContext[size]; - } - } - - private static final class CombinedDocValues { - private final NumericDocValues versionDV; - private final NumericDocValues seqNoDV; - private final NumericDocValues primaryTermDV; - private final NumericDocValues tombstoneDV; - private final NumericDocValues recoverySource; - - CombinedDocValues(LeafReader leafReader) throws IOException { - this.versionDV = Objects.requireNonNull(leafReader.getNumericDocValues(VersionFieldMapper.NAME), "VersionDV is missing"); - this.seqNoDV = Objects.requireNonNull(leafReader.getNumericDocValues(SeqNoFieldMapper.NAME), "SeqNoDV is missing"); - this.primaryTermDV = Objects.requireNonNull( - leafReader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME), "PrimaryTermDV is missing"); - this.tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); - this.recoverySource = leafReader.getNumericDocValues(SourceFieldMapper.RECOVERY_SOURCE_NAME); - } - - long docVersion(int segmentDocId) throws IOException { - assert versionDV.docID() < segmentDocId; - if (versionDV.advanceExact(segmentDocId) == false) { - throw new IllegalStateException("DocValues for field [" + VersionFieldMapper.NAME + "] is not found"); - } - return versionDV.longValue(); - } - - long docSeqNo(int segmentDocId) throws IOException { - assert seqNoDV.docID() < segmentDocId; - if (seqNoDV.advanceExact(segmentDocId) == false) { - throw new IllegalStateException("DocValues for field [" + SeqNoFieldMapper.NAME + "] is not found"); - } - return seqNoDV.longValue(); - } - - long docPrimaryTerm(int segmentDocId) throws IOException { - if (primaryTermDV == null) { - return -1L; - } - assert primaryTermDV.docID() < segmentDocId; - // Use -1 for docs which don't have primary term. The caller considers those docs as nested docs. - if (primaryTermDV.advanceExact(segmentDocId) == false) { - return -1; - } - return primaryTermDV.longValue(); - } - - boolean isTombstone(int segmentDocId) throws IOException { - if (tombstoneDV == null) { - return false; - } - assert tombstoneDV.docID() < segmentDocId; - return tombstoneDV.advanceExact(segmentDocId) && tombstoneDV.longValue() > 0; - } - - boolean hasRecoverySource(int segmentDocId) throws IOException { - if (recoverySource == null) { - return false; - } - assert recoverySource.docID() < segmentDocId; - return recoverySource.advanceExact(segmentDocId); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java deleted file mode 100644 index fde97562de8..00000000000 --- a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java +++ /dev/null @@ -1,292 +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.engine; - -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.codecs.StoredFieldsReader; -import org.apache.lucene.index.BinaryDocValues; -import org.apache.lucene.index.CodecReader; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FilterCodecReader; -import org.apache.lucene.index.FilterNumericDocValues; -import org.apache.lucene.index.MergePolicy; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.OneMergeWrappingMergePolicy; -import org.apache.lucene.index.SortedDocValues; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.index.SortedSetDocValues; -import org.apache.lucene.index.StoredFieldVisitor; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.ConjunctionDISI; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.DocValuesFieldExistsQuery; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.Weight; -import org.apache.lucene.util.BitSet; -import org.apache.lucene.util.BitSetIterator; - -import java.io.IOException; -import java.util.Arrays; -import java.util.function.Supplier; - -final class RecoverySourcePruneMergePolicy extends OneMergeWrappingMergePolicy { - RecoverySourcePruneMergePolicy(String recoverySourceField, Supplier retainSourceQuerySupplier, MergePolicy in) { - super(in, toWrap -> new OneMerge(toWrap.segments) { - @Override - public CodecReader wrapForMerge(CodecReader reader) throws IOException { - CodecReader wrapped = toWrap.wrapForMerge(reader); - return wrapReader(recoverySourceField, wrapped, retainSourceQuerySupplier); - } - }); - } - - // pkg private for testing - static CodecReader wrapReader(String recoverySourceField, CodecReader reader, Supplier retainSourceQuerySupplier) - throws IOException { - NumericDocValues recoverySource = reader.getNumericDocValues(recoverySourceField); - if (recoverySource == null || recoverySource.nextDoc() == DocIdSetIterator.NO_MORE_DOCS) { - return reader; // early terminate - nothing to do here since non of the docs has a recovery source anymore. - } - BooleanQuery.Builder builder = new BooleanQuery.Builder(); - builder.add(new DocValuesFieldExistsQuery(recoverySourceField), BooleanClause.Occur.FILTER); - builder.add(retainSourceQuerySupplier.get(), BooleanClause.Occur.FILTER); - IndexSearcher s = new IndexSearcher(reader); - s.setQueryCache(null); - Weight weight = s.createWeight(s.rewrite(builder.build()), false, 1.0f); - Scorer scorer = weight.scorer(reader.getContext()); - if (scorer != null) { - return new SourcePruningFilterCodecReader(recoverySourceField, reader, BitSet.of(scorer.iterator(), reader.maxDoc())); - } else { - return new SourcePruningFilterCodecReader(recoverySourceField, reader, null); - } - } - - private static class SourcePruningFilterCodecReader extends FilterCodecReader { - private final BitSet recoverySourceToKeep; - private final String recoverySourceField; - - SourcePruningFilterCodecReader(String recoverySourceField, CodecReader reader, BitSet recoverySourceToKeep) { - super(reader); - this.recoverySourceField = recoverySourceField; - this.recoverySourceToKeep = recoverySourceToKeep; - } - - @Override - public DocValuesProducer getDocValuesReader() { - DocValuesProducer docValuesReader = super.getDocValuesReader(); - return new FilterDocValuesProducer(docValuesReader) { - @Override - public NumericDocValues getNumeric(FieldInfo field) throws IOException { - NumericDocValues numeric = super.getNumeric(field); - if (recoverySourceField.equals(field.name)) { - assert numeric != null : recoverySourceField + " must have numeric DV but was null"; - final DocIdSetIterator intersection; - if (recoverySourceToKeep == null) { - // we can't return null here lucenes DocIdMerger expects an instance - intersection = DocIdSetIterator.empty(); - } else { - intersection = ConjunctionDISI.intersectIterators(Arrays.asList(numeric, - new BitSetIterator(recoverySourceToKeep, recoverySourceToKeep.length()))); - } - return new FilterNumericDocValues(numeric) { - @Override - public int nextDoc() throws IOException { - return intersection.nextDoc(); - } - - @Override - public int advance(int target) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean advanceExact(int target) { - throw new UnsupportedOperationException(); - } - }; - - } - return numeric; - } - }; - } - - @Override - public StoredFieldsReader getFieldsReader() { - StoredFieldsReader fieldsReader = super.getFieldsReader(); - return new FilterStoredFieldsReader(fieldsReader) { - @Override - public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException { - if (recoverySourceToKeep != null && recoverySourceToKeep.get(docID)) { - super.visitDocument(docID, visitor); - } else { - super.visitDocument(docID, new FilterStoredFieldVisitor(visitor) { - @Override - public Status needsField(FieldInfo fieldInfo) throws IOException { - if (recoverySourceField.equals(fieldInfo.name)) { - return Status.NO; - } - return super.needsField(fieldInfo); - } - }); - } - } - }; - } - - @Override - public CacheHelper getCoreCacheHelper() { - return null; - } - - @Override - public CacheHelper getReaderCacheHelper() { - return null; - } - - private static class FilterDocValuesProducer extends DocValuesProducer { - private final DocValuesProducer in; - - FilterDocValuesProducer(DocValuesProducer in) { - this.in = in; - } - - @Override - public NumericDocValues getNumeric(FieldInfo field) throws IOException { - return in.getNumeric(field); - } - - @Override - public BinaryDocValues getBinary(FieldInfo field) throws IOException { - return in.getBinary(field); - } - - @Override - public SortedDocValues getSorted(FieldInfo field) throws IOException { - return in.getSorted(field); - } - - @Override - public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { - return in.getSortedNumeric(field); - } - - @Override - public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { - return in.getSortedSet(field); - } - - @Override - public void checkIntegrity() throws IOException { - in.checkIntegrity(); - } - - @Override - public void close() throws IOException { - in.close(); - } - - @Override - public long ramBytesUsed() { - return in.ramBytesUsed(); - } - } - - private static class FilterStoredFieldsReader extends StoredFieldsReader { - - private final StoredFieldsReader fieldsReader; - - FilterStoredFieldsReader(StoredFieldsReader fieldsReader) { - this.fieldsReader = fieldsReader; - } - - @Override - public long ramBytesUsed() { - return fieldsReader.ramBytesUsed(); - } - - @Override - public void close() throws IOException { - fieldsReader.close(); - } - - @Override - public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException { - fieldsReader.visitDocument(docID, visitor); - } - - @Override - public StoredFieldsReader clone() { - return fieldsReader.clone(); - } - - @Override - public void checkIntegrity() throws IOException { - fieldsReader.checkIntegrity(); - } - } - - private static class FilterStoredFieldVisitor extends StoredFieldVisitor { - private final StoredFieldVisitor visitor; - - FilterStoredFieldVisitor(StoredFieldVisitor visitor) { - this.visitor = visitor; - } - - @Override - public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { - visitor.binaryField(fieldInfo, value); - } - - @Override - public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException { - visitor.stringField(fieldInfo, value); - } - - @Override - public void intField(FieldInfo fieldInfo, int value) throws IOException { - visitor.intField(fieldInfo, value); - } - - @Override - public void longField(FieldInfo fieldInfo, long value) throws IOException { - visitor.longField(fieldInfo, value); - } - - @Override - public void floatField(FieldInfo fieldInfo, float value) throws IOException { - visitor.floatField(fieldInfo, value); - } - - @Override - public void doubleField(FieldInfo fieldInfo, double value) throws IOException { - visitor.doubleField(fieldInfo, value); - } - - @Override - public Status needsField(FieldInfo fieldInfo) throws IOException { - return visitor.needsField(fieldInfo); - } - } - } -} diff --git a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java deleted file mode 100644 index af2ded8c466..00000000000 --- a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java +++ /dev/null @@ -1,120 +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.engine; - -import org.apache.lucene.document.LongPoint; -import org.apache.lucene.search.Query; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.index.translog.Translog; - -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.LongSupplier; - -/** - * A policy that controls how many soft-deleted documents should be retained for peer-recovery and querying history changes purpose. - */ -final class SoftDeletesPolicy { - private final LongSupplier globalCheckpointSupplier; - private long localCheckpointOfSafeCommit; - // This lock count is used to prevent `minRetainedSeqNo` from advancing. - private int retentionLockCount; - // The extra number of operations before the global checkpoint are retained - private long retentionOperations; - // The min seq_no value that is retained - ops after this seq# should exist in the Lucene index. - private long minRetainedSeqNo; - - SoftDeletesPolicy(LongSupplier globalCheckpointSupplier, long minRetainedSeqNo, long retentionOperations) { - this.globalCheckpointSupplier = globalCheckpointSupplier; - this.retentionOperations = retentionOperations; - this.minRetainedSeqNo = minRetainedSeqNo; - this.localCheckpointOfSafeCommit = SequenceNumbers.NO_OPS_PERFORMED; - this.retentionLockCount = 0; - } - - /** - * Updates the number of soft-deleted documents prior to the global checkpoint to be retained - * See {@link org.elasticsearch.index.IndexSettings#INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING} - */ - synchronized void setRetentionOperations(long retentionOperations) { - this.retentionOperations = retentionOperations; - } - - /** - * Sets the local checkpoint of the current safe commit - */ - synchronized void setLocalCheckpointOfSafeCommit(long newCheckpoint) { - if (newCheckpoint < this.localCheckpointOfSafeCommit) { - throw new IllegalArgumentException("Local checkpoint can't go backwards; " + - "new checkpoint [" + newCheckpoint + "]," + "current checkpoint [" + localCheckpointOfSafeCommit + "]"); - } - this.localCheckpointOfSafeCommit = newCheckpoint; - } - - /** - * Acquires a lock on soft-deleted documents to prevent them from cleaning up in merge processes. This is necessary to - * make sure that all operations that are being retained will be retained until the lock is released. - * This is a analogy to the translog's retention lock; see {@link Translog#acquireRetentionLock()} - */ - synchronized Releasable acquireRetentionLock() { - assert retentionLockCount >= 0 : "Invalid number of retention locks [" + retentionLockCount + "]"; - retentionLockCount++; - final AtomicBoolean released = new AtomicBoolean(); - return () -> { - if (released.compareAndSet(false, true)) { - releaseRetentionLock(); - } - }; - } - - private synchronized void releaseRetentionLock() { - assert retentionLockCount > 0 : "Invalid number of retention locks [" + retentionLockCount + "]"; - retentionLockCount--; - } - - /** - * Returns the min seqno that is retained in the Lucene index. - * Operations whose seq# is least this value should exist in the Lucene index. - */ - synchronized long getMinRetainedSeqNo() { - // Do not advance if the retention lock is held - if (retentionLockCount == 0) { - // This policy retains operations for two purposes: peer-recovery and querying changes history. - // - Peer-recovery is driven by the local checkpoint of the safe commit. In peer-recovery, the primary transfers a safe commit, - // then sends ops after the local checkpoint of that commit. This requires keeping all ops after localCheckpointOfSafeCommit; - // - Changes APIs are driven the combination of the global checkpoint and retention ops. Here we prefer using the global - // checkpoint instead of max_seqno because only operations up to the global checkpoint are exposed in the the changes APIs. - final long minSeqNoForQueryingChanges = globalCheckpointSupplier.getAsLong() - retentionOperations; - final long minSeqNoToRetain = Math.min(minSeqNoForQueryingChanges, localCheckpointOfSafeCommit) + 1; - // This can go backward as the retentionOperations value can be changed in settings. - minRetainedSeqNo = Math.max(minRetainedSeqNo, minSeqNoToRetain); - } - return minRetainedSeqNo; - } - - /** - * Returns a soft-deletes retention query that will be used in {@link org.apache.lucene.index.SoftDeletesRetentionMergePolicy} - * Documents including tombstones are soft-deleted and matched this query will be retained and won't cleaned up by merges. - */ - Query getRetentionQuery() { - return LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, getMinRetainedSeqNo(), Long.MAX_VALUE); - } -} diff --git a/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java b/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java index 462f8ce8e68..4c65635c61b 100644 --- a/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java +++ b/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java @@ -54,19 +54,13 @@ public class FieldsVisitor extends StoredFieldVisitor { RoutingFieldMapper.NAME)); private final boolean loadSource; - private final String sourceFieldName; private final Set requiredFields; protected BytesReference source; protected String type, id; protected Map> fieldsValues; public FieldsVisitor(boolean loadSource) { - this(loadSource, SourceFieldMapper.NAME); - } - - public FieldsVisitor(boolean loadSource, String sourceFieldName) { this.loadSource = loadSource; - this.sourceFieldName = sourceFieldName; requiredFields = new HashSet<>(); reset(); } @@ -109,7 +103,7 @@ public class FieldsVisitor extends StoredFieldVisitor { @Override public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { - if (sourceFieldName.equals(fieldInfo.name)) { + if (SourceFieldMapper.NAME.equals(fieldInfo.name)) { source = new BytesArray(value); } else if (IdFieldMapper.NAME.equals(fieldInfo.name)) { id = Uid.decodeId(value); @@ -181,7 +175,7 @@ public class FieldsVisitor extends StoredFieldVisitor { requiredFields.addAll(BASE_REQUIRED_FIELDS); if (loadSource) { - requiredFields.add(sourceFieldName); + requiredFields.add(SourceFieldMapper.NAME); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index 663aa7e6f9e..a0640ac68a9 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -19,14 +19,11 @@ package org.elasticsearch.index.mapper; -import org.apache.lucene.document.StoredField; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchGenerationException; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.text.Text; @@ -42,15 +39,12 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.stream.Stream; public class DocumentMapper implements ToXContentFragment { @@ -127,8 +121,6 @@ public class DocumentMapper implements ToXContentFragment { private final Map objectMappers; private final boolean hasNestedObjects; - private final MetadataFieldMapper[] deleteTombstoneMetadataFieldMappers; - private final MetadataFieldMapper[] noopTombstoneMetadataFieldMappers; public DocumentMapper(MapperService mapperService, Mapping mapping) { this.mapperService = mapperService; @@ -179,15 +171,6 @@ public class DocumentMapper implements ToXContentFragment { } catch (Exception e) { throw new ElasticsearchGenerationException("failed to serialize source for type [" + type + "]", e); } - - final Collection deleteTombstoneMetadataFields = Arrays.asList(VersionFieldMapper.NAME, IdFieldMapper.NAME, - TypeFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); - this.deleteTombstoneMetadataFieldMappers = Stream.of(mapping.metadataMappers) - .filter(field -> deleteTombstoneMetadataFields.contains(field.name())).toArray(MetadataFieldMapper[]::new); - final Collection noopTombstoneMetadataFields = Arrays.asList( - VersionFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME); - this.noopTombstoneMetadataFieldMappers = Stream.of(mapping.metadataMappers) - .filter(field -> noopTombstoneMetadataFields.contains(field.name())).toArray(MetadataFieldMapper[]::new); } public Mapping mapping() { @@ -259,22 +242,7 @@ public class DocumentMapper implements ToXContentFragment { } public ParsedDocument parse(SourceToParse source) throws MapperParsingException { - return documentParser.parseDocument(source, mapping.metadataMappers); - } - - public ParsedDocument createDeleteTombstoneDoc(String index, String type, String id) throws MapperParsingException { - final SourceToParse emptySource = SourceToParse.source(index, type, id, new BytesArray("{}"), XContentType.JSON); - return documentParser.parseDocument(emptySource, deleteTombstoneMetadataFieldMappers).toTombstone(); - } - - public ParsedDocument createNoopTombstoneDoc(String index, String reason) throws MapperParsingException { - final String id = ""; // _id won't be used. - final SourceToParse sourceToParse = SourceToParse.source(index, type, id, new BytesArray("{}"), XContentType.JSON); - final ParsedDocument parsedDoc = documentParser.parseDocument(sourceToParse, noopTombstoneMetadataFieldMappers).toTombstone(); - // Store the reason of a noop as a raw string in the _source field - final BytesRef byteRef = new BytesRef(reason); - parsedDoc.rootDoc().add(new StoredField(SourceFieldMapper.NAME, byteRef.bytes, byteRef.offset, byteRef.length)); - return parsedDoc; + return documentParser.parseDocument(source); } /** diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index 85123f602ed..0fd156c0905 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -55,7 +55,7 @@ final class DocumentParser { this.docMapper = docMapper; } - ParsedDocument parseDocument(SourceToParse source, MetadataFieldMapper[] metadataFieldsMappers) throws MapperParsingException { + ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException { validateType(source); final Mapping mapping = docMapper.mapping(); @@ -64,9 +64,9 @@ final class DocumentParser { try (XContentParser parser = XContentHelper.createParser(docMapperParser.getXContentRegistry(), LoggingDeprecationHandler.INSTANCE, source.source(), xContentType)) { - context = new ParseContext.InternalParseContext(indexSettings, docMapperParser, docMapper, source, parser); + context = new ParseContext.InternalParseContext(indexSettings.getSettings(), docMapperParser, docMapper, source, parser); validateStart(parser); - internalParseDocument(mapping, metadataFieldsMappers, context, parser); + internalParseDocument(mapping, context, parser); validateEnd(parser); } catch (Exception e) { throw wrapInMapperParsingException(source, e); @@ -81,11 +81,10 @@ final class DocumentParser { return parsedDocument(source, context, createDynamicUpdate(mapping, docMapper, context.getDynamicMappers())); } - private static void internalParseDocument(Mapping mapping, MetadataFieldMapper[] metadataFieldsMappers, - ParseContext.InternalParseContext context, XContentParser parser) throws IOException { + private static void internalParseDocument(Mapping mapping, ParseContext.InternalParseContext context, XContentParser parser) throws IOException { final boolean emptyDoc = isEmptyDoc(mapping, parser); - for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { + for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { metadataMapper.preParse(context); } @@ -96,7 +95,7 @@ final class DocumentParser { parseObjectOrNested(context, mapping.root); } - for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { + for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { metadataMapper.postParse(context); } } @@ -496,7 +495,7 @@ final class DocumentParser { if (builder == null) { builder = new ObjectMapper.Builder(currentFieldName).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); objectMapper = builder.build(builderContext); context.addDynamicMapper(objectMapper); context.path().add(currentFieldName); @@ -539,7 +538,7 @@ final class DocumentParser { if (builder == null) { parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName); } else { - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); mapper = builder.build(builderContext); assert mapper != null; if (mapper instanceof ArrayValueMapperParser) { @@ -697,13 +696,13 @@ final class DocumentParser { if (parseableAsLong && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { - builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); + builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); } return builder; } else if (parseableAsDouble && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DOUBLE); if (builder == null) { - builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); + builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); } return builder; } else if (parseableAsLong == false && parseableAsDouble == false && context.root().dateDetection()) { @@ -719,7 +718,7 @@ final class DocumentParser { } Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DATE); if (builder == null) { - builder = newDateBuilder(currentFieldName, dateTimeFormatter, context.indexSettings().getIndexVersionCreated()); + builder = newDateBuilder(currentFieldName, dateTimeFormatter, Version.indexCreated(context.indexSettings())); } if (builder instanceof DateFieldMapper.Builder) { DateFieldMapper.Builder dateBuilder = (DateFieldMapper.Builder) builder; @@ -742,7 +741,7 @@ final class DocumentParser { if (numberType == XContentParser.NumberType.INT || numberType == XContentParser.NumberType.LONG) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { - builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); + builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); } return builder; } else if (numberType == XContentParser.NumberType.FLOAT || numberType == XContentParser.NumberType.DOUBLE) { @@ -751,7 +750,7 @@ final class DocumentParser { // no templates are defined, we use float by default instead of double // since this is much more space-efficient and should be enough most of // the time - builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); + builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); } return builder; } @@ -786,7 +785,7 @@ final class DocumentParser { return; } final String path = context.path().pathAsText(currentFieldName); - final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); + final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); final MappedFieldType existingFieldType = context.mapperService().fullName(path); final Mapper.Builder builder; if (existingFieldType != null) { @@ -884,8 +883,8 @@ final class DocumentParser { if (builder == null) { builder = new ObjectMapper.Builder(paths[i]).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), - context.path()); mapper = (ObjectMapper) builder.build(builderContext); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + mapper = (ObjectMapper) builder.build(builderContext); if (mapper.nested() != ObjectMapper.Nested.NO) { throw new MapperParsingException("It is forbidden to create dynamic nested objects ([" + context.path().pathAsText(paths[i]) + "]) through `copy_to` or dots in field names"); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java index 8389a306270..606777392de 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java @@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; import org.apache.lucene.search.Query; import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.lucene.Lucene; @@ -204,12 +205,12 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper { } @Override - public void preParse(ParseContext context) { + public void preParse(ParseContext context) throws IOException { } @Override public void postParse(ParseContext context) throws IOException { - if (context.indexSettings().getIndexVersionCreated().before(Version.V_6_1_0)) { + if (context.indexSettings().getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).before(Version.V_6_1_0)) { super.parse(context); } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index cf8cc4022fd..b77ffee05ca 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -24,8 +24,9 @@ import com.carrotsearch.hppc.ObjectObjectMap; import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.IndexSettings; import java.util.ArrayList; import java.util.Collection; @@ -195,7 +196,7 @@ public abstract class ParseContext implements Iterable{ } @Override - public IndexSettings indexSettings() { + public Settings indexSettings() { return in.indexSettings(); } @@ -314,7 +315,8 @@ public abstract class ParseContext implements Iterable{ private final List documents; - private final IndexSettings indexSettings; + @Nullable + private final Settings indexSettings; private final SourceToParse sourceToParse; @@ -332,8 +334,8 @@ public abstract class ParseContext implements Iterable{ private final Set ignoredFields = new HashSet<>(); - public InternalParseContext(IndexSettings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, - SourceToParse source, XContentParser parser) { + public InternalParseContext(@Nullable Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, + SourceToParse source, XContentParser parser) { this.indexSettings = indexSettings; this.docMapper = docMapper; this.docMapperParser = docMapperParser; @@ -345,7 +347,7 @@ public abstract class ParseContext implements Iterable{ this.version = null; this.sourceToParse = source; this.dynamicMappers = new ArrayList<>(); - this.maxAllowedNumNestedDocs = indexSettings.getValue(MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING); + this.maxAllowedNumNestedDocs = MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING.get(indexSettings); this.numNestedDocs = 0L; } @@ -355,7 +357,8 @@ public abstract class ParseContext implements Iterable{ } @Override - public IndexSettings indexSettings() { + @Nullable + public Settings indexSettings() { return this.indexSettings; } @@ -562,7 +565,8 @@ public abstract class ParseContext implements Iterable{ return false; } - public abstract IndexSettings indexSettings(); + @Nullable + public abstract Settings indexSettings(); public abstract SourceToParse sourceToParse(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java index d2cf17ddd35..414cb3a98ec 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -83,17 +83,6 @@ public class ParsedDocument { this.seqID.primaryTerm.setLongValue(primaryTerm); } - /** - * Makes the processing document as a tombstone document rather than a regular document. - * Tombstone documents are stored in Lucene index to represent delete operations or Noops. - */ - ParsedDocument toTombstone() { - assert docs().size() == 1 : "Tombstone should have a single doc [" + docs() + "]"; - this.seqID.tombstoneField.setLongValue(1); - rootDoc().add(this.seqID.tombstoneField); - return this; - } - public String routing() { return this.routing; } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java index 5a0db4163bf..ac3ffe46272 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java @@ -69,29 +69,26 @@ public class SeqNoFieldMapper extends MetadataFieldMapper { public final Field seqNo; public final Field seqNoDocValue; public final Field primaryTerm; - public final Field tombstoneField; - public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm, Field tombstoneField) { + public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm) { Objects.requireNonNull(seqNo, "sequence number field cannot be null"); Objects.requireNonNull(seqNoDocValue, "sequence number dv field cannot be null"); Objects.requireNonNull(primaryTerm, "primary term field cannot be null"); this.seqNo = seqNo; this.seqNoDocValue = seqNoDocValue; this.primaryTerm = primaryTerm; - this.tombstoneField = tombstoneField; } public static SequenceIDFields emptySeqID() { return new SequenceIDFields(new LongPoint(NAME, SequenceNumbers.UNASSIGNED_SEQ_NO), new NumericDocValuesField(NAME, SequenceNumbers.UNASSIGNED_SEQ_NO), - new NumericDocValuesField(PRIMARY_TERM_NAME, 0), new NumericDocValuesField(TOMBSTONE_NAME, 0)); + new NumericDocValuesField(PRIMARY_TERM_NAME, 0)); } } public static final String NAME = "_seq_no"; public static final String CONTENT_TYPE = "_seq_no"; public static final String PRIMARY_TERM_NAME = "_primary_term"; - public static final String TOMBSTONE_NAME = "_tombstone"; public static class SeqNoDefaults { public static final String NAME = SeqNoFieldMapper.NAME; diff --git a/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java index 7bfe793bba4..f2090613c09 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.mapper; -import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; @@ -50,7 +49,6 @@ import java.util.function.Function; public class SourceFieldMapper extends MetadataFieldMapper { public static final String NAME = "_source"; - public static final String RECOVERY_SOURCE_NAME = "_recovery_source"; public static final String CONTENT_TYPE = "_source"; private final Function, Map> filter; @@ -226,8 +224,7 @@ public class SourceFieldMapper extends MetadataFieldMapper { @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { - BytesReference originalSource = context.sourceToParse().source(); - BytesReference source = originalSource; + BytesReference source = context.sourceToParse().source(); if (enabled && fieldType().stored() && source != null) { // Percolate and tv APIs may not set the source and that is ok, because these APIs will not index any data if (filter != null) { @@ -243,17 +240,8 @@ public class SourceFieldMapper extends MetadataFieldMapper { } BytesRef ref = source.toBytesRef(); fields.add(new StoredField(fieldType().name(), ref.bytes, ref.offset, ref.length)); - } else { - source = null; } - - if (originalSource != null && source != originalSource && context.indexSettings().isSoftDeleteEnabled()) { - // if we omitted source or modified it we add the _recovery_source to ensure we have it for ops based recovery - BytesRef ref = originalSource.toBytesRef(); - fields.add(new StoredField(RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); - fields.add(new NumericDocValuesField(RECOVERY_SOURCE_NAME, 1)); - } - } + } @Override protected String contentType() { diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index ef5f9ab0ef3..e030c95b56e 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -92,14 +92,12 @@ import org.elasticsearch.index.fielddata.ShardFieldData; 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.IdFieldMapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; @@ -1622,33 +1620,25 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl } /** - * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed + * Acquires a lock on the translog files, preventing them from being trimmed. */ - public Closeable acquireRetentionLockForPeerRecovery() { - return getEngine().acquireRetentionLockForPeerRecovery(); + public Closeable acquireTranslogRetentionLock() { + return getEngine().acquireTranslogRetentionLock(); } /** - * Returns the estimated number of history operations whose seq# at least the provided seq# in this shard. + * Creates a new translog snapshot for reading translog operations whose seq# at least the provided seq#. + * The caller has to close the returned snapshot after finishing the reading. */ - public int estimateNumberOfHistoryOperations(String source, long startingSeqNo) throws IOException { - return getEngine().estimateNumberOfHistoryOperations(source, mapperService, startingSeqNo); + public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { + return getEngine().newTranslogSnapshotFromMinSeqNo(minSeqNo); } /** - * Creates a new history snapshot for reading operations since the provided starting seqno (inclusive). - * The returned snapshot can be retrieved from either Lucene index or translog files. + * Returns the estimated number of operations in translog whose seq# at least the provided seq#. */ - public Translog.Snapshot getHistoryOperations(String source, long startingSeqNo) throws IOException { - return getEngine().readHistoryOperations(source, mapperService, startingSeqNo); - } - - /** - * Checks if we have a completed history of operations since the given starting seqno (inclusive). - * This method should be called after acquiring the retention lock; See {@link #acquireRetentionLockForPeerRecovery()} - */ - public boolean hasCompleteHistoryOperations(String source, long startingSeqNo) throws IOException { - return getEngine().hasCompleteOperationHistory(source, mapperService, startingSeqNo); + public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { + return getEngine().estimateTranslogOperationsFromMinSeq(minSeqNo); } public List segments(boolean verbose) { @@ -2219,7 +2209,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm, tombstoneDocSupplier()); + indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm); } /** @@ -2658,19 +2648,4 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl refreshMetric.inc(System.nanoTime() - currentRefreshStartTime); } } - - private EngineConfig.TombstoneDocSupplier tombstoneDocSupplier() { - final RootObjectMapper.Builder noopRootMapper = new RootObjectMapper.Builder("__noop"); - final DocumentMapper noopDocumentMapper = new DocumentMapper.Builder(noopRootMapper, mapperService).build(mapperService); - return new EngineConfig.TombstoneDocSupplier() { - @Override - public ParsedDocument newDeleteTombstoneDoc(String type, String id) { - return docMapper(type).getDocumentMapper().createDeleteTombstoneDoc(shardId.getIndexName(), type, id); - } - @Override - public ParsedDocument newNoopTombstoneDoc(String reason) { - return noopDocumentMapper.createNoopTombstoneDoc(shardId.getIndexName(), reason); - } - }; - } } diff --git a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java index 016a8afff69..1edc0eb5dca 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/shard/PrimaryReplicaSyncer.java @@ -89,7 +89,7 @@ public class PrimaryReplicaSyncer extends AbstractComponent { // Wrap translog snapshot to make it synchronized as it is accessed by different threads through SnapshotSender. // Even though those calls are not concurrent, snapshot.next() uses non-synchronized state and is not multi-thread-compatible // Also fail the resync early if the shard is shutting down - snapshot = indexShard.getHistoryOperations("resync", startingSeqNo); + snapshot = indexShard.newTranslogSnapshotFromMinSeqNo(startingSeqNo); final Translog.Snapshot originalSnapshot = snapshot; final Translog.Snapshot wrappedSnapshot = new Translog.Snapshot() { @Override diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index ae3f90e63e7..e9acfe3d8b0 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -156,7 +156,6 @@ final class StoreRecovery { final Directory hardLinkOrCopyTarget = new org.apache.lucene.store.HardlinkCopyDirectoryWrapper(target); IndexWriterConfig iwc = new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index 85975bc68c8..001e263ea8f 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1009,6 +1009,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref } final String segmentId = IndexFileNames.parseSegmentName(meta.name()); final String extension = IndexFileNames.getExtension(meta.name()); + assert FIELD_INFOS_FILE_EXTENSION.equals(extension) == false || IndexFileNames.stripExtension(IndexFileNames.stripSegmentName(meta.name())).isEmpty() : "FieldInfos are generational but updateable DV are not supported in elasticsearch"; if (IndexFileNames.SEGMENTS.equals(segmentId) || DEL_FILE_EXTENSION.equals(extension) || LIV_FILE_EXTENSION.equals(extension)) { // only treat del files as per-commit files fnm files are generational but only for upgradable DV perCommitStoreFiles.add(meta); @@ -1594,7 +1595,6 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref throws IOException { assert openMode == IndexWriterConfig.OpenMode.APPEND || commit == null : "can't specify create flag with a commit"; IndexWriterConfig iwc = new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) .setIndexCommit(commit) // we don't want merges to happen here - we call maybe merge on the engine diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index f17acac3789..618aa546e42 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1261,8 +1261,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC ", type='" + type + '\'' + ", seqNo=" + seqNo + ", primaryTerm=" + primaryTerm + - ", version=" + version + - ", autoGeneratedIdTimestamp=" + autoGeneratedIdTimestamp + '}'; } @@ -1405,7 +1403,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC "uid=" + uid + ", seqNo=" + seqNo + ", primaryTerm=" + primaryTerm + - ", version=" + version + '}'; } } diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index e0cfe9eaaff..f48f2ceb792 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -40,7 +40,6 @@ import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.util.HashMap; import java.util.Map; -import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongSupplier; @@ -193,24 +192,7 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable { new BufferedChecksumStreamInput(data.streamInput(), "assertion")); Translog.Operation prvOp = Translog.readOperation( new BufferedChecksumStreamInput(previous.v1().streamInput(), "assertion")); - // TODO: We haven't had timestamp for Index operations in Lucene yet, we need to loosen this check without timestamp. - final boolean sameOp; - if (newOp instanceof Translog.Index && prvOp instanceof Translog.Index) { - final Translog.Index o1 = (Translog.Index) prvOp; - final Translog.Index o2 = (Translog.Index) newOp; - sameOp = Objects.equals(o1.id(), o2.id()) && Objects.equals(o1.type(), o2.type()) - && Objects.equals(o1.source(), o2.source()) && Objects.equals(o1.routing(), o2.routing()) - && o1.primaryTerm() == o2.primaryTerm() && o1.seqNo() == o2.seqNo() - && o1.version() == o2.version(); - } else if (newOp instanceof Translog.Delete && prvOp instanceof Translog.Delete) { - final Translog.Delete o1 = (Translog.Delete) newOp; - final Translog.Delete o2 = (Translog.Delete) prvOp; - sameOp = Objects.equals(o1.id(), o2.id()) && Objects.equals(o1.type(), o2.type()) - && o1.primaryTerm() == o2.primaryTerm() && o1.seqNo() == o2.seqNo() && o1.version() == o2.version(); - } else { - sameOp = false; - } - if (sameOp == false) { + if (newOp.equals(prvOp) == false) { throw new AssertionError( "seqNo [" + seqNo + "] was processed twice in generation [" + generation + "], with different data. " + "prvOp [" + prvOp + "], newOp [" + newOp + "]", previous.v2()); diff --git a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java index a90f8af0af4..86995ae7c5a 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java @@ -32,7 +32,6 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.Lock; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.store.NativeFSLockFactory; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cli.EnvironmentAwareCommand; @@ -178,7 +177,6 @@ public class TruncateTranslogCommand extends EnvironmentAwareCommand { terminal.println("Marking index with the new history uuid"); // commit the new histroy id IndexWriterConfig iwc = new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 10f796e5e15..352f07d5764 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -146,11 +146,11 @@ public class RecoverySourceHandler { assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); - try (Closeable ignored = shard.acquireRetentionLockForPeerRecovery()) { + try (Closeable ignored = shard.acquireTranslogRetentionLock()) { final long startingSeqNo; final long requiredSeqNoRangeStart; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && - isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()); + isTargetSameHistory() && isTranslogReadyForSequenceNumberBasedRecovery(); if (isSequenceNumberBasedRecovery) { logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); startingSeqNo = request.startingSeqNo(); @@ -162,16 +162,14 @@ public class RecoverySourceHandler { } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); } - // We must have everything above the local checkpoint in the commit + // we set this to 0 to create a translog roughly according to the retention policy + // on the target. Note that it will still filter out legacy operations with no sequence numbers + startingSeqNo = 0; + // but we must have everything above the local checkpoint in the commit requiredSeqNoRangeStart = Long.parseLong(phase1Snapshot.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; - // If soft-deletes enabled, we need to transfer only operations after the local_checkpoint of the commit to have - // the same history on the target. However, with translog, we need to set this to 0 to create a translog roughly - // according to the retention policy on the target. Note that it will still filter out legacy operations without seqNo. - startingSeqNo = shard.indexSettings().isSoftDeleteEnabled() ? requiredSeqNoRangeStart : 0; try { - final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo); - phase1(phase1Snapshot.getIndexCommit(), () -> estimateNumOps); + phase1(phase1Snapshot.getIndexCommit(), () -> shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); } finally { @@ -188,8 +186,7 @@ public class RecoverySourceHandler { try { // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, - shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); + prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); } @@ -210,13 +207,11 @@ public class RecoverySourceHandler { */ cancellableThreads.execute(() -> shard.waitForOpsToComplete(endingSeqNo)); - if (logger.isTraceEnabled()) { - logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo); - logger.trace("snapshot translog for recovery; current size is [{}]", - shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); - } + logger.trace("all operations up to [{}] completed, which will be used as an ending sequence number", endingSeqNo); + + logger.trace("snapshot translog for recovery; current size is [{}]", shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); final long targetLocalCheckpoint; - try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) { + try(Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot); } catch (Exception e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); @@ -273,6 +268,36 @@ public class RecoverySourceHandler { }); } + /** + * Determines if the source translog is ready for a sequence-number-based peer recovery. The main condition here is that the source + * translog contains all operations above the local checkpoint on the target. We already know the that translog contains or will contain + * all ops above the source local checkpoint, so we can stop check there. + * + * @return {@code true} if the source is ready for a sequence-number-based recovery + * @throws IOException if an I/O exception occurred reading the translog snapshot + */ + boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException { + final long startingSeqNo = request.startingSeqNo(); + assert startingSeqNo >= 0; + final long localCheckpoint = shard.getLocalCheckpoint(); + logger.trace("testing sequence numbers in range: [{}, {}]", startingSeqNo, localCheckpoint); + // the start recovery request is initialized with the starting sequence number set to the target shard's local checkpoint plus one + if (startingSeqNo - 1 <= localCheckpoint) { + final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); + try (Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { + Translog.Operation operation; + while ((operation = snapshot.next()) != null) { + if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { + tracker.markSeqNoAsCompleted(operation.seqNo()); + } + } + } + return tracker.getCheckpoint() >= localCheckpoint; + } else { + return false; + } + } + /** * Perform phase1 of the recovery operations. Once this {@link IndexCommit} * snapshot has been performed no commit operations (files being fsync'd) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 9469f657c96..a4d6518e9af 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1492,7 +1492,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp // empty shard would cause exceptions to be thrown. Since there is no data to restore from an empty // shard anyway, we just create the empty shard here and then exit. IndexWriter writer = new IndexWriter(store.directory(), new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setOpenMode(IndexWriterConfig.OpenMode.CREATE) .setCommitOnClose(true)); writer.close(); diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 6acdbad2cce..702d63d0d94 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -64,7 +64,6 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.repositories.IndexId; @@ -121,8 +120,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateApp SETTING_NUMBER_OF_SHARDS, SETTING_VERSION_CREATED, SETTING_INDEX_UUID, - SETTING_CREATION_DATE, - IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey())); + SETTING_CREATION_DATE)); // It's OK to change some settings, but we shouldn't allow simply removing them private static final Set UNREMOVABLE_SETTINGS; diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 9786c0eaf52..90173455c3b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -392,7 +392,6 @@ public class PrimaryAllocationIT extends ESIntegTestCase { assertThat(shard.getLocalCheckpoint(), equalTo(numDocs + moreDocs)); } }, 30, TimeUnit.SECONDS); - internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); } } diff --git a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java index 890f6ef163b..753aedea01e 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java @@ -33,23 +33,18 @@ import org.apache.lucene.index.NoDeletionPolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; -import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; import org.apache.lucene.store.MMapDirectory; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.Bits; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.test.ESTestCase; import java.io.IOException; -import java.io.StringReader; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -58,8 +53,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; -import static org.hamcrest.Matchers.equalTo; - public class LuceneTests extends ESTestCase { public void testWaitForIndex() throws Exception { final MockDirectoryWrapper dir = newMockDirectory(); @@ -413,88 +406,4 @@ public class LuceneTests extends ESTestCase { // add assume's here if needed for certain platforms, but we should know if it does not work. assertTrue("MMapDirectory does not support unmapping: " + MMapDirectory.UNMAP_NOT_SUPPORTED_REASON, MMapDirectory.UNMAP_SUPPORTED); } - - public void testWrapAllDocsLive() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) - .setMergePolicy(new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, MatchAllDocsQuery::new, newMergePolicy())); - IndexWriter writer = new IndexWriter(dir, config); - int numDocs = between(1, 10); - Set liveDocs = new HashSet<>(); - for (int i = 0; i < numDocs; i++) { - String id = Integer.toString(i); - Document doc = new Document(); - doc.add(new StringField("id", id, Store.YES)); - writer.addDocument(doc); - liveDocs.add(id); - } - for (int i = 0; i < numDocs; i++) { - if (randomBoolean()) { - String id = Integer.toString(i); - Document doc = new Document(); - doc.add(new StringField("id", "v2-" + id, Store.YES)); - if (randomBoolean()) { - doc.add(Lucene.newSoftDeletesField()); - } - writer.softUpdateDocument(new Term("id", id), doc, Lucene.newSoftDeletesField()); - liveDocs.add("v2-" + id); - } - } - try (DirectoryReader unwrapped = DirectoryReader.open(writer)) { - DirectoryReader reader = Lucene.wrapAllDocsLive(unwrapped); - assertThat(reader.numDocs(), equalTo(liveDocs.size())); - IndexSearcher searcher = new IndexSearcher(reader); - Set actualDocs = new HashSet<>(); - TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), Integer.MAX_VALUE); - for (ScoreDoc scoreDoc : topDocs.scoreDocs) { - actualDocs.add(reader.document(scoreDoc.doc).get("id")); - } - assertThat(actualDocs, equalTo(liveDocs)); - } - IOUtils.close(writer, dir); - } - - public void testWrapLiveDocsNotExposeAbortedDocuments() throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig config = newIndexWriterConfig().setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) - .setMergePolicy(new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, MatchAllDocsQuery::new, newMergePolicy())); - IndexWriter writer = new IndexWriter(dir, config); - int numDocs = between(1, 10); - List liveDocs = new ArrayList<>(); - for (int i = 0; i < numDocs; i++) { - String id = Integer.toString(i); - Document doc = new Document(); - doc.add(new StringField("id", id, Store.YES)); - if (randomBoolean()) { - doc.add(Lucene.newSoftDeletesField()); - } - writer.addDocument(doc); - liveDocs.add(id); - } - int abortedDocs = between(1, 10); - for (int i = 0; i < abortedDocs; i++) { - try { - Document doc = new Document(); - doc.add(new StringField("id", "aborted-" + i, Store.YES)); - StringReader reader = new StringReader(""); - doc.add(new TextField("other", reader)); - reader.close(); // mark the indexing hit non-aborting error - writer.addDocument(doc); - fail("index should have failed"); - } catch (Exception ignored) { } - } - try (DirectoryReader unwrapped = DirectoryReader.open(writer)) { - DirectoryReader reader = Lucene.wrapAllDocsLive(unwrapped); - assertThat(reader.maxDoc(), equalTo(numDocs + abortedDocs)); - assertThat(reader.numDocs(), equalTo(liveDocs.size())); - IndexSearcher searcher = new IndexSearcher(reader); - List actualDocs = new ArrayList<>(); - TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), Integer.MAX_VALUE); - for (ScoreDoc scoreDoc : topDocs.scoreDocs) { - actualDocs.add(reader.document(scoreDoc.doc).get("id")); - } - assertThat(actualDocs, equalTo(liveDocs)); - } - IOUtils.close(writer, dir); - } } diff --git a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java index ac2f2b0d4f3..6bdd8ea3f2e 100644 --- a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java +++ b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java @@ -109,7 +109,6 @@ public abstract class AbstractDisruptionTestCase extends ESIntegTestCase { protected void beforeIndexDeletion() throws Exception { if (disableBeforeIndexDeletion == false) { super.beforeIndexDeletion(); - internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); assertSeqNos(); } } diff --git a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index b0b6c35f92a..d098c4918a7 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -40,7 +40,6 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; -import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -398,8 +397,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { .get(); logger.info("--> indexing docs"); - int numDocs = randomIntBetween(1, 1024); - for (int i = 0; i < numDocs; i++) { + for (int i = 0; i < randomIntBetween(1, 1024); i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -421,15 +419,12 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { } logger.info("--> restart replica node"); - boolean softDeleteEnabled = internalCluster().getInstance(IndicesService.class, primaryNode) - .indexServiceSafe(resolveIndex("test")).getShard(0).indexSettings().isSoftDeleteEnabled(); - int moreDocs = randomIntBetween(1, 1024); internalCluster().restartNode(replicaNode, new RestartCallback() { @Override public Settings onNodeStopped(String nodeName) throws Exception { // index some more documents; we expect to reuse the files that already exist on the replica - for (int i = 0; i < moreDocs; i++) { + for (int i = 0; i < randomIntBetween(1, 1024); i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -437,12 +432,8 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase { client(primaryNode).admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder() .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) ).get(); client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); - if (softDeleteEnabled) { // We need an extra flush to advance the min_retained_seqno of the SoftDeletesPolicy - client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); - } return super.onNodeStopped(nodeName); } }); diff --git a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java index b0b4ec3930a..28fa440d96a 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -307,7 +306,7 @@ public class IndexServiceTests extends ESSingleNodeTestCase { .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), -1)) .get(); IndexShard shard = indexService.getShard(0); - assertBusy(() -> assertThat(IndexShardTestCase.getTranslog(shard).totalOperations(), equalTo(0))); + assertBusy(() -> assertThat(shard.estimateTranslogOperationsFromMinSeq(0L), equalTo(0))); } public void testIllegalFsyncInterval() { diff --git a/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java b/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java index 64a2fa69bcb..b7da5add2ac 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java @@ -553,12 +553,4 @@ public class IndexSettingsTests extends ESTestCase { ); assertThat(index.getDefaultFields(), equalTo(Arrays.asList("body", "title"))); } - - public void testUpdateSoftDeletesFails() { - IndexScopedSettings settings = new IndexScopedSettings(Settings.EMPTY, IndexScopedSettings.BUILT_IN_INDEX_SETTINGS); - IllegalArgumentException error = expectThrows(IllegalArgumentException.class, () -> - settings.updateSettings(Settings.builder().put("index.soft_deletes.enabled", randomBoolean()).build(), - Settings.builder(), Settings.builder(), "index")); - assertThat(error.getMessage(), equalTo("final index setting [index.soft_deletes.enabled], not updateable")); - } } diff --git a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index 3f9fc9a0429..ea7de50b7b3 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -51,24 +51,20 @@ public class CombinedDeletionPolicyTests extends ESTestCase { public void testKeepCommitsAfterGlobalCheckpoint() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); - final int extraRetainedOps = between(0, 100); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, extraRetainedOps); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); final LongArrayList maxSeqNoList = new LongArrayList(); final LongArrayList translogGenList = new LongArrayList(); final List commitList = new ArrayList<>(); int totalCommits = between(2, 20); long lastMaxSeqNo = 0; - long lastCheckpoint = lastMaxSeqNo; long lastTranslogGen = 0; final UUID translogUUID = UUID.randomUUID(); for (int i = 0; i < totalCommits; i++) { lastMaxSeqNo += between(1, 10000); - lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); lastTranslogGen += between(1, 100); - commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); maxSeqNoList.add(lastMaxSeqNo); translogGenList.add(lastTranslogGen); } @@ -89,19 +85,14 @@ public class CombinedDeletionPolicyTests extends ESTestCase { } assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(translogGenList.get(keptIndex))); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(Math.min(getLocalCheckpoint(commitList.get(keptIndex)) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } public void testAcquireIndexCommit() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); - final int extraRetainedOps = between(0, 100); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, extraRetainedOps); final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); long lastMaxSeqNo = between(1, 1000); - long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); long lastTranslogGen = between(1, 20); int safeIndex = 0; List commitList = new ArrayList<>(); @@ -111,9 +102,8 @@ public class CombinedDeletionPolicyTests extends ESTestCase { int newCommits = between(1, 10); for (int n = 0; n < newCommits; n++) { lastMaxSeqNo += between(1, 1000); - lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); lastTranslogGen += between(1, 20); - commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); } // Advance the global checkpoint to between [safeIndex, safeIndex + 1) safeIndex = randomIntBetween(safeIndex, commitList.size() - 1); @@ -124,9 +114,6 @@ public class CombinedDeletionPolicyTests extends ESTestCase { globalCheckpoint.set(randomLongBetween(lower, upper)); commitList.forEach(this::resetDeletion); indexPolicy.onCommit(commitList); - IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(Math.min(getLocalCheckpoint(safeCommit) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); // Captures and releases some commits int captures = between(0, 5); for (int n = 0; n < captures; n++) { @@ -145,7 +132,7 @@ public class CombinedDeletionPolicyTests extends ESTestCase { snapshottingCommits.remove(snapshot); final long pendingSnapshots = snapshottingCommits.stream().filter(snapshot::equals).count(); final IndexCommit lastCommit = commitList.get(commitList.size() - 1); - safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); + final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); assertThat(indexPolicy.releaseCommit(snapshot), equalTo(pendingSnapshots == 0 && snapshot.equals(lastCommit) == false && snapshot.equals(safeCommit) == false)); } @@ -156,8 +143,6 @@ public class CombinedDeletionPolicyTests extends ESTestCase { equalTo(Long.parseLong(commitList.get(safeIndex).getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(Long.parseLong(commitList.get(commitList.size() - 1).getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(Math.min(getLocalCheckpoint(commitList.get(safeIndex)) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } snapshottingCommits.forEach(indexPolicy::releaseCommit); globalCheckpoint.set(randomLongBetween(lastMaxSeqNo, Long.MAX_VALUE)); @@ -169,27 +154,25 @@ public class CombinedDeletionPolicyTests extends ESTestCase { assertThat(commitList.get(commitList.size() - 1).isDeleted(), equalTo(false)); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(lastTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); - IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(Math.min(getLocalCheckpoint(safeCommit) + 1, globalCheckpoint.get() + 1 - extraRetainedOps))); } public void testLegacyIndex() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); long legacyTranslogGen = randomNonNegativeLong(); IndexCommit legacyCommit = mockLegacyIndexCommit(translogUUID, legacyTranslogGen); - assertThat(CombinedDeletionPolicy.findSafeCommitPoint(singletonList(legacyCommit), globalCheckpoint.get()), - equalTo(legacyCommit)); + indexPolicy.onCommit(singletonList(legacyCommit)); + verify(legacyCommit, never()).delete(); + assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(legacyTranslogGen)); + assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(legacyTranslogGen)); long safeTranslogGen = randomLongBetween(legacyTranslogGen, Long.MAX_VALUE); long maxSeqNo = randomLongBetween(1, Long.MAX_VALUE); - final IndexCommit freshCommit = mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, translogUUID, safeTranslogGen); + final IndexCommit freshCommit = mockIndexCommit(maxSeqNo, translogUUID, safeTranslogGen); globalCheckpoint.set(randomLongBetween(0, maxSeqNo - 1)); indexPolicy.onCommit(Arrays.asList(legacyCommit, freshCommit)); @@ -206,32 +189,25 @@ public class CombinedDeletionPolicyTests extends ESTestCase { verify(freshCommit, times(0)).delete(); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(safeTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(safeTranslogGen)); - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), equalTo(getLocalCheckpoint(freshCommit) + 1)); } public void testDeleteInvalidCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(randomNonNegativeLong()); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); final int invalidCommits = between(1, 10); final List commitList = new ArrayList<>(); for (int i = 0; i < invalidCommits; i++) { - long maxSeqNo = randomNonNegativeLong(); - commitList.add(mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, UUID.randomUUID(), randomNonNegativeLong())); + commitList.add(mockIndexCommit(randomNonNegativeLong(), UUID.randomUUID(), randomNonNegativeLong())); } final UUID expectedTranslogUUID = UUID.randomUUID(); long lastTranslogGen = 0; final int validCommits = between(1, 10); - long lastMaxSeqNo = between(1, 1000); - long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); for (int i = 0; i < validCommits; i++) { lastTranslogGen += between(1, 1000); - lastMaxSeqNo += between(1, 1000); - lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); - commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, expectedTranslogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(randomNonNegativeLong(), expectedTranslogUUID, lastTranslogGen)); } // We should never keep invalid commits regardless of the value of the global checkpoint. @@ -239,26 +215,21 @@ public class CombinedDeletionPolicyTests extends ESTestCase { for (int i = 0; i < invalidCommits - 1; i++) { verify(commitList.get(i), times(1)).delete(); } - assertThat(softDeletesPolicy.getMinRetainedSeqNo(), - equalTo(getLocalCheckpoint(CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get())) + 1)); } public void testCheckUnreferencedCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); - final SoftDeletesPolicy softDeletesPolicy = new SoftDeletesPolicy(globalCheckpoint::get, -1, 0); final UUID translogUUID = UUID.randomUUID(); final TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, globalCheckpoint::get); final List commitList = new ArrayList<>(); int totalCommits = between(2, 20); long lastMaxSeqNo = between(1, 1000); - long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); long lastTranslogGen = between(1, 50); for (int i = 0; i < totalCommits; i++) { lastMaxSeqNo += between(1, 10000); lastTranslogGen += between(1, 100); - lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); - commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); } IndexCommit safeCommit = randomFrom(commitList); globalCheckpoint.set(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO))); @@ -285,9 +256,8 @@ public class CombinedDeletionPolicyTests extends ESTestCase { } } - IndexCommit mockIndexCommit(long localCheckpoint, long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { + IndexCommit mockIndexCommit(long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); - userData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint)); userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGen)); @@ -308,10 +278,6 @@ public class CombinedDeletionPolicyTests extends ESTestCase { }).when(commit).delete(); } - private long getLocalCheckpoint(IndexCommit commit) throws IOException { - return Long.parseLong(commit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); - } - IndexCommit mockLegacyIndexCommit(UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); @@ -321,5 +287,4 @@ public class CombinedDeletionPolicyTests extends ESTestCase { resetDeletion(commit); return commit; } - } diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index d3aead9e44e..76e05ba1e0b 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.engine; -import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; import java.nio.charset.Charset; @@ -78,12 +77,10 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.LogByteSizeMergePolicy; import org.apache.lucene.index.LogDocMergePolicy; -import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PointValues; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.index.TieredMergePolicy; import org.apache.lucene.search.IndexSearcher; @@ -117,7 +114,6 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.Loggers; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; @@ -137,7 +133,6 @@ import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.ContentPath; 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; import org.elasticsearch.index.mapper.MetadataFieldMapper; import org.elasticsearch.index.mapper.ParseContext; @@ -177,10 +172,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.isIn; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -254,13 +247,8 @@ public class InternalEngineTests extends EngineTestCase { } public void testSegments() throws Exception { - Settings settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - IndexSettings indexSettings = IndexSettingsModule.newIndexSettings( - IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); try (Store store = createStore(); - InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null))) { + InternalEngine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { List segments = engine.segments(false); assertThat(segments.isEmpty(), equalTo(true)); assertThat(engine.segmentsStats(false).getCount(), equalTo(0L)); @@ -1323,13 +1311,9 @@ public class InternalEngineTests extends EngineTestCase { assertThat(indexResult.getVersion(), equalTo(1L)); } - public void testForceMergeWithoutSoftDeletes() throws IOException { - Settings settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + public void testForceMerge() throws IOException { try (Store store = createStore(); - Engine engine = createEngine(config(IndexSettingsModule.newIndexSettings(indexMetaData), store, createTempDir(), + Engine engine = createEngine(config(defaultSettings, store, createTempDir(), 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++) { @@ -1370,165 +1354,6 @@ public class InternalEngineTests extends EngineTestCase { } } - public void testForceMergeWithSoftDeletesRetention() throws Exception { - final long retainedExtraOps = randomLongBetween(0, 10); - Settings.Builder settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), retainedExtraOps); - final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - final MapperService mapperService = createMapperService("test"); - final Set liveDocs = new HashSet<>(); - try (Store store = createStore(); - InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get))) { - int numDocs = scaledRandomIntBetween(10, 100); - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); - engine.index(indexForDoc(doc)); - liveDocs.add(doc.id()); - } - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); - if (randomBoolean()) { - engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); - liveDocs.remove(doc.id()); - } - if (randomBoolean()) { - engine.index(indexForDoc(doc)); - liveDocs.add(doc.id()); - } - if (randomBoolean()) { - engine.flush(randomBoolean(), true); - } - } - engine.flush(); - - long localCheckpoint = engine.getLocalCheckpoint(); - globalCheckpoint.set(randomLongBetween(0, localCheckpoint)); - engine.syncTranslog(); - final long safeCommitCheckpoint; - try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { - safeCommitCheckpoint = Long.parseLong(safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); - } - engine.forceMerge(true, 1, false, false, false); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - Map ops = readAllOperationsInLucene(engine, mapperService) - .stream().collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); - for (long seqno = 0; seqno <= localCheckpoint; seqno++) { - long minSeqNoToRetain = Math.min(globalCheckpoint.get() + 1 - retainedExtraOps, safeCommitCheckpoint + 1); - String msg = "seq# [" + seqno + "], global checkpoint [" + globalCheckpoint + "], retained-ops [" + retainedExtraOps + "]"; - if (seqno < minSeqNoToRetain) { - Translog.Operation op = ops.get(seqno); - if (op != null) { - assertThat(op, instanceOf(Translog.Index.class)); - assertThat(msg, ((Translog.Index) op).id(), isIn(liveDocs)); - assertEquals(msg, ((Translog.Index) op).source(), B_1); - } - } else { - assertThat(msg, ops.get(seqno), notNullValue()); - } - } - settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0); - indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); - engine.onSettingsChanged(); - globalCheckpoint.set(localCheckpoint); - engine.syncTranslog(); - - engine.forceMerge(true, 1, false, false, false); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - assertThat(readAllOperationsInLucene(engine, mapperService), hasSize(liveDocs.size())); - } - } - - public void testForceMergeWithSoftDeletesRetentionAndRecoverySource() throws Exception { - final long retainedExtraOps = randomLongBetween(0, 10); - Settings.Builder settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), retainedExtraOps); - final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - final MapperService mapperService = createMapperService("test"); - final boolean omitSourceAllTheTime = randomBoolean(); - final Set liveDocs = new HashSet<>(); - final Set liveDocsWithSource = new HashSet<>(); - try (Store store = createStore(); - InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, - globalCheckpoint::get))) { - int numDocs = scaledRandomIntBetween(10, 100); - for (int i = 0; i < numDocs; i++) { - boolean useRecoverySource = randomBoolean() || omitSourceAllTheTime; - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null, useRecoverySource); - engine.index(indexForDoc(doc)); - liveDocs.add(doc.id()); - if (useRecoverySource == false) { - liveDocsWithSource.add(Integer.toString(i)); - } - } - for (int i = 0; i < numDocs; i++) { - boolean useRecoverySource = randomBoolean() || omitSourceAllTheTime; - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null, useRecoverySource); - if (randomBoolean()) { - engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); - liveDocs.remove(doc.id()); - liveDocsWithSource.remove(doc.id()); - } - if (randomBoolean()) { - engine.index(indexForDoc(doc)); - liveDocs.add(doc.id()); - if (useRecoverySource == false) { - liveDocsWithSource.add(doc.id()); - } else { - liveDocsWithSource.remove(doc.id()); - } - } - if (randomBoolean()) { - engine.flush(randomBoolean(), true); - } - } - engine.flush(); - globalCheckpoint.set(randomLongBetween(0, engine.getLocalCheckpoint())); - engine.syncTranslog(); - final long minSeqNoToRetain; - try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { - long safeCommitLocalCheckpoint = Long.parseLong( - safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); - minSeqNoToRetain = Math.min(globalCheckpoint.get() + 1 - retainedExtraOps, safeCommitLocalCheckpoint + 1); - } - engine.forceMerge(true, 1, false, false, false); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - Map ops = readAllOperationsInLucene(engine, mapperService) - .stream().collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); - for (long seqno = 0; seqno <= engine.getLocalCheckpoint(); seqno++) { - String msg = "seq# [" + seqno + "], global checkpoint [" + globalCheckpoint + "], retained-ops [" + retainedExtraOps + "]"; - if (seqno < minSeqNoToRetain) { - Translog.Operation op = ops.get(seqno); - if (op != null) { - assertThat(op, instanceOf(Translog.Index.class)); - assertThat(msg, ((Translog.Index) op).id(), isIn(liveDocs)); - } - } else { - Translog.Operation op = ops.get(seqno); - assertThat(msg, op, notNullValue()); - if (op instanceof Translog.Index) { - assertEquals(msg, ((Translog.Index) op).source(), B_1); - } - } - } - settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0); - indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); - engine.onSettingsChanged(); - globalCheckpoint.set(engine.getLocalCheckpoint()); - engine.syncTranslog(); - engine.forceMerge(true, 1, false, false, false); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - assertThat(readAllOperationsInLucene(engine, mapperService), hasSize(liveDocsWithSource.size())); - } - } - public void testForceMergeAndClose() throws IOException, InterruptedException { int numIters = randomIntBetween(2, 10); for (int j = 0; j < numIters; j++) { @@ -1597,10 +1422,126 @@ public class InternalEngineTests extends EngineTestCase { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } + protected List generateSingleDocHistory(boolean forReplica, VersionType versionType, + long primaryTerm, + int minOpCount, int maxOpCount, String docId) { + final int numOfOps = randomIntBetween(minOpCount, maxOpCount); + final List ops = new ArrayList<>(); + final Term id = newUid(docId); + final int startWithSeqNo = 0; + final String valuePrefix = (forReplica ? "r_" : "p_" ) + docId + "_"; + final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); + for (int i = 0; i < numOfOps; i++) { + final Engine.Operation op; + final long version; + switch (versionType) { + case INTERNAL: + version = forReplica ? i : Versions.MATCH_ANY; + break; + case EXTERNAL: + version = i; + break; + case EXTERNAL_GTE: + version = randomBoolean() ? Math.max(i - 1, 0) : i; + break; + case FORCE: + version = randomNonNegativeLong(); + break; + default: + throw new UnsupportedOperationException("unknown version type: " + versionType); + } + if (randomBoolean()) { + op = new Engine.Index(id, testParsedDocument(docId, null, testDocumentWithTextField(valuePrefix + i), B_1, null), + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, + version, + forReplica ? null : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis(), -1, false + ); + } else { + op = new Engine.Delete("test", docId, id, + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, + version, + forReplica ? null : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis()); + } + ops.add(op); + } + return ops; + } + public void testOutOfOrderDocsOnReplica() throws IOException { final List ops = generateSingleDocHistory(true, randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE, VersionType.FORCE), 2, 2, 20, "1"); - assertOpsOnReplica(ops, replicaEngine, true, logger); + assertOpsOnReplica(ops, replicaEngine, true); + } + + private void assertOpsOnReplica(List ops, InternalEngine replicaEngine, boolean shuffleOps) throws IOException { + final Engine.Operation lastOp = ops.get(ops.size() - 1); + final String lastFieldValue; + if (lastOp instanceof Engine.Index) { + Engine.Index index = (Engine.Index) lastOp; + lastFieldValue = index.docs().get(0).get("value"); + } else { + // delete + lastFieldValue = null; + } + if (shuffleOps) { + int firstOpWithSeqNo = 0; + while (firstOpWithSeqNo < ops.size() && ops.get(firstOpWithSeqNo).seqNo() < 0) { + firstOpWithSeqNo++; + } + // shuffle ops but make sure legacy ops are first + shuffle(ops.subList(0, firstOpWithSeqNo), random()); + shuffle(ops.subList(firstOpWithSeqNo, ops.size()), random()); + } + boolean firstOp = true; + for (Engine.Operation op : ops) { + logger.info("performing [{}], v [{}], seq# [{}], term [{}]", + op.operationType().name().charAt(0), op.version(), op.seqNo(), op.primaryTerm()); + if (op instanceof Engine.Index) { + Engine.IndexResult result = replicaEngine.index((Engine.Index) op); + // replicas don't really care to about creation status of documents + // this allows to ignore the case where a document was found in the live version maps in + // a delete state and return false for the created flag in favor of code simplicity + // as deleted or not. This check is just signal regression so a decision can be made if it's + // intentional + assertThat(result.isCreated(), equalTo(firstOp)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); + + } else { + Engine.DeleteResult result = replicaEngine.delete((Engine.Delete) op); + // Replicas don't really care to about found status of documents + // this allows to ignore the case where a document was found in the live version maps in + // a delete state and return true for the found flag in favor of code simplicity + // his check is just signal regression so a decision can be made if it's + // intentional + assertThat(result.isFound(), equalTo(firstOp == false)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); + } + if (randomBoolean()) { + engine.refresh("test"); + } + if (randomBoolean()) { + engine.flush(); + engine.refresh("test"); + } + firstOp = false; + } + + assertVisibleCount(replicaEngine, lastFieldValue == null ? 0 : 1); + if (lastFieldValue != null) { + try (Searcher searcher = replicaEngine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); + assertThat(collector.getTotalHits(), equalTo(1)); + } + } } public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, InterruptedException { @@ -1628,12 +1569,11 @@ public class InternalEngineTests extends EngineTestCase { } // randomly interleave final AtomicLong seqNoGenerator = new AtomicLong(); - BiFunction seqNoUpdater = (operation, newSeqNo) -> { + Function seqNoUpdater = operation -> { + final long newSeqNo = seqNoGenerator.getAndIncrement(); if (operation instanceof Engine.Index) { Engine.Index index = (Engine.Index) operation; - Document doc = testDocumentWithTextField(index.docs().get(0).get("value")); - ParsedDocument parsedDocument = testParsedDocument(index.id(), index.routing(), doc, index.source(), null); - return new Engine.Index(index.uid(), parsedDocument, newSeqNo, index.primaryTerm(), index.version(), + return new Engine.Index(index.uid(), index.parsedDoc(), newSeqNo, index.primaryTerm(), index.version(), index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry()); } else { Engine.Delete delete = (Engine.Delete) operation; @@ -1646,12 +1586,12 @@ public class InternalEngineTests extends EngineTestCase { Iterator iter2 = opsDoc2.iterator(); while (iter1.hasNext() && iter2.hasNext()) { final Engine.Operation next = randomBoolean() ? iter1.next() : iter2.next(); - allOps.add(seqNoUpdater.apply(next, seqNoGenerator.getAndIncrement())); + allOps.add(seqNoUpdater.apply(next)); } - iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); - iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); + iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); + iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); // insert some duplicates - randomSubsetOf(allOps).forEach(op -> allOps.add(seqNoUpdater.apply(op, op.seqNo()))); + allOps.addAll(randomSubsetOf(allOps)); shuffle(allOps, random()); concurrentlyApplyOps(allOps, engine); @@ -1683,6 +1623,42 @@ public class InternalEngineTests extends EngineTestCase { assertVisibleCount(engine, totalExpectedOps); } + private void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { + Thread[] thread = new Thread[randomIntBetween(3, 5)]; + CountDownLatch startGun = new CountDownLatch(thread.length); + AtomicInteger offset = new AtomicInteger(-1); + for (int i = 0; i < thread.length; i++) { + thread[i] = new Thread(() -> { + startGun.countDown(); + try { + startGun.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + int docOffset; + while ((docOffset = offset.incrementAndGet()) < ops.size()) { + try { + final Engine.Operation op = ops.get(docOffset); + if (op instanceof Engine.Index) { + engine.index((Engine.Index) op); + } else { + engine.delete((Engine.Delete) op); + } + if ((docOffset + 1) % 4 == 0) { + engine.refresh("test"); + } + } catch (IOException e) { + throw new AssertionError(e); + } + } + }); + thread[i].start(); + } + for (int i = 0; i < thread.length; i++) { + thread[i].join(); + } + } + public void testInternalVersioningOnPrimary() throws IOException { final List ops = generateSingleDocHistory(false, VersionType.INTERNAL, 2, 2, 20, "1"); assertOpsOnPrimary(ops, Versions.NOT_FOUND, true, engine); @@ -1893,7 +1869,7 @@ public class InternalEngineTests extends EngineTestCase { final boolean deletedOnReplica = lastReplicaOp instanceof Engine.Delete; final long finalReplicaVersion = lastReplicaOp.version(); final long finalReplicaSeqNo = lastReplicaOp.seqNo(); - assertOpsOnReplica(replicaOps, replicaEngine, true, logger); + assertOpsOnReplica(replicaOps, replicaEngine, true); final int opsOnPrimary = assertOpsOnPrimary(primaryOps, finalReplicaVersion, deletedOnReplica, replicaEngine); final long currentSeqNo = getSequenceID(replicaEngine, new Engine.Get(false, false, "type", lastReplicaOp.uid().text(), lastReplicaOp.uid())).v1(); @@ -2698,16 +2674,14 @@ public class InternalEngineTests extends EngineTestCase { Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } - EngineConfig config = engine.config(); assertVisibleCount(engine, numDocs); engine.close(); - trimUnsafeCommits(config); - try (InternalEngine engine = new InternalEngine(config)) { - engine.skipTranslogRecovery(); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(0L)); - } + trimUnsafeCommits(engine.config()); + engine = new InternalEngine(engine.config()); + engine.skipTranslogRecovery(); + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); + assertThat(topDocs.totalHits, equalTo(0L)); } } @@ -2837,7 +2811,7 @@ public class InternalEngineTests extends EngineTestCase { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get, tombstoneDocSupplier()); + new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get); try { InternalEngine internalEngine = new InternalEngine(brokenConfig); fail("translog belongs to a different engine"); @@ -2966,12 +2940,6 @@ public class InternalEngineTests extends EngineTestCase { } } - @Override - public long softUpdateDocument(Term term, Iterable doc, Field... softDeletes) throws IOException { - maybeThrowFailure(); - return super.softUpdateDocument(term, doc, softDeletes); - } - @Override public long deleteDocuments(Term... terms) throws IOException { maybeThrowFailure(); @@ -3172,10 +3140,10 @@ public class InternalEngineTests extends EngineTestCase { } public void testDoubleDeliveryReplicaAppendingOnly() throws IOException { - final Supplier doc = () -> testParsedDocument("1", null, testDocumentWithTextField(), + final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); - Engine.Index operation = appendOnlyReplica(doc.get(), false, 1, randomIntBetween(0, 5)); - Engine.Index retry = appendOnlyReplica(doc.get(), true, 1, randomIntBetween(0, 5)); + Engine.Index operation = appendOnlyReplica(doc, false, 1, randomIntBetween(0, 5)); + Engine.Index retry = appendOnlyReplica(doc, true, 1, randomIntBetween(0, 5)); // operations with a seq# equal or lower to the local checkpoint are not indexed to lucene // and the version lookup is skipped final boolean belowLckp = operation.seqNo() == 0 && retry.seqNo() == 0; @@ -3214,8 +3182,8 @@ public class InternalEngineTests extends EngineTestCase { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } - operation = randomAppendOnly(doc.get(), false, 1); - retry = randomAppendOnly(doc.get(), true, 1); + operation = randomAppendOnly(doc, false, 1); + retry = randomAppendOnly(doc, true, 1); if (randomBoolean()) { Engine.IndexResult indexResult = engine.index(operation); assertNotNull(indexResult.getTranslogLocation()); @@ -3280,8 +3248,6 @@ public class InternalEngineTests extends EngineTestCase { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } - List ops = readAllOperationsInLucene(engine, createMapperService("test")); - assertThat(ops.stream().map(o -> o.seqNo()).collect(Collectors.toList()), hasItem(20L)); } public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOException { @@ -3750,22 +3716,20 @@ public class InternalEngineTests extends EngineTestCase { final List operations = new ArrayList<>(); final int numberOfOperations = randomIntBetween(16, 32); + final Document document = testDocumentWithTextField(); final AtomicLong sequenceNumber = new AtomicLong(); final Engine.Operation.Origin origin = randomFrom(LOCAL_TRANSLOG_RECOVERY, PEER_RECOVERY, PRIMARY, REPLICA); final LongSupplier sequenceNumberSupplier = origin == PRIMARY ? () -> SequenceNumbers.UNASSIGNED_SEQ_NO : sequenceNumber::getAndIncrement; - final Supplier doc = () -> { - final Document document = testDocumentWithTextField(); - document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - return testParsedDocument("1", null, document, B_1, null); - }; - final Term uid = newUid("1"); + document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); + final ParsedDocument doc = testParsedDocument("1", null, document, B_1, null); + final Term uid = newUid(doc); final BiFunction searcherFactory = engine::acquireSearcher; for (int i = 0; i < numberOfOperations; i++) { if (randomBoolean()) { final Engine.Index index = new Engine.Index( uid, - doc.get(), + doc, sequenceNumberSupplier.getAsLong(), 1, i, @@ -3841,9 +3805,7 @@ public class InternalEngineTests extends EngineTestCase { maxSeqNo, localCheckpoint); trimUnsafeCommits(engine.config()); - EngineConfig noopEngineConfig = copy(engine.config(), new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, - () -> new MatchAllDocsQuery(), engine.config().getMergePolicy())); - noOpEngine = new InternalEngine(noopEngineConfig, supplier) { + noOpEngine = new InternalEngine(engine.config(), supplier) { @Override protected long doGenerateSeqNoForOperation(Operation operation) { throw new UnsupportedOperationException(); @@ -3851,7 +3813,7 @@ public class InternalEngineTests extends EngineTestCase { }; noOpEngine.recoverFromTranslog(Long.MAX_VALUE); final int gapsFilled = noOpEngine.fillSeqNoGaps(primaryTerm.get()); - final String reason = "filling gaps"; + final String reason = randomAlphaOfLength(16); noOpEngine.noOp(new Engine.NoOp(maxSeqNo + 1, primaryTerm.get(), LOCAL_TRANSLOG_RECOVERY, System.nanoTime(), reason)); assertThat(noOpEngine.getLocalCheckpoint(), equalTo((long) (maxSeqNo + 1))); assertThat(noOpEngine.getTranslog().stats().getUncommittedOperations(), equalTo(gapsFilled)); @@ -3873,77 +3835,11 @@ public class InternalEngineTests extends EngineTestCase { assertThat(noOp.seqNo(), equalTo((long) (maxSeqNo + 2))); assertThat(noOp.primaryTerm(), equalTo(primaryTerm.get())); assertThat(noOp.reason(), equalTo(reason)); - if (engine.engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - MapperService mapperService = createMapperService("test"); - List operationsFromLucene = readAllOperationsInLucene(noOpEngine, mapperService); - assertThat(operationsFromLucene, hasSize(maxSeqNo + 2 - localCheckpoint)); // fills n gap and 2 manual noop. - for (int i = 0; i < operationsFromLucene.size(); i++) { - assertThat(operationsFromLucene.get(i), equalTo(new Translog.NoOp(localCheckpoint + 1 + i, primaryTerm.get(), "filling gaps"))); - } - assertConsistentHistoryBetweenTranslogAndLuceneIndex(noOpEngine, mapperService); - } } finally { IOUtils.close(noOpEngine); } } - /** - * Verifies that a segment containing only no-ops can be used to look up _version and _seqno. - */ - public void testSegmentContainsOnlyNoOps() throws Exception { - Engine.NoOpResult noOpResult = engine.noOp(new Engine.NoOp(1, primaryTerm.get(), - randomFrom(Engine.Operation.Origin.values()), randomNonNegativeLong(), "test")); - assertThat(noOpResult.getFailure(), nullValue()); - engine.refresh("test"); - Engine.DeleteResult deleteResult = engine.delete(replicaDeleteForDoc("id", 1, 2, randomNonNegativeLong())); - assertThat(deleteResult.getFailure(), nullValue()); - engine.refresh("test"); - } - - /** - * A simple test to check that random combination of operations can coexist in segments and be lookup. - * This is needed as some fields in Lucene may not exist if a segment misses operation types and this code is to check for that. - * For example, a segment containing only no-ops does not have neither _uid or _version. - */ - public void testRandomOperations() throws Exception { - int numOps = between(10, 100); - for (int i = 0; i < numOps; i++) { - String id = Integer.toString(randomIntBetween(1, 10)); - ParsedDocument doc = createParsedDoc(id, null); - Engine.Operation.TYPE type = randomFrom(Engine.Operation.TYPE.values()); - switch (type) { - case INDEX: - Engine.IndexResult index = engine.index(replicaIndexForDoc(doc, between(1, 100), i, randomBoolean())); - assertThat(index.getFailure(), nullValue()); - break; - case DELETE: - Engine.DeleteResult delete = engine.delete(replicaDeleteForDoc(doc.id(), between(1, 100), i, randomNonNegativeLong())); - assertThat(delete.getFailure(), nullValue()); - break; - case NO_OP: - Engine.NoOpResult noOp = engine.noOp(new Engine.NoOp(i, primaryTerm.get(), - randomFrom(Engine.Operation.Origin.values()), randomNonNegativeLong(), "")); - assertThat(noOp.getFailure(), nullValue()); - break; - default: - throw new IllegalStateException("Invalid op [" + type + "]"); - } - if (randomBoolean()) { - engine.refresh("test"); - } - if (randomBoolean()) { - engine.flush(); - } - if (randomBoolean()) { - engine.forceMerge(randomBoolean(), between(1, 10), randomBoolean(), false, false); - } - } - if (engine.engineConfig.getIndexSettings().isSoftDeleteEnabled()) { - List operations = readAllOperationsInLucene(engine, createMapperService("test")); - assertThat(operations, hasSize(numOps)); - } - } - public void testMinGenerationForSeqNo() throws IOException, BrokenBarrierException, InterruptedException { engine.close(); final int numberOfTriplets = randomIntBetween(1, 32); @@ -4509,7 +4405,7 @@ public class InternalEngineTests extends EngineTestCase { globalCheckpoint.set(randomLongBetween(engine.getLocalCheckpoint(), Long.MAX_VALUE)); engine.syncTranslog(); assertThat(DirectoryReader.listCommits(store.directory()), contains(commits.get(commits.size() - 1))); - assertThat(engine.getTranslog().totalOperations(), equalTo(0)); + assertThat(engine.estimateTranslogOperationsFromMinSeq(0L), equalTo(0)); } } @@ -4872,154 +4768,6 @@ public class InternalEngineTests extends EngineTestCase { } } - public void testLuceneHistoryOnPrimary() throws Exception { - final List operations = generateSingleDocHistory(false, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "1"); - assertOperationHistoryInLucene(operations); - } - - public void testLuceneHistoryOnReplica() throws Exception { - final List operations = generateSingleDocHistory(true, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); - Randomness.shuffle(operations); - assertOperationHistoryInLucene(operations); - } - - private void assertOperationHistoryInLucene(List operations) throws IOException { - final MergePolicy keepSoftDeleteDocsMP = new SoftDeletesRetentionMergePolicy( - Lucene.SOFT_DELETES_FIELD, () -> new MatchAllDocsQuery(), engine.config().getMergePolicy()); - Settings.Builder settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); - final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); - Set expectedSeqNos = new HashSet<>(); - try (Store store = createStore(); - Engine engine = createEngine(config(indexSettings, store, createTempDir(), keepSoftDeleteDocsMP, null))) { - for (Engine.Operation op : operations) { - if (op instanceof Engine.Index) { - Engine.IndexResult indexResult = engine.index((Engine.Index) op); - assertThat(indexResult.getFailure(), nullValue()); - expectedSeqNos.add(indexResult.getSeqNo()); - } else { - Engine.DeleteResult deleteResult = engine.delete((Engine.Delete) op); - assertThat(deleteResult.getFailure(), nullValue()); - expectedSeqNos.add(deleteResult.getSeqNo()); - } - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(); - } - if (rarely()) { - engine.forceMerge(true); - } - } - MapperService mapperService = createMapperService("test"); - List actualOps = readAllOperationsInLucene(engine, mapperService); - assertThat(actualOps.stream().map(o -> o.seqNo()).collect(Collectors.toList()), containsInAnyOrder(expectedSeqNos.toArray())); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - } - } - - public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { - IOUtils.close(engine, store); - Settings.Builder settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); - final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - final List operations = generateSingleDocHistory(true, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); - Randomness.shuffle(operations); - Set existingSeqNos = new HashSet<>(); - store = createStore(); - engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get)); - assertThat(engine.getMinRetainedSeqNo(), equalTo(0L)); - long lastMinRetainedSeqNo = engine.getMinRetainedSeqNo(); - for (Engine.Operation op : operations) { - final Engine.Result result; - if (op instanceof Engine.Index) { - result = engine.index((Engine.Index) op); - } else { - result = engine.delete((Engine.Delete) op); - } - existingSeqNos.add(result.getSeqNo()); - if (randomBoolean()) { - globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); - } - if (rarely()) { - settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); - indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); - engine.onSettingsChanged(); - } - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(true, true); - assertThat(Long.parseLong(engine.getLastCommittedSegmentInfos().userData.get(Engine.MIN_RETAINED_SEQNO)), - equalTo(engine.getMinRetainedSeqNo())); - } - if (rarely()) { - engine.forceMerge(randomBoolean()); - } - try (Closeable ignored = engine.acquireRetentionLockForPeerRecovery()) { - long minRetainSeqNos = engine.getMinRetainedSeqNo(); - assertThat(minRetainSeqNos, lessThanOrEqualTo(globalCheckpoint.get() + 1)); - Long[] expectedOps = existingSeqNos.stream().filter(seqno -> seqno >= minRetainSeqNos).toArray(Long[]::new); - Set actualOps = readAllOperationsInLucene(engine, createMapperService("test")).stream() - .map(Translog.Operation::seqNo).collect(Collectors.toSet()); - assertThat(actualOps, containsInAnyOrder(expectedOps)); - } - try (Engine.IndexCommitRef commitRef = engine.acquireSafeIndexCommit()) { - IndexCommit safeCommit = commitRef.getIndexCommit(); - if (safeCommit.getUserData().containsKey(Engine.MIN_RETAINED_SEQNO)) { - lastMinRetainedSeqNo = Long.parseLong(safeCommit.getUserData().get(Engine.MIN_RETAINED_SEQNO)); - } - } - } - if (randomBoolean()) { - engine.close(); - } else { - engine.flushAndClose(); - } - trimUnsafeCommits(engine.config()); - try (InternalEngine recoveringEngine = new InternalEngine(engine.config())) { - assertThat(recoveringEngine.getMinRetainedSeqNo(), equalTo(lastMinRetainedSeqNo)); - } - } - - public void testLastRefreshCheckpoint() throws Exception { - AtomicBoolean done = new AtomicBoolean(); - Thread[] refreshThreads = new Thread[between(1, 8)]; - CountDownLatch latch = new CountDownLatch(refreshThreads.length); - for (int i = 0; i < refreshThreads.length; i++) { - latch.countDown(); - refreshThreads[i] = new Thread(() -> { - while (done.get() == false) { - long checkPointBeforeRefresh = engine.getLocalCheckpoint(); - engine.refresh("test", randomFrom(Engine.SearcherScope.values())); - assertThat(engine.lastRefreshedCheckpoint(), greaterThanOrEqualTo(checkPointBeforeRefresh)); - } - }); - refreshThreads[i].start(); - } - latch.await(); - List ops = generateSingleDocHistory(true, VersionType.EXTERNAL, 1, 10, 1000, "1"); - concurrentlyApplyOps(ops, engine); - done.set(true); - for (Thread thread : refreshThreads) { - thread.join(); - } - engine.refresh("test"); - assertThat(engine.lastRefreshedCheckpoint(), equalTo(engine.getLocalCheckpoint())); - } - private static void trimUnsafeCommits(EngineConfig config) throws IOException { final Store store = config.getStore(); final TranslogConfig translogConfig = config.getTranslogConfig(); diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java deleted file mode 100644 index 2d097366a27..00000000000 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ /dev/null @@ -1,289 +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.engine; - -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.translog.SnapshotMatchers; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.test.IndexSettingsModule; -import org.junit.Before; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; - -public class LuceneChangesSnapshotTests extends EngineTestCase { - private MapperService mapperService; - - @Before - public void createMapper() throws Exception { - mapperService = createMapperService("test"); - } - - @Override - protected Settings indexSettings() { - return Settings.builder().put(super.indexSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) // always enable soft-deletes - .build(); - } - - public void testBasics() throws Exception { - long fromSeqNo = randomNonNegativeLong(); - long toSeqNo = randomLongBetween(fromSeqNo, Long.MAX_VALUE); - // Empty engine - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, true)) { - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat(error.getMessage(), - containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found")); - } - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, false)) { - assertThat(snapshot, SnapshotMatchers.size(0)); - } - int numOps = between(1, 100); - int refreshedSeqNo = -1; - for (int i = 0; i < numOps; i++) { - String id = Integer.toString(randomIntBetween(i, i + 5)); - ParsedDocument doc = createParsedDoc(id, null, randomBoolean()); - if (randomBoolean()) { - engine.index(indexForDoc(doc)); - } else { - engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); - } - if (rarely()) { - if (randomBoolean()) { - engine.flush(); - } else { - engine.refresh("test"); - } - refreshedSeqNo = i; - } - } - if (refreshedSeqNo == -1) { - fromSeqNo = between(0, numOps); - toSeqNo = randomLongBetween(fromSeqNo, numOps * 2); - - Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, false)) { - searcher = null; - assertThat(snapshot, SnapshotMatchers.size(0)); - } finally { - IOUtils.close(searcher); - } - - searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { - searcher = null; - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat(error.getMessage(), - containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found")); - }finally { - IOUtils.close(searcher); - } - } else { - fromSeqNo = randomLongBetween(0, refreshedSeqNo); - toSeqNo = randomLongBetween(refreshedSeqNo + 1, numOps * 2); - Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, false)) { - searcher = null; - assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); - } finally { - IOUtils.close(searcher); - } - searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { - searcher = null; - IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); - assertThat(error.getMessage(), - containsString("Not all operations between from_seqno [" + fromSeqNo + "] and to_seqno [" + toSeqNo + "] found")); - }finally { - IOUtils.close(searcher); - } - toSeqNo = randomLongBetween(fromSeqNo, refreshedSeqNo); - searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - searcher, mapperService, between(1, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE), fromSeqNo, toSeqNo, true)) { - searcher = null; - assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); - } finally { - IOUtils.close(searcher); - } - } - // Get snapshot via engine will auto refresh - fromSeqNo = randomLongBetween(0, numOps - 1); - toSeqNo = randomLongBetween(fromSeqNo, numOps - 1); - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, randomBoolean())) { - assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); - } - } - - public void testDedupByPrimaryTerm() throws Exception { - Map latestOperations = new HashMap<>(); - List terms = Arrays.asList(between(1, 1000), between(1000, 2000)); - int totalOps = 0; - for (long term : terms) { - final List ops = generateSingleDocHistory(true, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE), term, 2, 20, "1"); - primaryTerm.set(Math.max(primaryTerm.get(), term)); - engine.rollTranslogGeneration(); - for (Engine.Operation op : ops) { - // We need to simulate a rollback here as only ops after local checkpoint get into the engine - if (op.seqNo() <= engine.getLocalCheckpointTracker().getCheckpoint()) { - engine.getLocalCheckpointTracker().resetCheckpoint(randomLongBetween(-1, op.seqNo() - 1)); - engine.rollTranslogGeneration(); - } - if (op instanceof Engine.Index) { - engine.index((Engine.Index) op); - } else if (op instanceof Engine.Delete) { - engine.delete((Engine.Delete) op); - } - latestOperations.put(op.seqNo(), op.primaryTerm()); - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(); - } - totalOps++; - } - } - long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapperService, 0, maxSeqNo, false)) { - Translog.Operation op; - while ((op = snapshot.next()) != null) { - assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); - } - assertThat(snapshot.skippedOperations(), equalTo(totalOps - latestOperations.size())); - } - } - - public void testUpdateAndReadChangesConcurrently() throws Exception { - Follower[] followers = new Follower[between(1, 3)]; - CountDownLatch readyLatch = new CountDownLatch(followers.length + 1); - AtomicBoolean isDone = new AtomicBoolean(); - for (int i = 0; i < followers.length; i++) { - followers[i] = new Follower(engine, isDone, readyLatch); - followers[i].start(); - } - boolean onPrimary = randomBoolean(); - List operations = new ArrayList<>(); - int numOps = scaledRandomIntBetween(1, 1000); - for (int i = 0; i < numOps; i++) { - String id = Integer.toString(randomIntBetween(1, 10)); - ParsedDocument doc = createParsedDoc(id, randomAlphaOfLengthBetween(1, 5), randomBoolean()); - final Engine.Operation op; - if (onPrimary) { - if (randomBoolean()) { - op = new Engine.Index(newUid(doc), primaryTerm.get(), doc); - } else { - op = new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get()); - } - } else { - if (randomBoolean()) { - op = replicaIndexForDoc(doc, randomNonNegativeLong(), i, randomBoolean()); - } else { - op = replicaDeleteForDoc(doc.id(), randomNonNegativeLong(), i, randomNonNegativeLong()); - } - } - operations.add(op); - } - readyLatch.countDown(); - concurrentlyApplyOps(operations, engine); - assertThat(engine.getLocalCheckpointTracker().getCheckpoint(), equalTo(operations.size() - 1L)); - isDone.set(true); - for (Follower follower : followers) { - follower.join(); - } - } - - class Follower extends Thread { - private final Engine leader; - private final TranslogHandler translogHandler; - private final AtomicBoolean isDone; - private final CountDownLatch readLatch; - - Follower(Engine leader, AtomicBoolean isDone, CountDownLatch readLatch) { - this.leader = leader; - this.isDone = isDone; - this.readLatch = readLatch; - this.translogHandler = new TranslogHandler(xContentRegistry(), IndexSettingsModule.newIndexSettings(shardId.getIndexName(), - engine.engineConfig.getIndexSettings().getSettings())); - } - - void pullOperations(Engine follower) throws IOException { - long leaderCheckpoint = leader.getLocalCheckpoint(); - long followerCheckpoint = follower.getLocalCheckpoint(); - if (followerCheckpoint < leaderCheckpoint) { - long fromSeqNo = followerCheckpoint + 1; - long batchSize = randomLongBetween(0, 100); - long toSeqNo = Math.min(fromSeqNo + batchSize, leaderCheckpoint); - try (Translog.Snapshot snapshot = leader.newChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, true)) { - translogHandler.run(follower, snapshot); - } - } - } - - @Override - public void run() { - try (Store store = createStore(); - InternalEngine follower = createEngine(store, createTempDir())) { - readLatch.countDown(); - readLatch.await(); - while (isDone.get() == false || - follower.getLocalCheckpointTracker().getCheckpoint() < leader.getLocalCheckpoint()) { - pullOperations(follower); - } - assertConsistentHistoryBetweenTranslogAndLuceneIndex(follower, mapperService); - assertThat(getDocIds(follower, true), equalTo(getDocIds(leader, true))); - } catch (Exception ex) { - throw new AssertionError(ex); - } - } - } - - private List drainAll(Translog.Snapshot snapshot) throws IOException { - List operations = new ArrayList<>(); - Translog.Operation op; - while ((op = snapshot.next()) != null) { - final Translog.Operation newOp = op; - logger.error("Reading [{}]", op); - assert operations.stream().allMatch(o -> o.seqNo() < newOp.seqNo()) : "Operations [" + operations + "], op [" + op + "]"; - operations.add(newOp); - } - return operations; - } -} diff --git a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java deleted file mode 100644 index c46b47b87d0..00000000000 --- a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java +++ /dev/null @@ -1,161 +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.engine; - -import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; -import org.apache.lucene.document.NumericDocValuesField; -import org.apache.lucene.document.StoredField; -import org.apache.lucene.document.StringField; -import org.apache.lucene.index.CodecReader; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.IndexableField; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.MergePolicy; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.SegmentCommitInfo; -import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.StandardDirectoryReader; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.MatchNoDocsQuery; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.store.Directory; -import org.apache.lucene.util.InfoStream; -import org.apache.lucene.util.NullInfoStream; -import org.elasticsearch.test.ESTestCase; - -import java.io.IOException; -import java.util.Collections; -import java.util.Set; -import java.util.stream.Collectors; - -public class RecoverySourcePruneMergePolicyTests extends ESTestCase { - - public void testPruneAll() throws IOException { - try (Directory dir = newDirectory()) { - IndexWriterConfig iwc = newIndexWriterConfig(); - RecoverySourcePruneMergePolicy mp = new RecoverySourcePruneMergePolicy("extra_source", MatchNoDocsQuery::new, - newLogMergePolicy()); - iwc.setMergePolicy(mp); - try (IndexWriter writer = new IndexWriter(dir, iwc)) { - for (int i = 0; i < 20; i++) { - if (i > 0 && randomBoolean()) { - writer.flush(); - } - Document doc = new Document(); - doc.add(new StoredField("source", "hello world")); - doc.add(new StoredField("extra_source", "hello world")); - doc.add(new NumericDocValuesField("extra_source", 1)); - writer.addDocument(doc); - } - writer.forceMerge(1); - writer.commit(); - try (DirectoryReader reader = DirectoryReader.open(writer)) { - for (int i = 0; i < reader.maxDoc(); i++) { - Document document = reader.document(i); - assertEquals(1, document.getFields().size()); - assertEquals("source", document.getFields().get(0).name()); - } - assertEquals(1, reader.leaves().size()); - LeafReader leafReader = reader.leaves().get(0).reader(); - NumericDocValues extra_source = leafReader.getNumericDocValues("extra_source"); - if (extra_source != null) { - assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); - } - if (leafReader instanceof CodecReader && reader instanceof StandardDirectoryReader) { - CodecReader codecReader = (CodecReader) leafReader; - StandardDirectoryReader sdr = (StandardDirectoryReader) reader; - SegmentInfos segmentInfos = sdr.getSegmentInfos(); - MergePolicy.MergeSpecification forcedMerges = mp.findForcedDeletesMerges(segmentInfos, - new MergePolicy.MergeContext() { - @Override - public int numDeletesToMerge(SegmentCommitInfo info) { - return info.info.maxDoc() - 1; - } - - @Override - public int numDeletedDocs(SegmentCommitInfo info) { - return info.info.maxDoc() - 1; - } - - @Override - public InfoStream getInfoStream() { - return new NullInfoStream(); - } - - @Override - public Set getMergingSegments() { - return Collections.emptySet(); - } - }); - // don't wrap if there is nothing to do - assertSame(codecReader, forcedMerges.merges.get(0).wrapForMerge(codecReader)); - } - } - } - } - } - - - public void testPruneSome() throws IOException { - try (Directory dir = newDirectory()) { - IndexWriterConfig iwc = newIndexWriterConfig(); - iwc.setMergePolicy(new RecoverySourcePruneMergePolicy("extra_source", - () -> new TermQuery(new Term("even", "true")), iwc.getMergePolicy())); - try (IndexWriter writer = new IndexWriter(dir, iwc)) { - for (int i = 0; i < 20; i++) { - if (i > 0 && randomBoolean()) { - writer.flush(); - } - Document doc = new Document(); - doc.add(new StringField("even", Boolean.toString(i % 2 == 0), Field.Store.YES)); - doc.add(new StoredField("source", "hello world")); - doc.add(new StoredField("extra_source", "hello world")); - doc.add(new NumericDocValuesField("extra_source", 1)); - writer.addDocument(doc); - } - writer.forceMerge(1); - writer.commit(); - try (DirectoryReader reader = DirectoryReader.open(writer)) { - assertEquals(1, reader.leaves().size()); - NumericDocValues extra_source = reader.leaves().get(0).reader().getNumericDocValues("extra_source"); - assertNotNull(extra_source); - for (int i = 0; i < reader.maxDoc(); i++) { - Document document = reader.document(i); - Set collect = document.getFields().stream().map(IndexableField::name).collect(Collectors.toSet()); - assertTrue(collect.contains("source")); - assertTrue(collect.contains("even")); - if (collect.size() == 3) { - assertTrue(collect.contains("extra_source")); - assertEquals("true", document.getField("even").stringValue()); - assertEquals(i, extra_source.nextDoc()); - } else { - assertEquals(2, document.getFields().size()); - } - } - assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); - } - } - } - } -} diff --git a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java deleted file mode 100644 index f3590100382..00000000000 --- a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java +++ /dev/null @@ -1,75 +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.engine; - -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.test.ESTestCase; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - -import static org.hamcrest.Matchers.equalTo; - -public class SoftDeletesPolicyTests extends ESTestCase { - /** - * Makes sure we won't advance the retained seq# if the retention lock is held - */ - public void testSoftDeletesRetentionLock() { - long retainedOps = between(0, 10000); - AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - long safeCommitCheckpoint = globalCheckpoint.get(); - SoftDeletesPolicy policy = new SoftDeletesPolicy(globalCheckpoint::get, between(1, 10000), retainedOps); - long minRetainedSeqNo = policy.getMinRetainedSeqNo(); - List locks = new ArrayList<>(); - int iters = scaledRandomIntBetween(10, 1000); - for (int i = 0; i < iters; i++) { - if (randomBoolean()) { - locks.add(policy.acquireRetentionLock()); - } - // Advances the global checkpoint and the local checkpoint of a safe commit - globalCheckpoint.addAndGet(between(0, 1000)); - safeCommitCheckpoint = randomLongBetween(safeCommitCheckpoint, globalCheckpoint.get()); - policy.setLocalCheckpointOfSafeCommit(safeCommitCheckpoint); - if (rarely()) { - retainedOps = between(0, 10000); - policy.setRetentionOperations(retainedOps); - } - // Release some locks - List releasingLocks = randomSubsetOf(locks); - locks.removeAll(releasingLocks); - releasingLocks.forEach(Releasable::close); - - // We only expose the seqno to the merge policy if the retention lock is not held. - policy.getRetentionQuery(); - if (locks.isEmpty()) { - long retainedSeqNo = Math.min(safeCommitCheckpoint, globalCheckpoint.get() - retainedOps) + 1; - minRetainedSeqNo = Math.max(minRetainedSeqNo, retainedSeqNo); - } - assertThat(policy.getMinRetainedSeqNo(), equalTo(minRetainedSeqNo)); - } - - locks.forEach(Releasable::close); - long retainedSeqNo = Math.min(safeCommitCheckpoint, globalCheckpoint.get() - retainedOps) + 1; - minRetainedSeqNo = Math.max(minRetainedSeqNo, retainedSeqNo); - assertThat(policy.getMinRetainedSeqNo(), equalTo(minRetainedSeqNo)); - } -} diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java index 5a46b9a889f..76ca6aa7ea8 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java @@ -31,7 +31,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -312,18 +311,15 @@ public class DocumentParserTests extends ESSingleNodeTestCase { // creates an object mapper, which is about 100x harder than it should be.... ObjectMapper createObjectMapper(MapperService mapperService, String name) throws Exception { - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); - ParseContext context = new ParseContext.InternalParseContext(settings, + ParseContext context = new ParseContext.InternalParseContext( + Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(), mapperService.documentMapperParser(), mapperService.documentMapper("type"), null, null); String[] nameParts = name.split("\\."); for (int i = 0; i < nameParts.length - 1; ++i) { context.path().add(nameParts[i]); } Mapper.Builder builder = new ObjectMapper.Builder(nameParts[nameParts.length - 1]).enabled(true); - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); return (ObjectMapper)builder.build(builderContext); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java index b11e4876f9e..cb2ed785699 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.BooleanFieldMapper.BooleanFieldType; import org.elasticsearch.index.mapper.DateFieldMapper.DateFieldType; import org.elasticsearch.index.mapper.NumberFieldMapper.NumberFieldType; @@ -216,10 +215,7 @@ public class DynamicMappingTests extends ESSingleNodeTestCase { } private Mapper parse(DocumentMapper mapper, DocumentMapperParser parser, XContentBuilder builder) throws Exception { - IndexMetaData build = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); - IndexSettings settings = new IndexSettings(build, Settings.EMPTY); + Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); SourceToParse source = SourceToParse.source("test", mapper.type(), "some_id", BytesReference.bytes(builder), builder.contentType()); try (XContentParser xContentParser = createParser(JsonXContent.jsonXContent, source.source())) { ParseContext.InternalParseContext ctx = new ParseContext.InternalParseContext(settings, parser, mapper, source, xContentParser); diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index fba71dd1e52..1d1e423afc1 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.index.replication; -import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; @@ -42,7 +41,6 @@ import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.engine.InternalEngineTests; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.engine.VersionConflictEngineException; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -142,9 +140,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase } public void testInheritMaxValidAutoIDTimestampOnRecovery() throws Exception { - //TODO: Enables this test with soft-deletes once we have timestamp - Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - try (ReplicationGroup shards = createGroup(0, settings)) { + try (ReplicationGroup shards = createGroup(0)) { shards.startAll(); final IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON); indexRequest.onRetry(); // force an update of the timestamp @@ -350,13 +346,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase final AtomicBoolean throwAfterIndexedOneDoc = new AtomicBoolean(); // need one document to trigger delete in IW. @Override public long addDocument(Iterable doc) throws IOException { - boolean isTombstone = false; - for (IndexableField field : doc) { - if (SeqNoFieldMapper.TOMBSTONE_NAME.equals(field.name())) { - isTombstone = true; - } - } - if (isTombstone == false && throwAfterIndexedOneDoc.getAndSet(true)) { + if (throwAfterIndexedOneDoc.getAndSet(true)) { throw indexException; } else { return super.addDocument(doc); @@ -366,10 +356,6 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase public long deleteDocuments(Term... terms) throws IOException { throw deleteException; } - @Override - public long softUpdateDocument(Term term, Iterable doc, Field...fields) throws IOException { - throw deleteException; // a delete uses softUpdateDocument API if soft-deletes enabled - } }, null, null, config); try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(0)) { @Override @@ -404,9 +390,6 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase try (Translog.Snapshot snapshot = getTranslog(shard).newSnapshot()) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); } - try (Translog.Snapshot snapshot = shard.getHistoryOperations("test", 0)) { - assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); - } } // unlike previous failures, these two failures replicated directly from the replication channel. indexResp = shards.index(new IndexRequest(index.getName(), "type", "any").source("{}", XContentType.JSON)); @@ -421,9 +404,6 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase try (Translog.Snapshot snapshot = getTranslog(shard).newSnapshot()) { assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); } - try (Translog.Snapshot snapshot = shard.getHistoryOperations("test", 0)) { - assertThat(snapshot, SnapshotMatchers.containsOperationsInAnyOrder(expectedTranslogOps)); - } } shards.assertAllEqual(1); } @@ -521,9 +501,8 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase recoverReplica(replica3, replica2, true); try (Translog.Snapshot snapshot = getTranslog(replica3).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(initDocs + 1)); - final List expectedOps = new ArrayList<>(initOperations); - expectedOps.add(op2); - assertThat(snapshot, containsOperationsInAnyOrder(expectedOps)); + assertThat(snapshot.next(), equalTo(op2)); + assertThat("Remaining of snapshot should contain init operations", snapshot, containsOperationsInAnyOrder(initOperations)); assertThat("Peer-recovery should not send overridden operations", snapshot.skippedOperations(), equalTo(0)); } // TODO: We should assert the content of shards in the ReplicationGroup. diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 28122665e9b..2d198c32ba7 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -98,8 +98,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC } public void testRecoveryOfDisconnectedReplica() throws Exception { - Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - try (ReplicationGroup shards = createGroup(1, settings)) { + try (ReplicationGroup shards = createGroup(1)) { shards.startAll(); int docs = shards.indexDocs(randomInt(50)); shards.flush(); @@ -267,7 +266,6 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC builder.settings(Settings.builder().put(newPrimary.indexSettings().getSettings()) .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) ); newPrimary.indexSettings().updateIndexMetaData(builder.build()); newPrimary.onSettingsChanged(); @@ -277,12 +275,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC shards.syncGlobalCheckpoint(); assertThat(newPrimary.getLastSyncedGlobalCheckpoint(), equalTo(newPrimary.seqNoStats().getMaxSeqNo())); }); - newPrimary.flush(new FlushRequest().force(true)); - if (replica.indexSettings().isSoftDeleteEnabled()) { - // We need an extra flush to advance the min_retained_seqno on the new primary so ops-based won't happen. - // The min_retained_seqno only advances when a merge asks for the retention query. - newPrimary.flush(new FlushRequest().force(true)); - } + newPrimary.flush(new FlushRequest()); uncommittedOpsOnPrimary = shards.indexDocs(randomIntBetween(0, 10)); totalDocs += uncommittedOpsOnPrimary; } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 50f95bf4d47..2228e1b017f 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -22,7 +22,6 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.TermQuery; @@ -31,7 +30,6 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; -import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Constants; import org.elasticsearch.Assertions; import org.elasticsearch.Version; @@ -91,13 +89,8 @@ import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; @@ -167,7 +160,6 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasKey; -import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.lessThan; @@ -245,8 +237,7 @@ public class IndexShardTests extends IndexShardTestCase { assertNotNull(shardPath); // fail shard shard.failShard("test shard fail", new CorruptIndexException("", "")); - shard.close("do not assert history", false); - shard.store().close(); + closeShards(shard); // check state file still exists ShardStateMetaData shardStateMetaData = load(logger, shardPath.getShardStatePath()); assertEquals(shardStateMetaData, getShardStateMetadata(shard)); @@ -2403,8 +2394,7 @@ public class IndexShardTests extends IndexShardTestCase { public void testDocStats() throws IOException, InterruptedException { IndexShard indexShard = null; try { - indexShard = newStartedShard( - Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0).build()); + indexShard = newStartedShard(); final long numDocs = randomIntBetween(2, 32); // at least two documents so we have docs to delete final long numDocsToDelete = randomLongBetween(1, numDocs); for (int i = 0; i < numDocs; i++) { @@ -2434,16 +2424,7 @@ public class IndexShardTests extends IndexShardTestCase { deleteDoc(indexShard, "_doc", id); indexDoc(indexShard, "_doc", id); } - // Need to update and sync the global checkpoint as the soft-deletes retention MergePolicy depends on it. - if (indexShard.indexSettings.isSoftDeleteEnabled()) { - if (indexShard.routingEntry().primary()) { - indexShard.updateGlobalCheckpointForShard(indexShard.routingEntry().allocationId().getId(), - indexShard.getLocalCheckpoint()); - } else { - indexShard.updateGlobalCheckpointOnReplica(indexShard.getLocalCheckpoint(), "test"); - } - indexShard.sync(); - } + // flush the buffered deletes final FlushRequest flushRequest = new FlushRequest(); flushRequest.force(false); @@ -2981,7 +2962,6 @@ public class IndexShardTests extends IndexShardTestCase { assertThat(breaker.getUsed(), greaterThan(preRefreshBytes)); indexDoc(primary, "_doc", "4", "{\"foo\": \"potato\"}"); - indexDoc(primary, "_doc", "5", "{\"foo\": \"potato\"}"); // Forces a refresh with the INTERNAL scope ((InternalEngine) primary.getEngine()).writeIndexingBuffer(); @@ -2993,13 +2973,6 @@ public class IndexShardTests extends IndexShardTestCase { // Deleting a doc causes its memory to be freed from the breaker deleteDoc(primary, "_doc", "0"); - // Here we are testing that a fully deleted segment should be dropped and its memory usage is freed. - // In order to instruct the merge policy not to keep a fully deleted segment, - // we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything. - if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { - primary.sync(); - flushShard(primary); - } primary.refresh("force refresh"); ss = primary.segmentStats(randomBoolean()); @@ -3091,7 +3064,6 @@ public class IndexShardTests extends IndexShardTestCase { // Close remaining searchers IOUtils.close(searchers); - primary.refresh("test"); SegmentsStats ss = primary.segmentStats(randomBoolean()); CircuitBreaker breaker = primary.circuitBreakerService.getBreaker(CircuitBreaker.ACCOUNTING); @@ -3209,28 +3181,4 @@ public class IndexShardTests extends IndexShardTestCase { } - public void testSupplyTombstoneDoc() throws Exception { - IndexShard shard = newStartedShard(); - String id = randomRealisticUnicodeOfLengthBetween(1, 10); - ParsedDocument deleteTombstone = shard.getEngine().config().getTombstoneDocSupplier().newDeleteTombstoneDoc("doc", id); - assertThat(deleteTombstone.docs(), hasSize(1)); - ParseContext.Document deleteDoc = deleteTombstone.docs().get(0); - assertThat(deleteDoc.getFields().stream().map(IndexableField::name).collect(Collectors.toList()), - containsInAnyOrder(IdFieldMapper.NAME, VersionFieldMapper.NAME, - SeqNoFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME, SeqNoFieldMapper.TOMBSTONE_NAME)); - assertThat(deleteDoc.getField(IdFieldMapper.NAME).binaryValue(), equalTo(Uid.encodeId(id))); - assertThat(deleteDoc.getField(SeqNoFieldMapper.TOMBSTONE_NAME).numericValue().longValue(), equalTo(1L)); - - final String reason = randomUnicodeOfLength(200); - ParsedDocument noopTombstone = shard.getEngine().config().getTombstoneDocSupplier().newNoopTombstoneDoc(reason); - assertThat(noopTombstone.docs(), hasSize(1)); - ParseContext.Document noopDoc = noopTombstone.docs().get(0); - assertThat(noopDoc.getFields().stream().map(IndexableField::name).collect(Collectors.toList()), - containsInAnyOrder(VersionFieldMapper.NAME, SourceFieldMapper.NAME, SeqNoFieldMapper.TOMBSTONE_NAME, - SeqNoFieldMapper.NAME, SeqNoFieldMapper.NAME, SeqNoFieldMapper.PRIMARY_TERM_NAME)); - assertThat(noopDoc.getField(SeqNoFieldMapper.TOMBSTONE_NAME).numericValue().longValue(), equalTo(1L)); - assertThat(noopDoc.getField(SourceFieldMapper.NAME).binaryValue(), equalTo(new BytesRef(reason))); - - closeShards(shard); - } } diff --git a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java index 29b16ca28f4..ae2cc84e487 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -106,22 +106,17 @@ public class PrimaryReplicaSyncerTests extends IndexShardTestCase { .isPresent(), is(false)); } + + assertEquals(globalCheckPoint == numDocs - 1 ? 0 : numDocs, resyncTask.getTotalOperations()); if (syncNeeded && globalCheckPoint < numDocs - 1) { - if (shard.indexSettings.isSoftDeleteEnabled()) { - assertThat(resyncTask.getSkippedOperations(), equalTo(0)); - assertThat(resyncTask.getResyncedOperations(), equalTo(resyncTask.getTotalOperations())); - assertThat(resyncTask.getTotalOperations(), equalTo(Math.toIntExact(numDocs - 1 - globalCheckPoint))); - } else { - int skippedOps = Math.toIntExact(globalCheckPoint + 1); // everything up to global checkpoint included - assertThat(resyncTask.getSkippedOperations(), equalTo(skippedOps)); - assertThat(resyncTask.getResyncedOperations(), equalTo(numDocs - skippedOps)); - assertThat(resyncTask.getTotalOperations(), equalTo(globalCheckPoint == numDocs - 1 ? 0 : numDocs)); - } + long skippedOps = globalCheckPoint + 1; // everything up to global checkpoint included + assertEquals(skippedOps, resyncTask.getSkippedOperations()); + assertEquals(numDocs - skippedOps, resyncTask.getResyncedOperations()); } else { - assertThat(resyncTask.getSkippedOperations(), equalTo(0)); - assertThat(resyncTask.getResyncedOperations(), equalTo(0)); - assertThat(resyncTask.getTotalOperations(), equalTo(0)); + assertEquals(0, resyncTask.getSkippedOperations()); + assertEquals(0, resyncTask.getResyncedOperations()); } + closeShards(shard); } diff --git a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index b93f170174c..774b272121a 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -42,7 +42,6 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; -import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; import org.elasticsearch.index.mapper.IdFieldMapper; @@ -131,8 +130,7 @@ public class RefreshListenersTests extends ESTestCase { indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, - (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm, - EngineTestCase.tombstoneDocSupplier()); + (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm); engine = new InternalEngine(config); engine.recoverFromTranslog(Long.MAX_VALUE); listeners.setCurrentRefreshLocationSupplier(engine::getTranslogLastWriteLocation); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 81afab4bb8f..89a8813e3e0 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -67,7 +67,6 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import org.junit.After; import java.io.IOException; import java.util.ArrayList; @@ -111,11 +110,6 @@ public class IndexRecoveryIT extends ESIntegTestCase { RecoverySettingsChunkSizePlugin.class); } - @After - public void assertConsistentHistoryInLuceneIndex() throws Exception { - internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); - } - private void assertRecoveryStateWithoutStage(RecoveryState state, int shardId, RecoverySource recoverySource, boolean primary, String sourceNode, String targetNode) { assertThat(state.getShardId().getId(), equalTo(shardId)); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java index b6f5a7b6451..4b1419375e6 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -25,7 +25,6 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.NoMergePolicy; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -92,7 +91,6 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase { replica.close("test", false); final List commits = DirectoryReader.listCommits(replica.store().directory()); IndexWriterConfig iwc = new IndexWriterConfig(null) - .setSoftDeletesField(Lucene.SOFT_DELETES_FIELD) .setCommitOnClose(false) .setMergePolicy(NoMergePolicy.INSTANCE) .setOpenMode(IndexWriterConfig.OpenMode.APPEND); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 0351111c305..f0644b029c3 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -411,6 +411,12 @@ public class RecoverySourceHandlerTests extends ESTestCase { recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY) { + + @Override + boolean isTranslogReadyForSequenceNumberBasedRecovery() throws IOException { + return randomBoolean(); + } + @Override public void phase1(final IndexCommit snapshot, final Supplier translogOps) { phase1Called.set(true); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index 45535e19672..5547a629ab2 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.SourceToParse; @@ -64,13 +63,13 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { int docs = shards.indexDocs(10); getTranslog(shards.getPrimary()).rollGeneration(); shards.flush(); - int moreDocs = shards.indexDocs(randomInt(10)); + if (randomBoolean()) { + docs += shards.indexDocs(10); + } shards.addReplica(); shards.startAll(); final IndexShard replica = shards.getReplicas().get(0); - boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); - assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? moreDocs : docs + moreDocs)); - shards.assertAllEqual(docs + moreDocs); + assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(docs)); } } @@ -102,12 +101,12 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { // rolling/flushing is async assertBusy(() -> { assertThat(replica.getLastSyncedGlobalCheckpoint(), equalTo(19L)); - assertThat(getTranslog(replica).totalOperations(), equalTo(0)); + assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(0)); }); } } - public void testRecoveryWithOutOfOrderDeleteWithTranslog() throws Exception { + public void testRecoveryWithOutOfOrderDelete() throws Exception { /* * The flow of this test: * - delete #1 @@ -119,8 +118,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { * - index #5 * - If flush and the translog retention disabled, delete #1 will be removed while index #0 is still retained and replayed. */ - Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); - try (ReplicationGroup shards = createGroup(1, settings)) { + try (ReplicationGroup shards = createGroup(1)) { shards.startAll(); // create out of order delete and index op on replica final IndexShard orgReplica = shards.getReplicas().get(0); @@ -172,63 +170,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(newReplica); shards.assertAllEqual(3); - assertThat(getTranslog(newReplica).totalOperations(), equalTo(translogOps)); - } - } - - public void testRecoveryWithOutOfOrderDeleteWithSoftDeletes() throws Exception { - Settings settings = Settings.builder() - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 10) - // If soft-deletes is enabled, delete#1 will be reclaimed because its segment (segment_1) is fully deleted - // index#0 will be retained if merge is disabled; otherwise it will be reclaimed because gcp=3 and retained_ops=0 - .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false).build(); - try (ReplicationGroup shards = createGroup(1, settings)) { - shards.startAll(); - // create out of order delete and index op on replica - final IndexShard orgReplica = shards.getReplicas().get(0); - final String indexName = orgReplica.shardId().getIndexName(); - - // delete #1 - orgReplica.applyDeleteOperationOnReplica(1, 2, "type", "id"); - orgReplica.flush(new FlushRequest().force(true)); // isolate delete#1 in its own translog generation and lucene segment - // index #0 - orgReplica.applyIndexOperationOnReplica(0, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(indexName, "type", "id", new BytesArray("{}"), XContentType.JSON)); - // index #3 - orgReplica.applyIndexOperationOnReplica(3, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(indexName, "type", "id-3", new BytesArray("{}"), XContentType.JSON)); - // Flushing a new commit with local checkpoint=1 allows to delete the translog gen #1. - orgReplica.flush(new FlushRequest().force(true).waitIfOngoing(true)); - // index #2 - orgReplica.applyIndexOperationOnReplica(2, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(indexName, "type", "id-2", new BytesArray("{}"), XContentType.JSON)); - orgReplica.updateGlobalCheckpointOnReplica(3L, "test"); - // index #5 -> force NoOp #4. - orgReplica.applyIndexOperationOnReplica(5, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(indexName, "type", "id-5", new BytesArray("{}"), XContentType.JSON)); - - if (randomBoolean()) { - if (randomBoolean()) { - logger.info("--> flushing shard (translog/soft-deletes will be trimmed)"); - IndexMetaData.Builder builder = IndexMetaData.builder(orgReplica.indexSettings().getIndexMetaData()); - builder.settings(Settings.builder().put(orgReplica.indexSettings().getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0)); - orgReplica.indexSettings().updateIndexMetaData(builder.build()); - orgReplica.onSettingsChanged(); - } - flushShard(orgReplica); - } - - final IndexShard orgPrimary = shards.getPrimary(); - shards.promoteReplicaToPrimary(orgReplica).get(); // wait for primary/replica sync to make sure seq# gap is closed. - - IndexShard newReplica = shards.addReplicaWithExistingPath(orgPrimary.shardPath(), orgPrimary.routingEntry().currentNodeId()); - shards.recoverReplica(newReplica); - shards.assertAllEqual(3); - try (Translog.Snapshot snapshot = newReplica.getHistoryOperations("test", 0)) { - assertThat(snapshot, SnapshotMatchers.size(6)); - } + assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(translogOps)); } } @@ -280,8 +222,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(newReplica); // file based recovery should be made assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty())); - boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); - assertThat(getTranslog(newReplica).totalOperations(), equalTo(softDeletesEnabled ? nonFlushedDocs : numDocs)); + assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); // history uuid was restored assertThat(newReplica.getHistoryUUID(), equalTo(historyUUID)); @@ -385,8 +326,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase { shards.recoverReplica(replica); // Make sure the flushing will eventually be completed (eg. `shouldPeriodicallyFlush` is false) assertBusy(() -> assertThat(getEngine(replica).shouldPeriodicallyFlush(), equalTo(false))); - boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); - assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? 0 : numDocs)); + assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); shards.assertAllEqual(numDocs); } } diff --git a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java index ce162b9600c..fa591411bba 100644 --- a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -43,7 +43,6 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexModule; -import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.MergeSchedulerConfig; @@ -51,7 +50,6 @@ import org.elasticsearch.index.VersionType; import org.elasticsearch.index.cache.query.QueryCacheStats; import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesQueryCache; import org.elasticsearch.indices.IndicesRequestCache; @@ -71,7 +69,6 @@ import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.Random; -import java.util.Set; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -118,7 +115,6 @@ public class IndexStatsIT extends ESIntegTestCase { return Settings.builder().put(super.indexSettings()) .put(IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.getKey(), true) .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) .build(); } @@ -1010,15 +1006,10 @@ public class IndexStatsIT extends ESIntegTestCase { @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/32506") public void testFilterCacheStats() throws Exception { - Settings settings = Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build(); - assertAcked(prepareCreate("index").setSettings(settings).get()); - indexRandom(false, true, + assertAcked(prepareCreate("index").setSettings(Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build()).get()); + indexRandom(true, client().prepareIndex("index", "type", "1").setSource("foo", "bar"), client().prepareIndex("index", "type", "2").setSource("foo", "baz")); - if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { - persistGlobalCheckpoint("index"); // Need to persist the global checkpoint for the soft-deletes retention MP. - } - refresh(); ensureGreen(); IndicesStatsResponse response = client().admin().indices().prepareStats("index").setQueryCache(true).get(); @@ -1049,13 +1040,6 @@ public class IndexStatsIT extends ESIntegTestCase { assertEquals(DocWriteResponse.Result.DELETED, client().prepareDelete("index", "type", "1").get().getResult()); assertEquals(DocWriteResponse.Result.DELETED, client().prepareDelete("index", "type", "2").get().getResult()); - // Here we are testing that a fully deleted segment should be dropped and its cached is evicted. - // In order to instruct the merge policy not to keep a fully deleted segment, - // we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything. - if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { - persistGlobalCheckpoint("index"); - flush("index"); - } refresh(); response = client().admin().indices().prepareStats("index").setQueryCache(true).get(); assertCumulativeQueryCacheStats(response); @@ -1189,21 +1173,4 @@ public class IndexStatsIT extends ESIntegTestCase { assertThat(executionFailures.get(), emptyCollectionOf(Exception.class)); } - - /** - * Persist the global checkpoint on all shards of the given index into disk. - * This makes sure that the persisted global checkpoint on those shards will equal to the in-memory value. - */ - private void persistGlobalCheckpoint(String index) throws Exception { - final Set nodes = internalCluster().nodesInclude(index); - for (String node : nodes) { - final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); - for (IndexService indexService : indexServices) { - for (IndexShard indexShard : indexService) { - indexShard.sync(); - assertThat(indexShard.getLastSyncedGlobalCheckpoint(), equalTo(indexShard.getGlobalCheckpoint())); - } - } - } - } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index c25cad61e07..23c56688e00 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -27,7 +27,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.snapshots.mockstore.MockRepository; import org.elasticsearch.test.ESIntegTestCase; -import org.junit.After; import java.io.IOException; import java.nio.file.FileVisitResult; @@ -59,11 +58,6 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase { return Arrays.asList(MockRepository.Plugin.class); } - @After - public void assertConsistentHistoryInLuceneIndex() throws Exception { - internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); - } - public static long getFailureCount(String repository) { long failureCount = 0; for (RepositoriesService repositoriesService : diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index 632a1ecbee1..1230d594b98 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -122,7 +122,6 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.IndexSettings.INDEX_REFRESH_INTERVAL_SETTING; -import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; import static org.elasticsearch.index.shard.IndexShardTests.getEngineFromShard; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -2049,9 +2048,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES))); // only one shard - final Settings indexSettings = Settings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).build(); - assertAcked(prepareCreate("test").setSettings(indexSettings)); + assertAcked(prepareCreate("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1))); ensureGreen(); logger.info("--> indexing"); @@ -2097,13 +2094,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas SnapshotStatus snapshotStatus = client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("test-2").get().getSnapshots().get(0); List shards = snapshotStatus.getShards(); for (SnapshotIndexShardStatus status : shards) { - // we flush before the snapshot such that we have to process the segments_N files plus the .del file - if (INDEX_SOFT_DELETES_SETTING.get(indexSettings)) { - // soft-delete generates DV files. - assertThat(status.getStats().getProcessedFileCount(), greaterThan(2)); - } else { - assertThat(status.getStats().getProcessedFileCount(), equalTo(2)); - } + assertThat(status.getStats().getProcessedFileCount(), equalTo(2)); // we flush before the snapshot such that we have to process the segments_N files plus the .del file } } } diff --git a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java index 588118db4ae..caf4f725fa4 100644 --- a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java +++ b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.VersionType; @@ -786,26 +785,4 @@ public class SimpleVersioningIT extends ESIntegTestCase { .getVersion(), equalTo(-1L)); } - - public void testSpecialVersioning() { - internalCluster().ensureAtLeastNumDataNodes(2); - createIndex("test", Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).build()); - IndexResponse doc1 = client().prepareIndex("test", "type", "1").setSource("field", "value1") - .setVersion(0).setVersionType(VersionType.EXTERNAL).execute().actionGet(); - assertThat(doc1.getVersion(), equalTo(0L)); - IndexResponse doc2 = client().prepareIndex("test", "type", "1").setSource("field", "value2") - .setVersion(Versions.MATCH_ANY).setVersionType(VersionType.INTERNAL).execute().actionGet(); - assertThat(doc2.getVersion(), equalTo(1L)); - client().prepareDelete("test", "type", "1").get(); //v2 - IndexResponse doc3 = client().prepareIndex("test", "type", "1").setSource("field", "value3") - .setVersion(Versions.MATCH_DELETED).setVersionType(VersionType.INTERNAL).execute().actionGet(); - assertThat(doc3.getVersion(), equalTo(3L)); - IndexResponse doc4 = client().prepareIndex("test", "type", "1").setSource("field", "value4") - .setVersion(4L).setVersionType(VersionType.EXTERNAL_GTE).execute().actionGet(); - assertThat(doc4.getVersion(), equalTo(4L)); - // Make sure that these versions are replicated correctly - client().admin().indices().prepareUpdateSettings("test") - .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)).get(); - ensureGreen("test"); - } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index b558cd1ba90..b5ba5f18b39 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -19,18 +19,14 @@ package org.elasticsearch.index.engine; -import org.apache.logging.log4j.Logger; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StoredField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.Term; @@ -38,41 +34,32 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.Sort; -import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.store.Directory; -import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.MapperTestUtils; -import org.elasticsearch.index.VersionType; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MapperService; 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.SourceFieldMapper; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -93,30 +80,17 @@ import org.junit.Before; import java.io.IOException; import java.nio.charset.Charset; import java.nio.file.Path; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; -import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.ToLongBiFunction; -import java.util.stream.Collectors; import static java.util.Collections.emptyList; -import static java.util.Collections.shuffle; -import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; -import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; public abstract class EngineTestCase extends ESTestCase { @@ -154,20 +128,6 @@ public abstract class EngineTestCase extends ESTestCase { } } - protected Settings indexSettings() { - // TODO randomize more settings - return Settings.builder() - .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.CURRENT) - .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), - between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), - randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) - .build(); - } - @Override @Before public void setUp() throws Exception { @@ -182,7 +142,13 @@ public abstract class EngineTestCase extends ESTestCase { } else { codecName = "default"; } - defaultSettings = IndexSettingsModule.newIndexSettings("test", indexSettings()); + defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() + .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.CURRENT) + .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), + between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) + .build()); // TODO randomize more settings threadPool = new TestThreadPool(getClass().getName()); store = createStore(); storeReplica = createStore(); @@ -214,7 +180,7 @@ public abstract class EngineTestCase extends ESTestCase { new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), globalCheckpointSupplier, config.getPrimaryTermSupplier(), tombstoneDocSupplier()); + config.getCircuitBreakerService(), globalCheckpointSupplier, config.getPrimaryTermSupplier()); } public EngineConfig copy(EngineConfig config, Analyzer analyzer) { @@ -223,18 +189,7 @@ public abstract class EngineTestCase extends ESTestCase { new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier(), - config.getTombstoneDocSupplier()); - } - - public EngineConfig copy(EngineConfig config, MergePolicy mergePolicy) { - return new EngineConfig(config.getShardId(), config.getAllocationId(), config.getThreadPool(), config.getIndexSettings(), - config.getWarmer(), config.getStore(), mergePolicy, config.getAnalyzer(), config.getSimilarity(), - new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), - config.getTranslogConfig(), config.getFlushMergesAfter(), - config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier(), - config.getTombstoneDocSupplier()); + config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier(), config.getPrimaryTermSupplier()); } @Override @@ -243,11 +198,9 @@ public abstract class EngineTestCase extends ESTestCase { super.tearDown(); if (engine != null && engine.isClosed.get() == false) { engine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs(); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, createMapperService("test")); } if (replicaEngine != null && replicaEngine.isClosed.get() == false) { replicaEngine.getTranslog().getDeletionPolicy().assertNoOpenTranslogRefs(); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(replicaEngine, createMapperService("test")); } IOUtils.close( replicaEngine, storeReplica, @@ -275,18 +228,8 @@ public abstract class EngineTestCase extends ESTestCase { return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null); } - public static ParsedDocument createParsedDoc(String id, String routing, boolean recoverySource) { - return testParsedDocument(id, routing, testDocumentWithTextField(), new BytesArray("{ \"value\" : \"test\" }"), null, - recoverySource); - } - protected static ParsedDocument testParsedDocument( String id, String routing, ParseContext.Document document, BytesReference source, Mapping mappingUpdate) { - return testParsedDocument(id, routing, document, source, mappingUpdate, false); - } - protected static ParsedDocument testParsedDocument( - String id, String routing, ParseContext.Document document, BytesReference source, Mapping mappingUpdate, - boolean recoverySource) { Field uidField = new Field("_id", Uid.encodeId(id), IdFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); @@ -296,57 +239,11 @@ public abstract class EngineTestCase extends ESTestCase { document.add(seqID.seqNoDocValue); document.add(seqID.primaryTerm); BytesRef ref = source.toBytesRef(); - if (recoverySource) { - document.add(new StoredField(SourceFieldMapper.RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); - document.add(new NumericDocValuesField(SourceFieldMapper.RECOVERY_SOURCE_NAME, 1)); - } else { - document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); - } + document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); return new ParsedDocument(versionField, seqID, id, "test", routing, Arrays.asList(document), source, XContentType.JSON, mappingUpdate); } - /** - * Creates a tombstone document that only includes uid, seq#, term and version fields. - */ - public static EngineConfig.TombstoneDocSupplier tombstoneDocSupplier(){ - return new EngineConfig.TombstoneDocSupplier() { - @Override - public ParsedDocument newDeleteTombstoneDoc(String type, String id) { - final ParseContext.Document doc = new ParseContext.Document(); - Field uidField = new Field(IdFieldMapper.NAME, Uid.encodeId(id), IdFieldMapper.Defaults.FIELD_TYPE); - doc.add(uidField); - Field versionField = new NumericDocValuesField(VersionFieldMapper.NAME, 0); - doc.add(versionField); - SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); - doc.add(seqID.seqNo); - doc.add(seqID.seqNoDocValue); - doc.add(seqID.primaryTerm); - seqID.tombstoneField.setLongValue(1); - doc.add(seqID.tombstoneField); - return new ParsedDocument(versionField, seqID, id, type, null, - Collections.singletonList(doc), new BytesArray("{}"), XContentType.JSON, null); - } - - @Override - public ParsedDocument newNoopTombstoneDoc(String reason) { - final ParseContext.Document doc = new ParseContext.Document(); - SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); - doc.add(seqID.seqNo); - doc.add(seqID.seqNoDocValue); - doc.add(seqID.primaryTerm); - seqID.tombstoneField.setLongValue(1); - doc.add(seqID.tombstoneField); - Field versionField = new NumericDocValuesField(VersionFieldMapper.NAME, 0); - doc.add(versionField); - BytesRef byteRef = new BytesRef(reason); - doc.add(new StoredField(SourceFieldMapper.NAME, byteRef.bytes, byteRef.offset, byteRef.length)); - return new ParsedDocument(versionField, seqID, null, null, null, - Collections.singletonList(doc), null, XContentType.JSON, null); - } - }; - } - protected Store createStore() throws IOException { return createStore(newDirectory()); } @@ -564,7 +461,7 @@ public abstract class EngineTestCase extends ESTestCase { new NoneCircuitBreakerService(), globalCheckpointSupplier == null ? new ReplicationTracker(shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED, update -> {}) : - globalCheckpointSupplier, primaryTerm::get, tombstoneDocSupplier()); + globalCheckpointSupplier, primaryTerm::get); return config; } @@ -577,7 +474,7 @@ public abstract class EngineTestCase extends ESTestCase { return new BytesArray(string.getBytes(Charset.defaultCharset())); } - protected static Term newUid(String id) { + protected Term newUid(String id) { return new Term("_id", Uid.encodeId(id)); } @@ -602,279 +499,6 @@ public abstract class EngineTestCase extends ESTestCase { protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, long startTime) { return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, null, Engine.Operation.Origin.REPLICA, startTime); } - protected static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { - assertVisibleCount(engine, numDocs, true); - } - - protected static void assertVisibleCount(InternalEngine engine, int numDocs, boolean refresh) throws IOException { - if (refresh) { - engine.refresh("test"); - } - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - final TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.searcher().search(new MatchAllDocsQuery(), collector); - assertThat(collector.getTotalHits(), equalTo(numDocs)); - } - } - - public static List generateSingleDocHistory(boolean forReplica, VersionType versionType, - long primaryTerm, int minOpCount, int maxOpCount, String docId) { - final int numOfOps = randomIntBetween(minOpCount, maxOpCount); - final List ops = new ArrayList<>(); - final Term id = newUid(docId); - final int startWithSeqNo = 0; - final String valuePrefix = (forReplica ? "r_" : "p_" ) + docId + "_"; - final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); - for (int i = 0; i < numOfOps; i++) { - final Engine.Operation op; - final long version; - switch (versionType) { - case INTERNAL: - version = forReplica ? i : Versions.MATCH_ANY; - break; - case EXTERNAL: - version = i; - break; - case EXTERNAL_GTE: - version = randomBoolean() ? Math.max(i - 1, 0) : i; - break; - case FORCE: - version = randomNonNegativeLong(); - break; - default: - throw new UnsupportedOperationException("unknown version type: " + versionType); - } - if (randomBoolean()) { - op = new Engine.Index(id, testParsedDocument(docId, null, testDocumentWithTextField(valuePrefix + i), B_1, null), - forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, - version, - forReplica ? null : versionType, - forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis(), -1, false - ); - } else { - op = new Engine.Delete("test", docId, id, - forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, - version, - forReplica ? null : versionType, - forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis()); - } - ops.add(op); - } - return ops; - } - - public static void assertOpsOnReplica( - final List ops, - final InternalEngine replicaEngine, - boolean shuffleOps, - final Logger logger) throws IOException { - final Engine.Operation lastOp = ops.get(ops.size() - 1); - final String lastFieldValue; - if (lastOp instanceof Engine.Index) { - Engine.Index index = (Engine.Index) lastOp; - lastFieldValue = index.docs().get(0).get("value"); - } else { - // delete - lastFieldValue = null; - } - if (shuffleOps) { - int firstOpWithSeqNo = 0; - while (firstOpWithSeqNo < ops.size() && ops.get(firstOpWithSeqNo).seqNo() < 0) { - firstOpWithSeqNo++; - } - // shuffle ops but make sure legacy ops are first - shuffle(ops.subList(0, firstOpWithSeqNo), random()); - shuffle(ops.subList(firstOpWithSeqNo, ops.size()), random()); - } - boolean firstOp = true; - for (Engine.Operation op : ops) { - logger.info("performing [{}], v [{}], seq# [{}], term [{}]", - op.operationType().name().charAt(0), op.version(), op.seqNo(), op.primaryTerm()); - if (op instanceof Engine.Index) { - Engine.IndexResult result = replicaEngine.index((Engine.Index) op); - // replicas don't really care to about creation status of documents - // this allows to ignore the case where a document was found in the live version maps in - // a delete state and return false for the created flag in favor of code simplicity - // as deleted or not. This check is just signal regression so a decision can be made if it's - // intentional - assertThat(result.isCreated(), equalTo(firstOp)); - assertThat(result.getVersion(), equalTo(op.version())); - assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); - - } else { - Engine.DeleteResult result = replicaEngine.delete((Engine.Delete) op); - // Replicas don't really care to about found status of documents - // this allows to ignore the case where a document was found in the live version maps in - // a delete state and return true for the found flag in favor of code simplicity - // his check is just signal regression so a decision can be made if it's - // intentional - assertThat(result.isFound(), equalTo(firstOp == false)); - assertThat(result.getVersion(), equalTo(op.version())); - assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); - } - if (randomBoolean()) { - replicaEngine.refresh("test"); - } - if (randomBoolean()) { - replicaEngine.flush(); - replicaEngine.refresh("test"); - } - firstOp = false; - } - - assertVisibleCount(replicaEngine, lastFieldValue == null ? 0 : 1); - if (lastFieldValue != null) { - try (Engine.Searcher searcher = replicaEngine.acquireSearcher("test")) { - final TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); - assertThat(collector.getTotalHits(), equalTo(1)); - } - } - } - - protected void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { - Thread[] thread = new Thread[randomIntBetween(3, 5)]; - CountDownLatch startGun = new CountDownLatch(thread.length); - AtomicInteger offset = new AtomicInteger(-1); - for (int i = 0; i < thread.length; i++) { - thread[i] = new Thread(() -> { - startGun.countDown(); - try { - startGun.await(); - } catch (InterruptedException e) { - throw new AssertionError(e); - } - int docOffset; - while ((docOffset = offset.incrementAndGet()) < ops.size()) { - try { - final Engine.Operation op = ops.get(docOffset); - if (op instanceof Engine.Index) { - engine.index((Engine.Index) op); - } else if (op instanceof Engine.Delete){ - engine.delete((Engine.Delete) op); - } else { - engine.noOp((Engine.NoOp) op); - } - if ((docOffset + 1) % 4 == 0) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(); - } - } catch (IOException e) { - throw new AssertionError(e); - } - } - }); - thread[i].start(); - } - for (int i = 0; i < thread.length; i++) { - thread[i].join(); - } - } - - /** - * Gets all docId from the given engine. - */ - public static Set getDocIds(Engine engine, boolean refresh) throws IOException { - if (refresh) { - engine.refresh("test_get_doc_ids"); - } - try (Engine.Searcher searcher = engine.acquireSearcher("test_get_doc_ids")) { - 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(IdFieldMapper.NAME)); - BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); - ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); - } - } - } - return ids; - } - } - - /** - * Reads all engine operations that have been processed by the engine from Lucene index. - * The returned operations are sorted and de-duplicated, thus each sequence number will be have at most one operation. - */ - public static List readAllOperationsInLucene(Engine engine, MapperService mapper) throws IOException { - final List operations = new ArrayList<>(); - long maxSeqNo = Math.max(0, ((InternalEngine)engine).getLocalCheckpointTracker().getMaxSeqNo()); - try (Translog.Snapshot snapshot = engine.newChangesSnapshot("test", mapper, 0, maxSeqNo, false)) { - Translog.Operation op; - while ((op = snapshot.next()) != null){ - operations.add(op); - } - } - return operations; - } - - /** - * Asserts the provided engine has a consistent document history between translog and Lucene index. - */ - public static void assertConsistentHistoryBetweenTranslogAndLuceneIndex(Engine engine, MapperService mapper) throws IOException { - if (mapper.documentMapper() == null || engine.config().getIndexSettings().isSoftDeleteEnabled() == false) { - return; - } - final long maxSeqNo = ((InternalEngine) engine).getLocalCheckpointTracker().getMaxSeqNo(); - if (maxSeqNo < 0) { - return; // nothing to check - } - final Map translogOps = new HashMap<>(); - try (Translog.Snapshot snapshot = EngineTestCase.getTranslog(engine).newSnapshot()) { - Translog.Operation op; - while ((op = snapshot.next()) != null) { - translogOps.put(op.seqNo(), op); - } - } - final Map luceneOps = readAllOperationsInLucene(engine, mapper).stream() - .collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity())); - final long globalCheckpoint = EngineTestCase.getTranslog(engine).getLastSyncedGlobalCheckpoint(); - final long retainedOps = engine.config().getIndexSettings().getSoftDeleteRetentionOperations(); - final long seqNoForRecovery; - try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) { - seqNoForRecovery = Long.parseLong(safeCommit.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; - } - final long minSeqNoToRetain = Math.min(seqNoForRecovery, globalCheckpoint + 1 - retainedOps); - for (Translog.Operation translogOp : translogOps.values()) { - final Translog.Operation luceneOp = luceneOps.get(translogOp.seqNo()); - if (luceneOp == null) { - if (minSeqNoToRetain <= translogOp.seqNo() && translogOp.seqNo() <= maxSeqNo) { - fail("Operation not found seq# [" + translogOp.seqNo() + "], global checkpoint [" + globalCheckpoint + "], " + - "retention policy [" + retainedOps + "], maxSeqNo [" + maxSeqNo + "], translog op [" + translogOp + "]"); - } else { - continue; - } - } - assertThat(luceneOp, notNullValue()); - assertThat(luceneOp.toString(), luceneOp.primaryTerm(), equalTo(translogOp.primaryTerm())); - assertThat(luceneOp.opType(), equalTo(translogOp.opType())); - if (luceneOp.opType() == Translog.Operation.Type.INDEX) { - assertThat(luceneOp.getSource().source, equalTo(translogOp.getSource().source)); - } - } - } - - protected MapperService createMapperService(String type) throws IOException { - IndexMetaData indexMetaData = IndexMetaData.builder("test") - .settings(Settings.builder() - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)) - .putMapping(type, "{\"properties\": {}}") - .build(); - MapperService mapperService = MapperTestUtils.newMapperService(new NamedXContentRegistry(ClusterModule.getNamedXWriteables()), - createTempDir(), Settings.EMPTY, "test"); - mapperService.merge(indexMetaData, MapperService.MergeReason.MAPPING_UPDATE); - return mapperService; - } /** * Exposes a translog associated with the given engine for testing purpose. diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index f2afdff9c3a..3f1f5daf514 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -60,7 +60,6 @@ import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; @@ -100,14 +99,10 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase protected final Index index = new Index("test", "uuid"); private final ShardId shardId = new ShardId(index, 0); - protected final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); + private final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); protected ReplicationGroup createGroup(int replicas) throws IOException { - return createGroup(replicas, Settings.EMPTY); - } - - protected ReplicationGroup createGroup(int replicas, Settings settings) throws IOException { - IndexMetaData metaData = buildIndexMetaData(replicas, settings, indexMapping); + IndexMetaData metaData = buildIndexMetaData(replicas); return new ReplicationGroup(metaData); } @@ -116,17 +111,9 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } protected IndexMetaData buildIndexMetaData(int replicas, Map mappings) throws IOException { - return buildIndexMetaData(replicas, Settings.EMPTY, mappings); - } - - protected IndexMetaData buildIndexMetaData(int replicas, Settings indexSettings, Map mappings) throws IOException { Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, replicas) .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), - randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) - .put(indexSettings) .build(); IndexMetaData.Builder metaData = IndexMetaData.builder(index.getName()) .settings(settings) @@ -159,7 +146,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } }); - protected ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { + ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { final ShardRouting primaryRouting = this.createShardRouting("s0", true); primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting), () -> {}); replicas = new CopyOnWriteArrayList<>(); @@ -461,7 +448,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } } - protected abstract class ReplicationAction, + abstract class ReplicationAction, ReplicaRequest extends ReplicationRequest, Response extends ReplicationResponse> { private final Request request; @@ -469,7 +456,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase private final ReplicationGroup replicationGroup; private final String opType; - protected ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { + ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { this.request = request; this.listener = listener; this.replicationGroup = group; @@ -595,11 +582,11 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase } } - protected class PrimaryResult implements ReplicationOperation.PrimaryResult { + class PrimaryResult implements ReplicationOperation.PrimaryResult { final ReplicaRequest replicaRequest; final Response finalResponse; - public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { + PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { this.replicaRequest = replicaRequest; this.finalResponse = finalResponse; } 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 2f4a3dfd6c1..d2a84589669 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 @@ -18,8 +18,13 @@ */ package org.elasticsearch.index.shard; +import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexNotFoundException; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; @@ -52,8 +57,10 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.engine.InternalEngineFactory; +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.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.similarity.SimilarityService; @@ -173,63 +180,37 @@ public abstract class IndexShardTestCase extends ESTestCase { } /** - * Creates a new initializing shard. The shard will have its own unique data path. + * creates a new initializing shard. The shard will have its own unique data path. * - * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica (ready to recover from - * another shard) + * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica + * (ready to recover from another shard) */ protected IndexShard newShard(boolean primary) throws IOException { - return newShard(primary, Settings.EMPTY, new InternalEngineFactory()); + ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), primary, + ShardRoutingState.INITIALIZING, + primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); + return newShard(shardRouting); } /** - * Creates a new initializing shard. The shard will have its own unique data path. + * creates a new initializing shard. The shard will have its own unique data path. * - * @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica (ready to recover from - * another shard) - * @param settings the settings to use for this shard - * @param engineFactory the engine factory to use for this shard - */ - protected IndexShard newShard(boolean primary, Settings settings, EngineFactory engineFactory) throws IOException { - final RecoverySource recoverySource = - primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE; - final ShardRouting shardRouting = - TestShardRouting.newShardRouting( - new ShardId("index", "_na_", 0), randomAlphaOfLength(10), primary, ShardRoutingState.INITIALIZING, recoverySource); - return newShard(shardRouting, settings, engineFactory); - } - - protected IndexShard newShard(ShardRouting shardRouting, final IndexingOperationListener... listeners) throws IOException { - return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners); - } - - /** - * Creates a new initializing shard. The shard will have its own unique data path. - * - * @param shardRouting the {@link ShardRouting} to use for this shard - * @param settings the settings to use for this shard - * @param engineFactory the engine factory to use for this shard - * @param listeners an optional set of listeners to add to the shard + * @param shardRouting the {@link ShardRouting} to use for this shard + * @param listeners an optional set of listeners to add to the shard */ protected IndexShard newShard( final ShardRouting shardRouting, - final Settings settings, - final EngineFactory engineFactory, final IndexingOperationListener... listeners) throws IOException { assert shardRouting.initializing() : shardRouting; - 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(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), - randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) - .put(settings) - .build(); + Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .build(); IndexMetaData.Builder metaData = IndexMetaData.builder(shardRouting.getIndexName()) - .settings(indexSettings) + .settings(settings) .primaryTerm(0, primaryTerm) .putMapping("_doc", "{ \"properties\": {} }"); - return newShard(shardRouting, metaData.build(), engineFactory, listeners); + return newShard(shardRouting, metaData.build(), listeners); } /** @@ -244,7 +225,7 @@ public abstract class IndexShardTestCase extends ESTestCase { ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, randomAlphaOfLength(5), primary, ShardRoutingState.INITIALIZING, primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners); + return newShard(shardRouting, listeners); } /** @@ -284,10 +265,9 @@ public abstract class IndexShardTestCase extends ESTestCase { * @param indexMetaData indexMetaData for the shard, including any mapping * @param listeners an optional set of listeners to add to the shard */ - protected IndexShard newShard( - ShardRouting routing, IndexMetaData indexMetaData, EngineFactory engineFactory, IndexingOperationListener... listeners) + protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, IndexingOperationListener... listeners) throws IOException { - return newShard(routing, indexMetaData, null, engineFactory, () -> {}, listeners); + return newShard(routing, indexMetaData, null, new InternalEngineFactory(), () -> {}, listeners); } /** @@ -392,39 +372,19 @@ public abstract class IndexShardTestCase extends ESTestCase { } /** - * Creates a new empty shard and starts it. The shard will randomly be a replica or a primary. + * creates a new empyu shard and starts it. The shard will be either a replica or a primary. */ protected IndexShard newStartedShard() throws IOException { return newStartedShard(randomBoolean()); } /** - * Creates a new empty shard and starts it - * @param settings the settings to use for this shard - */ - protected IndexShard newStartedShard(Settings settings) throws IOException { - return newStartedShard(randomBoolean(), settings, new InternalEngineFactory()); - } - - /** - * Creates a new empty shard and starts it. + * creates a new empty shard and starts it. * * @param primary controls whether the shard will be a primary or a replica. */ - protected IndexShard newStartedShard(final boolean primary) throws IOException { - return newStartedShard(primary, Settings.EMPTY, new InternalEngineFactory()); - } - - /** - * Creates a new empty shard with the specified settings and engine factory and starts it. - * - * @param primary controls whether the shard will be a primary or a replica. - * @param settings the settings to use for this shard - * @param engineFactory the engine factory to use for this shard - */ - protected IndexShard newStartedShard( - final boolean primary, final Settings settings, final EngineFactory engineFactory) throws IOException { - IndexShard shard = newShard(primary, settings, engineFactory); + protected IndexShard newStartedShard(boolean primary) throws IOException { + IndexShard shard = newShard(primary); if (primary) { recoverShardFromStore(shard); } else { @@ -441,7 +401,6 @@ public abstract class IndexShardTestCase extends ESTestCase { for (IndexShard shard : shards) { if (shard != null) { try { - assertConsistentHistoryBetweenTranslogAndLucene(shard); shard.close("test", false); } finally { IOUtils.close(shard.store()); @@ -623,7 +582,22 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Set getShardDocUIDs(final IndexShard shard) throws IOException { - return EngineTestCase.getDocIds(shard.getEngine(), true); + shard.refresh("get_uids"); + try (Engine.Searcher searcher = shard.acquireSearcher("test")) { + 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(IdFieldMapper.NAME)); + BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); + ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); + } + } + } + return ids; + } } protected void assertDocCount(IndexShard shard, int docDount) throws IOException { @@ -636,12 +610,6 @@ public abstract class IndexShardTestCase extends ESTestCase { assertThat(shardDocUIDs, hasSize(ids.length)); } - public static void assertConsistentHistoryBetweenTranslogAndLucene(IndexShard shard) throws IOException { - final Engine engine = shard.getEngineOrNull(); - if (engine != null) { - EngineTestCase.assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, shard.mapperService()); - } - } protected Engine.IndexResult indexDoc(IndexShard shard, String type, String id) throws IOException { return indexDoc(shard, type, id, "{}"); @@ -685,14 +653,11 @@ public abstract class IndexShardTestCase extends ESTestCase { } protected Engine.DeleteResult deleteDoc(IndexShard shard, String type, String id) throws IOException { - final Engine.DeleteResult result; if (shard.routingEntry().primary()) { - result = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); - shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(), shard.getEngine().getLocalCheckpoint()); + return shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); } else { - result = shard.applyDeleteOperationOnReplica(shard.seqNoStats().getMaxSeqNo() + 1, 0L, type, id); + return shard.applyDeleteOperationOnReplica(shard.seqNoStats().getMaxSeqNo() + 1, 0L, type, id); } - return result; } protected void flushShard(IndexShard shard) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index be9e40ab420..322e2a128c9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -723,10 +723,6 @@ public abstract class ESIntegTestCase extends ESTestCase { } // always default delayed allocation to 0 to make sure we have tests are not delayed builder.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), 0); - builder.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()); - if (randomBoolean()) { - builder.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), between(0, 1000)); - } return builder.build(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index 19290f8cf11..9633f56dea9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -41,7 +41,6 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.node.MockNode; @@ -88,14 +87,6 @@ public abstract class ESSingleNodeTestCase extends ESTestCase { .setOrder(0) .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)).get(); - client().admin().indices() - .preparePutTemplate("random-soft-deletes-template") - .setPatterns(Collections.singletonList("*")) - .setOrder(0) - .setSettings(Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean()) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), - randomBoolean() ? IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.get(Settings.EMPTY) : between(0, 1000)) - ).get(); } private static void stopNode() throws IOException { diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 4c813372fae..306f79e5e16 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -1163,26 +1163,6 @@ public final class InternalTestCluster extends TestCluster { }); } - /** - * Asserts that the document history in Lucene index is consistent with Translog's on every index shard of the cluster. - * This assertion might be expensive, thus we prefer not to execute on every test but only interesting tests. - */ - public void assertConsistentHistoryBetweenTranslogAndLuceneIndex() throws IOException { - final Collection nodesAndClients = nodes.values(); - for (NodeAndClient nodeAndClient : nodesAndClients) { - IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); - for (IndexService indexService : indexServices) { - for (IndexShard indexShard : indexService) { - try { - IndexShardTestCase.assertConsistentHistoryBetweenTranslogAndLucene(indexShard); - } catch (AlreadyClosedException ignored) { - // shard is closed - } - } - } - } - } - private void randomlyResetClients() throws IOException { // only reset the clients on nightly tests, it causes heavy load... if (RandomizedTest.isNightly() && rarely(random)) {