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