From 5ad569abe0e416ecd0aa65a94373eab372d0e636 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 28 Aug 2018 15:22:23 -0400 Subject: [PATCH] Integrates soft-deletes into Elasticsearch This PR integrates Lucene soft-deletes (LUCENE-8200) into Elasticsearch. Highlight works in this PR include: 1. Replace hard-deletes by soft-deletes in InternalEngine 2. Use _recovery_source if _source is disabled or modified (#31106) 3. Soft-deletes retention policy based on the global checkpoint (#30335) 4. Read operation history from Lucene instead of translog (#30120) 5. Use Lucene history in peer-recovery (#30522) These works have been done by the whole team; however, these individuals (lexical order) have significant contribution in coding and reviewing: Co-authored-by: Adrien Grand Co-authored-by: Boaz Leskes Co-authored-by: Jason Tedor Co-authored-by: Martijn van Groningen Co-authored-by: Nhat Nguyen Co-authored-by: Simon Willnauer --- docs/reference/indices/flush.asciidoc | 3 +- .../percolator/CandidateQueryTests.java | 8 +- .../PercolatorFieldMapperTests.java | 30 +- .../elasticsearch/common/lucene/Lucene.java | 86 ++- .../uid/PerThreadIDVersionAndSeqNoLookup.java | 21 +- .../common/settings/IndexScopedSettings.java | 2 + .../elasticsearch/index/IndexSettings.java | 38 ++ .../index/engine/CombinedDeletionPolicy.java | 12 +- .../elasticsearch/index/engine/Engine.java | 28 +- .../index/engine/EngineConfig.java | 27 +- .../index/engine/InternalEngine.java | 358 +++++++++-- .../index/engine/LuceneChangesSnapshot.java | 369 +++++++++++ .../RecoverySourcePruneMergePolicy.java | 292 +++++++++ .../index/engine/SoftDeletesPolicy.java | 120 ++++ .../index/fieldvisitor/FieldsVisitor.java | 10 +- .../index/mapper/DocumentMapper.java | 34 +- .../index/mapper/DocumentParser.java | 33 +- .../index/mapper/FieldNamesFieldMapper.java | 5 +- .../index/mapper/ParseContext.java | 20 +- .../index/mapper/ParsedDocument.java | 11 + .../index/mapper/SeqNoFieldMapper.java | 7 +- .../index/mapper/SourceFieldMapper.java | 16 +- .../elasticsearch/index/shard/IndexShard.java | 47 +- .../index/shard/PrimaryReplicaSyncer.java | 2 +- .../index/shard/StoreRecovery.java | 1 + .../org/elasticsearch/index/store/Store.java | 2 +- .../index/translog/Translog.java | 3 + .../index/translog/TranslogWriter.java | 21 +- .../translog/TruncateTranslogCommand.java | 2 + .../recovery/RecoverySourceHandler.java | 59 +- .../blobstore/BlobStoreRepository.java | 1 + .../cluster/routing/PrimaryAllocationIT.java | 1 + .../common/lucene/LuceneTests.java | 91 +++ .../discovery/AbstractDisruptionTestCase.java | 1 + .../gateway/RecoveryFromGatewayIT.java | 13 +- .../index/IndexServiceTests.java | 3 +- .../index/IndexSettingsTests.java | 8 + .../engine/CombinedDeletionPolicyTests.java | 69 +- .../index/engine/InternalEngineTests.java | 594 ++++++++++++------ .../engine/LuceneChangesSnapshotTests.java | 289 +++++++++ .../RecoverySourcePruneMergePolicyTests.java | 161 +++++ .../index/engine/SoftDeletesPolicyTests.java | 75 +++ .../index/mapper/DocumentParserTests.java | 10 +- .../index/mapper/DynamicMappingTests.java | 6 +- .../IndexLevelReplicationTests.java | 24 +- .../RecoveryDuringReplicationTests.java | 11 +- .../index/shard/IndexShardTests.java | 58 +- .../shard/PrimaryReplicaSyncerTests.java | 21 +- .../index/shard/RefreshListenersTests.java | 4 +- .../indices/recovery/IndexRecoveryIT.java | 6 + .../PeerRecoveryTargetServiceTests.java | 2 + .../recovery/RecoverySourceHandlerTests.java | 6 - .../indices/recovery/RecoveryTests.java | 80 ++- .../indices/stats/IndexStatsIT.java | 37 +- .../AbstractSnapshotIntegTestCase.java | 6 + .../SharedClusterSnapshotRestoreIT.java | 13 +- .../versioning/SimpleVersioningIT.java | 23 + .../index/engine/EngineTestCase.java | 400 +++++++++++- .../ESIndexLevelReplicationTestCase.java | 27 +- .../index/shard/IndexShardTestCase.java | 131 ++-- .../elasticsearch/test/ESIntegTestCase.java | 4 + .../test/ESSingleNodeTestCase.java | 9 + .../test/InternalTestCluster.java | 20 + 63 files changed, 3376 insertions(+), 495 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java create mode 100644 server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java create mode 100644 server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index 8583afc96ab1f..37d48eec2a215 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -101,7 +101,8 @@ which returns something similar to: "translog_generation" : "2", "max_seq_no" : "-1", "sync_id" : "AVvFY-071siAOuFGEO9P", <1> - "max_unsafe_auto_id_timestamp" : "-1" + "max_unsafe_auto_id_timestamp" : "-1", + "min_retained_seq_no": "0" }, "num_docs" : 0 } 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 e6d637aabb141..9c8979601e8dc 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/CandidateQueryTests.java @@ -77,6 +77,7 @@ 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; @@ -87,6 +88,7 @@ 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; @@ -1109,7 +1111,11 @@ private void duelRun(PercolateQuery.QueryStore queryStore, MemoryIndex memoryInd } private void addQuery(Query query, List docs) { - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + 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, 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 ecff48b344c53..80524a2f862fe 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java @@ -42,6 +42,7 @@ 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; @@ -58,6 +59,7 @@ 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; @@ -182,7 +184,11 @@ public void testExtractTerms() throws Exception { DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + 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, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -204,7 +210,7 @@ public void testExtractTerms() throws Exception { bq.add(termQuery1, Occur.MUST); bq.add(termQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, mapperService.documentMapperParser(), + parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -232,8 +238,12 @@ public void testExtractRanges() throws Exception { 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.EMPTY, + ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); ParseContext.Document document = parseContext.doc(); @@ -259,7 +269,7 @@ public void testExtractRanges() throws Exception { .rangeQuery(15, 20, true, true, null, null, null, null); bq.add(rangeQuery2, Occur.MUST); - parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + parseContext = new ParseContext.InternalParseContext(settings, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(bq.build(), parseContext); document = parseContext.doc(); @@ -283,7 +293,11 @@ public void testExtractTermsAndRanges_failed() throws Exception { 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); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + 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, mapperService.documentMapperParser(), documentMapper, null, null); fieldMapper.processQuery(query, parseContext); ParseContext.Document document = parseContext.doc(); @@ -298,7 +312,11 @@ public void testExtractTermsAndRanges_partial() throws Exception { PhraseQuery phraseQuery = new PhraseQuery("field", "term"); DocumentMapper documentMapper = mapperService.documentMapper("doc"); PercolatorFieldMapper fieldMapper = (PercolatorFieldMapper) documentMapper.mappers().getMapper(fieldName); - ParseContext.InternalParseContext parseContext = new ParseContext.InternalParseContext(Settings.EMPTY, + 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, 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 a24a6aea07fcd..d807f8fc12e4b 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -27,8 +27,10 @@ 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; @@ -96,6 +98,8 @@ 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()); @@ -140,7 +144,7 @@ public static Iterable files(SegmentInfos infos) throws IOException { public static int getNumDocs(SegmentInfos info) { int numDocs = 0; for (SegmentCommitInfo si : info) { - numDocs += si.info.maxDoc() - si.getDelCount(); + numDocs += si.info.maxDoc() - si.getDelCount() - si.getSoftDelCount(); } return numDocs; } @@ -197,6 +201,7 @@ public static SegmentInfos pruneUnreferencedFiles(String segmentsFileName, Direc } 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) @@ -220,6 +225,7 @@ public static void cleanLuceneIndex(Directory directory) throws IOException { } } 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... @@ -829,4 +835,82 @@ public int length() { } }; } + + /** + * 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: Avoid recalculate numDocs everytime. + 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 38fcdfe5f1b62..3a037bed62b7f 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,6 +28,7 @@ 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; @@ -66,15 +67,22 @@ final class PerThreadIDVersionAndSeqNoLookup { */ PerThreadIDVersionAndSeqNoLookup(LeafReader reader, String uidField) throws IOException { this.uidField = uidField; - Terms terms = reader.terms(uidField); + final Terms terms = reader.terms(uidField); if (terms == null) { - throw new IllegalArgumentException("reader misses the [" + uidField + "] field"); + // 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(); } - termsEnum = terms.iterator(); if (reader.getNumericDocValues(VersionFieldMapper.NAME) == null) { - throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field"); + throw new IllegalArgumentException("reader misses the [" + VersionFieldMapper.NAME + "] field; _uid terms [" + terms + "]"); } - Object readerKey = null; assert (readerKey = reader.getCoreCacheHelper().getKey()) != null; this.readerKey = readerKey; @@ -111,7 +119,8 @@ public DocIdAndVersion lookupVersion(BytesRef id, LeafReaderContext context) * {@link DocIdSetIterator#NO_MORE_DOCS} is returned if not found * */ private int getDocID(BytesRef id, Bits liveDocs) throws IOException { - if (termsEnum.seekExact(id)) { + // termsEnum can possibly be null here if this leaf contains only no-ops. + if (termsEnum != null && 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 137378f509d6f..57d176ba41435 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -129,6 +129,8 @@ 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 44cd743bbd428..4e1176c902e8d 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -237,6 +237,21 @@ 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", true, 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. */ @@ -289,6 +304,8 @@ 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; @@ -400,6 +417,8 @@ public IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSetti 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_6_5_0) && 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); @@ -458,6 +477,7 @@ public IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSetti 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; } @@ -841,4 +861,22 @@ public String getDefaultPipeline() { 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 d0575c8a8c977..d10690379eddd 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -46,14 +46,17 @@ 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, LongSupplier globalCheckpointSupplier) { + CombinedDeletionPolicy(Logger logger, TranslogDeletionPolicy translogDeletionPolicy, + SoftDeletesPolicy softDeletesPolicy, LongSupplier globalCheckpointSupplier) { this.logger = logger; this.translogDeletionPolicy = translogDeletionPolicy; + this.softDeletesPolicy = softDeletesPolicy; this.globalCheckpointSupplier = globalCheckpointSupplier; this.snapshottedCommits = new ObjectIntHashMap<>(); } @@ -80,7 +83,7 @@ public synchronized void onCommit(List commits) throws IO deleteCommit(commits.get(i)); } } - updateTranslogDeletionPolicy(); + updateRetentionPolicy(); } private void deleteCommit(IndexCommit commit) throws IOException { @@ -90,7 +93,7 @@ private void deleteCommit(IndexCommit commit) throws IOException { assert commit.isDeleted() : "Deletion commit [" + commitDescription(commit) + "] was suppressed"; } - private void updateTranslogDeletionPolicy() throws IOException { + private void updateRetentionPolicy() 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"; @@ -101,6 +104,9 @@ private void updateTranslogDeletionPolicy() throws IOException { 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 4d95cf89ef00e..a5d5521415080 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -58,6 +58,7 @@ 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; @@ -97,6 +98,7 @@ 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; @@ -585,18 +587,32 @@ public enum SearcherScope { public abstract void syncTranslog() throws IOException; - public abstract Closeable acquireTranslogRetentionLock(); + /** + * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed + */ + public abstract Closeable acquireRetentionLockForPeerRecovery(); + + /** + * Creates a new history snapshot from Lucene for reading operations whose seqno in the requesting seqno range (both inclusive) + */ + public abstract Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService mapperService, + long minSeqNo, long maxSeqNo, boolean requiredFullRange) 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. + */ + public abstract Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; /** - * 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. + * Returns the estimated number of history operations whose seq# at least {@code startingSeqNo}(inclusive) in this engine. */ - public abstract Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException; + public abstract int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException; /** - * Returns the estimated number of translog operations in this engine whose seq# at least the provided seq#. + * Checks if this engine has every operations since {@code startingSeqNo}(inclusive) in its history (either Lucene or translog) */ - public abstract int estimateTranslogOperationsFromMinSeq(long minSeqNo); + public abstract boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException; 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 2deae61bd52e9..23a90553f60a8 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -34,6 +34,7 @@ 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; @@ -80,6 +81,7 @@ 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. @@ -126,7 +128,8 @@ public EngineConfig(ShardId shardId, String allocationId, ThreadPool threadPool, List externalRefreshListener, List internalRefreshListener, Sort indexSort, TranslogRecoveryRunner translogRecoveryRunner, CircuitBreakerService circuitBreakerService, - LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier) { + LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier, + TombstoneDocSupplier tombstoneDocSupplier) { this.shardId = shardId; this.allocationId = allocationId; this.indexSettings = indexSettings; @@ -164,6 +167,7 @@ public EngineConfig(ShardId shardId, String allocationId, ThreadPool threadPool, this.circuitBreakerService = circuitBreakerService; this.globalCheckpointSupplier = globalCheckpointSupplier; this.primaryTermSupplier = primaryTermSupplier; + this.tombstoneDocSupplier = tombstoneDocSupplier; } /** @@ -373,4 +377,25 @@ public CircuitBreakerService getCircuitBreakerService() { 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 023e659ffabeb..fd06655f566b1 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -21,16 +21,20 @@ 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; @@ -42,6 +46,7 @@ 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; @@ -61,7 +66,11 @@ 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; @@ -140,6 +149,10 @@ 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 @@ -184,8 +197,10 @@ public InternalEngine(EngineConfig engineConfig) { 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, translog::getLastSyncedGlobalCheckpoint); + new CombinedDeletionPolicy(logger, translogDeletionPolicy, softDeletesPolicy, translog::getLastSyncedGlobalCheckpoint); writer = createWriter(); bootstrapAppendOnlyInfoFromWriter(writer); historyUUID = loadHistoryUUID(writer); @@ -215,6 +230,8 @@ public InternalEngine(EngineConfig engineConfig) { 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) { @@ -240,6 +257,18 @@ private LocalCheckpointTracker createLocalCheckpointTracker( 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 @@ -451,19 +480,35 @@ public void syncTranslog() throws IOException { 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 Closeable acquireTranslogRetentionLock() { - return getTranslog().acquireRetentionLock(); - } - - @Override - public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { - return getTranslog().newSnapshotFromMinSeqNo(minSeqNo); + public Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + return newLuceneChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false); + } else { + return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo); + } } + /** + * Returns the estimated number of history operations whose seq# at least the provided seq# in this engine. + */ @Override - public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { - return getTranslog().estimateTotalOperationsFromMinSeq(minSeqNo); + public int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException { + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + try (Translog.Snapshot snapshot = + newLuceneChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false)) { + return snapshot.totalOperations(); + } catch (IOException ex) { + maybeFailEngine(source, ex); + throw ex; + } + } else { + return getTranslog().estimateTotalOperationsFromMinSeq(startingSeqNo); + } } @Override @@ -790,7 +835,7 @@ public IndexResult index(Index index) throws IOException { if (plan.earlyResultOnPreFlightError.isPresent()) { indexResult = plan.earlyResultOnPreFlightError.get(); assert indexResult.getResultType() == Result.Type.FAILURE : indexResult.getResultType(); - } else if (plan.indexIntoLucene) { + } else if (plan.indexIntoLucene || plan.addStaleOpToLucene) { indexResult = indexIntoLucene(index, plan); } else { indexResult = new IndexResult( @@ -802,7 +847,9 @@ public IndexResult index(Index index) throws IOException { 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 with the generated seq_no - location = translog.add(new Translog.NoOp(indexResult.getSeqNo(), index.primaryTerm(), indexResult.getFailure().toString())); + final NoOp noOp = new NoOp(indexResult.getSeqNo(), index.primaryTerm(), index.origin(), + index.startTime(), indexResult.getFailure().toString()); + location = innerNoOp(noOp).getTranslogLocation(); } else { location = null; } @@ -854,7 +901,6 @@ private IndexingStrategy planIndexingAsNonPrimary(Index index) throws IOExceptio // 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 @@ -863,16 +909,15 @@ private IndexingStrategy planIndexingAsNonPrimary(Index index) throws IOExceptio // 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 { - plan = IndexingStrategy.processNormally( - opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, index.seqNo(), index.version() - ); + 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()); + } } } return plan; @@ -921,7 +966,7 @@ private IndexResult indexIntoLucene(Index index, IndexingStrategy plan) 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; + assert plan.indexIntoLucene || plan.addStaleOpToLucene; /* 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. @@ -929,7 +974,9 @@ private IndexResult indexIntoLucene(Index index, IndexingStrategy plan) index.parsedDoc().updateSeqID(plan.seqNoForIndexing, index.primaryTerm()); index.parsedDoc().version().setLongValue(plan.versionForIndexing); try { - if (plan.useLuceneUpdateDocument) { + if (plan.addStaleOpToLucene) { + addStaleDocs(index.docs(), indexWriter); + } else 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 @@ -993,16 +1040,27 @@ private void addDocs(final List docs, final IndexWriter i numDocAppends.inc(docs.size()); } - private static final class IndexingStrategy { + private void addStaleDocs(final List docs, final IndexWriter indexWriter) throws IOException { + assert softDeleteEnabled : "Add history documents but soft-deletes is disabled"; + docs.forEach(d -> d.add(softDeletesField)); + if (docs.size() > 1) { + indexWriter.addDocuments(docs); + } else { + indexWriter.addDocument(docs.get(0)); + } + } + + protected 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, long seqNoForIndexing, + boolean indexIntoLucene, boolean addStaleOpToLucene, long seqNoForIndexing, long versionForIndexing, IndexResult earlyResultOnPreFlightError) { assert useLuceneUpdateDocument == false || indexIntoLucene : "use lucene update is set to true, but we're not indexing into lucene"; @@ -1015,37 +1073,40 @@ private IndexingStrategy(boolean currentNotFoundOrDeleted, boolean useLuceneUpda 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, seqNoForIndexing, 1, null); + return new IndexingStrategy(true, false, true, false, 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, SequenceNumbers.UNASSIGNED_SEQ_NO, Versions.NOT_FOUND, result); + currentNotFoundOrDeleted, false, 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, seqNoForIndexing, versionForIndexing, null); + true, false, seqNoForIndexing, versionForIndexing, null); } static IndexingStrategy overrideExistingAsIfNotThere( long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(true, true, true, seqNoForIndexing, versionForIndexing, null); + return new IndexingStrategy(true, true, true, false, seqNoForIndexing, versionForIndexing, null); + } + + static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, long seqNoForIndexing, long versionForIndexing) { + return new IndexingStrategy(currentNotFoundOrDeleted, false, false, false, seqNoForIndexing, versionForIndexing, null); } - static IndexingStrategy processButSkipLucene(boolean currentNotFoundOrDeleted, - long seqNoForIndexing, long versionForIndexing) { - return new IndexingStrategy(currentNotFoundOrDeleted, false, - false, seqNoForIndexing, versionForIndexing, null); + static IndexingStrategy processAsStaleOp(boolean addStaleOpToLucene, long seqNoForIndexing, long versionForIndexing) { + return new IndexingStrategy(false, false, false, addStaleOpToLucene, seqNoForIndexing, versionForIndexing, null); } } @@ -1072,10 +1133,18 @@ private boolean assertDocDoesNotExist(final Index index, final boolean allowDele } private void updateDocs(final Term uid, final List docs, final IndexWriter indexWriter) throws IOException { - if (docs.size() > 1) { - indexWriter.updateDocuments(uid, docs); + if (softDeleteEnabled) { + if (docs.size() > 1) { + indexWriter.softUpdateDocuments(uid, docs, softDeletesField); + } else { + indexWriter.softUpdateDocument(uid, docs.get(0), softDeletesField); + } } else { - indexWriter.updateDocument(uid, docs.get(0)); + if (docs.size() > 1) { + indexWriter.updateDocuments(uid, docs); + } else { + indexWriter.updateDocument(uid, docs.get(0)); + } } numDocUpdates.inc(docs.size()); } @@ -1099,7 +1168,7 @@ public DeleteResult delete(Delete delete) throws IOException { if (plan.earlyResultOnPreflightError.isPresent()) { deleteResult = plan.earlyResultOnPreflightError.get(); - } else if (plan.deleteFromLucene) { + } else if (plan.deleteFromLucene || plan.addStaleOpToLucene) { deleteResult = deleteInLucene(delete, plan); } else { deleteResult = new DeleteResult( @@ -1142,7 +1211,7 @@ private DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws IOExcept // 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 OpVsLuceneDocStatus opVsLucene; + final DeletionStrategy plan; 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 @@ -1151,18 +1220,15 @@ private DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws IOExcept // 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 { - plan = DeletionStrategy.processNormally( - opVsLucene == OpVsLuceneDocStatus.LUCENE_DOC_NOT_FOUND, - delete.seqNo(), delete.version()); + 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()); + } } return plan; } @@ -1197,15 +1263,31 @@ private DeletionStrategy planDeletionAsPrimary(Delete delete) throws IOException private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) throws IOException { try { - if (plan.currentlyDeleted == false) { + 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) { // 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) { @@ -1219,15 +1301,16 @@ private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) } } - private static final class DeletionStrategy { + protected 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 currentlyDeleted, + private DeletionStrategy(boolean deleteFromLucene, boolean addStaleOpToLucene, boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion, DeleteResult earlyResultOnPreflightError) { assert (deleteFromLucene && earlyResultOnPreflightError != null) == false : @@ -1235,6 +1318,7 @@ private DeletionStrategy(boolean deleteFromLucene, boolean currentlyDeleted, "deleteFromLucene: " + deleteFromLucene + " earlyResultOnPreFlightError:" + earlyResultOnPreflightError; this.deleteFromLucene = deleteFromLucene; + this.addStaleOpToLucene = addStaleOpToLucene; this.currentlyDeleted = currentlyDeleted; this.seqNoOfDeletion = seqNoOfDeletion; this.versionOfDeletion = versionOfDeletion; @@ -1246,16 +1330,22 @@ static DeletionStrategy skipDueToVersionConflict( 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, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); + return new DeletionStrategy(false, false, currentlyDeleted, unassignedSeqNo, Versions.NOT_FOUND, deleteResult); } static DeletionStrategy processNormally(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(true, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); + return new DeletionStrategy(true, false, currentlyDeleted, seqNoOfDeletion, versionOfDeletion, null); } - public static DeletionStrategy processButSkipLucene(boolean currentlyDeleted, long seqNoOfDeletion, long versionOfDeletion) { - return new DeletionStrategy(false, 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); } } @@ -1284,7 +1374,28 @@ private NoOpResult innerNoOp(final NoOp noOp) throws IOException { assert noOp.seqNo() > SequenceNumbers.NO_OPS_PERFORMED; final long seqNo = noOp.seqNo(); try { - final NoOpResult noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo()); + 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()); 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); @@ -1930,7 +2041,11 @@ private IndexWriter createWriter() throws IOException { // pkg-private for testing IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException { - return new IndexWriter(directory, iwc); + if (Assertions.ENABLED) { + return new AssertingIndexWriter(directory, iwc); + } else { + return new IndexWriter(directory, iwc); + } } private IndexWriterConfig getIndexWriterConfig() { @@ -1946,11 +2061,15 @@ private IndexWriterConfig getIndexWriterConfig() { } 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 = new ElasticsearchMergePolicy(mergePolicy); - iwc.setMergePolicy(mergePolicy); + 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)); iwc.setSimilarity(engineConfig.getSimilarity()); iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac()); iwc.setCodec(engineConfig.getCodec()); @@ -2147,6 +2266,9 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl 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(); }); @@ -2202,6 +2324,7 @@ public void onSettingsChanged() { final IndexSettings indexSettings = engineConfig.getIndexSettings(); translogDeletionPolicy.setRetentionAgeInMillis(indexSettings.getTranslogRetentionAge().getMillis()); translogDeletionPolicy.setRetentionSizeInBytes(indexSettings.getTranslogRetentionSize().getBytes()); + softDeletesPolicy.setRetentionOperations(indexSettings.getSoftDeleteRetentionOperations()); } public MergeStats getMergeStats() { @@ -2296,6 +2419,60 @@ long getNumDocUpdates() { return numDocUpdates.count(); } + @Override + public Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService mapperService, + long minSeqNo, long maxSeqNo, boolean requiredFullRange) throws IOException { + // TODO: Should we defer the refresh until we really need it? + ensureOpen(); + if (lastRefreshedCheckpoint() < maxSeqNo) { + refresh(source, SearcherScope.INTERNAL); + } + Searcher searcher = acquireSearcher(source, SearcherScope.INTERNAL); + try { + LuceneChangesSnapshot snapshot = new LuceneChangesSnapshot( + searcher, mapperService, LuceneChangesSnapshot.DEFAULT_BATCH_SIZE, minSeqNo, maxSeqNo, requiredFullRange); + searcher = null; + return snapshot; + } 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() { + final Closeable translogLock = translog.acquireRetentionLock(); + final Releasable softDeletesLock = softDeletesPolicy.acquireRetentionLock(); + return () -> IOUtils.close(translogLock, softDeletesLock); + } + @Override public boolean isRecovering() { return pendingTranslogRecovery.get(); @@ -2311,4 +2488,59 @@ private static Map commitDataAsMap(final IndexWriter indexWriter } 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() { + pendingCheckpoint = localCheckpointTracker.getCheckpoint(); // All change until this point should be visible after refresh + } + @Override + public void afterRefresh(boolean didRefresh) { + if (didRefresh) { + refreshedCheckpoint.set(pendingCheckpoint); + } + } + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java new file mode 100644 index 0000000000000..0a13ddb3bb6ce --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -0,0 +1,369 @@ +/* + * 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.SortedNumericSortField; +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 not be negative [" + 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, fromSeqNo, toSeqNo); + final Sort sortedBySeqNoThenByTerm = new Sort( + new SortedNumericSortField(SeqNoFieldMapper.NAME, SortField.Type.LONG), + new SortedNumericSortField(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 new file mode 100644 index 0000000000000..72f41f110dd27 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java @@ -0,0 +1,292 @@ +/* + * 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(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 new file mode 100644 index 0000000000000..af2ded8c46620 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java @@ -0,0 +1,120 @@ +/* + * 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 4c65635c61b3b..462f8ce8e68b4 100644 --- a/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java +++ b/server/src/main/java/org/elasticsearch/index/fieldvisitor/FieldsVisitor.java @@ -54,13 +54,19 @@ 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(); } @@ -103,7 +109,7 @@ public void postProcess(MapperService mapperService) { @Override public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { - if (SourceFieldMapper.NAME.equals(fieldInfo.name)) { + if (sourceFieldName.equals(fieldInfo.name)) { source = new BytesArray(value); } else if (IdFieldMapper.NAME.equals(fieldInfo.name)) { id = Uid.decodeId(value); @@ -175,7 +181,7 @@ public void reset() { requiredFields.addAll(BASE_REQUIRED_FIELDS); if (loadSource) { - requiredFields.add(SourceFieldMapper.NAME); + requiredFields.add(sourceFieldName); } } 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 a0640ac68a992..663aa7e6f9e10 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -19,11 +19,14 @@ 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; @@ -39,12 +42,15 @@ 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 { @@ -121,6 +127,8 @@ public DocumentMapper build(MapperService mapperService) { 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; @@ -171,6 +179,15 @@ public DocumentMapper(MapperService mapperService, Mapping mapping) { } 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() { @@ -242,7 +259,22 @@ public Map objectMappers() { } public ParsedDocument parse(SourceToParse source) throws MapperParsingException { - return documentParser.parseDocument(source); + 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; } /** 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 0fd156c090531..85123f602edf8 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) throws MapperParsingException { + ParsedDocument parseDocument(SourceToParse source, MetadataFieldMapper[] metadataFieldsMappers) throws MapperParsingException { validateType(source); final Mapping mapping = docMapper.mapping(); @@ -64,9 +64,9 @@ ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException try (XContentParser parser = XContentHelper.createParser(docMapperParser.getXContentRegistry(), LoggingDeprecationHandler.INSTANCE, source.source(), xContentType)) { - context = new ParseContext.InternalParseContext(indexSettings.getSettings(), docMapperParser, docMapper, source, parser); + context = new ParseContext.InternalParseContext(indexSettings, docMapperParser, docMapper, source, parser); validateStart(parser); - internalParseDocument(mapping, context, parser); + internalParseDocument(mapping, metadataFieldsMappers, context, parser); validateEnd(parser); } catch (Exception e) { throw wrapInMapperParsingException(source, e); @@ -81,10 +81,11 @@ ParsedDocument parseDocument(SourceToParse source) throws MapperParsingException return parsedDocument(source, context, createDynamicUpdate(mapping, docMapper, context.getDynamicMappers())); } - private static void internalParseDocument(Mapping mapping, ParseContext.InternalParseContext context, XContentParser parser) throws IOException { + private static void internalParseDocument(Mapping mapping, MetadataFieldMapper[] metadataFieldsMappers, + ParseContext.InternalParseContext context, XContentParser parser) throws IOException { final boolean emptyDoc = isEmptyDoc(mapping, parser); - for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { + for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { metadataMapper.preParse(context); } @@ -95,7 +96,7 @@ private static void internalParseDocument(Mapping mapping, ParseContext.Internal parseObjectOrNested(context, mapping.root); } - for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { + for (MetadataFieldMapper metadataMapper : metadataFieldsMappers) { metadataMapper.postParse(context); } } @@ -495,7 +496,7 @@ private static void parseObject(final ParseContext context, ObjectMapper mapper, if (builder == null) { builder = new ObjectMapper.Builder(currentFieldName).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); objectMapper = builder.build(builderContext); context.addDynamicMapper(objectMapper); context.path().add(currentFieldName); @@ -538,7 +539,7 @@ private static void parseArray(ParseContext context, ObjectMapper parentMapper, if (builder == null) { parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName); } else { - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); mapper = builder.build(builderContext); assert mapper != null; if (mapper instanceof ArrayValueMapperParser) { @@ -696,13 +697,13 @@ private static Mapper.Builder createBuilderFromDynamicValue(final ParseCont if (parseableAsLong && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG); if (builder == null) { - builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } else if (parseableAsDouble && context.root().numericDetection()) { Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DOUBLE); if (builder == null) { - builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings())); + builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } else if (parseableAsLong == false && parseableAsDouble == false && context.root().dateDetection()) { @@ -718,7 +719,7 @@ private static Mapper.Builder createBuilderFromDynamicValue(final ParseCont } Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DATE); if (builder == null) { - builder = newDateBuilder(currentFieldName, dateTimeFormatter, Version.indexCreated(context.indexSettings())); + builder = newDateBuilder(currentFieldName, dateTimeFormatter, context.indexSettings().getIndexVersionCreated()); } if (builder instanceof DateFieldMapper.Builder) { DateFieldMapper.Builder dateBuilder = (DateFieldMapper.Builder) builder; @@ -741,7 +742,7 @@ private static Mapper.Builder createBuilderFromDynamicValue(final ParseCont 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, Version.indexCreated(context.indexSettings())); + builder = newLongBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } else if (numberType == XContentParser.NumberType.FLOAT || numberType == XContentParser.NumberType.DOUBLE) { @@ -750,7 +751,7 @@ private static Mapper.Builder createBuilderFromDynamicValue(final ParseCont // 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, Version.indexCreated(context.indexSettings())); + builder = newFloatBuilder(currentFieldName, context.indexSettings().getIndexVersionCreated()); } return builder; } @@ -785,7 +786,7 @@ private static void parseDynamicValue(final ParseContext context, ObjectMapper p return; } final String path = context.path().pathAsText(currentFieldName); - final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); + final Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), context.path()); final MappedFieldType existingFieldType = context.mapperService().fullName(path); final Mapper.Builder builder; if (existingFieldType != null) { @@ -883,8 +884,8 @@ private static Tuple getDynamicParentMapper(ParseContext if (builder == null) { builder = new ObjectMapper.Builder(paths[i]).enabled(true); } - Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings(), context.path()); - mapper = (ObjectMapper) builder.build(builderContext); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), + 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 606777392dec5..8389a30627018 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldNamesFieldMapper.java @@ -24,7 +24,6 @@ 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; @@ -205,12 +204,12 @@ public FieldNamesFieldType fieldType() { } @Override - public void preParse(ParseContext context) throws IOException { + public void preParse(ParseContext context) { } @Override public void postParse(ParseContext context) throws IOException { - if (context.indexSettings().getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).before(Version.V_6_1_0)) { + if (context.indexSettings().getIndexVersionCreated().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 b77ffee05caf0..cf8cc4022fd8a 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -24,9 +24,8 @@ 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; @@ -196,7 +195,7 @@ public boolean isWithinMultiFields() { } @Override - public Settings indexSettings() { + public IndexSettings indexSettings() { return in.indexSettings(); } @@ -315,8 +314,7 @@ public static class InternalParseContext extends ParseContext { private final List documents; - @Nullable - private final Settings indexSettings; + private final IndexSettings indexSettings; private final SourceToParse sourceToParse; @@ -334,8 +332,8 @@ public static class InternalParseContext extends ParseContext { private final Set ignoredFields = new HashSet<>(); - public InternalParseContext(@Nullable Settings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, - SourceToParse source, XContentParser parser) { + public InternalParseContext(IndexSettings indexSettings, DocumentMapperParser docMapperParser, DocumentMapper docMapper, + SourceToParse source, XContentParser parser) { this.indexSettings = indexSettings; this.docMapper = docMapper; this.docMapperParser = docMapperParser; @@ -347,7 +345,7 @@ public InternalParseContext(@Nullable Settings indexSettings, DocumentMapperPars this.version = null; this.sourceToParse = source; this.dynamicMappers = new ArrayList<>(); - this.maxAllowedNumNestedDocs = MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING.get(indexSettings); + this.maxAllowedNumNestedDocs = indexSettings.getValue(MapperService.INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING); this.numNestedDocs = 0L; } @@ -357,8 +355,7 @@ public DocumentMapperParser docMapperParser() { } @Override - @Nullable - public Settings indexSettings() { + public IndexSettings indexSettings() { return this.indexSettings; } @@ -565,8 +562,7 @@ public boolean isWithinMultiFields() { return false; } - @Nullable - public abstract Settings indexSettings(); + public abstract IndexSettings 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 414cb3a98ecab..d2cf17ddd3503 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -83,6 +83,17 @@ public void updateSeqID(long sequenceNumber, long primaryTerm) { 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 ac3ffe4627238..5a0db4163bf28 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java @@ -69,26 +69,29 @@ public static class SequenceIDFields { public final Field seqNo; public final Field seqNoDocValue; public final Field primaryTerm; + public final Field tombstoneField; - public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm) { + public SequenceIDFields(Field seqNo, Field seqNoDocValue, Field primaryTerm, Field tombstoneField) { 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(PRIMARY_TERM_NAME, 0), new NumericDocValuesField(TOMBSTONE_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 f2090613c0965..7bfe793bba4a4 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/SourceFieldMapper.java @@ -19,6 +19,7 @@ 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; @@ -49,6 +50,7 @@ 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; @@ -224,7 +226,8 @@ public Mapper parse(ParseContext context) throws IOException { @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { - BytesReference source = context.sourceToParse().source(); + BytesReference originalSource = context.sourceToParse().source(); + BytesReference source = originalSource; 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) { @@ -240,8 +243,17 @@ protected void parseCreateField(ParseContext context, List field } 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 e030c95b56e34..ef5f9ab0ef3ec 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -92,12 +92,14 @@ 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; @@ -1620,25 +1622,33 @@ public void onSettingsChanged() { } /** - * Acquires a lock on the translog files, preventing them from being trimmed. + * Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed */ - public Closeable acquireTranslogRetentionLock() { - return getEngine().acquireTranslogRetentionLock(); + public Closeable acquireRetentionLockForPeerRecovery() { + return getEngine().acquireRetentionLockForPeerRecovery(); } /** - * 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. + * Returns the estimated number of history operations whose seq# at least the provided seq# in this shard. */ - public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { - return getEngine().newTranslogSnapshotFromMinSeqNo(minSeqNo); + public int estimateNumberOfHistoryOperations(String source, long startingSeqNo) throws IOException { + return getEngine().estimateNumberOfHistoryOperations(source, mapperService, startingSeqNo); } /** - * Returns the estimated number of operations in translog whose seq# at least the provided seq#. + * 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. */ - public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { - return getEngine().estimateTranslogOperationsFromMinSeq(minSeqNo); + 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 List segments(boolean verbose) { @@ -2209,7 +2219,7 @@ private EngineConfig newEngineConfig() { IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm); + indexSort, this::runTranslogRecovery, circuitBreakerService, replicationTracker, () -> operationPrimaryTerm, tombstoneDocSupplier()); } /** @@ -2648,4 +2658,19 @@ public void afterRefresh(boolean didRefresh) throws IOException { 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 1edc0eb5dcafe..016a8afff6964 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 void resync(final IndexShard indexShard, final ActionListener // 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.newTranslogSnapshotFromMinSeqNo(startingSeqNo); + snapshot = indexShard.getHistoryOperations("resync", 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 e9acfe3d8b06f..ae3f90e63e7d1 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -156,6 +156,7 @@ void addIndices(final RecoveryState.Index indexRecoveryStats, final Directory ta 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 001e263ea8ffb..85975bc68c85f 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1009,7 +1009,6 @@ public RecoveryDiff recoveryDiff(MetadataSnapshot recoveryTargetSnapshot) { } 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); @@ -1595,6 +1594,7 @@ private static IndexWriter newIndexWriter(final IndexWriterConfig.OpenMode openM 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 618aa546e4250..f17acac37896d 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1261,6 +1261,8 @@ public String toString() { ", type='" + type + '\'' + ", seqNo=" + seqNo + ", primaryTerm=" + primaryTerm + + ", version=" + version + + ", autoGeneratedIdTimestamp=" + autoGeneratedIdTimestamp + '}'; } @@ -1403,6 +1405,7 @@ public String toString() { "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 f48f2ceb79278..0fa9237f6031e 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -40,6 +40,7 @@ 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; @@ -192,7 +193,25 @@ private synchronized boolean assertNoSeqNumberConflict(long seqNo, BytesReferenc new BufferedChecksumStreamInput(data.streamInput(), "assertion")); Translog.Operation prvOp = Translog.readOperation( new BufferedChecksumStreamInput(previous.v1().streamInput(), "assertion")); - if (newOp.equals(prvOp) == false) { + // TODO: We haven't had timestamp for Index operations in Lucene yet, we need to loosen this check without timestamp. + // We don't store versionType in Lucene index, we need to exclude it from this check + 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) { 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 86995ae7c5a99..a90f8af0af42c 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TruncateTranslogCommand.java @@ -32,6 +32,7 @@ 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; @@ -177,6 +178,7 @@ protected void execute(Terminal terminal, OptionSet options, Environment env) th 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 352f07d57649d..10f796e5e1551 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 RecoveryResponse recoverToTarget() throws IOException { 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.acquireTranslogRetentionLock()) { + try (Closeable ignored = shard.acquireRetentionLockForPeerRecovery()) { final long startingSeqNo; final long requiredSeqNoRangeStart; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && - isTargetSameHistory() && isTranslogReadyForSequenceNumberBasedRecovery(); + isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()); if (isSequenceNumberBasedRecovery) { logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); startingSeqNo = request.startingSeqNo(); @@ -162,14 +162,16 @@ public RecoveryResponse recoverToTarget() throws IOException { } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); } - // 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 + // 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 { - phase1(phase1Snapshot.getIndexCommit(), () -> shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); + final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo); + phase1(phase1Snapshot.getIndexCommit(), () -> estimateNumOps); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); } finally { @@ -186,7 +188,8 @@ public RecoveryResponse recoverToTarget() throws IOException { try { // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, shard.estimateTranslogOperationsFromMinSeq(startingSeqNo)); + prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, + shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); } @@ -207,11 +210,13 @@ public RecoveryResponse recoverToTarget() throws IOException { */ cancellableThreads.execute(() -> shard.waitForOpsToComplete(endingSeqNo)); - 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)); + 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)); + } final long targetLocalCheckpoint; - try(Translog.Snapshot snapshot = shard.newTranslogSnapshotFromMinSeqNo(startingSeqNo)) { + try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) { targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot); } catch (Exception e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); @@ -268,36 +273,6 @@ public void onFailure(Exception e) { }); } - /** - * 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 a4d6518e9af92..9469f657c96bd 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -1492,6 +1492,7 @@ public void restore() throws IOException { // 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/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 90173455c3be3..9786c0eaf5290 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -392,6 +392,7 @@ public void testPrimaryReplicaResyncFailed() throws Exception { 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 753aedea01e02..890f6ef163b33 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java @@ -33,18 +33,23 @@ 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; @@ -53,6 +58,8 @@ 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(); @@ -406,4 +413,88 @@ public void testMMapHackSupported() throws Exception { // 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 6bdd8ea3f2e07..ac2f2b0d4f32e 100644 --- a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java +++ b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java @@ -109,6 +109,7 @@ public void setDisruptionScheme(ServiceDisruptionScheme scheme) { 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 d098c4918a76f..b0b6c35f92a1a 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -40,6 +40,7 @@ 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; @@ -397,7 +398,8 @@ public void testReuseInFileBasedPeerRecovery() throws Exception { .get(); logger.info("--> indexing docs"); - for (int i = 0; i < randomIntBetween(1, 1024); i++) { + int numDocs = randomIntBetween(1, 1024); + for (int i = 0; i < numDocs; i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -419,12 +421,15 @@ public void testReuseInFileBasedPeerRecovery() throws Exception { } 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 < randomIntBetween(1, 1024); i++) { + for (int i = 0; i < moreDocs; i++) { client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } @@ -432,8 +437,12 @@ public Settings onNodeStopped(String nodeName) throws Exception { 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 28fa440d96ac2..b0b4ec3930adf 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java @@ -32,6 +32,7 @@ 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; @@ -306,7 +307,7 @@ public void testAsyncTranslogTrimActuallyWorks() throws Exception { .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), -1)) .get(); IndexShard shard = indexService.getShard(0); - assertBusy(() -> assertThat(shard.estimateTranslogOperationsFromMinSeq(0L), equalTo(0))); + assertBusy(() -> assertThat(IndexShardTestCase.getTranslog(shard).totalOperations(), 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 b7da5add2acf6..64a2fa69bcbd5 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexSettingsTests.java @@ -553,4 +553,12 @@ public void testQueryDefaultField() { ); 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 ea7de50b7b34c..3f9fc9a0429b7 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -51,20 +51,24 @@ 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, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, 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(lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); maxSeqNoList.add(lastMaxSeqNo); translogGenList.add(lastTranslogGen); } @@ -85,14 +89,19 @@ public void testKeepCommitsAfterGlobalCheckpoint() throws Exception { } 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, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); long lastMaxSeqNo = between(1, 1000); + long lastCheckpoint = randomLongBetween(-1, lastMaxSeqNo); long lastTranslogGen = between(1, 20); int safeIndex = 0; List commitList = new ArrayList<>(); @@ -102,8 +111,9 @@ public void testAcquireIndexCommit() throws Exception { 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(lastMaxSeqNo, translogUUID, lastTranslogGen)); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); } // Advance the global checkpoint to between [safeIndex, safeIndex + 1) safeIndex = randomIntBetween(safeIndex, commitList.size() - 1); @@ -114,6 +124,9 @@ public void testAcquireIndexCommit() throws Exception { 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++) { @@ -132,7 +145,7 @@ public void testAcquireIndexCommit() throws Exception { snapshottingCommits.remove(snapshot); final long pendingSnapshots = snapshottingCommits.stream().filter(snapshot::equals).count(); final IndexCommit lastCommit = commitList.get(commitList.size() - 1); - final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); + safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commitList, globalCheckpoint.get()); assertThat(indexPolicy.releaseCommit(snapshot), equalTo(pendingSnapshots == 0 && snapshot.equals(lastCommit) == false && snapshot.equals(safeCommit) == false)); } @@ -143,6 +156,8 @@ public void testAcquireIndexCommit() throws Exception { 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)); @@ -154,25 +169,27 @@ public void testAcquireIndexCommit() throws Exception { 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, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); long legacyTranslogGen = randomNonNegativeLong(); IndexCommit legacyCommit = mockLegacyIndexCommit(translogUUID, legacyTranslogGen); - indexPolicy.onCommit(singletonList(legacyCommit)); - verify(legacyCommit, never()).delete(); - assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(legacyTranslogGen)); - assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(legacyTranslogGen)); + assertThat(CombinedDeletionPolicy.findSafeCommitPoint(singletonList(legacyCommit), globalCheckpoint.get()), + equalTo(legacyCommit)); long safeTranslogGen = randomLongBetween(legacyTranslogGen, Long.MAX_VALUE); long maxSeqNo = randomLongBetween(1, Long.MAX_VALUE); - final IndexCommit freshCommit = mockIndexCommit(maxSeqNo, translogUUID, safeTranslogGen); + final IndexCommit freshCommit = mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, translogUUID, safeTranslogGen); globalCheckpoint.set(randomLongBetween(0, maxSeqNo - 1)); indexPolicy.onCommit(Arrays.asList(legacyCommit, freshCommit)); @@ -189,25 +206,32 @@ public void testLegacyIndex() throws Exception { 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, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, globalCheckpoint::get); final int invalidCommits = between(1, 10); final List commitList = new ArrayList<>(); for (int i = 0; i < invalidCommits; i++) { - commitList.add(mockIndexCommit(randomNonNegativeLong(), UUID.randomUUID(), randomNonNegativeLong())); + long maxSeqNo = randomNonNegativeLong(); + commitList.add(mockIndexCommit(randomLongBetween(-1, maxSeqNo), maxSeqNo, 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); - commitList.add(mockIndexCommit(randomNonNegativeLong(), expectedTranslogUUID, lastTranslogGen)); + lastMaxSeqNo += between(1, 1000); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, expectedTranslogUUID, lastTranslogGen)); } // We should never keep invalid commits regardless of the value of the global checkpoint. @@ -215,21 +239,26 @@ public void testDeleteInvalidCommits() throws Exception { 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, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(logger, translogPolicy, softDeletesPolicy, 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); - commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); + lastCheckpoint = randomLongBetween(lastCheckpoint, lastMaxSeqNo); + commitList.add(mockIndexCommit(lastCheckpoint, lastMaxSeqNo, translogUUID, lastTranslogGen)); } IndexCommit safeCommit = randomFrom(commitList); globalCheckpoint.set(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO))); @@ -256,8 +285,9 @@ public void testCheckUnreferencedCommits() throws Exception { } } - IndexCommit mockIndexCommit(long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { + IndexCommit mockIndexCommit(long localCheckpoint, 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)); @@ -278,6 +308,10 @@ void resetDeletion(IndexCommit commit) { }).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()); @@ -287,4 +321,5 @@ IndexCommit mockLegacyIndexCommit(UUID translogUUID, long translogGen) throws IO 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 76e05ba1e0b58..8da58ffc3a378 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.engine; +import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; import java.nio.charset.Charset; @@ -77,10 +78,12 @@ 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; @@ -114,6 +117,7 @@ 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; @@ -133,6 +137,7 @@ 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; @@ -172,8 +177,10 @@ 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; @@ -247,8 +254,13 @@ public void testVersionMapAfterAutoIDDocument() throws IOException { } 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(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { + InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null))) { List segments = engine.segments(false); assertThat(segments.isEmpty(), equalTo(true)); assertThat(engine.segmentsStats(false).getCount(), equalTo(0L)); @@ -1311,9 +1323,13 @@ public void testVersioningNewIndex() throws IOException { assertThat(indexResult.getVersion(), equalTo(1L)); } - public void testForceMerge() throws IOException { + 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(); try (Store store = createStore(); - Engine engine = createEngine(config(defaultSettings, store, createTempDir(), + Engine engine = createEngine(config(IndexSettingsModule.newIndexSettings(indexMetaData), 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++) { @@ -1354,6 +1370,165 @@ public void testForceMerge() throws IOException { } } + 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++) { @@ -1422,126 +1597,10 @@ public void testVersioningCreateExistsException() throws IOException { 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); - } - - 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)); - } - } + assertOpsOnReplica(ops, replicaEngine, true, logger); } public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, InterruptedException { @@ -1569,11 +1628,12 @@ public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, Interrup } // randomly interleave final AtomicLong seqNoGenerator = new AtomicLong(); - Function seqNoUpdater = operation -> { - final long newSeqNo = seqNoGenerator.getAndIncrement(); + BiFunction seqNoUpdater = (operation, newSeqNo) -> { if (operation instanceof Engine.Index) { Engine.Index index = (Engine.Index) operation; - return new Engine.Index(index.uid(), index.parsedDoc(), newSeqNo, index.primaryTerm(), index.version(), + 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(), index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry()); } else { Engine.Delete delete = (Engine.Delete) operation; @@ -1586,12 +1646,12 @@ public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, Interrup Iterator iter2 = opsDoc2.iterator(); while (iter1.hasNext() && iter2.hasNext()) { final Engine.Operation next = randomBoolean() ? iter1.next() : iter2.next(); - allOps.add(seqNoUpdater.apply(next)); + allOps.add(seqNoUpdater.apply(next, seqNoGenerator.getAndIncrement())); } - iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); - iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o))); + iter1.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); + iter2.forEachRemaining(o -> allOps.add(seqNoUpdater.apply(o, seqNoGenerator.getAndIncrement()))); // insert some duplicates - allOps.addAll(randomSubsetOf(allOps)); + randomSubsetOf(allOps).forEach(op -> allOps.add(seqNoUpdater.apply(op, op.seqNo()))); shuffle(allOps, random()); concurrentlyApplyOps(allOps, engine); @@ -1623,42 +1683,6 @@ public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, Interrup 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); @@ -1869,7 +1893,7 @@ public void testVersioningPromotedReplica() throws IOException { final boolean deletedOnReplica = lastReplicaOp instanceof Engine.Delete; final long finalReplicaVersion = lastReplicaOp.version(); final long finalReplicaSeqNo = lastReplicaOp.seqNo(); - assertOpsOnReplica(replicaOps, replicaEngine, true); + assertOpsOnReplica(replicaOps, replicaEngine, true, logger); 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(); @@ -2674,14 +2698,16 @@ public void testSkipTranslogReplay() throws IOException { Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } + EngineConfig config = engine.config(); assertVisibleCount(engine, numDocs); engine.close(); - 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)); + 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)); + } } } @@ -2811,7 +2837,7 @@ public void testRecoverFromForeignTranslog() throws IOException { 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); + new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get, tombstoneDocSupplier()); try { InternalEngine internalEngine = new InternalEngine(brokenConfig); fail("translog belongs to a different engine"); @@ -2940,6 +2966,12 @@ private void maybeThrowFailure() throws IOException { } } + @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(); @@ -3140,10 +3172,10 @@ public void testDoubleDeliveryReplicaAppendingAndDeleteOnly() throws IOException } public void testDoubleDeliveryReplicaAppendingOnly() throws IOException { - final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), + final Supplier doc = () -> testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); - Engine.Index operation = appendOnlyReplica(doc, false, 1, randomIntBetween(0, 5)); - Engine.Index retry = appendOnlyReplica(doc, true, 1, randomIntBetween(0, 5)); + Engine.Index operation = appendOnlyReplica(doc.get(), false, 1, randomIntBetween(0, 5)); + Engine.Index retry = appendOnlyReplica(doc.get(), 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; @@ -3182,8 +3214,8 @@ public void testDoubleDeliveryReplicaAppendingOnly() throws IOException { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(1, topDocs.totalHits); } - operation = randomAppendOnly(doc, false, 1); - retry = randomAppendOnly(doc, true, 1); + operation = randomAppendOnly(doc.get(), false, 1); + retry = randomAppendOnly(doc.get(), true, 1); if (randomBoolean()) { Engine.IndexResult indexResult = engine.index(operation); assertNotNull(indexResult.getTranslogLocation()); @@ -3248,6 +3280,8 @@ public void testDoubleDeliveryReplica() throws IOException { 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 { @@ -3716,20 +3750,22 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio 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; - 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 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"); final BiFunction searcherFactory = engine::acquireSearcher; for (int i = 0; i < numberOfOperations; i++) { if (randomBoolean()) { final Engine.Index index = new Engine.Index( uid, - doc, + doc.get(), sequenceNumberSupplier.getAsLong(), 1, i, @@ -3805,7 +3841,9 @@ public void testNoOps() throws IOException { maxSeqNo, localCheckpoint); trimUnsafeCommits(engine.config()); - noOpEngine = new InternalEngine(engine.config(), supplier) { + EngineConfig noopEngineConfig = copy(engine.config(), new SoftDeletesRetentionMergePolicy(Lucene.SOFT_DELETES_FIELD, + () -> new MatchAllDocsQuery(), engine.config().getMergePolicy())); + noOpEngine = new InternalEngine(noopEngineConfig, supplier) { @Override protected long doGenerateSeqNoForOperation(Operation operation) { throw new UnsupportedOperationException(); @@ -3813,7 +3851,7 @@ protected long doGenerateSeqNoForOperation(Operation operation) { }; noOpEngine.recoverFromTranslog(Long.MAX_VALUE); final int gapsFilled = noOpEngine.fillSeqNoGaps(primaryTerm.get()); - final String reason = randomAlphaOfLength(16); + final String reason = "filling gaps"; 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)); @@ -3835,11 +3873,77 @@ protected long doGenerateSeqNoForOperation(Operation operation) { 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); @@ -4405,7 +4509,7 @@ public void testCleanUpCommitsWhenGlobalCheckpointAdvanced() throws Exception { globalCheckpoint.set(randomLongBetween(engine.getLocalCheckpoint(), Long.MAX_VALUE)); engine.syncTranslog(); assertThat(DirectoryReader.listCommits(store.directory()), contains(commits.get(commits.size() - 1))); - assertThat(engine.estimateTranslogOperationsFromMinSeq(0L), equalTo(0)); + assertThat(engine.getTranslog().totalOperations(), equalTo(0)); } } @@ -4768,6 +4872,128 @@ public void testTrimUnsafeCommits() throws Exception { } } + 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)); + } + } + 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 new file mode 100644 index 0000000000000..7de086a3be239 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -0,0 +1,289 @@ +/* + * 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.newLuceneChangesSnapshot("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.newLuceneChangesSnapshot("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.newLuceneChangesSnapshot("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.newLuceneChangesSnapshot("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.newLuceneChangesSnapshot("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 new file mode 100644 index 0000000000000..c46b47b87d06e --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java @@ -0,0 +1,161 @@ +/* + * 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 new file mode 100644 index 0000000000000..f359010038284 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java @@ -0,0 +1,75 @@ +/* + * 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 76ca6aa7ea8d4..5a46b9a889fd0 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java @@ -31,6 +31,7 @@ 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; @@ -311,15 +312,18 @@ DocumentMapper createDummyMapping(MapperService mapperService) throws Exception // creates an object mapper, which is about 100x harder than it should be.... ObjectMapper createObjectMapper(MapperService mapperService, String name) throws Exception { - ParseContext context = new ParseContext.InternalParseContext( - Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(), + 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, 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(), context.path()); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(context.indexSettings().getSettings(), 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 cb2ed785699c8..b11e4876f9eaf 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java @@ -34,6 +34,7 @@ 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; @@ -215,7 +216,10 @@ private String serialize(ToXContent mapper) throws Exception { } private Mapper parse(DocumentMapper mapper, DocumentMapperParser parser, XContentBuilder builder) throws Exception { - Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); + 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); 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 1d1e423afc1b6..9f3f3f64129ed 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -18,6 +18,7 @@ */ 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; @@ -41,6 +42,7 @@ 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; @@ -140,7 +142,9 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa } public void testInheritMaxValidAutoIDTimestampOnRecovery() throws Exception { - try (ReplicationGroup shards = createGroup(0)) { + //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)) { shards.startAll(); final IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON); indexRequest.onRetry(); // force an update of the timestamp @@ -338,6 +342,7 @@ public void testReplicaOperationWithConcurrentPrimaryPromotion() throws Exceptio * test document failures (failures after seq_no generation) are added as noop operation to the translog * for primary and replica shards */ + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/33217") public void testDocumentFailureReplication() throws Exception { final IOException indexException = new IOException("simulated indexing failure"); final IOException deleteException = new IOException("simulated deleting failure"); @@ -346,7 +351,13 @@ public void testDocumentFailureReplication() throws Exception { final AtomicBoolean throwAfterIndexedOneDoc = new AtomicBoolean(); // need one document to trigger delete in IW. @Override public long addDocument(Iterable doc) throws IOException { - if (throwAfterIndexedOneDoc.getAndSet(true)) { + boolean isTombstone = false; + for (IndexableField field : doc) { + if (SeqNoFieldMapper.TOMBSTONE_NAME.equals(field.name())) { + isTombstone = true; + } + } + if (isTombstone == false && throwAfterIndexedOneDoc.getAndSet(true)) { throw indexException; } else { return super.addDocument(doc); @@ -356,6 +367,10 @@ public long addDocument(Iterable doc) throws IOExcepti 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 @@ -501,8 +516,9 @@ public void testSeqNoCollision() throws Exception { recoverReplica(replica3, replica2, true); try (Translog.Snapshot snapshot = getTranslog(replica3).newSnapshot()) { assertThat(snapshot.totalOperations(), equalTo(initDocs + 1)); - assertThat(snapshot.next(), equalTo(op2)); - assertThat("Remaining of snapshot should contain init operations", snapshot, containsOperationsInAnyOrder(initOperations)); + final List expectedOps = new ArrayList<>(initOperations); + expectedOps.add(op2); + assertThat(snapshot, containsOperationsInAnyOrder(expectedOps)); 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 2d198c32ba74f..28122665e9bb6 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -98,7 +98,8 @@ public void testIndexingDuringFileRecovery() throws Exception { } public void testRecoveryOfDisconnectedReplica() throws Exception { - try (ReplicationGroup shards = createGroup(1)) { + Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build(); + try (ReplicationGroup shards = createGroup(1, settings)) { shards.startAll(); int docs = shards.indexDocs(randomInt(50)); shards.flush(); @@ -266,6 +267,7 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { 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(); @@ -275,7 +277,12 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { shards.syncGlobalCheckpoint(); assertThat(newPrimary.getLastSyncedGlobalCheckpoint(), equalTo(newPrimary.seqNoStats().getMaxSeqNo())); }); - newPrimary.flush(new FlushRequest()); + 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)); + } 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 2228e1b017fd4..50f95bf4d4738 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -22,6 +22,7 @@ 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; @@ -30,6 +31,7 @@ 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; @@ -89,8 +91,13 @@ 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; @@ -160,6 +167,7 @@ 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; @@ -237,7 +245,8 @@ public void testFailShard() throws Exception { assertNotNull(shardPath); // fail shard shard.failShard("test shard fail", new CorruptIndexException("", "")); - closeShards(shard); + shard.close("do not assert history", false); + shard.store().close(); // check state file still exists ShardStateMetaData shardStateMetaData = load(logger, shardPath.getShardStatePath()); assertEquals(shardStateMetaData, getShardStateMetadata(shard)); @@ -2394,7 +2403,8 @@ public void testRecoverFromLocalShard() throws IOException { public void testDocStats() throws IOException, InterruptedException { IndexShard indexShard = null; try { - indexShard = newStartedShard(); + indexShard = newStartedShard( + Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0).build()); 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++) { @@ -2424,7 +2434,16 @@ public void testDocStats() throws IOException, InterruptedException { 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); @@ -2962,6 +2981,7 @@ public void testSegmentMemoryTrackedInBreaker() throws Exception { 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(); @@ -2973,6 +2993,13 @@ public void testSegmentMemoryTrackedInBreaker() throws Exception { // 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()); @@ -3064,6 +3091,7 @@ public void testSegmentMemoryTrackedWithRandomSearchers() throws Exception { // Close remaining searchers IOUtils.close(searchers); + primary.refresh("test"); SegmentsStats ss = primary.segmentStats(randomBoolean()); CircuitBreaker breaker = primary.circuitBreakerService.getBreaker(CircuitBreaker.ACCOUNTING); @@ -3181,4 +3209,28 @@ public void testOnCloseStats() throws IOException { } + 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 ae2cc84e4870c..29b16ca28f4da 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -106,17 +106,22 @@ public void testSyncerSendsOffCorrectDocuments() throws Exception { .isPresent(), is(false)); } - - assertEquals(globalCheckPoint == numDocs - 1 ? 0 : numDocs, resyncTask.getTotalOperations()); if (syncNeeded && globalCheckPoint < numDocs - 1) { - long skippedOps = globalCheckPoint + 1; // everything up to global checkpoint included - assertEquals(skippedOps, resyncTask.getSkippedOperations()); - assertEquals(numDocs - skippedOps, resyncTask.getResyncedOperations()); + 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)); + } } else { - assertEquals(0, resyncTask.getSkippedOperations()); - assertEquals(0, resyncTask.getResyncedOperations()); + assertThat(resyncTask.getSkippedOperations(), equalTo(0)); + assertThat(resyncTask.getResyncedOperations(), equalTo(0)); + assertThat(resyncTask.getTotalOperations(), equalTo(0)); } - 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 774b272121a5c..b93f170174c3c 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -42,6 +42,7 @@ 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; @@ -130,7 +131,8 @@ public void onFailedEngine(String reason, @Nullable Exception e) { 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); + (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, () -> primaryTerm, + EngineTestCase.tombstoneDocSupplier()); 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 89a8813e3e07b..81afab4bb8f7e 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -67,6 +67,7 @@ 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; @@ -110,6 +111,11 @@ protected Collection> nodePlugins() { 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 4b1419375e6e5..b6f5a7b645169 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -25,6 +25,7 @@ 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; @@ -91,6 +92,7 @@ public void testGetStartingSeqNo() throws Exception { 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 f0644b029c3dc..0351111c305ce 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -411,12 +411,6 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE 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 5547a629ab2a1..45535e19672c7 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -34,6 +34,7 @@ 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; @@ -63,13 +64,13 @@ public void testTranslogHistoryTransferred() throws Exception { int docs = shards.indexDocs(10); getTranslog(shards.getPrimary()).rollGeneration(); shards.flush(); - if (randomBoolean()) { - docs += shards.indexDocs(10); - } + int moreDocs = shards.indexDocs(randomInt(10)); shards.addReplica(); shards.startAll(); final IndexShard replica = shards.getReplicas().get(0); - assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(docs)); + boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); + assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? moreDocs : docs + moreDocs)); + shards.assertAllEqual(docs + moreDocs); } } @@ -101,12 +102,12 @@ public void testRetentionPolicyChangeDuringRecovery() throws Exception { // rolling/flushing is async assertBusy(() -> { assertThat(replica.getLastSyncedGlobalCheckpoint(), equalTo(19L)); - assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(0)); + assertThat(getTranslog(replica).totalOperations(), equalTo(0)); }); } } - public void testRecoveryWithOutOfOrderDelete() throws Exception { + public void testRecoveryWithOutOfOrderDeleteWithTranslog() throws Exception { /* * The flow of this test: * - delete #1 @@ -118,7 +119,8 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { * - index #5 * - If flush and the translog retention disabled, delete #1 will be removed while index #0 is still retained and replayed. */ - try (ReplicationGroup shards = createGroup(1)) { + Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), 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); @@ -170,7 +172,63 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { shards.recoverReplica(newReplica); shards.assertAllEqual(3); - assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(translogOps)); + 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)); + } } } @@ -222,7 +280,8 @@ public void testDifferentHistoryUUIDDisablesOPsRecovery() throws Exception { shards.recoverReplica(newReplica); // file based recovery should be made assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty())); - assertThat(newReplica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); + boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); + assertThat(getTranslog(newReplica).totalOperations(), equalTo(softDeletesEnabled ? nonFlushedDocs : numDocs)); // history uuid was restored assertThat(newReplica.getHistoryUUID(), equalTo(historyUUID)); @@ -326,7 +385,8 @@ public void testShouldFlushAfterPeerRecovery() throws Exception { shards.recoverReplica(replica); // Make sure the flushing will eventually be completed (eg. `shouldPeriodicallyFlush` is false) assertBusy(() -> assertThat(getEngine(replica).shouldPeriodicallyFlush(), equalTo(false))); - assertThat(replica.estimateTranslogOperationsFromMinSeq(0), equalTo(numDocs)); + boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled(); + assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? 0 : 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 fa591411bba11..ce162b9600cf4 100644 --- a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -43,6 +43,7 @@ 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; @@ -50,6 +51,7 @@ 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; @@ -69,6 +71,7 @@ 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; @@ -115,6 +118,7 @@ public Settings indexSettings() { 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(); } @@ -1006,10 +1010,15 @@ private void assertCumulativeQueryCacheStats(IndicesStatsResponse response) { @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/32506") public void testFilterCacheStats() throws Exception { - assertAcked(prepareCreate("index").setSettings(Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build()).get()); - indexRandom(true, + Settings settings = Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build(); + assertAcked(prepareCreate("index").setSettings(settings).get()); + indexRandom(false, 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(); @@ -1040,6 +1049,13 @@ public void testFilterCacheStats() throws Exception { 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); @@ -1173,4 +1189,21 @@ public void testConcurrentIndexingAndStatsRequests() throws BrokenBarrierExcepti 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 23c56688e00b4..c25cad61e0740 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/server/src/test/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -27,6 +27,7 @@ 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; @@ -58,6 +59,11 @@ protected Collection> nodePlugins() { 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 1230d594b98a2..632a1ecbee1ae 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -122,6 +122,7 @@ 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; @@ -2048,7 +2049,9 @@ public void testSnapshotMoreThanOnce() throws ExecutionException, InterruptedExc .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES))); // only one shard - assertAcked(prepareCreate("test").setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1))); + final Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).build(); + assertAcked(prepareCreate("test").setSettings(indexSettings)); ensureGreen(); logger.info("--> indexing"); @@ -2094,7 +2097,13 @@ public void testSnapshotMoreThanOnce() throws ExecutionException, InterruptedExc SnapshotStatus snapshotStatus = client.admin().cluster().prepareSnapshotStatus("test-repo").setSnapshots("test-2").get().getSnapshots().get(0); List shards = snapshotStatus.getShards(); for (SnapshotIndexShardStatus status : shards) { - 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 + // 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)); + } } } } diff --git a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java index caf4f725fa453..588118db4aef3 100644 --- a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java +++ b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java @@ -26,6 +26,7 @@ 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; @@ -785,4 +786,26 @@ public void testGCDeletesZero() throws Exception { .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 b5ba5f18b395c..8c2af5c1d3c16 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,14 +19,18 @@ 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; @@ -34,32 +38,41 @@ 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; @@ -80,17 +93,30 @@ 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 { @@ -128,6 +154,20 @@ protected static void assertVisibleCount(Engine engine, int numDocs, boolean ref } } + 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 { @@ -142,13 +182,7 @@ public void setUp() throws Exception { } else { codecName = "default"; } - 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 + defaultSettings = IndexSettingsModule.newIndexSettings("test", indexSettings()); threadPool = new TestThreadPool(getClass().getName()); store = createStore(); storeReplica = createStore(); @@ -180,7 +214,7 @@ public EngineConfig copy(EngineConfig config, LongSupplier globalCheckpointSuppl 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()); + config.getCircuitBreakerService(), globalCheckpointSupplier, config.getPrimaryTermSupplier(), tombstoneDocSupplier()); } public EngineConfig copy(EngineConfig config, Analyzer analyzer) { @@ -189,7 +223,18 @@ public EngineConfig copy(EngineConfig config, Analyzer analyzer) { 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.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()); } @Override @@ -198,9 +243,11 @@ public void tearDown() throws Exception { 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, @@ -228,8 +275,18 @@ public static ParsedDocument createParsedDoc(String id, String routing) { 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(); @@ -239,11 +296,57 @@ protected static ParsedDocument testParsedDocument( document.add(seqID.seqNoDocValue); document.add(seqID.primaryTerm); BytesRef ref = source.toBytesRef(); - document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); + 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)); + } 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()); } @@ -461,7 +564,7 @@ public void onFailedEngine(String reason, @Nullable Exception e) { new NoneCircuitBreakerService(), globalCheckpointSupplier == null ? new ReplicationTracker(shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED, update -> {}) : - globalCheckpointSupplier, primaryTerm::get); + globalCheckpointSupplier, primaryTerm::get, tombstoneDocSupplier()); return config; } @@ -474,7 +577,7 @@ protected static BytesArray bytesArray(String string) { return new BytesArray(string.getBytes(Charset.defaultCharset())); } - protected Term newUid(String id) { + protected static Term newUid(String id) { return new Term("_id", Uid.encodeId(id)); } @@ -499,6 +602,279 @@ protected Engine.Index replicaIndexForDoc(ParsedDocument doc, long version, long 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.newLuceneChangesSnapshot("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 3f1f5daf5148c..f2afdff9c3a3a 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,6 +60,7 @@ 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; @@ -99,10 +100,14 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase protected final Index index = new Index("test", "uuid"); private final ShardId shardId = new ShardId(index, 0); - private final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); + protected final Map indexMapping = Collections.singletonMap("type", "{ \"type\": {} }"); protected ReplicationGroup createGroup(int replicas) throws IOException { - IndexMetaData metaData = buildIndexMetaData(replicas); + return createGroup(replicas, Settings.EMPTY); + } + + protected ReplicationGroup createGroup(int replicas, Settings settings) throws IOException { + IndexMetaData metaData = buildIndexMetaData(replicas, settings, indexMapping); return new ReplicationGroup(metaData); } @@ -111,9 +116,17 @@ protected IndexMetaData buildIndexMetaData(int replicas) throws IOException { } 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) @@ -146,7 +159,7 @@ protected class ReplicationGroup implements AutoCloseable, Iterable } }); - ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { + protected ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { final ShardRouting primaryRouting = this.createShardRouting("s0", true); primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting), () -> {}); replicas = new CopyOnWriteArrayList<>(); @@ -448,7 +461,7 @@ private void updateAllocationIDsOnPrimary() throws IOException { } } - abstract class ReplicationAction, + protected abstract class ReplicationAction, ReplicaRequest extends ReplicationRequest, Response extends ReplicationResponse> { private final Request request; @@ -456,7 +469,7 @@ abstract class ReplicationAction, private final ReplicationGroup replicationGroup; private final String opType; - ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { + protected ReplicationAction(Request request, ActionListener listener, ReplicationGroup group, String opType) { this.request = request; this.listener = listener; this.replicationGroup = group; @@ -582,11 +595,11 @@ public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, R } } - class PrimaryResult implements ReplicationOperation.PrimaryResult { + protected class PrimaryResult implements ReplicationOperation.PrimaryResult { final ReplicaRequest replicaRequest; final Response finalResponse; - PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) { + public 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 d2a84589669a6..2f4a3dfd6c12c 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,13 +18,8 @@ */ 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; @@ -57,10 +52,8 @@ 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; @@ -180,37 +173,63 @@ public Directory newDirectory() throws IOException { } /** - * 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 { - 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); + return newShard(primary, Settings.EMPTY, new InternalEngineFactory()); } /** - * 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 listeners an optional set of listeners to add to the shard + * @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 */ protected IndexShard newShard( final ShardRouting shardRouting, + final Settings settings, + final EngineFactory engineFactory, final IndexingOperationListener... listeners) throws IOException { assert shardRouting.initializing() : shardRouting; - 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(); + 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(); IndexMetaData.Builder metaData = IndexMetaData.builder(shardRouting.getIndexName()) - .settings(settings) + .settings(indexSettings) .primaryTerm(0, primaryTerm) .putMapping("_doc", "{ \"properties\": {} }"); - return newShard(shardRouting, metaData.build(), listeners); + return newShard(shardRouting, metaData.build(), engineFactory, listeners); } /** @@ -225,7 +244,7 @@ protected IndexShard newShard(ShardId shardId, boolean primary, IndexingOperatio ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, randomAlphaOfLength(5), primary, ShardRoutingState.INITIALIZING, primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting, listeners); + return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners); } /** @@ -265,9 +284,10 @@ protected IndexShard newShard(ShardId shardId, boolean primary, String nodeId, I * @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, IndexingOperationListener... listeners) + protected IndexShard newShard( + ShardRouting routing, IndexMetaData indexMetaData, EngineFactory engineFactory, IndexingOperationListener... listeners) throws IOException { - return newShard(routing, indexMetaData, null, new InternalEngineFactory(), () -> {}, listeners); + return newShard(routing, indexMetaData, null, engineFactory, () -> {}, listeners); } /** @@ -372,19 +392,39 @@ protected IndexShard reinitShard(IndexShard current, ShardRouting routing, Index } /** - * creates a new empyu shard and starts it. The shard will be either a replica or a primary. + * Creates a new empty shard and starts it. The shard will randomly be a replica or a primary. */ protected IndexShard newStartedShard() throws IOException { return newStartedShard(randomBoolean()); } /** - * creates a new empty shard and starts it. + * 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. * * @param primary controls whether the shard will be a primary or a replica. */ - protected IndexShard newStartedShard(boolean primary) throws IOException { - IndexShard shard = newShard(primary); + 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); if (primary) { recoverShardFromStore(shard); } else { @@ -401,6 +441,7 @@ protected void closeShards(Iterable shards) throws IOException { for (IndexShard shard : shards) { if (shard != null) { try { + assertConsistentHistoryBetweenTranslogAndLucene(shard); shard.close("test", false); } finally { IOUtils.close(shard.store()); @@ -582,22 +623,7 @@ private Store.MetadataSnapshot getMetadataSnapshotOrEmpty(IndexShard replica) th } protected Set getShardDocUIDs(final IndexShard shard) throws IOException { - 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; - } + return EngineTestCase.getDocIds(shard.getEngine(), true); } protected void assertDocCount(IndexShard shard, int docDount) throws IOException { @@ -610,6 +636,12 @@ protected void assertDocs(IndexShard shard, String... ids) throws IOException { 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, "{}"); @@ -653,11 +685,14 @@ protected void updateMappings(IndexShard shard, IndexMetaData indexMetadata) { } protected Engine.DeleteResult deleteDoc(IndexShard shard, String type, String id) throws IOException { + final Engine.DeleteResult result; if (shard.routingEntry().primary()) { - return shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); + result = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); + shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(), shard.getEngine().getLocalCheckpoint()); } else { - return shard.applyDeleteOperationOnReplica(shard.seqNoStats().getMaxSeqNo() + 1, 0L, type, id); + result = 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 1f51ad495e182..d692d9fd68426 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -725,6 +725,10 @@ public Settings indexSettings() { } // 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 9633f56dea941..19290f8cf118d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -41,6 +41,7 @@ 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; @@ -87,6 +88,14 @@ protected void startNode(long seed) throws Exception { .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 306f79e5e16ec..4c813372fae31 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -1163,6 +1163,26 @@ private void assertOpenTranslogReferences() throws Exception { }); } + /** + * 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)) {