From f2211c1e5a29dbfd73154dd11afabc9cb4c56d55 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 6 Mar 2017 21:07:12 -0800 Subject: [PATCH 01/14] initial version --- ... => PerThreadIDAndVersionSeqNoLookup.java} | 74 ++- .../common/lucene/uid/Versions.java | 231 --------- .../lucene/uid/VersionsAndSeqNoResolver.java | 183 ++++++++ .../index/engine/DeleteVersionValue.java | 14 +- .../elasticsearch/index/engine/Engine.java | 19 +- .../index/engine/InternalEngine.java | 440 ++++++++++-------- .../index/engine/LiveVersionMap.java | 12 +- .../VersionConflictEngineException.java | 4 + .../index/engine/VersionValue.java | 25 +- .../index/get/ShardGetService.java | 4 +- .../index/mapper/ParseContext.java | 15 +- .../index/mapper/ParsedDocument.java | 4 +- .../index/mapper/SeqNoFieldMapper.java | 25 +- .../index/termvectors/TermVectorsService.java | 4 +- .../common/lucene/uid/VersionLookupTests.java | 30 +- .../common/lucene/uid/VersionsTests.java | 56 +-- .../index/IndexingSlowLogTests.java | 2 +- .../index/engine/InternalEngineTests.java | 438 +++++++++++++---- .../index/engine/LiveVersionMapTests.java | 4 +- .../index/engine/ShadowEngineTests.java | 2 +- .../index/engine/VersionValueTests.java | 6 +- .../index/shard/IndexShardIT.java | 2 +- .../index/shard/IndexShardTests.java | 2 +- .../index/shard/RefreshListenersTests.java | 2 +- .../index/translog/TranslogTests.java | 2 +- .../recovery/RecoverySourceHandlerTests.java | 3 +- 26 files changed, 954 insertions(+), 649 deletions(-) rename core/src/main/java/org/elasticsearch/common/lucene/uid/{PerThreadIDAndVersionLookup.java => PerThreadIDAndVersionSeqNoLookup.java} (55%) create mode 100644 core/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java diff --git a/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionLookup.java b/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionSeqNoLookup.java similarity index 55% rename from core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionLookup.java rename to core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionSeqNoLookup.java index 6e4e3e1923d5c..20ea37dbcaa98 100644 --- a/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionLookup.java +++ b/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionSeqNoLookup.java @@ -29,7 +29,9 @@ import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.lucene.uid.Versions.DocIdAndVersion; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; @@ -43,7 +45,7 @@ * in more than one document! It will only return the first one it * finds. */ -final class PerThreadIDAndVersionLookup { +final class PerThreadIDAndVersionSeqNoLookup { // TODO: do we really need to store all this stuff? some if it might not speed up anything. // we keep it around for now, to reduce the amount of e.g. hash lookups by field and stuff @@ -51,49 +53,73 @@ final class PerThreadIDAndVersionLookup { private final TermsEnum termsEnum; /** _version data */ private final NumericDocValues versions; + /** _seq_no data */ + private final NumericDocValues seqNos; + /** _primary_term data */ + private final NumericDocValues primaryTerms; /** Reused for iteration (when the term exists) */ private PostingsEnum docsEnum; + private final Object readerKey; + /** * Initialize lookup for the provided segment */ - PerThreadIDAndVersionLookup(LeafReader reader) throws IOException { - TermsEnum termsEnum = null; - NumericDocValues versions = null; - + PerThreadIDAndVersionSeqNoLookup(LeafReader reader) throws IOException { Fields fields = reader.fields(); - if (fields != null) { - Terms terms = fields.terms(UidFieldMapper.NAME); - if (terms != null) { - termsEnum = terms.iterator(); - assert termsEnum != null; - versions = reader.getNumericDocValues(VersionFieldMapper.NAME); - assert versions != null; - } - } - - this.versions = versions; - this.termsEnum = termsEnum; + Terms terms = fields.terms(UidFieldMapper.NAME); + termsEnum = terms.iterator(); + assert termsEnum != null; + versions = reader.getNumericDocValues(VersionFieldMapper.NAME); + assert versions != null; + seqNos = reader.getNumericDocValues(SeqNoFieldMapper.NAME); + primaryTerms = reader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME); + readerKey = reader.getCoreCacheKey(); } /** Return null if id is not found. */ - public DocIdAndVersion lookup(BytesRef id, Bits liveDocs, LeafReaderContext context) throws IOException { + public DocIdAndVersion lookupVersion(BytesRef id, Bits liveDocs, LeafReaderContext context) throws IOException { + assert context.reader().getCoreCacheKey().equals(readerKey); + int docID = getDocID(id, liveDocs); + + if (docID != DocIdSetIterator.NO_MORE_DOCS) { + return new DocIdAndVersion(docID, versions.get(docID), context); + } else { + return null; + } + } + + /** returns the internal lucene doc id for the given id bytes. {@link DocIdSetIterator#NO_MORE_DOCS} is returned if not found */ + private int getDocID(BytesRef id, Bits liveDocs) throws IOException { 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); - int docID = DocIdSetIterator.NO_MORE_DOCS; for (int d = docsEnum.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d = docsEnum.nextDoc()) { if (liveDocs != null && liveDocs.get(d) == false) { continue; } docID = d; } + return docID; + } else { + return DocIdSetIterator.NO_MORE_DOCS; + } + } - if (docID != DocIdSetIterator.NO_MORE_DOCS) { - return new DocIdAndVersion(docID, versions.get(docID), context); - } + /** Return null if id is not found. */ + public DocIdAndSeqNo lookupSequenceNo(BytesRef id, Bits liveDocs, LeafReaderContext context) throws IOException { + assert context.reader().getCoreCacheKey().equals(readerKey); + int docID = seqNos == null ? DocIdSetIterator.NO_MORE_DOCS : getDocID(id, liveDocs); + if (docID != DocIdSetIterator.NO_MORE_DOCS) { + return new DocIdAndSeqNo(docID, seqNos.get(docID), context); + } else { + return null; } + } - return null; + /** returns 0 if the primary term is not found */ + public long lookUpPrimaryTerm(int docID) throws IOException { + return primaryTerms == null ? 0 : primaryTerms.get(docID); } } diff --git a/core/src/main/java/org/elasticsearch/common/lucene/uid/Versions.java b/core/src/main/java/org/elasticsearch/common/lucene/uid/Versions.java index 6c5ffed0938e6..8cc612421551c 100644 --- a/core/src/main/java/org/elasticsearch/common/lucene/uid/Versions.java +++ b/core/src/main/java/org/elasticsearch/common/lucene/uid/Versions.java @@ -19,31 +19,6 @@ package org.elasticsearch.common.lucene.uid; -import org.apache.lucene.index.Fields; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReader.CoreClosedListener; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.PostingsEnum; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.Terms; -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.apache.lucene.util.CloseableThreadLocal; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.UidFieldMapper; -import org.elasticsearch.index.seqno.SequenceNumbersService; - -import java.io.IOException; -import java.util.List; -import java.util.concurrent.ConcurrentMap; - -/** Utility class to resolve the Lucene doc ID and version for a given uid. */ public class Versions { /** used to indicate the write operation should succeed regardless of current version **/ @@ -59,210 +34,4 @@ public class Versions { * i.e., not found in the index and/or found as deleted (with version) in the version map */ public static final long MATCH_DELETED = -4L; - - // TODO: is there somewhere else we can store these? - static final ConcurrentMap> lookupStates = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(); - - // Evict this reader from lookupStates once it's closed: - private static final CoreClosedListener removeLookupState = new CoreClosedListener() { - @Override - public void onClose(Object key) { - CloseableThreadLocal ctl = lookupStates.remove(key); - if (ctl != null) { - ctl.close(); - } - } - }; - - private static PerThreadIDAndVersionLookup getLookupState(LeafReader reader) throws IOException { - Object key = reader.getCoreCacheKey(); - CloseableThreadLocal ctl = lookupStates.get(key); - if (ctl == null) { - // First time we are seeing this reader's core; make a - // new CTL: - ctl = new CloseableThreadLocal<>(); - CloseableThreadLocal other = lookupStates.putIfAbsent(key, ctl); - if (other == null) { - // Our CTL won, we must remove it when the - // core is closed: - reader.addCoreClosedListener(removeLookupState); - } else { - // Another thread beat us to it: just use - // their CTL: - ctl = other; - } - } - - PerThreadIDAndVersionLookup lookupState = ctl.get(); - if (lookupState == null) { - lookupState = new PerThreadIDAndVersionLookup(reader); - ctl.set(lookupState); - } - - return lookupState; - } - - private Versions() { - } - - /** Wraps an {@link LeafReaderContext}, a doc ID relative to the context doc base and a version. */ - public static class DocIdAndVersion { - public final int docId; - public final long version; - public final LeafReaderContext context; - - public DocIdAndVersion(int docId, long version, LeafReaderContext context) { - this.docId = docId; - this.version = version; - this.context = context; - } - } - - /** - * Load the internal doc ID and version for the uid from the reader, returning
    - *
  • null if the uid wasn't found, - *
  • a doc ID and a version otherwise - *
- */ - public static DocIdAndVersion loadDocIdAndVersion(IndexReader reader, Term term) throws IOException { - assert term.field().equals(UidFieldMapper.NAME); - List leaves = reader.leaves(); - if (leaves.isEmpty()) { - return null; - } - // iterate backwards to optimize for the frequently updated documents - // which are likely to be in the last segments - for (int i = leaves.size() - 1; i >= 0; i--) { - LeafReaderContext context = leaves.get(i); - LeafReader leaf = context.reader(); - PerThreadIDAndVersionLookup lookup = getLookupState(leaf); - DocIdAndVersion result = lookup.lookup(term.bytes(), leaf.getLiveDocs(), context); - if (result != null) { - return result; - } - } - return null; - } - - /** - * Load the version for the uid from the reader, returning
    - *
  • {@link #NOT_FOUND} if no matching doc exists, - *
  • the version associated with the provided uid otherwise - *
- */ - public static long loadVersion(IndexReader reader, Term term) throws IOException { - final DocIdAndVersion docIdAndVersion = loadDocIdAndVersion(reader, term); - return docIdAndVersion == null ? NOT_FOUND : docIdAndVersion.version; - } - - - /** - * Returns the sequence number for the given uid term, returning - * {@code SequenceNumbersService.UNASSIGNED_SEQ_NO} if none is found. - */ - public static long loadSeqNo(IndexReader reader, Term term) throws IOException { - assert term.field().equals(UidFieldMapper.NAME) : "can only load _seq_no by uid"; - List leaves = reader.leaves(); - if (leaves.isEmpty()) { - return SequenceNumbersService.UNASSIGNED_SEQ_NO; - } - - // iterate backwards to optimize for the frequently updated documents - // which are likely to be in the last segments - for (int i = leaves.size() - 1; i >= 0; i--) { - LeafReader leaf = leaves.get(i).reader(); - Bits liveDocs = leaf.getLiveDocs(); - - TermsEnum termsEnum = null; - SortedNumericDocValues dvField = null; - PostingsEnum docsEnum = null; - - final Fields fields = leaf.fields(); - if (fields != null) { - Terms terms = fields.terms(UidFieldMapper.NAME); - if (terms != null) { - termsEnum = terms.iterator(); - assert termsEnum != null; - dvField = leaf.getSortedNumericDocValues(SeqNoFieldMapper.NAME); - assert dvField != null; - - final BytesRef id = term.bytes(); - if (termsEnum.seekExact(id)) { - // 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); - int docID = DocIdSetIterator.NO_MORE_DOCS; - for (int d = docsEnum.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d = docsEnum.nextDoc()) { - if (liveDocs != null && liveDocs.get(d) == false) { - continue; - } - docID = d; - } - - if (docID != DocIdSetIterator.NO_MORE_DOCS) { - dvField.setDocument(docID); - assert dvField.count() == 1 : "expected only a single value for _seq_no but got " + - dvField.count(); - return dvField.valueAt(0); - } - } - } - } - - } - return SequenceNumbersService.UNASSIGNED_SEQ_NO; - } - - /** - * Returns the primary term for the given uid term, returning {@code 0} if none is found. - */ - public static long loadPrimaryTerm(IndexReader reader, Term term) throws IOException { - assert term.field().equals(UidFieldMapper.NAME) : "can only load _primary_term by uid"; - List leaves = reader.leaves(); - if (leaves.isEmpty()) { - return 0; - } - - // iterate backwards to optimize for the frequently updated documents - // which are likely to be in the last segments - for (int i = leaves.size() - 1; i >= 0; i--) { - LeafReader leaf = leaves.get(i).reader(); - Bits liveDocs = leaf.getLiveDocs(); - - TermsEnum termsEnum = null; - NumericDocValues dvField = null; - PostingsEnum docsEnum = null; - - final Fields fields = leaf.fields(); - if (fields != null) { - Terms terms = fields.terms(UidFieldMapper.NAME); - if (terms != null) { - termsEnum = terms.iterator(); - assert termsEnum != null; - dvField = leaf.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME); - assert dvField != null; - - final BytesRef id = term.bytes(); - if (termsEnum.seekExact(id)) { - // 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); - int docID = DocIdSetIterator.NO_MORE_DOCS; - for (int d = docsEnum.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d = docsEnum.nextDoc()) { - if (liveDocs != null && liveDocs.get(d) == false) { - continue; - } - docID = d; - } - - if (docID != DocIdSetIterator.NO_MORE_DOCS) { - return dvField.get(docID); - } - } - } - } - - } - return 0; - } } diff --git a/core/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java b/core/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java new file mode 100644 index 0000000000000..a402f9817385a --- /dev/null +++ b/core/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java @@ -0,0 +1,183 @@ +/* + * 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.common.lucene.uid; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReader.CoreClosedListener; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.Term; +import org.apache.lucene.util.CloseableThreadLocal; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.index.mapper.UidFieldMapper; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ConcurrentMap; + +import static org.elasticsearch.common.lucene.uid.Versions.NOT_FOUND; + +/** Utility class to resolve the Lucene doc ID, version, seqNo and primaryTerms for a given uid. */ +public class VersionsAndSeqNoResolver { + + static final ConcurrentMap> lookupStates = + ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(); + + // Evict this reader from lookupStates once it's closed: + private static final CoreClosedListener removeLookupState = key -> { + CloseableThreadLocal ctl = lookupStates.remove(key); + if (ctl != null) { + ctl.close(); + } + }; + + private static PerThreadIDAndVersionSeqNoLookup getLookupState(LeafReader reader) throws IOException { + Object key = reader.getCoreCacheKey(); + CloseableThreadLocal ctl = lookupStates.get(key); + if (ctl == null) { + // First time we are seeing this reader's core; make a + // new CTL: + ctl = new CloseableThreadLocal<>(); + CloseableThreadLocal other = lookupStates.putIfAbsent(key, ctl); + if (other == null) { + // Our CTL won, we must remove it when the + // core is closed: + reader.addCoreClosedListener(removeLookupState); + } else { + // Another thread beat us to it: just use + // their CTL: + ctl = other; + } + } + + PerThreadIDAndVersionSeqNoLookup lookupState = ctl.get(); + if (lookupState == null) { + lookupState = new PerThreadIDAndVersionSeqNoLookup(reader); + ctl.set(lookupState); + } + + return lookupState; + } + + private VersionsAndSeqNoResolver() { + } + + /** Wraps an {@link LeafReaderContext}, a doc ID relative to the context doc base and a version. */ + public static class DocIdAndVersion { + public final int docId; + public final long version; + public final LeafReaderContext context; + + public DocIdAndVersion(int docId, long version, LeafReaderContext context) { + this.docId = docId; + this.version = version; + this.context = context; + } + } + + /** Wraps an {@link LeafReaderContext}, a doc ID relative to the context doc base and a seqNo. */ + public static class DocIdAndSeqNo { + public final int docId; + public final long seqNo; + public final LeafReaderContext context; + + public DocIdAndSeqNo(int docId, long seqNo, LeafReaderContext context) { + this.docId = docId; + this.seqNo = seqNo; + this.context = context; + } + } + + + /** + * Load the internal doc ID and version for the uid from the reader, returning
    + *
  • null if the uid wasn't found, + *
  • a doc ID and a version otherwise + *
+ */ + public static DocIdAndVersion loadDocIdAndVersion(IndexReader reader, Term term) throws IOException { + assert term.field().equals(UidFieldMapper.NAME); + List leaves = reader.leaves(); + if (leaves.isEmpty()) { + return null; + } + // iterate backwards to optimize for the frequently updated documents + // which are likely to be in the last segments + for (int i = leaves.size() - 1; i >= 0; i--) { + LeafReaderContext context = leaves.get(i); + LeafReader leaf = context.reader(); + PerThreadIDAndVersionSeqNoLookup lookup = getLookupState(leaf); + DocIdAndVersion result = lookup.lookupVersion(term.bytes(), leaf.getLiveDocs(), context); + if (result != null) { + return result; + } + } + return null; + } + + /** + * Load the internal doc ID and sequence number for the uid from the reader, returning
    + *
  • null if the uid wasn't found, + *
  • a doc ID and the associated seqNo otherwise + *
+ */ + public static DocIdAndSeqNo loadDocIdAndSeqNo(IndexReader reader, Term term) throws IOException { + assert term.field().equals(UidFieldMapper.NAME); + List leaves = reader.leaves(); + if (leaves.isEmpty()) { + return null; + } + // iterate backwards to optimize for the frequently updated documents + // which are likely to be in the last segments + for (int i = leaves.size() - 1; i >= 0; i--) { + LeafReaderContext context = leaves.get(i); + LeafReader leaf = context.reader(); + PerThreadIDAndVersionSeqNoLookup lookup = getLookupState(leaf); + DocIdAndSeqNo result = lookup.lookupSequenceNo(term.bytes(), leaf.getLiveDocs(), context); + if (result != null) { + return result; + } + } + return null; + } + + /** + * Load the primaryTerm associated with the given {@link DocIdAndSeqNo} + */ + public static long loadPrimaryTerm(DocIdAndSeqNo docIdAndSeqNo) throws IOException { + LeafReader leaf = docIdAndSeqNo.context.reader(); + PerThreadIDAndVersionSeqNoLookup lookup = getLookupState(leaf); + long result = lookup.lookUpPrimaryTerm(docIdAndSeqNo.docId); + assert result > 0 : "should always resolve a primary term for a resolved sequence number. primary_term [" + result + "]" + + " docId [" + docIdAndSeqNo.docId + "] seqNo [" + docIdAndSeqNo.seqNo + "]"; + return result; + } + + /** + * Load the version for the uid from the reader, returning
    + *
  • {@link Versions#NOT_FOUND} if no matching doc exists, + *
  • the version associated with the provided uid otherwise + *
+ */ + public static long loadVersion(IndexReader reader, Term term) throws IOException { + final DocIdAndVersion docIdAndVersion = loadDocIdAndVersion(reader, term); + return docIdAndVersion == null ? NOT_FOUND : docIdAndVersion.version; + } +} diff --git a/core/src/main/java/org/elasticsearch/index/engine/DeleteVersionValue.java b/core/src/main/java/org/elasticsearch/index/engine/DeleteVersionValue.java index dbfb2416baed6..9ba084a1a9f69 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/DeleteVersionValue.java +++ b/core/src/main/java/org/elasticsearch/index/engine/DeleteVersionValue.java @@ -29,18 +29,18 @@ class DeleteVersionValue extends VersionValue { private final long time; - DeleteVersionValue(long version, long time) { - super(version); + DeleteVersionValue(long version,long seqNo, long term, long time) { + super(version, seqNo, term); this.time = time; } @Override - public long time() { + public long getTime() { return this.time; } @Override - public boolean delete() { + public boolean isDelete() { return true; } @@ -52,8 +52,10 @@ public long ramBytesUsed() { @Override public String toString() { return "DeleteVersionValue{" + - "version=" + version() + ", " + - "time=" + time + + "version=" + getVersion() + + ", seqNo=" + getSeqNo() + + ", term=" + getTerm() + + ",time=" + time + '}'; } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index f6b452502a5cc..2f4d6243454ff 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -55,6 +55,8 @@ import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.Versions; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; @@ -460,9 +462,9 @@ public enum SyncedFlushResult { protected final GetResult getFromSearcher(Get get, Function searcherFactory) throws EngineException { final Searcher searcher = searcherFactory.apply("get"); - final Versions.DocIdAndVersion docIdAndVersion; + final DocIdAndVersion docIdAndVersion; try { - docIdAndVersion = Versions.loadDocIdAndVersion(searcher.reader(), get.uid()); + docIdAndVersion = VersionsAndSeqNoResolver.loadDocIdAndVersion(searcher.reader(), get.uid()); } catch (Exception e) { Releasables.closeWhileHandlingException(searcher); //TODO: A better exception goes here @@ -1037,9 +1039,10 @@ public Index(Term uid, ParsedDocument doc) { } // TEST ONLY Index(Term uid, ParsedDocument doc, long version) { - this(uid, doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, version, VersionType.INTERNAL, + // use a primary term of 2 to allow tests to reduce it to a valid >0 term + this(uid, doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, version, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime(), -1, false); - } + } // TEST ONLY public ParsedDocument parsedDoc() { return this.doc; @@ -1227,12 +1230,12 @@ public Get versionType(VersionType versionType) { public static class GetResult implements Releasable { private final boolean exists; private final long version; - private final Versions.DocIdAndVersion docIdAndVersion; + private final DocIdAndVersion docIdAndVersion; private final Searcher searcher; public static final GetResult NOT_EXISTS = new GetResult(false, Versions.NOT_FOUND, null, null); - private GetResult(boolean exists, long version, Versions.DocIdAndVersion docIdAndVersion, Searcher searcher) { + private GetResult(boolean exists, long version, DocIdAndVersion docIdAndVersion, Searcher searcher) { this.exists = exists; this.version = version; this.docIdAndVersion = docIdAndVersion; @@ -1242,7 +1245,7 @@ private GetResult(boolean exists, long version, Versions.DocIdAndVersion docIdAn /** * Build a non-realtime get result from the searcher. */ - public GetResult(Searcher searcher, Versions.DocIdAndVersion docIdAndVersion) { + public GetResult(Searcher searcher, DocIdAndVersion docIdAndVersion) { this(true, docIdAndVersion.version, docIdAndVersion, searcher); } @@ -1258,7 +1261,7 @@ public Searcher searcher() { return this.searcher; } - public Versions.DocIdAndVersion docIdAndVersion() { + public DocIdAndVersion docIdAndVersion() { return docIdAndVersion; } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 0fa6855ce0867..c09124e545024 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -51,6 +51,8 @@ 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; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -385,13 +387,13 @@ public GetResult get(Get get, Function searcherFactory) throws if (get.realtime()) { VersionValue versionValue = versionMap.getUnderLock(get.uid()); if (versionValue != null) { - if (versionValue.delete()) { + if (versionValue.isDelete()) { return GetResult.NOT_EXISTS; } - if (get.versionType().isVersionConflictForReads(versionValue.version(), get.version())) { + if (get.versionType().isVersionConflictForReads(versionValue.getVersion(), get.version())) { Uid uid = Uid.createUid(get.uid().text()); throw new VersionConflictEngineException(shardId, uid.type(), uid.id(), - get.versionType().explainConflictForReads(versionValue.version(), get.version())); + get.versionType().explainConflictForReads(versionValue.getVersion(), get.version())); } refresh("realtime_get"); } @@ -402,54 +404,72 @@ public GetResult get(Get get, Function searcherFactory) throws } } - /** - * Checks for version conflicts. If a non-critical version conflict exists true is returned. In the case of a critical - * version conflict (if operation origin is primary) a {@link VersionConflictEngineException} is thrown. - * - * @param op the operation - * @param currentVersion the current version - * @param expectedVersion the expected version - * @param deleted {@code true} if the current version is not found or represents a delete - * @return true iff a non-critical version conflict (origin recovery or replica) is found otherwise false - * @throws VersionConflictEngineException if a critical version conflict was found where the operation origin is primary - * @throws IllegalArgumentException if an unsupported version type is used. - */ - private boolean checkVersionConflict(final Operation op, final long currentVersion, final long expectedVersion, final boolean deleted) { - if (op.versionType() == VersionType.FORCE) { - if (engineConfig.getIndexSettings().getIndexVersionCreated().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { - // If index was created in 5.0 or later, 'force' is not allowed at all - throw new IllegalArgumentException("version type [FORCE] may not be used for indices created after 6.0"); - } else if (op.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { - // For earlier indices, 'force' is only allowed for translog recovery - throw new IllegalArgumentException("version type [FORCE] may not be used for non-translog operations"); + enum LuceneOpStatus { + NEWER_OR_EQUAL, + STALE, + NOT_FOUND + } + + private LuceneOpStatus checkLuceneOpStatusBasedOnSeq(final Operation op) throws IOException { + assert op.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO; + final LuceneOpStatus status; + final VersionValue versionValue = versionMap.getUnderLock(op.uid()); + assert incrementVersionLookup(); + if (versionValue != null) { + if (op.seqNo() > versionValue.getSeqNo() || + (op.seqNo() == versionValue.getSeqNo() && op.primaryTerm() > versionValue.getTerm())) + status = LuceneOpStatus.STALE; + else { + status = LuceneOpStatus.NEWER_OR_EQUAL; + } + } else { + // load from index + assert incrementIndexVersionLookup(); + try (Searcher searcher = acquireSearcher("load_seq_no")) { + DocIdAndSeqNo docAndSeqNo = VersionsAndSeqNoResolver.loadDocIdAndSeqNo(searcher.reader(), op.uid()); + if (docAndSeqNo == null) { + status = LuceneOpStatus.NOT_FOUND; + } else if (op.seqNo() > docAndSeqNo.seqNo) { + status = LuceneOpStatus.STALE; + } else if (op.seqNo() == docAndSeqNo.seqNo) { + // load term to tie break + final long existingTerm = VersionsAndSeqNoResolver.loadPrimaryTerm(docAndSeqNo); + if (op.primaryTerm() > existingTerm) { + status = LuceneOpStatus.STALE; + } else { + status = LuceneOpStatus.NEWER_OR_EQUAL; + } + } else { + status = LuceneOpStatus.NEWER_OR_EQUAL; + } } } + return status; + } - if (op.versionType().isVersionConflictForWrites(currentVersion, expectedVersion, deleted)) { - if (op.origin() == Operation.Origin.PRIMARY) { - // fatal version conflict - throw new VersionConflictEngineException( - shardId, - op.type(), - op.id(), - op.versionType().explainConflictForWrites(currentVersion, expectedVersion, deleted)); - - } else { - /* Version conflicts during recovery and on replicas are normal due to asynchronous execution; as such, we should return a - * successful result.*/ - return true; - } + private LuceneOpStatus checkLuceneOpStatusBasedOnVersions(final Operation op) throws IOException { + assert op.seqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO; + assert op.version() >= 0; + final LuceneOpStatus status; + final long currentVersion; + final VersionValue versionValue = versionMap.getUnderLock(op.uid()); + if (versionValue != null) { + currentVersion = versionValue.getVersion(); } else { - return false; + // load from index + currentVersion = loadCurrentVersionFromIndex(op.uid()); + } + return op.version() > currentVersion ? LuceneOpStatus.STALE : LuceneOpStatus.NEWER_OR_EQUAL; } + private long checkDeletedAndGCed(VersionValue versionValue) { long currentVersion; - if (engineConfig.isEnableGcDeletes() && versionValue.delete() && (engineConfig.getThreadPool().relativeTimeInMillis() - versionValue.time()) > getGcDeletesInMillis()) { + if (engineConfig.isEnableGcDeletes() && versionValue.isDelete() && (engineConfig.getThreadPool().relativeTimeInMillis() - versionValue.getTime()) > getGcDeletesInMillis()) { currentVersion = Versions.NOT_FOUND; // deleted, and GC } else { - currentVersion = versionValue.version(); + currentVersion = versionValue.getVersion(); } return currentVersion; } @@ -492,7 +512,7 @@ private boolean assertVersionType(final Engine.Operation operation) { return true; } - private boolean assertSequenceNumber(final Engine.Operation.Origin origin, final long seqNo) { + private boolean assertIncomingSequenceNumber(final Engine.Operation.Origin origin, final long seqNo) { if (engineConfig.getIndexSettings().getIndexVersionCreated().before(Version.V_6_0_0_alpha1_UNRELEASED) && origin == Operation.Origin.LOCAL_TRANSLOG_RECOVERY) { // legacy support assert seqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO : "old op recovering but it already has a seq no.;" + @@ -507,20 +527,29 @@ private boolean assertSequenceNumber(final Engine.Operation.Origin origin, final return true; } + private boolean assertSequenceNumberBeforeIndexing(final Engine.Operation.Origin origin, final long seqNo) { + if (engineConfig.getIndexSettings().getIndexVersionCreated().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED) || + origin == Operation.Origin.PRIMARY) { + // sequence number should be set when operation origin is primary or when all shards are on new nodes + assert seqNo >= 0 : "ops should have an assigned seq no.; origin: " + origin; + } + return true; + } + + @Override public IndexResult index(Index index) throws IOException { final boolean doThrottle = index.origin().isRecovery() == false; try (ReleasableLock releasableLock = readLock.acquire()) { ensureOpen(); - assert assertSequenceNumber(index.origin(), index.seqNo()); + assert assertIncomingSequenceNumber(index.origin(), index.seqNo()); assert assertVersionType(index); - final Translog.Location location; - long seqNo = index.seqNo(); try (Releasable ignored = acquireLock(index.uid()); Releasable indexThrottle = doThrottle ? () -> {} : throttle.acquireThrottle()) { lastWriteNanos = index.startTime(); - /* if we have an autoGeneratedID that comes into the engine we can potentially optimize - * and just use addDocument instead of updateDocument and skip the entire version and index lookup across the board. + /* A NOTE ABOUT APPEND ONLY OPTIMIZATIONS: + * if we have an autoGeneratedID that comes into the engine we can potentially optimize + * and just use addDocument instead of updateDocument and skip the entire version and index lookupVersion across the board. * Yet, we have to deal with multiple document delivery, for this we use a property of the document that is added * to detect if it has potentially been added before. We use the documents timestamp for this since it's something * that: @@ -543,62 +572,102 @@ public IndexResult index(Index index) throws IOException { * if A arrives on the shard first we use addDocument since maxUnsafeAutoIdTimestamp is < 10. A` will then just be skipped or calls * updateDocument. */ - long currentVersion; - final boolean deleted; - // if anything is fishy here ie. there is a retry we go and force updateDocument below so we are updating the document in the - // lucene index without checking the version map but we still do the version check - final boolean forceUpdateDocument; + final boolean currentNotFoundOrDeleted; + final boolean useLuceneUpdateDocument; final boolean canOptimizeAddDocument = canOptimizeAddDocument(index); - if (canOptimizeAddDocument) { - forceUpdateDocument = isForceUpdateDocument(index); - currentVersion = Versions.NOT_FOUND; - deleted = true; + final long seqNoForIndexing; + final long versionForIndexing; + final boolean addToLucene; + Optional earlyResultOnPreFlightError = Optional.empty(); + + if (index.origin() == Operation.Origin.PRIMARY) { + // resolve an external operation into an internal one which is safe to replay + if (canOptimizeAddDocument) { + useLuceneUpdateDocument = mayHaveBeenIndexedBefore(index); + versionForIndexing = 1L; + currentNotFoundOrDeleted = true; + seqNoForIndexing = seqNoService().generateSeqNo(); + addToLucene = true; + } else { + // resolves incoming version + final VersionValue versionValue = versionMap.getUnderLock(index.uid()); + final long currentVersion; + assert incrementVersionLookup(); + if (versionValue == null) { + currentVersion = loadCurrentVersionFromIndex(index.uid()); + currentNotFoundOrDeleted = currentVersion == Versions.NOT_FOUND; + useLuceneUpdateDocument = currentVersion != Versions.NOT_FOUND; + } else { + currentVersion = checkDeletedAndGCed(versionValue); + currentNotFoundOrDeleted = versionValue.isDelete(); + // nocommit: previously we based this on whether version is not found. That can mean however + // that the last op was deleted, but was also GCed. So presumably it is safe + // to sue use add after a delete. + useLuceneUpdateDocument = versionValue.isDelete() == false; // last operation is not delete + } + if (index.versionType().isVersionConflictForWrites(currentVersion, index.version(), currentNotFoundOrDeleted)) { + earlyResultOnPreFlightError = Optional.of(new IndexResult( + new VersionConflictEngineException(shardId, index, currentVersion, currentNotFoundOrDeleted), + currentVersion + )); + versionForIndexing = Versions.NOT_FOUND; + seqNoForIndexing = SequenceNumbersService.UNASSIGNED_SEQ_NO; + addToLucene = false; + } else { + versionForIndexing = index.versionType().updateVersion(currentVersion, index.version()); + seqNoForIndexing = seqNoService().generateSeqNo(); + addToLucene = true; + } + } + } else if (canOptimizeAddDocument && mayHaveBeenIndexedBefore(index) == false) { + // no need to deal with out of order delivery - we never saw this one + // nocommit: verify this is tested! + useLuceneUpdateDocument = false; + currentNotFoundOrDeleted = true; + assert index.version() == 1L : "can optimize on replicas but incoming version is [" + index.version() + "]"; + seqNoForIndexing = index.seqNo(); + versionForIndexing = index.version(); + addToLucene = true; } else { - // update the document - forceUpdateDocument = false; // we don't force it - it depends on the version - final VersionValue versionValue = versionMap.getUnderLock(index.uid()); - assert incrementVersionLookup(); - if (versionValue == null) { - currentVersion = loadCurrentVersionFromIndex(index.uid()); - deleted = currentVersion == Versions.NOT_FOUND; + // drop out of order operations + assert index.versionType().versionTypeForReplicationAndRecovery() == index.versionType() : + "resolving out of order delivery based on versioning but version type isn't fit for it. got [" + + index.versionType() + "]"; + final LuceneOpStatus luceneOpStatus; + if (index.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + luceneOpStatus = checkLuceneOpStatusBasedOnSeq(index); } else { - currentVersion = checkDeletedAndGCed(versionValue); - deleted = versionValue.delete(); + luceneOpStatus = checkLuceneOpStatusBasedOnVersions(index); } - } - final long expectedVersion = index.version(); - Optional resultOnVersionConflict; - try { - final boolean isVersionConflict = checkVersionConflict(index, currentVersion, expectedVersion, deleted); - resultOnVersionConflict = isVersionConflict ? Optional.of(new IndexResult(currentVersion, index.seqNo(), false)) - : Optional.empty(); - } catch (IllegalArgumentException | VersionConflictEngineException ex) { - resultOnVersionConflict = Optional.of(new IndexResult(ex, currentVersion, index.seqNo())); + useLuceneUpdateDocument = luceneOpStatus != LuceneOpStatus.NOT_FOUND; + currentNotFoundOrDeleted = luceneOpStatus == LuceneOpStatus.NOT_FOUND; + seqNoForIndexing = index.seqNo(); + versionForIndexing = index.version(); + addToLucene = luceneOpStatus != LuceneOpStatus.NEWER_OR_EQUAL; } final IndexResult indexResult; - if (resultOnVersionConflict.isPresent()) { - indexResult = resultOnVersionConflict.get(); + if (earlyResultOnPreFlightError.isPresent()) { + indexResult = earlyResultOnPreFlightError.get(); + assert indexResult.hasFailure(); + } else if (addToLucene) { + indexResult = indexIntoLucene(index, seqNoForIndexing, versionForIndexing, currentNotFoundOrDeleted, + useLuceneUpdateDocument); } else { - // no version conflict - if (index.origin() == Operation.Origin.PRIMARY) { - seqNo = seqNoService().generateSeqNo(); - } - indexResult = indexIntoLucene(index, seqNo, currentVersion, deleted, forceUpdateDocument, canOptimizeAddDocument, expectedVersion); + indexResult = new IndexResult(versionForIndexing, seqNoForIndexing, false); } if (indexResult.hasFailure() == false) { - location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY + Translog.Location location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(new Translog.Index(index, indexResult)) : null; indexResult.setTranslogLocation(location); } + if (indexResult.getSeqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + seqNoService().markSeqNoAsCompleted(indexResult.getSeqNo()); + } indexResult.setTook(System.nanoTime() - index.startTime()); indexResult.freeze(); return indexResult; - } finally { - if (seqNo != SequenceNumbersService.UNASSIGNED_SEQ_NO) { - seqNoService().markSeqNoAsCompleted(seqNo); - } } } catch (RuntimeException | IOException e) { try { @@ -610,24 +679,27 @@ public IndexResult index(Index index) throws IOException { } } - private IndexResult indexIntoLucene(Index index, long seqNo, long currentVersion, boolean deleted, boolean forceUpdateDocument, boolean canOptimizeAddDocument, long expectedVersion) throws IOException { + private IndexResult indexIntoLucene(Index index, long seqNo, long newVersion, boolean markDocAsCreated, + boolean useLuceneUpdateDocument) + throws IOException { + assertSequenceNumberBeforeIndexing(index.origin(), seqNo); + assert newVersion >= 0; /* 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. */ index.parsedDoc().updateSeqID(seqNo, index.primaryTerm()); - final long updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion); - index.parsedDoc().version().setLongValue(updatedVersion); + index.parsedDoc().version().setLongValue(newVersion); try { - if (currentVersion == Versions.NOT_FOUND && forceUpdateDocument == false) { + if (useLuceneUpdateDocument) { + update(index.uid(), index.docs(), indexWriter); + } else { // document does not exists, we can optimize for create, but double check if assertions are running - assert assertDocDoesNotExist(index, canOptimizeAddDocument == false); + assert assertDocDoesNotExist(index, canOptimizeAddDocument(index) == false); index(index.docs(), indexWriter); - } else { - update(index.uid(), index.docs(), indexWriter); } - versionMap.putUnderLock(index.uid().bytes(), new VersionValue(updatedVersion)); - return new IndexResult(updatedVersion, seqNo, deleted); + versionMap.putUnderLock(index.uid().bytes(), new VersionValue(newVersion, seqNo, index.primaryTerm())); + return new IndexResult(newVersion, seqNo, markDocAsCreated); } catch (Exception ex) { if (indexWriter.getTragicException() == null) { /* There is no tragic event recorded so this must be a document failure. @@ -640,18 +712,19 @@ private IndexResult indexIntoLucene(Index index, long seqNo, long currentVersion * `indexWriter.getTragicException()` will be null otherwise we have to rethrow and treat it as fatal or rather * non-document failure */ - return new IndexResult(ex, currentVersion, index.seqNo()); + return new IndexResult(ex, Versions.MATCH_ANY, index.seqNo()); } else { throw ex; } } } - private boolean isForceUpdateDocument(Index index) { - boolean forceUpdateDocument; + private boolean mayHaveBeenIndexedBefore(Index index) { + assert canOptimizeAddDocument(index); + boolean mayHaveBeenIndexBefore; long deOptimizeTimestamp = maxUnsafeAutoIdTimestamp.get(); if (index.isRetry()) { - forceUpdateDocument = true; + mayHaveBeenIndexBefore = true; do { deOptimizeTimestamp = maxUnsafeAutoIdTimestamp.get(); if (deOptimizeTimestamp >= index.getAutoGeneratedIdTimestamp()) { @@ -662,9 +735,9 @@ private boolean isForceUpdateDocument(Index index) { assert maxUnsafeAutoIdTimestamp.get() >= index.getAutoGeneratedIdTimestamp(); } else { // in this case we force - forceUpdateDocument = deOptimizeTimestamp >= index.getAutoGeneratedIdTimestamp(); + mayHaveBeenIndexBefore = deOptimizeTimestamp >= index.getAutoGeneratedIdTimestamp(); } - return forceUpdateDocument; + return mayHaveBeenIndexBefore; } private static void index(final List docs, final IndexWriter indexWriter) throws IOException { @@ -681,7 +754,7 @@ private static void index(final List docs, final IndexWri private boolean assertDocDoesNotExist(final Index index, final boolean allowDeleted) throws IOException { final VersionValue versionValue = versionMap.getUnderLock(index.uid()); if (versionValue != null) { - if (versionValue.delete() == false || allowDeleted == false) { + if (versionValue.isDelete() == false || allowDeleted == false) { throw new AssertionError("doc [" + index.type() + "][" + index.id() + "] exists in version map (version " + versionValue + ")"); } } else { @@ -705,108 +778,105 @@ private static void update(final Term uid, final List doc @Override public DeleteResult delete(Delete delete) throws IOException { - DeleteResult result; - try (ReleasableLock ignored = readLock.acquire()) { - assert assertVersionType(delete); + assert assertVersionType(delete); + assert assertIncomingSequenceNumber(delete.origin(), delete.seqNo()); + final DeleteResult deleteResult; + // NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments: + try (ReleasableLock ignored = readLock.acquire(); Releasable ignored2 = acquireLock(delete.uid())) { ensureOpen(); - // NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments: - result = innerDelete(delete); - } catch (RuntimeException | IOException e) { - try { - maybeFailEngine("index", e); - } catch (Exception inner) { - e.addSuppressed(inner); - } - throw e; - } - maybePruneDeletedTombstones(); - return result; - } - - private void maybePruneDeletedTombstones() { - // It's expensive to prune because we walk the deletes map acquiring dirtyLock for each uid so we only do it - // every 1/4 of gcDeletesInMillis: - if (engineConfig.isEnableGcDeletes() && engineConfig.getThreadPool().relativeTimeInMillis() - lastDeleteVersionPruneTimeMSec > getGcDeletesInMillis() * 0.25) { - pruneDeletedTombstones(); - } - } - - private DeleteResult innerDelete(Delete delete) throws IOException { - assert assertSequenceNumber(delete.origin(), delete.seqNo()); - final Translog.Location location; - final long updatedVersion; - final boolean found; - long seqNo = delete.seqNo(); - try (Releasable ignored = acquireLock(delete.uid())) { lastWriteNanos = delete.startTime(); - final long currentVersion; - final boolean deleted; - final VersionValue versionValue = versionMap.getUnderLock(delete.uid()); - assert incrementVersionLookup(); - if (versionValue == null) { - currentVersion = loadCurrentVersionFromIndex(delete.uid()); - deleted = currentVersion == Versions.NOT_FOUND; + final boolean deleteInLucene; + final boolean currentFound; + final long seqNoForIndexing; + final long versionForIndexing; + Optional earlyResultOnPreflightError = Optional.empty(); + if (delete.origin() == Operation.Origin.PRIMARY) { + // resolve operation from external to internal + final VersionValue versionValue = versionMap.getUnderLock(delete.uid()); + assert incrementVersionLookup(); + final long currentVersion; + final boolean currentIsDeleted; + if (versionValue == null) { + currentVersion = loadCurrentVersionFromIndex(delete.uid()); + currentIsDeleted = currentVersion == Versions.NOT_FOUND; + } else { + currentVersion = checkDeletedAndGCed(versionValue); + currentIsDeleted = versionValue.isDelete(); + } + currentFound = currentVersion != Versions.NOT_FOUND; + if (delete.versionType().isVersionConflictForWrites(currentVersion, delete.version(), currentIsDeleted)) { + earlyResultOnPreflightError = Optional.of(new DeleteResult( + new VersionConflictEngineException(shardId, delete, currentVersion, currentIsDeleted), + currentVersion, delete.seqNo() + )); + deleteInLucene = false; + seqNoForIndexing = SequenceNumbersService.UNASSIGNED_SEQ_NO; + versionForIndexing = -1; + } else { + versionForIndexing = delete.versionType().updateVersion(currentVersion, delete.version()); + seqNoForIndexing = seqNoService().generateSeqNo(); + deleteInLucene = true; + } } else { - currentVersion = checkDeletedAndGCed(versionValue); - deleted = versionValue.delete(); + // drop out of order operations + assert delete.versionType().versionTypeForReplicationAndRecovery() == delete.versionType() : + "resolving out of order delivery based on versioning but version type isn't fit for it. got [" + + delete.versionType() + "]"; + final LuceneOpStatus luceneOpStatus; + if (delete.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + luceneOpStatus = checkLuceneOpStatusBasedOnSeq(delete); + } else { + luceneOpStatus = checkLuceneOpStatusBasedOnVersions(delete); + } + // the found vs deleted distinction is not relevant on replicas, just set it + currentFound = luceneOpStatus != LuceneOpStatus.NOT_FOUND; + seqNoForIndexing = delete.seqNo(); + versionForIndexing = delete.version(); + deleteInLucene = luceneOpStatus != LuceneOpStatus.NEWER_OR_EQUAL; } - final long expectedVersion = delete.version(); - Optional resultOnVersionConflict; - try { - final boolean isVersionConflict = checkVersionConflict(delete, currentVersion, expectedVersion, deleted); - resultOnVersionConflict = isVersionConflict ? Optional.of(new DeleteResult(expectedVersion, delete.seqNo(), true)) - : Optional.empty(); - } catch (IllegalArgumentException | VersionConflictEngineException ex) { - resultOnVersionConflict = Optional.of(new DeleteResult(ex, expectedVersion, delete.seqNo())); - } - final DeleteResult deleteResult; - if (resultOnVersionConflict.isPresent()) { - deleteResult = resultOnVersionConflict.get(); + if (earlyResultOnPreflightError.isPresent()) { + deleteResult = earlyResultOnPreflightError.get(); } else { - if (delete.origin() == Operation.Origin.PRIMARY) { - seqNo = seqNoService().generateSeqNo(); + if (deleteInLucene) { + // 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()); + versionMap.putUnderLock(delete.uid().bytes(), + new DeleteVersionValue(versionForIndexing, seqNoForIndexing, delete.primaryTerm(), + engineConfig.getThreadPool().relativeTimeInMillis())); } - updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion); - found = deleteIfFound(delete.uid(), currentVersion, deleted, versionValue); - deleteResult = new DeleteResult(updatedVersion, seqNo, found); - - versionMap.putUnderLock(delete.uid().bytes(), - new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().relativeTimeInMillis())); + deleteResult = new DeleteResult(versionForIndexing, seqNoForIndexing, currentFound); } if (!deleteResult.hasFailure()) { - location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY + Translog.Location location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(new Translog.Delete(delete, deleteResult)) : null; deleteResult.setTranslogLocation(location); } + if (deleteResult.getSeqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + seqNoService().markSeqNoAsCompleted(deleteResult.getSeqNo()); + } deleteResult.setTook(System.nanoTime() - delete.startTime()); deleteResult.freeze(); - return deleteResult; - } finally { - if (seqNo != SequenceNumbersService.UNASSIGNED_SEQ_NO) { - seqNoService().markSeqNoAsCompleted(seqNo); + } catch (RuntimeException | IOException e) { + try { + maybeFailEngine("index", e); + } catch (Exception inner) { + e.addSuppressed(inner); } + throw e; } + maybePruneDeletedTombstones(); + return deleteResult; } - private boolean deleteIfFound(Term uid, long currentVersion, boolean deleted, VersionValue versionValue) throws IOException { - assert uid != null : "uid must not be null"; - final boolean found; - if (currentVersion == Versions.NOT_FOUND) { - // doc does not exist and no prior deletes - found = false; - } else if (versionValue != null && deleted) { - // a "delete on delete", in this case, we still increment the version, log it, and return that version - found = false; - } else { - // we deleted a currently existing document - // 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(uid); - found = true; + private void maybePruneDeletedTombstones() { + // It's expensive to prune because we walk the deletes map acquiring dirtyLock for each uid so we only do it + // every 1/4 of gcDeletesInMillis: + if (engineConfig.isEnableGcDeletes() && engineConfig.getThreadPool().relativeTimeInMillis() - lastDeleteVersionPruneTimeMSec > getGcDeletesInMillis() * 0.25) { + pruneDeletedTombstones(); } - return found; } @Override @@ -1059,7 +1129,7 @@ private void pruneDeletedTombstones() { // Must re-get it here, vs using entry.getValue(), in case the uid was indexed/deleted since we pulled the iterator: VersionValue versionValue = versionMap.getTombstoneUnderLock(uid); if (versionValue != null) { - if (timeMSec - versionValue.time() > getGcDeletesInMillis()) { + if (timeMSec - versionValue.getTime() > getGcDeletesInMillis()) { versionMap.removeTombstoneUnderLock(uid); } } @@ -1302,7 +1372,7 @@ private Releasable acquireLock(Term uid) { private long loadCurrentVersionFromIndex(Term uid) throws IOException { assert incrementIndexVersionLookup(); try (Searcher searcher = acquireSearcher("load_version")) { - return Versions.loadVersion(searcher.reader(), uid); + return VersionsAndSeqNoResolver.loadVersion(searcher.reader(), uid); } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java b/core/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java index 44b302e85cdd7..cd27635091e2d 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java +++ b/core/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java @@ -164,7 +164,7 @@ void putUnderLock(BytesRef uid, VersionValue version) { if (prev != null) { // Deduct RAM for the version we just replaced: long prevBytes = BASE_BYTES_PER_CHM_ENTRY; - if (prev.delete() == false) { + if (prev.isDelete() == false) { prevBytes += prev.ramBytesUsed() + uidRAMBytesUsed; } ramBytesUsedCurrent.addAndGet(-prevBytes); @@ -172,13 +172,13 @@ void putUnderLock(BytesRef uid, VersionValue version) { // Add RAM for the new version: long newBytes = BASE_BYTES_PER_CHM_ENTRY; - if (version.delete() == false) { + if (version.isDelete() == false) { newBytes += version.ramBytesUsed() + uidRAMBytesUsed; } ramBytesUsedCurrent.addAndGet(newBytes); final VersionValue prevTombstone; - if (version.delete()) { + if (version.isDelete()) { // Also enroll the delete into tombstones, and account for its RAM too: prevTombstone = tombstones.put(uid, version); @@ -187,7 +187,7 @@ void putUnderLock(BytesRef uid, VersionValue version) { // the accounting to current: ramBytesUsedTombstones.addAndGet(BASE_BYTES_PER_CHM_ENTRY + version.ramBytesUsed() + uidRAMBytesUsed); - if (prevTombstone == null && prev != null && prev.delete()) { + if (prevTombstone == null && prev != null && prev.isDelete()) { // If prev was a delete that had already been removed from tombstones, then current was already accounting for the // BytesRef/VersionValue RAM, so we now deduct that as well: ramBytesUsedCurrent.addAndGet(-(prev.ramBytesUsed() + uidRAMBytesUsed)); @@ -211,12 +211,12 @@ void removeTombstoneUnderLock(BytesRef uid) { final VersionValue prev = tombstones.remove(uid); if (prev != null) { - assert prev.delete(); + assert prev.isDelete(); long v = ramBytesUsedTombstones.addAndGet(-(BASE_BYTES_PER_CHM_ENTRY + prev.ramBytesUsed() + uidRAMBytesUsed)); assert v >= 0: "bytes=" + v; } final VersionValue curVersion = maps.current.get(uid); - if (curVersion != null && curVersion.delete()) { + if (curVersion != null && curVersion.isDelete()) { // We now shift accounting of the BytesRef from tombstones to current, because a refresh would clear this RAM. This should be // uncommon, because with the default refresh=1s and gc_deletes=60s, deletes should be cleared from current long before we drop // them from tombstones: diff --git a/core/src/main/java/org/elasticsearch/index/engine/VersionConflictEngineException.java b/core/src/main/java/org/elasticsearch/index/engine/VersionConflictEngineException.java index b743141d0c7f8..f829e35af8912 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/VersionConflictEngineException.java +++ b/core/src/main/java/org/elasticsearch/index/engine/VersionConflictEngineException.java @@ -26,6 +26,10 @@ public class VersionConflictEngineException extends EngineException { + public VersionConflictEngineException(ShardId shardId, Engine.Operation op, long currentVersion, boolean deleted) { + this(shardId, op.type(), op.id(), op.versionType().explainConflictForWrites(currentVersion, op.version(), deleted)); + } + public VersionConflictEngineException(ShardId shardId, String type, String id, String explanation) { this(shardId, null, type, id, explanation); } diff --git a/core/src/main/java/org/elasticsearch/index/engine/VersionValue.java b/core/src/main/java/org/elasticsearch/index/engine/VersionValue.java index 6ee484b2b38fd..f6844ee792bf2 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/VersionValue.java +++ b/core/src/main/java/org/elasticsearch/index/engine/VersionValue.java @@ -29,24 +29,39 @@ class VersionValue implements Accountable { private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(VersionValue.class); + /** the version of the document. used for versioned indexed operations and as a BWC layer, where no seq# are set yet */ private final long version; - VersionValue(long version) { + /** the seq number of the operation that last changed the associated uuid */ + private final long seqNo; + /** the the term of the operation that last changed the associated uuid */ + private final long term; + + VersionValue(long version, long seqNo, long term) { this.version = version; + this.seqNo = seqNo; + this.term = term; } - public long time() { + public long getTime() { throw new UnsupportedOperationException(); } - public long version() { + public long getVersion() { return version; } - public boolean delete() { + public boolean isDelete() { return false; } + public long getSeqNo() { + return seqNo; + } + + public long getTerm() { + return term; + } @Override public long ramBytesUsed() { @@ -62,6 +77,8 @@ public Collection getChildResources() { public String toString() { return "VersionValue{" + "version=" + version + + ", seqNo=" + seqNo + + ", term=" + term + '}'; } } diff --git a/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java b/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java index cfab3382c18d5..6d3e1e3ab6a56 100644 --- a/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java +++ b/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java @@ -24,7 +24,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.lucene.uid.Versions; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.metrics.MeanMetric; import org.elasticsearch.common.util.set.Sets; @@ -179,7 +179,7 @@ private GetResult innerGet(String type, String id, String[] gFields, boolean rea private GetResult innerGetLoadFromStoredFields(String type, String id, String[] gFields, FetchSourceContext fetchSourceContext, Engine.GetResult get, MapperService mapperService) { Map fields = null; BytesReference source = null; - Versions.DocIdAndVersion docIdAndVersion = get.docIdAndVersion(); + DocIdAndVersion docIdAndVersion = get.docIdAndVersion(); FieldsVisitor fieldVisitor = buildFieldsVisitors(gFields, fetchSourceContext); if (fieldVisitor != null) { try { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index ad21cce66749d..64c4932e47017 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -29,7 +29,6 @@ import org.elasticsearch.common.lucene.all.AllEntries; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; import java.util.ArrayList; import java.util.Iterator; @@ -255,12 +254,12 @@ public void version(Field version) { } @Override - public SeqNoFieldMapper.SequenceID seqID() { + public SeqNoFieldMapper.SequenceIDFields seqID() { return in.seqID(); } @Override - public void seqID(SeqNoFieldMapper.SequenceID seqID) { + public void seqID(SeqNoFieldMapper.SequenceIDFields seqID) { in.seqID(seqID); } @@ -311,7 +310,7 @@ public static class InternalParseContext extends ParseContext { private Field version; - private SeqNoFieldMapper.SequenceID seqID; + private SeqNoFieldMapper.SequenceIDFields seqID; private final AllEntries allEntries; @@ -405,12 +404,12 @@ public void version(Field version) { } @Override - public SeqNoFieldMapper.SequenceID seqID() { + public SeqNoFieldMapper.SequenceIDFields seqID() { return this.seqID; } @Override - public void seqID(SeqNoFieldMapper.SequenceID seqID) { + public void seqID(SeqNoFieldMapper.SequenceIDFields seqID) { this.seqID = seqID; } @@ -540,9 +539,9 @@ public boolean isWithinMultiFields() { public abstract void version(Field version); - public abstract SeqNoFieldMapper.SequenceID seqID(); + public abstract SeqNoFieldMapper.SequenceIDFields seqID(); - public abstract void seqID(SeqNoFieldMapper.SequenceID seqID); + public abstract void seqID(SeqNoFieldMapper.SequenceIDFields seqID); public final boolean includeInAll(Boolean includeInAll, FieldMapper mapper) { return includeInAll(includeInAll, mapper.fieldType().indexOptions() != IndexOptions.NONE); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java b/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java index f7d5804be0d53..91cf2aa4fa4a8 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -36,7 +36,7 @@ public class ParsedDocument { private final String id, type; private final BytesRef uid; - private final SeqNoFieldMapper.SequenceID seqID; + private final SeqNoFieldMapper.SequenceIDFields seqID; private final String routing; @@ -50,7 +50,7 @@ public class ParsedDocument { private String parent; public ParsedDocument(Field version, - SeqNoFieldMapper.SequenceID seqID, + SeqNoFieldMapper.SequenceIDFields seqID, String id, String type, String routing, diff --git a/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java index c160f2b8cb92f..ff7c164dceb69 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java @@ -26,32 +26,19 @@ import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PointValues; -import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType; import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData; -import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.Mapper; -import org.elasticsearch.index.mapper.MapperParsingException; -import org.elasticsearch.index.mapper.MetadataFieldMapper; -import org.elasticsearch.index.mapper.ParseContext; -import org.elasticsearch.index.mapper.Mapper.TypeParser.ParserContext; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.query.QueryShardContext; -import org.elasticsearch.index.query.QueryShardException; import org.elasticsearch.index.seqno.SequenceNumbersService; import java.io.IOException; @@ -79,13 +66,13 @@ public class SeqNoFieldMapper extends MetadataFieldMapper { * A sequence ID, which is made up of a sequence number (both the searchable * and doc_value version of the field) and the primary term. */ - public static class SequenceID { + public static class SequenceIDFields { public final Field seqNo; public final Field seqNoDocValue; public final Field primaryTerm; - public SequenceID(Field seqNo, Field seqNoDocValue, Field primaryTerm) { + 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"); @@ -94,9 +81,9 @@ public SequenceID(Field seqNo, Field seqNoDocValue, Field primaryTerm) { this.primaryTerm = primaryTerm; } - public static SequenceID emptySeqID() { - return new SequenceID(new LongPoint(NAME, SequenceNumbersService.UNASSIGNED_SEQ_NO), - new SortedNumericDocValuesField(NAME, SequenceNumbersService.UNASSIGNED_SEQ_NO), + public static SequenceIDFields emptySeqID() { + return new SequenceIDFields(new LongPoint(NAME, SequenceNumbersService.UNASSIGNED_SEQ_NO), + new NumericDocValuesField(NAME, SequenceNumbersService.UNASSIGNED_SEQ_NO), new NumericDocValuesField(PRIMARY_TERM_NAME, 0)); } } @@ -255,7 +242,7 @@ public void preParse(ParseContext context) throws IOException { protected void parseCreateField(ParseContext context, List fields) throws IOException { // see InternalEngine.innerIndex to see where the real version value is set // also see ParsedDocument.updateSeqID (called by innerIndex) - SequenceID seqID = SequenceID.emptySeqID(); + SequenceIDFields seqID = SequenceIDFields.emptySeqID(); context.seqID(seqID); fields.add(seqID.seqNo); fields.add(seqID.seqNoDocValue); diff --git a/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java b/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java index 7981c89124280..6351282a38a18 100644 --- a/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java +++ b/core/src/main/java/org/elasticsearch/index/termvectors/TermVectorsService.java @@ -34,7 +34,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.lucene.uid.Versions; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.XContentMapValues; @@ -98,7 +98,7 @@ static TermVectorsResponse getTermVectors(IndexShard indexShard, TermVectorsRequ final Engine.Searcher searcher = indexShard.acquireSearcher("term_vector"); try { Fields topLevelFields = MultiFields.getFields(get.searcher() != null ? get.searcher().reader() : searcher.reader()); - Versions.DocIdAndVersion docIdAndVersion = get.docIdAndVersion(); + DocIdAndVersion docIdAndVersion = get.docIdAndVersion(); /* from an artificial document */ if (request.doc() != null) { termVectorsByField = generateTermVectorsFromDoc(indexShard, request); diff --git a/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java b/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java index a67a728edccc8..05922ee85ec95 100644 --- a/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java +++ b/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java @@ -31,17 +31,17 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.FixedBitSet; import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.lucene.uid.Versions.DocIdAndVersion; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.test.ESTestCase; -/** - * test per-segment lookup of version-related datastructures +/** + * test per-segment lookup of version-related data structures */ public class VersionLookupTests extends ESTestCase { - /** + /** * test version lookup actually works */ public void testSimple() throws Exception { @@ -53,22 +53,22 @@ public void testSimple() throws Exception { writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); LeafReaderContext segment = reader.leaves().get(0); - PerThreadIDAndVersionLookup lookup = new PerThreadIDAndVersionLookup(segment.reader()); + PerThreadIDAndVersionSeqNoLookup lookup = new PerThreadIDAndVersionSeqNoLookup(segment.reader()); // found doc - DocIdAndVersion result = lookup.lookup(new BytesRef("6"), null, segment); + DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), null, segment); assertNotNull(result); assertEquals(87, result.version); assertEquals(0, result.docId); // not found doc - assertNull(lookup.lookup(new BytesRef("7"), null, segment)); + assertNull(lookup.lookupVersion(new BytesRef("7"), null, segment)); // deleted doc - assertNull(lookup.lookup(new BytesRef("6"), new Bits.MatchNoBits(1), segment)); + assertNull(lookup.lookupVersion(new BytesRef("6"), new Bits.MatchNoBits(1), segment)); reader.close(); writer.close(); dir.close(); } - - /** + + /** * test version lookup with two documents matching the ID */ public void testTwoDocuments() throws Exception { @@ -81,28 +81,28 @@ public void testTwoDocuments() throws Exception { writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); LeafReaderContext segment = reader.leaves().get(0); - PerThreadIDAndVersionLookup lookup = new PerThreadIDAndVersionLookup(segment.reader()); + PerThreadIDAndVersionSeqNoLookup lookup = new PerThreadIDAndVersionSeqNoLookup(segment.reader()); // return the last doc when there are duplicates - DocIdAndVersion result = lookup.lookup(new BytesRef("6"), null, segment); + DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), null, segment); assertNotNull(result); assertEquals(87, result.version); assertEquals(1, result.docId); // delete the first doc only FixedBitSet live = new FixedBitSet(2); live.set(1); - result = lookup.lookup(new BytesRef("6"), live, segment); + result = lookup.lookupVersion(new BytesRef("6"), live, segment); assertNotNull(result); assertEquals(87, result.version); assertEquals(1, result.docId); // delete the second doc only live.clear(1); live.set(0); - result = lookup.lookup(new BytesRef("6"), live, segment); + result = lookup.lookupVersion(new BytesRef("6"), live, segment); assertNotNull(result); assertEquals(87, result.version); assertEquals(0, result.docId); // delete both docs - assertNull(lookup.lookup(new BytesRef("6"), new Bits.MatchNoBits(2), segment)); + assertNull(lookup.lookupVersion(new BytesRef("6"), new Bits.MatchNoBits(2), segment)); reader.close(); writer.close(); dir.close(); diff --git a/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java b/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java index 45693e101d462..c5e66a3bf2ad5 100644 --- a/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java +++ b/core/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java @@ -38,6 +38,8 @@ import java.util.ArrayList; import java.util.List; +import static org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.loadDocIdAndVersion; +import static org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.loadVersion; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; @@ -61,15 +63,15 @@ public void testVersions() throws Exception { Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); DirectoryReader directoryReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "_na_", 1)); - MatcherAssert.assertThat(Versions.loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(Versions.NOT_FOUND)); + MatcherAssert.assertThat(loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(Versions.NOT_FOUND)); Document doc = new Document(); doc.add(new Field(UidFieldMapper.NAME, "1", UidFieldMapper.Defaults.FIELD_TYPE)); doc.add(new NumericDocValuesField(VersionFieldMapper.NAME, 1)); writer.updateDocument(new Term(UidFieldMapper.NAME, "1"), doc); directoryReader = reopen(directoryReader); - assertThat(Versions.loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(1L)); - assertThat(Versions.loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(1L)); + assertThat(loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(1L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(1L)); doc = new Document(); Field uid = new Field(UidFieldMapper.NAME, "1", UidFieldMapper.Defaults.FIELD_TYPE); @@ -78,8 +80,8 @@ public void testVersions() throws Exception { doc.add(version); writer.updateDocument(new Term(UidFieldMapper.NAME, "1"), doc); directoryReader = reopen(directoryReader); - assertThat(Versions.loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(2L)); - assertThat(Versions.loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(2L)); + assertThat(loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(2L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(2L)); // test reuse of uid field doc = new Document(); @@ -89,13 +91,13 @@ public void testVersions() throws Exception { writer.updateDocument(new Term(UidFieldMapper.NAME, "1"), doc); directoryReader = reopen(directoryReader); - assertThat(Versions.loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(3L)); - assertThat(Versions.loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(3L)); + assertThat(loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(3L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(3L)); writer.deleteDocuments(new Term(UidFieldMapper.NAME, "1")); directoryReader = reopen(directoryReader); - assertThat(Versions.loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(Versions.NOT_FOUND)); - assertThat(Versions.loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), nullValue()); + assertThat(loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(Versions.NOT_FOUND)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), nullValue()); directoryReader.close(); writer.close(); dir.close(); @@ -121,21 +123,21 @@ public void testNestedDocuments() throws IOException { writer.updateDocuments(new Term(UidFieldMapper.NAME, "1"), docs); DirectoryReader directoryReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "_na_", 1)); - assertThat(Versions.loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(5L)); - assertThat(Versions.loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(5L)); + assertThat(loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(5L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(5L)); version.setLongValue(6L); writer.updateDocuments(new Term(UidFieldMapper.NAME, "1"), docs); version.setLongValue(7L); writer.updateDocuments(new Term(UidFieldMapper.NAME, "1"), docs); directoryReader = reopen(directoryReader); - assertThat(Versions.loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(7L)); - assertThat(Versions.loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(7L)); + assertThat(loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(7L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")).version, equalTo(7L)); writer.deleteDocuments(new Term(UidFieldMapper.NAME, "1")); directoryReader = reopen(directoryReader); - assertThat(Versions.loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(Versions.NOT_FOUND)); - assertThat(Versions.loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), nullValue()); + assertThat(loadVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), equalTo(Versions.NOT_FOUND)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(UidFieldMapper.NAME, "1")), nullValue()); directoryReader.close(); writer.close(); dir.close(); @@ -143,7 +145,7 @@ public void testNestedDocuments() throws IOException { /** Test that version map cache works, is evicted on close, etc */ public void testCache() throws Exception { - int size = Versions.lookupStates.size(); + int size = VersionsAndSeqNoResolver.lookupStates.size(); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); @@ -153,22 +155,22 @@ public void testCache() throws Exception { writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); // should increase cache size by 1 - assertEquals(87, Versions.loadVersion(reader, new Term(UidFieldMapper.NAME, "6"))); - assertEquals(size+1, Versions.lookupStates.size()); + assertEquals(87, loadVersion(reader, new Term(UidFieldMapper.NAME, "6"))); + assertEquals(size+1, VersionsAndSeqNoResolver.lookupStates.size()); // should be cache hit - assertEquals(87, Versions.loadVersion(reader, new Term(UidFieldMapper.NAME, "6"))); - assertEquals(size+1, Versions.lookupStates.size()); + assertEquals(87, loadVersion(reader, new Term(UidFieldMapper.NAME, "6"))); + assertEquals(size+1, VersionsAndSeqNoResolver.lookupStates.size()); reader.close(); writer.close(); // core should be evicted from the map - assertEquals(size, Versions.lookupStates.size()); + assertEquals(size, VersionsAndSeqNoResolver.lookupStates.size()); dir.close(); } /** Test that version map cache behaves properly with a filtered reader */ public void testCacheFilterReader() throws Exception { - int size = Versions.lookupStates.size(); + int size = VersionsAndSeqNoResolver.lookupStates.size(); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); @@ -177,18 +179,18 @@ public void testCacheFilterReader() throws Exception { doc.add(new NumericDocValuesField(VersionFieldMapper.NAME, 87)); writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); - assertEquals(87, Versions.loadVersion(reader, new Term(UidFieldMapper.NAME, "6"))); - assertEquals(size+1, Versions.lookupStates.size()); + assertEquals(87, loadVersion(reader, new Term(UidFieldMapper.NAME, "6"))); + assertEquals(size+1, VersionsAndSeqNoResolver.lookupStates.size()); // now wrap the reader DirectoryReader wrapped = ElasticsearchDirectoryReader.wrap(reader, new ShardId("bogus", "_na_", 5)); - assertEquals(87, Versions.loadVersion(wrapped, new Term(UidFieldMapper.NAME, "6"))); + assertEquals(87, loadVersion(wrapped, new Term(UidFieldMapper.NAME, "6"))); // same size map: core cache key is shared - assertEquals(size+1, Versions.lookupStates.size()); + assertEquals(size+1, VersionsAndSeqNoResolver.lookupStates.size()); reader.close(); writer.close(); // core should be evicted from the map - assertEquals(size, Versions.lookupStates.size()); + assertEquals(size, VersionsAndSeqNoResolver.lookupStates.size()); dir.close(); } } diff --git a/core/src/test/java/org/elasticsearch/index/IndexingSlowLogTests.java b/core/src/test/java/org/elasticsearch/index/IndexingSlowLogTests.java index c815b2b55f9f3..daf196da7ce86 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexingSlowLogTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexingSlowLogTests.java @@ -41,7 +41,7 @@ public class IndexingSlowLogTests extends ESTestCase { public void testSlowLogParsedDocumentPrinterSourceToLog() throws IOException { BytesReference source = JsonXContent.contentBuilder().startObject().field("foo", "bar").endObject().bytes(); - ParsedDocument pd = new ParsedDocument(new NumericDocValuesField("version", 1), SeqNoFieldMapper.SequenceID.emptySeqID(), "id", + ParsedDocument pd = new ParsedDocument(new NumericDocValuesField("version", 1), SeqNoFieldMapper.SequenceIDFields.emptySeqID(), "id", "test", null, null, source, XContentType.JSON, null); Index index = new Index("foo", "123"); // Turning off document logging doesn't log source[] diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 6c7d2dd810fe9..0119aea0663c2 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -79,10 +79,11 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.FileSystemUtils; -import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.Versions; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; +import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; @@ -166,7 +167,6 @@ import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThan; @@ -265,11 +265,16 @@ public void tearDown() throws Exception { private static Document testDocumentWithTextField() { + return testDocumentWithTextField("test"); + } + + private static Document testDocumentWithTextField(String value) { Document document = testDocument(); - document.add(new TextField("value", "test", Field.Store.YES)); + document.add(new TextField("value", value, Field.Store.YES)); return document; } + private static Document testDocument() { return new Document(); } @@ -281,7 +286,7 @@ public static ParsedDocument createParsedDoc(String id, String type, String rout private static ParsedDocument testParsedDocument(String id, String type, String routing, Document document, BytesReference source, Mapping mappingUpdate) { Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); - SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID(); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); document.add(uidField); document.add(versionField); document.add(seqID.seqNo); @@ -1234,33 +1239,6 @@ public void testExternalVersioningIndexConflict() throws IOException { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } - public void testForceVersioningNotAllowedExceptForOlderIndices() throws Exception { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 42, VersionType.FORCE, PRIMARY, 0, -1, false); - - Engine.IndexResult indexResult = engine.index(index); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(IllegalArgumentException.class)); - assertThat(indexResult.getFailure().getMessage(), containsString("version type [FORCE] may not be used for indices created after 6.0")); - - IndexSettings oldIndexSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_5_0_0_beta1) - .build()); - try (Store store = createStore(); - Engine engine = createEngine(oldIndexSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 84, VersionType.FORCE, PRIMARY, 0, -1, false); - Engine.IndexResult result = engine.index(index); - assertTrue(result.hasFailure()); - assertThat(result.getFailure(), instanceOf(IllegalArgumentException.class)); - assertThat(result.getFailure().getMessage(), containsString("version type [FORCE] may not be used for non-translog operations")); - - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 84, VersionType.FORCE, - Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY, 0, -1, false); - result = engine.index(index); - assertThat(result.getVersion(), equalTo(84L)); - } - } - public void testVersioningIndexConflictWithFlush() throws IOException { ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); Engine.Index index = indexForDoc(doc); @@ -1499,19 +1477,180 @@ public void testVersioningCreateExistsExceptionWithFlush() throws IOException { } public void testVersioningReplicaConflict1() throws IOException { + IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", 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.V_5_4_0_UNRELEASED) + .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), + between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) + .build()); + + try (Store oldStore = createStore(); Store oldReplicaStore = createStore(); + InternalEngine engine = createEngine(oldSettings, oldStore, createTempDir("translog-old"), newMergePolicy()); + InternalEngine replicaEngine = + createEngine(oldSettings, oldReplicaStore, createTempDir("translog-old-replica"), newMergePolicy())) { + + final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField("v_1") , B_1, null); + final Engine.Index v1Index = indexForDoc(doc); + final Engine.IndexResult v1Result = engine.index(v1Index); + assertThat(v1Result.getVersion(), equalTo(1L)); + + final Engine.Index v2Index = indexForDoc(doc); + final Engine.IndexResult v2Result = engine.index(v2Index); + assertThat(v2Result.getVersion(), equalTo(2L)); + + // apply the second index to the replica, should work fine + final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2") , B_1, null); + final Engine.Index replicaV2Index = new Engine.Index( + newUid(docV2), + docV2, + SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup + v2Index.primaryTerm(), + v2Result.getVersion(), + VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), + REPLICA, + 0, + -1, + false); + final Engine.IndexResult replicaV2Result = replicaEngine.index(replicaV2Index); + assertThat(replicaV2Result.getVersion(), equalTo(2L)); + + // now, the old one should produce an indexing result + final Engine.Index replicaV1Index = new Engine.Index( + newUid(doc), + doc, + SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup + v1Index.primaryTerm(), + v1Result.getVersion(), + VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), + REPLICA, + 0, + -1, + false); + final Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); + assertFalse(replicaV1Result.hasFailure()); + assertFalse(replicaV1Result.isCreated()); + assertThat(replicaV1Result.getVersion(), equalTo(1L)); + + // second version on replica should fail as well + final Engine.IndexResult replicaV2ReplayResult = replicaEngine.index(replicaV2Index); + assertFalse(replicaV2Result.hasFailure()); + assertFalse(replicaV1Result.isCreated()); + assertThat(replicaV2ReplayResult.getVersion(), equalTo(2L)); + + replicaEngine.refresh("test"); + try (Searcher searchResult = replicaEngine.acquireSearcher("test")) { + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits( + new TermQuery(new Term("value", "v_2")), 1)); + } + } + } + + public void testVersioningReplicaConflict2() throws IOException { + IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", 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.V_5_4_0_UNRELEASED) + .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), + between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) + .build()); + + try (Store oldStore = createStore(); Store oldReplicaStore = createStore(); + InternalEngine engine = createEngine(oldSettings, oldStore, createTempDir("translog-old"), newMergePolicy()); + InternalEngine replicaEngine = + createEngine(oldSettings, oldReplicaStore, createTempDir("translog-old-replica"), newMergePolicy())) { + + final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField("v_1") , B_1, null); + final Engine.Index v1Index = indexForDoc(doc); + final Engine.IndexResult v1Result = engine.index(v1Index); + assertThat(v1Result.getVersion(), equalTo(1L)); + + // apply the first index to the replica, should work fine + final Engine.Index replicaV1Index = new Engine.Index( + newUid(doc), + doc, + SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup + v1Index.primaryTerm(), + v1Result.getVersion(), + VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), + REPLICA, + 0, + -1, + false); + Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); + assertThat(replicaV1Result.getVersion(), equalTo(1L)); + + // index it again + final Engine.Index v2Index = indexForDoc(doc); + final Engine.IndexResult v2Result = engine.index(v2Index); + assertThat(v2Result.getVersion(), equalTo(2L)); + + // now delete it + final Engine.Delete delete = new Engine.Delete("test", "1", newUid(doc)); + final Engine.DeleteResult deleteResult = engine.delete(delete); + assertThat(deleteResult.getVersion(), equalTo(3L)); + + // apply the delete on the replica (skipping the second index) + final Engine.Delete replicaDelete = new Engine.Delete( + "test", + "1", + newUid(doc), + SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup + delete.primaryTerm(), + deleteResult.getVersion(), + VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), + REPLICA, + 0); + final Engine.DeleteResult replicaDeleteResult = replicaEngine.delete(replicaDelete); + assertThat(replicaDeleteResult.getVersion(), equalTo(3L)); + + // second time delete with same version should just produce the same version + final Engine.DeleteResult deleteReplayResult = replicaEngine.delete(replicaDelete); + assertFalse(deleteReplayResult.hasFailure()); + assertTrue(deleteReplayResult.isFound()); + assertThat(deleteReplayResult.getVersion(), equalTo(3L)); + + // now do the second index on the replica, it should not change things + final Engine.Index replicaV2Index = new Engine.Index( + newUid(doc), + doc, + SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup + v2Index.primaryTerm(), + v2Result.getVersion(), + VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), + REPLICA, + 0, + -1, + false); + final Engine.IndexResult replicaV2Result = replicaEngine.index(replicaV2Index); + assertFalse(replicaV2Result.hasFailure()); + assertFalse(replicaV2Result.isCreated()); + assertThat(replicaV2Result.getVersion(), equalTo(2L)); + + replicaEngine.refresh("test"); + assertVisibleCount(replicaEngine, 0); + } + } + + public void testSeqNoReplicaConflict1() throws IOException { + final ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); final Engine.Index v1Index = indexForDoc(doc); final Engine.IndexResult v1Result = engine.index(v1Index); assertThat(v1Result.getVersion(), equalTo(1L)); + assertThat(v1Result.getSeqNo(), equalTo(0L)); final Engine.Index v2Index = indexForDoc(doc); final Engine.IndexResult v2Result = engine.index(v2Index); assertThat(v2Result.getVersion(), equalTo(2L)); + assertThat(v2Result.getSeqNo(), equalTo(1L)); // apply the second index to the replica, should work fine + final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2") , B_1, null); final Engine.Index replicaV2Index = new Engine.Index( - newUid(doc), - doc, + newUid(docV2), + docV2, v2Result.getSeqNo(), v2Index.primaryTerm(), v2Result.getVersion(), @@ -1524,11 +1663,12 @@ public void testVersioningReplicaConflict1() throws IOException { assertThat(replicaV2Result.getVersion(), equalTo(2L)); // now, the old one should produce an indexing result + boolean rejectByTerm = randomBoolean(); final Engine.Index replicaV1Index = new Engine.Index( newUid(doc), doc, - v1Result.getSeqNo(), - v1Index.primaryTerm(), + rejectByTerm ? v2Result.getSeqNo() : v1Result.getSeqNo(), + rejectByTerm || randomBoolean() ? v2Index.primaryTerm() - 1 : v1Index.primaryTerm(), v1Result.getVersion(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, @@ -1538,16 +1678,47 @@ public void testVersioningReplicaConflict1() throws IOException { final Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); assertFalse(replicaV1Result.hasFailure()); assertFalse(replicaV1Result.isCreated()); - assertThat(replicaV1Result.getVersion(), equalTo(2L)); + assertThat(replicaV1Result.getVersion(), equalTo(1L)); // second version on replica should fail as well final Engine.IndexResult replicaV2ReplayResult = replicaEngine.index(replicaV2Index); - assertFalse(replicaV2Result.hasFailure()); - assertFalse(replicaV1Result.isCreated()); + assertFalse(replicaV2ReplayResult.hasFailure()); + assertFalse(replicaV2ReplayResult.isCreated()); assertThat(replicaV2ReplayResult.getVersion(), equalTo(2L)); + + replicaEngine.refresh("test"); + try (Searcher searchResult = replicaEngine.acquireSearcher("test")) { + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits( + new TermQuery(new Term("value", "v_2")), 1)); + } + + // now index, resolving by primaryTerm + final ParsedDocument docV3 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_3") , B_1, null); + final Engine.Index replicaV3Index = new Engine.Index( + newUid(docV3), + docV3, + v2Result.getSeqNo(), + v2Index.primaryTerm() + 1, + v2Result.getVersion(), + VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), + REPLICA, + 0, + -1, + false); + final Engine.IndexResult replicaV3Result = replicaEngine.index(replicaV3Index); + assertFalse(replicaV3Result.hasFailure()); + assertFalse(replicaV3Result.isCreated()); + assertThat(replicaV3Result.getVersion(), equalTo(v2Result.getVersion())); + replicaEngine.refresh("test"); + try (Searcher searchResult = replicaEngine.acquireSearcher("test")) { + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits( + new TermQuery(new Term("value", "v_3")), 1)); + } } - public void testVersioningReplicaConflict2() throws IOException { + public void testSeqNoReplicaConflict2() throws IOException { final ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); final Engine.Index v1Index = indexForDoc(doc); final Engine.IndexResult v1Result = engine.index(v1Index); @@ -1598,7 +1769,7 @@ public void testVersioningReplicaConflict2() throws IOException { assertTrue(deleteReplayResult.isFound()); assertThat(deleteReplayResult.getVersion(), equalTo(3L)); - // now do the second index on the replica, it should result in the current version + // now do the second index on the replica, it should not change things final Engine.Index replicaV2Index = new Engine.Index( newUid(doc), doc, @@ -1613,7 +1784,10 @@ public void testVersioningReplicaConflict2() throws IOException { final Engine.IndexResult replicaV2Result = replicaEngine.index(replicaV2Index); assertFalse(replicaV2Result.hasFailure()); assertFalse(replicaV2Result.isCreated()); - assertThat(replicaV2Result.getVersion(), equalTo(3L)); + assertThat(replicaV2Result.getVersion(), equalTo(2L)); + + replicaEngine.refresh("test"); + assertVisibleCount(replicaEngine, 0); } public void testBasicCreatedFlag() throws IOException { @@ -2123,10 +2297,7 @@ public void testTranslogReplayWithFailure() throws IOException { assertThat(indexResult.getVersion(), equalTo(1L)); } engine.refresh("test"); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(numDocs)); - } + assertVisibleCount(engine, numDocs); engine.close(); final MockDirectoryWrapper directory = DirectoryUtils.getLeaf(store.directory(), MockDirectoryWrapper.class); if (directory != null) { @@ -2158,9 +2329,14 @@ public void testTranslogReplayWithFailure() throws IOException { // no mock directory, no fun. engine = createEngine(store, primaryTranslogDir); } - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(numDocs)); + assertVisibleCount(engine, numDocs); + } + + private static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { + try (Searcher searcher = engine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new MatchAllDocsQuery(), collector); + assertThat(collector.getTotalHits(), equalTo(numDocs)); } } @@ -2173,10 +2349,7 @@ public void testSkipTranslogReplay() throws IOException { assertThat(indexResult.getVersion(), equalTo(1L)); } engine.refresh("test"); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(numDocs)); - } + assertVisibleCount(engine, numDocs); engine.close(); engine = new InternalEngine(engine.config()); @@ -2295,10 +2468,7 @@ public void testTranslogReplay() throws IOException { assertThat(indexResult.getVersion(), equalTo(1L)); } engine.refresh("test"); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(numDocs)); - } + assertVisibleCount(engine, numDocs); TranslogHandler parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer(); parser.mappingUpdate = dynamicUpdate(); @@ -2307,10 +2477,7 @@ public void testTranslogReplay() throws IOException { engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); // we need to reuse the engine config unless the parser.mappingModified won't work engine.recoverFromTranslog(); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(numDocs)); - } + assertVisibleCount(engine, numDocs); parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer(); assertEquals(numDocs, parser.recoveredOps.get()); if (parser.mappingUpdate != null) { @@ -2322,10 +2489,7 @@ public void testTranslogReplay() throws IOException { engine.close(); engine = createEngine(store, primaryTranslogDir); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(numDocs)); - } + assertVisibleCount(engine, numDocs); parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer(); assertEquals(0, parser.recoveredOps.get()); @@ -2412,10 +2576,7 @@ public void testRecoverFromForeignTranslog() throws IOException { assertThat(index.getVersion(), equalTo(1L)); } engine.refresh("test"); - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(numDocs)); - } + assertVisibleCount(engine, numDocs); Translog.TranslogGeneration generation = engine.getTranslog().getGeneration(); engine.close(); @@ -2444,10 +2605,7 @@ public void testRecoverFromForeignTranslog() throws IOException { } engine = createEngine(store, primaryTranslogDir); // and recover again! - try (Engine.Searcher searcher = engine.acquireSearcher("test")) { - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); - assertThat(topDocs.totalHits, equalTo(numDocs)); - } + assertVisibleCount(engine, numDocs); } public void testShardNotAvailableExceptionWhenEngineClosedConcurrently() throws IOException, InterruptedException { @@ -2667,10 +2825,10 @@ public BytesRef binaryValue() { } } - public void testDoubleDelivery() throws IOException { + public void testDoubleDeliveryPrimary() throws IOException { final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); - Engine.Index operation = randomAppendOnly(doc, false, 1); - Engine.Index retry = randomAppendOnly(doc, true, 1); + Engine.Index operation = randomAppendOnlyPrimary(doc, false, 1); + Engine.Index retry = randomAppendOnlyPrimary(doc, true, 1); if (randomBoolean()) { Engine.IndexResult indexResult = engine.index(operation); assertFalse(engine.indexWriterHasDeletions()); @@ -2721,6 +2879,61 @@ public void testDoubleDelivery() throws IOException { } } + public void testDoubleDeliveryReplica() throws IOException { + final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField(), + new BytesArray("{}".getBytes(Charset.defaultCharset())), null); + Engine.Index operation = randomAppendOnlyReplica(doc, false, 1, randomIntBetween(0, 20)); + Engine.Index retry = randomAppendOnlyReplica(doc, true, 1, operation.seqNo()); + if (randomBoolean()) { + Engine.IndexResult indexResult = engine.index(operation); + assertFalse(engine.indexWriterHasDeletions()); + assertEquals(0, engine.getNumVersionLookups()); + assertNotNull(indexResult.getTranslogLocation()); + Engine.IndexResult retryResult = engine.index(retry); + assertFalse(engine.indexWriterHasDeletions()); + assertEquals(1, engine.getNumVersionLookups()); + assertNotNull(retryResult.getTranslogLocation()); + assertTrue(retryResult.getTranslogLocation().compareTo(indexResult.getTranslogLocation()) > 0); + } else { + Engine.IndexResult retryResult = engine.index(retry); + assertFalse(engine.indexWriterHasDeletions()); + assertEquals(1, engine.getNumVersionLookups()); + assertNotNull(retryResult.getTranslogLocation()); + Engine.IndexResult indexResult = engine.index(operation); + assertFalse(engine.indexWriterHasDeletions()); + assertEquals(2, engine.getNumVersionLookups()); + assertNotNull(retryResult.getTranslogLocation()); + assertTrue(retryResult.getTranslogLocation().compareTo(indexResult.getTranslogLocation()) < 0); + } + + engine.refresh("test"); + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); + assertEquals(1, topDocs.totalHits); + } + operation = randomAppendOnly(doc, false, 1); + retry = randomAppendOnly(doc, true, 1); + if (randomBoolean()) { + Engine.IndexResult indexResult = engine.index(operation); + assertNotNull(indexResult.getTranslogLocation()); + Engine.IndexResult retryResult = engine.index(retry); + assertNotNull(retryResult.getTranslogLocation()); + assertTrue(retryResult.getTranslogLocation().compareTo(indexResult.getTranslogLocation()) > 0); + } else { + Engine.IndexResult retryResult = engine.index(retry); + assertNotNull(retryResult.getTranslogLocation()); + Engine.IndexResult indexResult = engine.index(operation); + assertNotNull(retryResult.getTranslogLocation()); + assertTrue(retryResult.getTranslogLocation().compareTo(indexResult.getTranslogLocation()) < 0); + } + + engine.refresh("test"); + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); + assertEquals(1, topDocs.totalHits); + } + } + public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOException { @@ -2791,10 +3004,19 @@ public void testRetryWithAutogeneratedIdsAndWrongOrderWorksAndNoDuplicateDocs() public Engine.Index randomAppendOnly(ParsedDocument doc, boolean retry, final long autoGeneratedIdTimestamp) { if (randomBoolean()) { - return new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, - VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, retry); + return randomAppendOnlyPrimary(doc, retry, autoGeneratedIdTimestamp); + } else { + return randomAppendOnlyReplica(doc, retry, autoGeneratedIdTimestamp, 0); } - return new Engine.Index(newUid(doc), doc, 0, 0, 1, VersionType.EXTERNAL, + } + + public Engine.Index randomAppendOnlyPrimary(ParsedDocument doc, boolean retry, final long autoGeneratedIdTimestamp) { + return new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, + VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, retry); + } + + public Engine.Index randomAppendOnlyReplica(ParsedDocument doc, boolean retry, final long autoGeneratedIdTimestamp, final long seqNo) { + return new Engine.Index(newUid(doc), doc, seqNo, 2, 1, VersionType.EXTERNAL, Engine.Operation.Origin.REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, retry); } @@ -2802,10 +3024,18 @@ public void testRetryConcurrently() throws InterruptedException, IOException { Thread[] thread = new Thread[randomIntBetween(3, 5)]; int numDocs = randomIntBetween(1000, 10000); List docs = new ArrayList<>(); + final boolean primary = randomBoolean(); for (int i = 0; i < numDocs; i++) { final ParsedDocument doc = testParsedDocument(Integer.toString(i), "test", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); - Engine.Index originalIndex = randomAppendOnly(doc, false, i); - Engine.Index retryIndex = randomAppendOnly(doc, true, i); + final Engine.Index originalIndex; + final Engine.Index retryIndex; + if (primary) { + originalIndex = randomAppendOnlyPrimary(doc, false, i); + retryIndex = randomAppendOnlyPrimary(doc, true, i); + } else { + originalIndex = randomAppendOnlyReplica(doc, false, i, i * 2); + retryIndex = randomAppendOnlyReplica(doc, true, i, i * 2); + } docs.add(originalIndex); docs.add(retryIndex); } @@ -2834,14 +3064,26 @@ public void testRetryConcurrently() throws InterruptedException, IOException { for (int i = 0; i < thread.length; i++) { thread[i].join(); } - assertEquals(0, engine.getNumVersionLookups()); - assertEquals(0, engine.getNumIndexVersionsLookups()); + if (primary) { + assertEquals(0, engine.getNumVersionLookups()); + assertEquals(0, engine.getNumIndexVersionsLookups()); + } else { + // we don't really know what order the operations will arrive and thus can't predict how many + // version lookups will be needed + assertThat(engine.getNumIndexVersionsLookups(), lessThanOrEqualTo(engine.getNumVersionLookups())); + } engine.refresh("test"); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); assertEquals(numDocs, topDocs.totalHits); } - assertTrue(engine.indexWriterHasDeletions()); + if (primary) { + // primaries rely on lucene dedup and may index the same document twice + assertTrue(engine.indexWriterHasDeletions()); + } else { + // replicas rely on seq# based dedup and in this setup (same seq#) should never rely on lucene + assertFalse(engine.indexWriterHasDeletions()); + } } public void testEngineMaxTimestampIsInitialized() throws IOException { @@ -3001,7 +3243,7 @@ public void testSequenceIDs() throws Exception { seqID = getSequenceID(engine, new Engine.Get(false, newUid(doc))); logger.info("--> got seqID: {}", seqID); assertThat(seqID.v1(), equalTo(0L)); - assertThat(seqID.v2(), equalTo(0L)); + assertThat(seqID.v2(), equalTo(2L)); // Index the same document again document = testDocumentWithTextField(); @@ -3013,13 +3255,13 @@ public void testSequenceIDs() throws Exception { seqID = getSequenceID(engine, new Engine.Get(false, newUid(doc))); logger.info("--> got seqID: {}", seqID); assertThat(seqID.v1(), equalTo(1L)); - assertThat(seqID.v2(), equalTo(0L)); + assertThat(seqID.v2(), equalTo(2L)); // Index the same document for the third time, this time changing the primary term document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); doc = testParsedDocument("1", "test", null, document, B_1, null); - engine.index(new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1, + engine.index(new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 3, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false)); engine.refresh("test"); @@ -3027,7 +3269,7 @@ public void testSequenceIDs() throws Exception { seqID = getSequenceID(engine, new Engine.Get(false, newUid(doc))); logger.info("--> got seqID: {}", seqID); assertThat(seqID.v1(), equalTo(2L)); - assertThat(seqID.v2(), equalTo(1L)); + assertThat(seqID.v2(), equalTo(3L)); // we can query by the _seq_no Engine.Searcher searchResult = engine.acquireSearcher("test"); @@ -3113,7 +3355,7 @@ public long generateSeqNo() { } public void testSequenceNumberAdvancesToMaxSeqNoOnEngineOpenOnReplica() throws IOException { - final long v = Versions.MATCH_ANY; + final long v = 1; final VersionType t = VersionType.EXTERNAL; final long ts = IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP; final int docs = randomIntBetween(1, 32); @@ -3278,16 +3520,20 @@ public long generateSeqNo() { * second is the primary term. */ private Tuple getSequenceID(Engine engine, Engine.Get get) throws EngineException { - final Searcher searcher = engine.acquireSearcher("get"); - try { - long seqNum = Versions.loadSeqNo(searcher.reader(), get.uid()); - long primaryTerm = Versions.loadPrimaryTerm(searcher.reader(), get.uid()); - return new Tuple(seqNum, primaryTerm); + try (Searcher searcher = engine.acquireSearcher("get")) { + final long primaryTerm; + final long seqNo; + DocIdAndSeqNo docIdAndSeqNo = VersionsAndSeqNoResolver.loadDocIdAndSeqNo(searcher.reader(), get.uid()); + if (docIdAndSeqNo == null) { + primaryTerm = 0; + seqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; + } else { + seqNo = docIdAndSeqNo.seqNo; + primaryTerm = VersionsAndSeqNoResolver.loadPrimaryTerm(docIdAndSeqNo); + } + return new Tuple<>(seqNo, primaryTerm); } catch (Exception e) { - Releasables.closeWhileHandlingException(searcher); throw new EngineException(shardId, "unable to retrieve sequence id", e); - } finally { - searcher.close(); } } diff --git a/core/src/test/java/org/elasticsearch/index/engine/LiveVersionMapTests.java b/core/src/test/java/org/elasticsearch/index/engine/LiveVersionMapTests.java index 9161bc413c86c..97799f8c46a62 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/LiveVersionMapTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/LiveVersionMapTests.java @@ -33,7 +33,7 @@ public void testRamBytesUsed() throws Exception { for (int i = 0; i < 100000; ++i) { BytesRefBuilder uid = new BytesRefBuilder(); uid.copyChars(TestUtil.randomSimpleString(random(), 10, 20)); - VersionValue version = new VersionValue(randomLong()); + VersionValue version = new VersionValue(randomLong(), randomLong(), randomLong()); map.putUnderLock(uid.toBytesRef(), version); } long actualRamBytesUsed = RamUsageTester.sizeOf(map); @@ -48,7 +48,7 @@ public void testRamBytesUsed() throws Exception { for (int i = 0; i < 100000; ++i) { BytesRefBuilder uid = new BytesRefBuilder(); uid.copyChars(TestUtil.randomSimpleString(random(), 10, 20)); - VersionValue version = new VersionValue(randomLong()); + VersionValue version = new VersionValue(randomLong(), randomLong(), randomLong()); map.putUnderLock(uid.toBytesRef(), version); } actualRamBytesUsed = RamUsageTester.sizeOf(map); diff --git a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 53708b28dfb0f..f709f4349512b 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -173,7 +173,7 @@ private ParseContext.Document testDocument() { private ParsedDocument testParsedDocument(String id, String type, String routing, ParseContext.Document document, BytesReference source, Mapping mappingsUpdate) { Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); - SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID(); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); document.add(uidField); document.add(versionField); document.add(seqID.seqNo); diff --git a/core/src/test/java/org/elasticsearch/index/engine/VersionValueTests.java b/core/src/test/java/org/elasticsearch/index/engine/VersionValueTests.java index e66f55ff676c4..3b953edece1b4 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/VersionValueTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/VersionValueTests.java @@ -20,19 +20,17 @@ package org.elasticsearch.index.engine; import org.apache.lucene.util.RamUsageTester; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.TranslogTests; import org.elasticsearch.test.ESTestCase; public class VersionValueTests extends ESTestCase { public void testRamBytesUsed() { - VersionValue versionValue = new VersionValue(randomLong()); + VersionValue versionValue = new VersionValue(randomLong(), randomLong(), randomLong()); assertEquals(RamUsageTester.sizeOf(versionValue), versionValue.ramBytesUsed()); } public void testDeleteRamBytesUsed() { - DeleteVersionValue versionValue = new DeleteVersionValue(randomLong(), randomLong()); + DeleteVersionValue versionValue = new DeleteVersionValue(randomLong(), randomLong(), randomLong(), randomLong()); assertEquals(RamUsageTester.sizeOf(versionValue), versionValue.ramBytesUsed()); } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index 97c96c8af12f7..b8f178ca53658 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -107,7 +107,7 @@ private ParsedDocument testParsedDocument(String id, String type, String routing Mapping mappingUpdate) { Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); - SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID(); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); document.add(uidField); document.add(versionField); document.add(seqID.seqNo); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index b106a308098c9..05920621e9fb5 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -551,7 +551,7 @@ private ParsedDocument testParsedDocument(String id, String type, String routing ParseContext.Document document, BytesReference source, Mapping mappingUpdate) { Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); - SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID(); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); document.add(uidField); document.add(versionField); document.add(seqID.seqNo); diff --git a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index 846ca6be2014a..b7e20cf75c83c 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -332,7 +332,7 @@ private Engine.IndexResult index(String id, String testFieldValue) throws IOExce document.add(new TextField("test", testFieldValue, Field.Store.YES)); Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", Versions.MATCH_ANY); - SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID(); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); document.add(uidField); document.add(versionField); document.add(seqID.seqNo); diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index e47a5652b2431..197be925173d1 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -2032,7 +2032,7 @@ public static Translog.Location randomTranslogLocation() { public void testTranslogOpSerialization() throws Exception { BytesReference B_1 = new BytesArray(new byte[]{1}); - SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID(); + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); assert Version.CURRENT.major <= 6 : "Using UNASSIGNED_SEQ_NO can be removed in 7.0, because 6.0+ nodes have actual sequence numbers"; long randomSeqNum = randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : randomNonNegativeLong(); long randomPrimaryTerm = randomBoolean() ? 0 : randomNonNegativeLong(); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 3858b6647f988..40a92b11e7372 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -49,7 +49,6 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.SegmentsStats; -import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -205,7 +204,7 @@ private Engine.Index getIndex(final String id) { document.add(new TextField("test", "test", Field.Store.YES)); final Field uidField = new Field("_uid", Uid.createUid(type, id), UidFieldMapper.Defaults.FIELD_TYPE); final Field versionField = new NumericDocValuesField("_version", Versions.MATCH_ANY); - final SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID(); + final SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); document.add(uidField); document.add(versionField); document.add(seqID.seqNo); From baa7e51f3eb9faae09e8bc8f6cda683638d4a79d Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 7 Mar 2017 15:36:34 -0800 Subject: [PATCH 02/14] fix testAppendWhileRecovering --- .../index/replication/IndexLevelReplicationTests.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index 39d8778c2a4b8..7a11f89b73b2e 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -98,13 +98,9 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa future.get(); thread.join(); shards.assertAllEqual(numDocs); - Engine engine = IndexShardTests.getEngineFromShard(replica); - assertEquals("expected at no version lookups ", InternalEngineTests.getNumVersionLookups((InternalEngine) engine), 0); - for (IndexShard shard : shards) { - engine = IndexShardTests.getEngineFromShard(shard); - assertEquals(0, InternalEngineTests.getNumIndexVersionsLookups((InternalEngine) engine)); - assertEquals(0, InternalEngineTests.getNumVersionLookups((InternalEngine) engine)); - } + Engine engine = IndexShardTests.getEngineFromShard(shards.getPrimary()); + assertEquals(0, InternalEngineTests.getNumIndexVersionsLookups((InternalEngine) engine)); + assertEquals(0, InternalEngineTests.getNumVersionLookups((InternalEngine) engine)); } } From 14b4b4b450dd3498ba819240643bcfe1279db994 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 7 Mar 2017 15:45:59 -0800 Subject: [PATCH 03/14] lingering reference to SortedNumericDocValuesField --- .../java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java index ff7c164dceb69..9612d94e661ce 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/SeqNoFieldMapper.java @@ -22,7 +22,6 @@ import org.apache.lucene.document.Field; import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.NumericDocValuesField; -import org.apache.lucene.document.SortedNumericDocValuesField; import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; @@ -264,7 +263,7 @@ public void postParse(ParseContext context) throws IOException { for (int i = 1; i < context.docs().size(); i++) { final Document doc = context.docs().get(i); doc.add(new LongPoint(NAME, 1)); - doc.add(new SortedNumericDocValuesField(NAME, 1L)); + doc.add(new NumericDocValuesField(NAME, 1L)); doc.add(new NumericDocValuesField(PRIMARY_TERM_NAME, 0L)); } } From 2b835f73175808940605ab92f971978a6893acee Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 7 Mar 2017 19:32:15 -0800 Subject: [PATCH 04/14] currectly load sequence numbers if the index doesn't have the right doc values --- .../uid/PerThreadIDAndVersionSeqNoLookup.java | 5 +- .../index/engine/InternalEngineTests.java | 63 +++++++++++++++++-- 2 files changed, 62 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionSeqNoLookup.java b/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionSeqNoLookup.java index 20ea37dbcaa98..bfc110cd9febc 100644 --- a/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionSeqNoLookup.java +++ b/core/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDAndVersionSeqNoLookup.java @@ -34,6 +34,7 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; +import org.elasticsearch.index.seqno.SequenceNumbersService; import java.io.IOException; @@ -110,9 +111,9 @@ private int getDocID(BytesRef id, Bits liveDocs) throws IOException { /** Return null if id is not found. */ public DocIdAndSeqNo lookupSequenceNo(BytesRef id, Bits liveDocs, LeafReaderContext context) throws IOException { assert context.reader().getCoreCacheKey().equals(readerKey); - int docID = seqNos == null ? DocIdSetIterator.NO_MORE_DOCS : getDocID(id, liveDocs); + int docID = getDocID(id, liveDocs); if (docID != DocIdSetIterator.NO_MORE_DOCS) { - return new DocIdAndSeqNo(docID, seqNos.get(docID), context); + return new DocIdAndSeqNo(docID, seqNos == null ? SequenceNumbersService.UNASSIGNED_SEQ_NO : seqNos.get(docID), context); } else { return null; } diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 0119aea0663c2..a05ef4b415074 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1476,6 +1476,7 @@ public void testVersioningCreateExistsExceptionWithFlush() throws IOException { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } + // simulate a new replica getting data from an old priamry public void testVersioningReplicaConflict1() throws IOException { IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", Settings.builder() .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us @@ -1495,12 +1496,12 @@ public void testVersioningReplicaConflict1() throws IOException { final Engine.IndexResult v1Result = engine.index(v1Index); assertThat(v1Result.getVersion(), equalTo(1L)); - final Engine.Index v2Index = indexForDoc(doc); + final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2") , B_1, null); + final Engine.Index v2Index = indexForDoc(docV2); final Engine.IndexResult v2Result = engine.index(v2Index); assertThat(v2Result.getVersion(), equalTo(2L)); // apply the second index to the replica, should work fine - final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2") , B_1, null); final Engine.Index replicaV2Index = new Engine.Index( newUid(docV2), docV2, @@ -1633,6 +1634,60 @@ public void testVersioningReplicaConflict2() throws IOException { } } + // This models a new primary and a new replica, running on old data but generating seq# for the new data + public void testVersioningReplicaSomeDocsWithSeqNoSomeWithout() throws IOException { + IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", 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.V_5_4_0_UNRELEASED) + .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), + between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) + .build()); + + try (Store oldReplicaStore = createStore(); + InternalEngine replicaEngine = + createEngine(oldSettings, oldReplicaStore, createTempDir("translog-old-replica"), newMergePolicy())) { + + final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField("v_1"), B_1, null); + // insert the first operation into replica, simulating an old primary + final Engine.Index replicaV1Index = new Engine.Index(newUid(doc), doc, + SequenceNumbersService.UNASSIGNED_SEQ_NO, // old op, no seq no + 0, 1, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); + final Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); + assertFalse(replicaV1Result.hasFailure()); + assertFalse(replicaV1Result.isCreated()); + assertThat(replicaV1Result.getVersion(), equalTo(1L)); + + switch (randomIntBetween(1, 3)) { + case 1: + replicaEngine.refresh("test"); + break; + case 2: + replicaEngine.flush(); + break; + default: + break; + } + + final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2"), B_1, null); + + final Engine.Index replicaV2Index = new Engine.Index(newUid(docV2), docV2, + 10, // a real sequence number + 1, 2, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), + REPLICA, 0, -1, false); + + final Engine.IndexResult replicaV2Result = replicaEngine.index(replicaV2Index); + assertThat(replicaV2Result.getVersion(), equalTo(2L)); + + replicaEngine.refresh("test"); + try (Searcher searchResult = replicaEngine.acquireSearcher("test")) { + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits( + new TermQuery(new Term("value", "v_2")), 1)); + } + } + } + public void testSeqNoReplicaConflict1() throws IOException { final ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); @@ -1641,13 +1696,13 @@ public void testSeqNoReplicaConflict1() throws IOException { assertThat(v1Result.getVersion(), equalTo(1L)); assertThat(v1Result.getSeqNo(), equalTo(0L)); - final Engine.Index v2Index = indexForDoc(doc); + final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2") , B_1, null); + final Engine.Index v2Index = indexForDoc(docV2); final Engine.IndexResult v2Result = engine.index(v2Index); assertThat(v2Result.getVersion(), equalTo(2L)); assertThat(v2Result.getSeqNo(), equalTo(1L)); // apply the second index to the replica, should work fine - final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2") , B_1, null); final Engine.Index replicaV2Index = new Engine.Index( newUid(docV2), docV2, From 6b9e3caab98b9c06b0abeb82a23fce1dd3e510b0 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 8 Mar 2017 11:02:49 -0800 Subject: [PATCH 05/14] share more version look up code --- .../index/engine/InternalEngine.java | 100 +++++++++--------- .../index/engine/InternalEngineTests.java | 2 +- 2 files changed, 53 insertions(+), 49 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index c09124e545024..96e26ccfa99b0 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -447,31 +447,32 @@ private LuceneOpStatus checkLuceneOpStatusBasedOnSeq(final Operation op) throws return status; } - private LuceneOpStatus checkLuceneOpStatusBasedOnVersions(final Operation op) throws IOException { - assert op.seqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO; - assert op.version() >= 0; - final LuceneOpStatus status; - final long currentVersion; - final VersionValue versionValue = versionMap.getUnderLock(op.uid()); - if (versionValue != null) { - currentVersion = versionValue.getVersion(); - } else { - // load from index - currentVersion = loadCurrentVersionFromIndex(op.uid()); - + /** resolves the current version of the document, returning null if not found */ + private VersionValue resolveDocVersion(final Operation op) throws IOException { + assert incrementVersionLookup(); + VersionValue versionValue = versionMap.getUnderLock(op.uid()); + if (versionValue == null) { + assert incrementIndexVersionLookup(); + final long currentVersion = loadCurrentVersionFromIndex(op.uid()); + if (currentVersion != Versions.NOT_FOUND) { + versionValue = new VersionValue(currentVersion, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0L); + } + } else if (engineConfig.isEnableGcDeletes() && versionValue.isDelete() && + (engineConfig.getThreadPool().relativeTimeInMillis() - versionValue.getTime()) > getGcDeletesInMillis()) { + versionValue = null; } - return op.version() > currentVersion ? LuceneOpStatus.STALE : LuceneOpStatus.NEWER_OR_EQUAL; + return versionValue; } - - private long checkDeletedAndGCed(VersionValue versionValue) { - long currentVersion; - if (engineConfig.isEnableGcDeletes() && versionValue.isDelete() && (engineConfig.getThreadPool().relativeTimeInMillis() - versionValue.getTime()) > getGcDeletesInMillis()) { - currentVersion = Versions.NOT_FOUND; // deleted, and GC + private LuceneOpStatus checkLuceneOpStatusBasedOnVersions(final Operation op) throws IOException { + assert op.seqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO; + assert op.version() >= 0; + final VersionValue versionValue = resolveDocVersion(op); + if (versionValue == null) { + return LuceneOpStatus.NOT_FOUND; } else { - currentVersion = versionValue.getVersion(); + return op.version() > versionValue.getVersion() ? LuceneOpStatus.STALE : LuceneOpStatus.NEWER_OR_EQUAL; } - return currentVersion; } private boolean canOptimizeAddDocument(Index index) { @@ -590,19 +591,19 @@ public IndexResult index(Index index) throws IOException { addToLucene = true; } else { // resolves incoming version - final VersionValue versionValue = versionMap.getUnderLock(index.uid()); + final VersionValue versionValue = resolveDocVersion(index); final long currentVersion; assert incrementVersionLookup(); if (versionValue == null) { - currentVersion = loadCurrentVersionFromIndex(index.uid()); - currentNotFoundOrDeleted = currentVersion == Versions.NOT_FOUND; - useLuceneUpdateDocument = currentVersion != Versions.NOT_FOUND; - } else { - currentVersion = checkDeletedAndGCed(versionValue); - currentNotFoundOrDeleted = versionValue.isDelete(); // nocommit: previously we based this on whether version is not found. That can mean however // that the last op was deleted, but was also GCed. So presumably it is safe // to sue use add after a delete. + currentVersion = Versions.NOT_FOUND; + currentNotFoundOrDeleted = true; + useLuceneUpdateDocument = false; + } else { + currentVersion = versionValue.getVersion(); + currentNotFoundOrDeleted = versionValue.isDelete(); useLuceneUpdateDocument = versionValue.isDelete() == false; // last operation is not delete } if (index.versionType().isVersionConflictForWrites(currentVersion, index.version(), currentNotFoundOrDeleted)) { @@ -621,7 +622,6 @@ public IndexResult index(Index index) throws IOException { } } else if (canOptimizeAddDocument && mayHaveBeenIndexedBefore(index) == false) { // no need to deal with out of order delivery - we never saw this one - // nocommit: verify this is tested! useLuceneUpdateDocument = false; currentNotFoundOrDeleted = true; assert index.version() == 1L : "can optimize on replicas but incoming version is [" + index.version() + "]"; @@ -785,37 +785,38 @@ public DeleteResult delete(Delete delete) throws IOException { try (ReleasableLock ignored = readLock.acquire(); Releasable ignored2 = acquireLock(delete.uid())) { ensureOpen(); lastWriteNanos = delete.startTime(); - final boolean deleteInLucene; - final boolean currentFound; + // TODO: we can separate the two in the case that a document is already deleted but we want to update + // the version. For now, prefer to keep the code paths simpler (they are hairy enough) at the expense + // of a rare double delete + final boolean performOperation; + final boolean currentlyDeleted; final long seqNoForIndexing; final long versionForIndexing; Optional earlyResultOnPreflightError = Optional.empty(); if (delete.origin() == Operation.Origin.PRIMARY) { // resolve operation from external to internal - final VersionValue versionValue = versionMap.getUnderLock(delete.uid()); + final VersionValue versionValue = resolveDocVersion(delete); assert incrementVersionLookup(); final long currentVersion; - final boolean currentIsDeleted; if (versionValue == null) { - currentVersion = loadCurrentVersionFromIndex(delete.uid()); - currentIsDeleted = currentVersion == Versions.NOT_FOUND; + currentVersion = Versions.NOT_FOUND; + currentlyDeleted = true; } else { - currentVersion = checkDeletedAndGCed(versionValue); - currentIsDeleted = versionValue.isDelete(); + currentVersion = versionValue.getVersion(); + currentlyDeleted = versionValue.isDelete(); } - currentFound = currentVersion != Versions.NOT_FOUND; - if (delete.versionType().isVersionConflictForWrites(currentVersion, delete.version(), currentIsDeleted)) { + if (delete.versionType().isVersionConflictForWrites(currentVersion, delete.version(), currentlyDeleted)) { earlyResultOnPreflightError = Optional.of(new DeleteResult( - new VersionConflictEngineException(shardId, delete, currentVersion, currentIsDeleted), + new VersionConflictEngineException(shardId, delete, currentVersion, currentlyDeleted), currentVersion, delete.seqNo() )); - deleteInLucene = false; + performOperation = false; seqNoForIndexing = SequenceNumbersService.UNASSIGNED_SEQ_NO; versionForIndexing = -1; } else { versionForIndexing = delete.versionType().updateVersion(currentVersion, delete.version()); seqNoForIndexing = seqNoService().generateSeqNo(); - deleteInLucene = true; + performOperation = true; } } else { // drop out of order operations @@ -829,24 +830,27 @@ public DeleteResult delete(Delete delete) throws IOException { luceneOpStatus = checkLuceneOpStatusBasedOnVersions(delete); } // the found vs deleted distinction is not relevant on replicas, just set it - currentFound = luceneOpStatus != LuceneOpStatus.NOT_FOUND; + currentlyDeleted = luceneOpStatus == LuceneOpStatus.NOT_FOUND; seqNoForIndexing = delete.seqNo(); versionForIndexing = delete.version(); - deleteInLucene = luceneOpStatus != LuceneOpStatus.NEWER_OR_EQUAL; + performOperation = luceneOpStatus != LuceneOpStatus.NEWER_OR_EQUAL; } if (earlyResultOnPreflightError.isPresent()) { deleteResult = earlyResultOnPreflightError.get(); + assert performOperation == false; } else { - if (deleteInLucene) { - // 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 (performOperation) { + if (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()); + } versionMap.putUnderLock(delete.uid().bytes(), new DeleteVersionValue(versionForIndexing, seqNoForIndexing, delete.primaryTerm(), engineConfig.getThreadPool().relativeTimeInMillis())); } - deleteResult = new DeleteResult(versionForIndexing, seqNoForIndexing, currentFound); + deleteResult = new DeleteResult(versionForIndexing, seqNoForIndexing, currentlyDeleted == false); } if (!deleteResult.hasFailure()) { Translog.Location location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index a05ef4b415074..f1c1247653900 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1655,7 +1655,7 @@ public void testVersioningReplicaSomeDocsWithSeqNoSomeWithout() throws IOExcepti 0, 1, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); final Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); assertFalse(replicaV1Result.hasFailure()); - assertFalse(replicaV1Result.isCreated()); + assertTrue(replicaV1Result.isCreated()); assertThat(replicaV1Result.getVersion(), equalTo(1L)); switch (randomIntBetween(1, 3)) { From cde619ce86d80a6c1a9b4724338ab389b4f209e9 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 8 Mar 2017 13:22:02 -0800 Subject: [PATCH 06/14] remove no commit --- .../java/org/elasticsearch/index/engine/InternalEngine.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 96e26ccfa99b0..69103d8a27899 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -595,9 +595,6 @@ public IndexResult index(Index index) throws IOException { final long currentVersion; assert incrementVersionLookup(); if (versionValue == null) { - // nocommit: previously we based this on whether version is not found. That can mean however - // that the last op was deleted, but was also GCed. So presumably it is safe - // to sue use add after a delete. currentVersion = Versions.NOT_FOUND; currentNotFoundOrDeleted = true; useLuceneUpdateDocument = false; From d81bc8c45b9535d76e8595244135567d06e34d55 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 8 Mar 2017 18:35:46 -0800 Subject: [PATCH 07/14] minor tweaks --- .../index/engine/InternalEngine.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 69103d8a27899..15f2a5283189e 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -406,7 +406,7 @@ public GetResult get(Get get, Function searcherFactory) throws enum LuceneOpStatus { NEWER_OR_EQUAL, - STALE, + OLDER, NOT_FOUND } @@ -418,7 +418,7 @@ private LuceneOpStatus checkLuceneOpStatusBasedOnSeq(final Operation op) throws if (versionValue != null) { if (op.seqNo() > versionValue.getSeqNo() || (op.seqNo() == versionValue.getSeqNo() && op.primaryTerm() > versionValue.getTerm())) - status = LuceneOpStatus.STALE; + status = LuceneOpStatus.OLDER; else { status = LuceneOpStatus.NEWER_OR_EQUAL; } @@ -430,12 +430,12 @@ private LuceneOpStatus checkLuceneOpStatusBasedOnSeq(final Operation op) throws if (docAndSeqNo == null) { status = LuceneOpStatus.NOT_FOUND; } else if (op.seqNo() > docAndSeqNo.seqNo) { - status = LuceneOpStatus.STALE; + status = LuceneOpStatus.OLDER; } else if (op.seqNo() == docAndSeqNo.seqNo) { // load term to tie break final long existingTerm = VersionsAndSeqNoResolver.loadPrimaryTerm(docAndSeqNo); if (op.primaryTerm() > existingTerm) { - status = LuceneOpStatus.STALE; + status = LuceneOpStatus.OLDER; } else { status = LuceneOpStatus.NEWER_OR_EQUAL; } @@ -471,7 +471,7 @@ private LuceneOpStatus checkLuceneOpStatusBasedOnVersions(final Operation op) th if (versionValue == null) { return LuceneOpStatus.NOT_FOUND; } else { - return op.version() > versionValue.getVersion() ? LuceneOpStatus.STALE : LuceneOpStatus.NEWER_OR_EQUAL; + return op.version() > versionValue.getVersion() ? LuceneOpStatus.OLDER : LuceneOpStatus.NEWER_OR_EQUAL; } } @@ -593,7 +593,6 @@ public IndexResult index(Index index) throws IOException { // resolves incoming version final VersionValue versionValue = resolveDocVersion(index); final long currentVersion; - assert incrementVersionLookup(); if (versionValue == null) { currentVersion = Versions.NOT_FOUND; currentNotFoundOrDeleted = true; @@ -839,8 +838,8 @@ public DeleteResult delete(Delete delete) throws IOException { } else { if (performOperation) { if (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. + // 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()); } versionMap.putUnderLock(delete.uid().bytes(), @@ -861,6 +860,7 @@ public DeleteResult delete(Delete delete) throws IOException { deleteResult.setTook(System.nanoTime() - delete.startTime()); deleteResult.freeze(); } catch (RuntimeException | IOException e) { + assert indexWriter.getTragicException() != null : "unexpected exception in delete\n" + e; try { maybeFailEngine("index", e); } catch (Exception inner) { From a27352ddb5d2e701879ad23fe9b28ed9eaeb05f6 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 10 Mar 2017 14:42:02 -0800 Subject: [PATCH 08/14] internal versioning test for primary. Fix found flag on version conflict in deletes --- .../elasticsearch/index/engine/Engine.java | 4 +- .../index/engine/InternalEngine.java | 31 ++- .../index/engine/LiveVersionMap.java | 5 + .../index/engine/InternalEngineTests.java | 234 +++++++++++++++++- 4 files changed, 252 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 2f4d6243454ff..59655abf2894c 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -421,9 +421,9 @@ public DeleteResult(long version, long seqNo, boolean found) { this.found = found; } - public DeleteResult(Exception failure, long version, long seqNo) { + public DeleteResult(Exception failure, long version, long seqNo, boolean found) { super(Operation.TYPE.DELETE, failure, version, seqNo); - this.found = false; + this.found = found; } public boolean isFound() { diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 15f2a5283189e..511b73b639547 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -578,7 +578,7 @@ public IndexResult index(Index index) throws IOException { final boolean canOptimizeAddDocument = canOptimizeAddDocument(index); final long seqNoForIndexing; final long versionForIndexing; - final boolean addToLucene; + final boolean performOperation; Optional earlyResultOnPreFlightError = Optional.empty(); if (index.origin() == Operation.Origin.PRIMARY) { @@ -588,7 +588,7 @@ public IndexResult index(Index index) throws IOException { versionForIndexing = 1L; currentNotFoundOrDeleted = true; seqNoForIndexing = seqNoService().generateSeqNo(); - addToLucene = true; + performOperation = true; } else { // resolves incoming version final VersionValue versionValue = resolveDocVersion(index); @@ -609,11 +609,11 @@ public IndexResult index(Index index) throws IOException { )); versionForIndexing = Versions.NOT_FOUND; seqNoForIndexing = SequenceNumbersService.UNASSIGNED_SEQ_NO; - addToLucene = false; + performOperation = false; } else { versionForIndexing = index.versionType().updateVersion(currentVersion, index.version()); seqNoForIndexing = seqNoService().generateSeqNo(); - addToLucene = true; + performOperation = true; } } } else if (canOptimizeAddDocument && mayHaveBeenIndexedBefore(index) == false) { @@ -623,7 +623,7 @@ public IndexResult index(Index index) throws IOException { assert index.version() == 1L : "can optimize on replicas but incoming version is [" + index.version() + "]"; seqNoForIndexing = index.seqNo(); versionForIndexing = index.version(); - addToLucene = true; + performOperation = true; } else { // drop out of order operations assert index.versionType().versionTypeForReplicationAndRecovery() == index.versionType() : @@ -635,18 +635,21 @@ public IndexResult index(Index index) throws IOException { } else { luceneOpStatus = checkLuceneOpStatusBasedOnVersions(index); } - useLuceneUpdateDocument = luceneOpStatus != LuceneOpStatus.NOT_FOUND; + // 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 currentNotFoundOrDeleted = luceneOpStatus == LuceneOpStatus.NOT_FOUND; + useLuceneUpdateDocument = luceneOpStatus != LuceneOpStatus.NOT_FOUND; seqNoForIndexing = index.seqNo(); versionForIndexing = index.version(); - addToLucene = luceneOpStatus != LuceneOpStatus.NEWER_OR_EQUAL; + performOperation = luceneOpStatus != LuceneOpStatus.NEWER_OR_EQUAL; } final IndexResult indexResult; if (earlyResultOnPreFlightError.isPresent()) { indexResult = earlyResultOnPreFlightError.get(); assert indexResult.hasFailure(); - } else if (addToLucene) { + } else if (performOperation) { indexResult = indexIntoLucene(index, seqNoForIndexing, versionForIndexing, currentNotFoundOrDeleted, useLuceneUpdateDocument); } else { @@ -804,7 +807,7 @@ public DeleteResult delete(Delete delete) throws IOException { if (delete.versionType().isVersionConflictForWrites(currentVersion, delete.version(), currentlyDeleted)) { earlyResultOnPreflightError = Optional.of(new DeleteResult( new VersionConflictEngineException(shardId, delete, currentVersion, currentlyDeleted), - currentVersion, delete.seqNo() + currentVersion, delete.seqNo(), currentlyDeleted == false )); performOperation = false; seqNoForIndexing = SequenceNumbersService.UNASSIGNED_SEQ_NO; @@ -823,9 +826,12 @@ public DeleteResult delete(Delete delete) throws IOException { if (delete.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { luceneOpStatus = checkLuceneOpStatusBasedOnSeq(delete); } else { + // added a comment luceneOpStatus = checkLuceneOpStatusBasedOnVersions(delete); } - // the found vs deleted distinction is not relevant on replicas, just set it + // 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 currentlyDeleted = luceneOpStatus == LuceneOpStatus.NOT_FOUND; seqNoForIndexing = delete.seqNo(); versionForIndexing = delete.version(); @@ -1140,6 +1146,11 @@ private void pruneDeletedTombstones() { lastDeleteVersionPruneTimeMSec = timeMSec; } + // testing + void clearDeletedTombstones() { + versionMap.clearTombstones(); + } + @Override public void forceMerge(final boolean flush, int maxNumSegments, boolean onlyExpungeDeletes, final boolean upgrade, final boolean upgradeOnlyAncientSegments) throws EngineException, IOException { diff --git a/core/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java b/core/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java index cd27635091e2d..7233420309c72 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java +++ b/core/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java @@ -234,6 +234,11 @@ Iterable> getAllTombstones() { return tombstones.entrySet(); } + /** clears all tombstones ops */ + void clearTombstones() { + tombstones.clear(); + } + /** Called when this index is closed. */ synchronized void clear() { maps = new Maps(); diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index f1c1247653900..34f20950e8ed2 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -158,10 +158,12 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; import java.util.function.LongSupplier; import java.util.function.Supplier; import static java.util.Collections.emptyMap; +import static java.util.Collections.shuffle; import static org.elasticsearch.index.engine.Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PEER_RECOVERY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; @@ -1476,6 +1478,216 @@ public void testVersioningCreateExistsExceptionWithFlush() throws IOException { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } + + protected List generateSingleDocHistory(boolean forReplica, boolean externalVersioning, boolean partialOldPrimary) { + final int numOfOps = randomIntBetween(2, 20); + final List ops = new ArrayList<>(); + final Term id = newUid(Uid.createUid("test", "1")); + final int startWithSeqNo; + if (partialOldPrimary) { + startWithSeqNo = randomBoolean() ? numOfOps - 1 : randomIntBetween(0, numOfOps - 1); + } else { + startWithSeqNo = 0; + } + final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); + final VersionType versionType = externalVersioning ? VersionType.EXTERNAL : VersionType.INTERNAL; + for (int i = 0; i < numOfOps; i++) { + final Engine.Operation op; + if (randomBoolean()) { + op = new Engine.Index(id, testParsedDocument("1", "test", null, testDocumentWithTextField("v_" + i), B_1, null), + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbersService.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? 2 : 1, + forReplica || externalVersioning ? i : Versions.MATCH_ANY, + forReplica ? versionType.versionTypeForReplicationAndRecovery() : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis(), -1, false + ); + } else { + op = new Engine.Delete("test", "1", id, + forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbersService.UNASSIGNED_SEQ_NO, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? 2 : 1, + forReplica || externalVersioning ? i : Versions.MATCH_ANY, + forReplica ? versionType.versionTypeForReplicationAndRecovery() : versionType, + forReplica ? REPLICA : PRIMARY, + System.currentTimeMillis()); + } + ops.add(op); + } + return ops; + } + + public void testOutOfOrderDocsOnReplica() throws IOException { + final List ops = generateSingleDocHistory(true, true, false); + testOpsOnReplica(ops, replicaEngine); + } + + public void testOutOfOrderDocsOnReplicaOldPrimary() throws IOException { + IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", 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.V_5_4_0_UNRELEASED) + .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), + between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) + .build()); + + try (Store oldReplicaStore = createStore(); + InternalEngine replicaEngine = + createEngine(oldSettings, oldReplicaStore, createTempDir("translog-old-replica"), newMergePolicy())) { + final List ops = generateSingleDocHistory(true, true, true); + testOpsOnReplica(ops, replicaEngine); + } + } + + public void testInternalVersioningOnPrimary() throws IOException { + final List ops = generateSingleDocHistory(false, false, false); + String lastFieldValue = null; + long lastOpVersion = Versions.NOT_FOUND; + boolean docDeleted = true; + BiFunction indexWithVersion = (version, index) -> new Engine.Index(index.uid(), index.parsedDoc(), + index.seqNo(), index.primaryTerm(), version, index.versionType(), index.origin(), index.startTime(), + index.getAutoGeneratedIdTimestamp(), index.isRetry()); + BiFunction delWithVersion = (version, delete) -> new Engine.Delete(delete.type(), delete.id(), + delete.uid(), delete.seqNo(), delete.primaryTerm(), version, delete.versionType(), delete.origin(), delete.startTime()); + for (Engine.Operation op : ops) { + final boolean versionConflict = rarely(); + final boolean versionedOp = versionConflict || randomBoolean(); + logger.info("performing [{}]{}", + op.operationType().name().charAt(0), versionConflict ? " (conflict)" : versionedOp ? " (versioned)" : ""); + if (op instanceof Engine.Index) { + final Engine.Index index = (Engine.Index) op; + if (versionConflict) { + // generate a conflict + Engine.IndexResult result = engine.index(indexWithVersion.apply(lastOpVersion - 1, index)); + assertThat(result.isCreated(), equalTo(false)); + assertThat(result.getVersion(), equalTo(lastOpVersion)); + assertThat(result.hasFailure(), equalTo(true)); + assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); + } else { + Engine.IndexResult result = engine.index(versionedOp ? indexWithVersion.apply(lastOpVersion, index) : index); + assertThat(result.isCreated(), equalTo(docDeleted)); + assertThat(result.getVersion(), equalTo(Math.max(lastOpVersion + 1, 1))); + assertThat(result.hasFailure(), equalTo(false)); + assertThat(result.getFailure(), nullValue()); + lastFieldValue = index.docs().get(0).get("value"); + docDeleted = false; + lastOpVersion = result.getVersion(); + } + } else { + final Engine.Delete delete = (Engine.Delete) op; + if (versionConflict) { + // generate a conflict + Engine.DeleteResult result = engine.delete(delWithVersion.apply(lastOpVersion - 1, delete)); + assertThat(result.isFound(), equalTo(docDeleted == false)); + assertThat(result.getVersion(), equalTo(lastOpVersion)); + assertThat(result.hasFailure(), equalTo(true)); + assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); + } else { + Engine.DeleteResult result = engine.delete(versionedOp ? delWithVersion.apply(lastOpVersion, delete) : delete); + assertThat(result.isFound(), equalTo(docDeleted == false)); + assertThat(result.getVersion(), equalTo(Math.max(lastOpVersion + 1, 1))); + assertThat(result.hasFailure(), equalTo(false)); + assertThat(result.getFailure(), nullValue()); + docDeleted = true; + lastOpVersion = result.getVersion(); + } + } + if (randomBoolean()) { + // refresh and take the chance to check everything is ok so far + assertVisibleCount(engine, docDeleted ? 0 : 1); + if (docDeleted == false) { + try (Searcher searcher = engine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); + assertThat(collector.getTotalHits(), equalTo(1)); + } + } + } + if (randomBoolean()) { + engine.flush(); + } + + if (rarely()) { + // simulate GC deletes + engine.refresh("gc_simulation"); + engine.clearDeletedTombstones(); + if (docDeleted) { + lastOpVersion = Versions.NOT_FOUND; + } + } + } + + assertVisibleCount(engine, docDeleted ? 0 : 1); + if (docDeleted == false) { + try (Searcher searcher = engine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); + assertThat(collector.getTotalHits(), equalTo(1)); + } + } + } + + + private void testOpsOnReplica(List ops, InternalEngine replicaEngine) 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; + } + 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.hasFailure(), equalTo(false)); + + } 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.hasFailure(), equalTo(false)); + } + if (randomBoolean()) { + engine.refresh("test"); + } if (randomBoolean()) { + engine.flush(); + } + 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)); + } + } + } + // simulate a new replica getting data from an old priamry public void testVersioningReplicaConflict1() throws IOException { IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", Settings.builder() @@ -1629,7 +1841,6 @@ public void testVersioningReplicaConflict2() throws IOException { assertFalse(replicaV2Result.isCreated()); assertThat(replicaV2Result.getVersion(), equalTo(2L)); - replicaEngine.refresh("test"); assertVisibleCount(replicaEngine, 0); } } @@ -1841,7 +2052,6 @@ public void testSeqNoReplicaConflict2() throws IOException { assertFalse(replicaV2Result.isCreated()); assertThat(replicaV2Result.getVersion(), equalTo(2L)); - replicaEngine.refresh("test"); assertVisibleCount(replicaEngine, 0); } @@ -2351,7 +2561,6 @@ public void testTranslogReplayWithFailure() throws IOException { Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } - engine.refresh("test"); assertVisibleCount(engine, numDocs); engine.close(); final MockDirectoryWrapper directory = DirectoryUtils.getLeaf(store.directory(), MockDirectoryWrapper.class); @@ -2384,10 +2593,17 @@ public void testTranslogReplayWithFailure() throws IOException { // no mock directory, no fun. engine = createEngine(store, primaryTranslogDir); } - assertVisibleCount(engine, numDocs); + assertVisibleCount(engine, numDocs, false); } private static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { + assertVisibleCount(engine, numDocs, true); + } + + private static void assertVisibleCount(InternalEngine engine, int numDocs, boolean refresh) throws IOException { + if (refresh) { + engine.refresh("test"); + } try (Searcher searcher = engine.acquireSearcher("test")) { final TotalHitCountCollector collector = new TotalHitCountCollector(); searcher.searcher().search(new MatchAllDocsQuery(), collector); @@ -2403,7 +2619,6 @@ public void testSkipTranslogReplay() throws IOException { Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } - engine.refresh("test"); assertVisibleCount(engine, numDocs); engine.close(); engine = new InternalEngine(engine.config()); @@ -2522,7 +2737,6 @@ public void testTranslogReplay() throws IOException { Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } - engine.refresh("test"); assertVisibleCount(engine, numDocs); TranslogHandler parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer(); @@ -2532,7 +2746,7 @@ public void testTranslogReplay() throws IOException { engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); // we need to reuse the engine config unless the parser.mappingModified won't work engine.recoverFromTranslog(); - assertVisibleCount(engine, numDocs); + assertVisibleCount(engine, numDocs, false); parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer(); assertEquals(numDocs, parser.recoveredOps.get()); if (parser.mappingUpdate != null) { @@ -2544,7 +2758,7 @@ public void testTranslogReplay() throws IOException { engine.close(); engine = createEngine(store, primaryTranslogDir); - assertVisibleCount(engine, numDocs); + assertVisibleCount(engine, numDocs, false); parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer(); assertEquals(0, parser.recoveredOps.get()); @@ -2630,7 +2844,6 @@ public void testRecoverFromForeignTranslog() throws IOException { Engine.IndexResult index = engine.index(firstIndexRequest); assertThat(index.getVersion(), equalTo(1L)); } - engine.refresh("test"); assertVisibleCount(engine, numDocs); Translog.TranslogGeneration generation = engine.getTranslog().getGeneration(); engine.close(); @@ -2660,7 +2873,7 @@ public void testRecoverFromForeignTranslog() throws IOException { } engine = createEngine(store, primaryTranslogDir); // and recover again! - assertVisibleCount(engine, numDocs); + assertVisibleCount(engine, numDocs, false); } public void testShardNotAvailableExceptionWhenEngineClosedConcurrently() throws IOException, InterruptedException { @@ -3447,6 +3660,7 @@ public void testSequenceNumberAdvancesToMaxSeqNoOnEngineOpenOnReplica() throws I } } + /** java docs */ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOException { final List operations = new ArrayList<>(); From 7b58f1daf5e48397cc5b4361db3076684996e612 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 10 Mar 2017 15:28:25 -0800 Subject: [PATCH 09/14] fix delete error handling --- .../index/engine/InternalEngine.java | 50 ++++++++++++------- .../index/engine/InternalEngineTests.java | 5 +- 2 files changed, 35 insertions(+), 20 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 511b73b639547..cbbabe5080457 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -789,8 +789,8 @@ public DeleteResult delete(Delete delete) throws IOException { // of a rare double delete final boolean performOperation; final boolean currentlyDeleted; - final long seqNoForIndexing; - final long versionForIndexing; + final long seqNoOfDeletion; + final long versionOfDeletion; Optional earlyResultOnPreflightError = Optional.empty(); if (delete.origin() == Operation.Origin.PRIMARY) { // resolve operation from external to internal @@ -810,11 +810,11 @@ public DeleteResult delete(Delete delete) throws IOException { currentVersion, delete.seqNo(), currentlyDeleted == false )); performOperation = false; - seqNoForIndexing = SequenceNumbersService.UNASSIGNED_SEQ_NO; - versionForIndexing = -1; + seqNoOfDeletion = SequenceNumbersService.UNASSIGNED_SEQ_NO; + versionOfDeletion = -1; } else { - versionForIndexing = delete.versionType().updateVersion(currentVersion, delete.version()); - seqNoForIndexing = seqNoService().generateSeqNo(); + versionOfDeletion = delete.versionType().updateVersion(currentVersion, delete.version()); + seqNoOfDeletion = seqNoService().generateSeqNo(); performOperation = true; } } else { @@ -833,8 +833,8 @@ public DeleteResult delete(Delete delete) throws IOException { // 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 currentlyDeleted = luceneOpStatus == LuceneOpStatus.NOT_FOUND; - seqNoForIndexing = delete.seqNo(); - versionForIndexing = delete.version(); + seqNoOfDeletion = delete.seqNo(); + versionOfDeletion = delete.version(); performOperation = luceneOpStatus != LuceneOpStatus.NEWER_OR_EQUAL; } @@ -843,16 +843,10 @@ public DeleteResult delete(Delete delete) throws IOException { assert performOperation == false; } else { if (performOperation) { - if (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()); - } - versionMap.putUnderLock(delete.uid().bytes(), - new DeleteVersionValue(versionForIndexing, seqNoForIndexing, delete.primaryTerm(), - engineConfig.getThreadPool().relativeTimeInMillis())); + deleteResult = deleteInLucene(delete, currentlyDeleted, seqNoOfDeletion, versionOfDeletion); + } else { + deleteResult = new DeleteResult(versionOfDeletion, seqNoOfDeletion, currentlyDeleted == false); } - deleteResult = new DeleteResult(versionForIndexing, seqNoForIndexing, currentlyDeleted == false); } if (!deleteResult.hasFailure()) { Translog.Location location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY @@ -866,7 +860,6 @@ public DeleteResult delete(Delete delete) throws IOException { deleteResult.setTook(System.nanoTime() - delete.startTime()); deleteResult.freeze(); } catch (RuntimeException | IOException e) { - assert indexWriter.getTragicException() != null : "unexpected exception in delete\n" + e; try { maybeFailEngine("index", e); } catch (Exception inner) { @@ -878,6 +871,27 @@ public DeleteResult delete(Delete delete) throws IOException { return deleteResult; } + private DeleteResult deleteInLucene(Delete delete, boolean currentlyDeleted, long seqNo, long version) + throws IOException { + try { + if (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()); + } + versionMap.putUnderLock(delete.uid().bytes(), new DeleteVersionValue(version, seqNo, delete.primaryTerm(), + engineConfig.getThreadPool().relativeTimeInMillis())); + return new DeleteResult(version, seqNo, currentlyDeleted == false); + } catch (Exception ex) { + if (indexWriter.getTragicException() == null) { + // there is no tragic event and such it must be a document level failure + return new DeleteResult(ex, version, seqNo, currentlyDeleted == false); + } else { + throw ex; + } + } + } + private void maybePruneDeletedTombstones() { // It's expensive to prune because we walk the deletes map acquiring dirtyLock for each uid so we only do it // every 1/4 of gcDeletesInMillis: diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 34f20950e8ed2..38855b6e6c4af 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -3051,10 +3051,11 @@ public void testHandleDocumentFailure() throws Exception { // all these simulated exceptions are not fatal to the IW so we treat them as document failures if (randomBoolean()) { throwingIndexWriter.get().setThrowFailure(() -> new IOException("simulated")); - expectThrows(IOException.class, () -> engine.delete(new Engine.Delete("test", "1", newUid(doc1)))); + assertThat(engine.delete(new Engine.Delete("test", "1", newUid(doc1))).getFailure(), instanceOf(IOException.class)); } else { throwingIndexWriter.get().setThrowFailure(() -> new IllegalArgumentException("simulated max token length")); - expectThrows(IllegalArgumentException.class, () -> engine.delete(new Engine.Delete("test", "1", newUid(doc1)))); + assertThat(engine.delete(new Engine.Delete("test", "1", newUid(doc1))).getFailure(), + instanceOf(IllegalArgumentException.class)); } // test non document level failure is thrown From b626e959eaf8853801f90b1a20741dd7fde54859 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 10 Mar 2017 16:10:18 -0800 Subject: [PATCH 10/14] add primary external versioning test --- .../index/engine/InternalEngineTests.java | 140 +++++++++++++----- 1 file changed, 106 insertions(+), 34 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 38855b6e6c4af..08ad361ff4ec3 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1538,6 +1538,67 @@ public void testOutOfOrderDocsOnReplicaOldPrimary() throws IOException { } } + private void testOpsOnReplica(List ops, InternalEngine replicaEngine) 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; + } + 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.hasFailure(), equalTo(false)); + + } 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.hasFailure(), equalTo(false)); + } + if (randomBoolean()) { + engine.refresh("test"); + } if (randomBoolean()) { + engine.flush(); + } + 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 testInternalVersioningOnPrimary() throws IOException { final List ops = generateSingleDocHistory(false, false, false); String lastFieldValue = null; @@ -1626,8 +1687,8 @@ public void testInternalVersioningOnPrimary() throws IOException { } } - - private void testOpsOnReplica(List ops, InternalEngine replicaEngine) throws IOException { + public void testExternalVersioningOnPrimary() throws IOException { + final List ops = generateSingleDocHistory(false, true, false); final Engine.Operation lastOp = ops.get(ops.size() - 1); final String lastFieldValue; if (lastOp instanceof Engine.Index) { @@ -1637,50 +1698,61 @@ private void testOpsOnReplica(List ops, InternalEngine replica // delete lastFieldValue = null; } - 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; + shuffle(ops, random()); + long highestOpVersion = Versions.NOT_FOUND; + long seqNo = -1; + boolean docDeleted = 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.hasFailure(), equalTo(false)); - + final Engine.Index index = (Engine.Index) op; + Engine.IndexResult result = engine.index(index); + if (op.version() > highestOpVersion) { + seqNo++; + assertThat(result.getSeqNo(), equalTo(seqNo)); + assertThat(result.isCreated(), equalTo(docDeleted)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.hasFailure(), equalTo(false)); + assertThat(result.getFailure(), nullValue()); + docDeleted = false; + highestOpVersion = op.version(); + } else { + assertThat(result.isCreated(), equalTo(false)); + assertThat(result.getVersion(), equalTo(highestOpVersion)); + assertThat(result.hasFailure(), equalTo(true)); + assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); + } } 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.hasFailure(), equalTo(false)); + final Engine.Delete delete = (Engine.Delete) op; + Engine.DeleteResult result = engine.delete(delete); + if (op.version() > highestOpVersion) { + seqNo++; + assertThat(result.getSeqNo(), equalTo(seqNo)); + assertThat(result.isFound(), equalTo(docDeleted == false)); + assertThat(result.getVersion(), equalTo(op.version())); + assertThat(result.hasFailure(), equalTo(false)); + assertThat(result.getFailure(), nullValue()); + docDeleted = true; + highestOpVersion = op.version(); + } else { + assertThat(result.isFound(), equalTo(docDeleted == false)); + assertThat(result.getVersion(), equalTo(highestOpVersion)); + assertThat(result.hasFailure(), equalTo(true)); + assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); + } } if (randomBoolean()) { engine.refresh("test"); - } if (randomBoolean()) { + } + if (randomBoolean()) { engine.flush(); } - firstOp = false; } - assertVisibleCount(replicaEngine, lastFieldValue == null ? 0 : 1); - if (lastFieldValue != null) { - try (Searcher searcher = replicaEngine.acquireSearcher("test")) { + assertVisibleCount(engine, docDeleted ? 0 : 1); + if (docDeleted == false) { + try (Searcher searcher = engine.acquireSearcher("test")) { final TotalHitCountCollector collector = new TotalHitCountCollector(); searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); assertThat(collector.getTotalHits(), equalTo(1)); From eca78726bdeeb008105f159a3162d8fb458fce4a Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 10 Mar 2017 16:26:15 -0800 Subject: [PATCH 11/14] added testVersioningPromotedReplica --- .../index/engine/InternalEngineTests.java | 56 ++++++++++++++----- 1 file changed, 43 insertions(+), 13 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 08ad361ff4ec3..27ef0a92ce4ee 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1479,7 +1479,8 @@ public void testVersioningCreateExistsExceptionWithFlush() throws IOException { } - protected List generateSingleDocHistory(boolean forReplica, boolean externalVersioning, boolean partialOldPrimary) { + protected List generateSingleDocHistory(boolean forReplica, boolean externalVersioning, boolean partialOldPrimary, + long primaryTerm) { final int numOfOps = randomIntBetween(2, 20); final List ops = new ArrayList<>(); final Term id = newUid(Uid.createUid("test", "1")); @@ -1489,14 +1490,15 @@ protected List generateSingleDocHistory(boolean forReplica, bo } else { startWithSeqNo = 0; } + final String valuePrefix = forReplica ? "r_" : "p_"; final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); final VersionType versionType = externalVersioning ? VersionType.EXTERNAL : VersionType.INTERNAL; for (int i = 0; i < numOfOps; i++) { final Engine.Operation op; if (randomBoolean()) { - op = new Engine.Index(id, testParsedDocument("1", "test", null, testDocumentWithTextField("v_" + i), B_1, null), + op = new Engine.Index(id, testParsedDocument("1", "test", null, testDocumentWithTextField(valuePrefix + i), B_1, null), forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbersService.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? 2 : 1, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, forReplica || externalVersioning ? i : Versions.MATCH_ANY, forReplica ? versionType.versionTypeForReplicationAndRecovery() : versionType, forReplica ? REPLICA : PRIMARY, @@ -1505,7 +1507,7 @@ protected List generateSingleDocHistory(boolean forReplica, bo } else { op = new Engine.Delete("test", "1", id, forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbersService.UNASSIGNED_SEQ_NO, - forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? 2 : 1, + forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, forReplica || externalVersioning ? i : Versions.MATCH_ANY, forReplica ? versionType.versionTypeForReplicationAndRecovery() : versionType, forReplica ? REPLICA : PRIMARY, @@ -1517,8 +1519,8 @@ protected List generateSingleDocHistory(boolean forReplica, bo } public void testOutOfOrderDocsOnReplica() throws IOException { - final List ops = generateSingleDocHistory(true, true, false); - testOpsOnReplica(ops, replicaEngine); + final List ops = generateSingleDocHistory(true, true, false, 2); + assertOpsOnReplica(ops, replicaEngine); } public void testOutOfOrderDocsOnReplicaOldPrimary() throws IOException { @@ -1533,12 +1535,12 @@ public void testOutOfOrderDocsOnReplicaOldPrimary() throws IOException { try (Store oldReplicaStore = createStore(); InternalEngine replicaEngine = createEngine(oldSettings, oldReplicaStore, createTempDir("translog-old-replica"), newMergePolicy())) { - final List ops = generateSingleDocHistory(true, true, true); - testOpsOnReplica(ops, replicaEngine); + final List ops = generateSingleDocHistory(true, true, true, 2); + assertOpsOnReplica(ops, replicaEngine); } } - private void testOpsOnReplica(List ops, InternalEngine replicaEngine) throws IOException { + private void assertOpsOnReplica(List ops, InternalEngine replicaEngine) throws IOException { final Engine.Operation lastOp = ops.get(ops.size() - 1); final String lastFieldValue; if (lastOp instanceof Engine.Index) { @@ -1600,10 +1602,15 @@ private void testOpsOnReplica(List ops, InternalEngine replica } public void testInternalVersioningOnPrimary() throws IOException { - final List ops = generateSingleDocHistory(false, false, false); + final List ops = generateSingleDocHistory(false, false, false, 2); + assertOpsOnPrimary(ops, Versions.NOT_FOUND, true, engine); + } + + private int assertOpsOnPrimary(List ops, long currentOpVersion, boolean docDeleted, InternalEngine engine) + throws IOException { String lastFieldValue = null; - long lastOpVersion = Versions.NOT_FOUND; - boolean docDeleted = true; + int opsPerformed = 0; + long lastOpVersion = currentOpVersion; BiFunction indexWithVersion = (version, index) -> new Engine.Index(index.uid(), index.parsedDoc(), index.seqNo(), index.primaryTerm(), version, index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry()); @@ -1632,6 +1639,7 @@ public void testInternalVersioningOnPrimary() throws IOException { lastFieldValue = index.docs().get(0).get("value"); docDeleted = false; lastOpVersion = result.getVersion(); + opsPerformed++; } } else { final Engine.Delete delete = (Engine.Delete) op; @@ -1650,6 +1658,7 @@ public void testInternalVersioningOnPrimary() throws IOException { assertThat(result.getFailure(), nullValue()); docDeleted = true; lastOpVersion = result.getVersion(); + opsPerformed++; } } if (randomBoolean()) { @@ -1685,10 +1694,11 @@ public void testInternalVersioningOnPrimary() throws IOException { assertThat(collector.getTotalHits(), equalTo(1)); } } + return opsPerformed; } public void testExternalVersioningOnPrimary() throws IOException { - final List ops = generateSingleDocHistory(false, true, false); + final List ops = generateSingleDocHistory(false, true, false, 2); final Engine.Operation lastOp = ops.get(ops.size() - 1); final String lastFieldValue; if (lastOp instanceof Engine.Index) { @@ -1760,6 +1770,26 @@ public void testExternalVersioningOnPrimary() throws IOException { } } + public void testVersioningPromotedReplica() throws IOException { + final List replicaOps = generateSingleDocHistory(true, true, false, 1); + List primaryOps = generateSingleDocHistory(false, false, false, 2); + Engine.Operation lastReplicaOp = replicaOps.get(replicaOps.size() - 1); + final boolean deletedOnReplica = lastReplicaOp instanceof Engine.Delete; + final long finalReplicaVersion = lastReplicaOp.version(); + final long finalReplicaSeqNo = lastReplicaOp.seqNo(); + assertOpsOnReplica(replicaOps, replicaEngine); + final int opsOnPrimary = assertOpsOnPrimary(primaryOps, finalReplicaVersion, deletedOnReplica, replicaEngine); + final long currentSeqNo = getSequenceID(replicaEngine, new Engine.Get(false, lastReplicaOp.uid())).v1(); + try (Searcher searcher = engine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new MatchAllDocsQuery(), collector); + if (collector.getTotalHits() > 0) { + // last op wasn't delete + assertThat(currentSeqNo, equalTo(finalReplicaSeqNo + opsOnPrimary)); + } + } + } + // simulate a new replica getting data from an old priamry public void testVersioningReplicaConflict1() throws IOException { IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", Settings.builder() From 0e4a2f6e6ed9b3bdb358cadcbc0158a9e8344f88 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 10 Mar 2017 21:49:00 -0800 Subject: [PATCH 12/14] add concurrency tests --- .../index/engine/InternalEngineTests.java | 102 ++++++++++++++++-- 1 file changed, 93 insertions(+), 9 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 27ef0a92ce4ee..2c3c5b1cdd9c8 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1480,8 +1480,8 @@ public void testVersioningCreateExistsExceptionWithFlush() throws IOException { protected List generateSingleDocHistory(boolean forReplica, boolean externalVersioning, boolean partialOldPrimary, - long primaryTerm) { - final int numOfOps = randomIntBetween(2, 20); + long primaryTerm, int minOpCount, int maxOpCount) { + final int numOfOps = randomIntBetween(minOpCount, maxOpCount); final List ops = new ArrayList<>(); final Term id = newUid(Uid.createUid("test", "1")); final int startWithSeqNo; @@ -1519,7 +1519,7 @@ protected List generateSingleDocHistory(boolean forReplica, bo } public void testOutOfOrderDocsOnReplica() throws IOException { - final List ops = generateSingleDocHistory(true, true, false, 2); + final List ops = generateSingleDocHistory(true, true, false, 2, 2, 20); assertOpsOnReplica(ops, replicaEngine); } @@ -1535,7 +1535,7 @@ public void testOutOfOrderDocsOnReplicaOldPrimary() throws IOException { try (Store oldReplicaStore = createStore(); InternalEngine replicaEngine = createEngine(oldSettings, oldReplicaStore, createTempDir("translog-old-replica"), newMergePolicy())) { - final List ops = generateSingleDocHistory(true, true, true, 2); + final List ops = generateSingleDocHistory(true, true, true, 2, 2, 20); assertOpsOnReplica(ops, replicaEngine); } } @@ -1601,8 +1601,68 @@ private void assertOpsOnReplica(List ops, InternalEngine repli } } + public void testConcurrentOutOfDocsOnReplica() throws IOException, InterruptedException { + final List ops = generateSingleDocHistory(true, true, false, 2, 100, 300); + 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; + } + shuffle(ops, random()); + concurrentlyApplyOps(ops, engine); + + assertVisibleCount(engine, lastFieldValue == null ? 0 : 1); + if (lastFieldValue != null) { + try (Searcher searcher = engine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); + assertThat(collector.getTotalHits(), equalTo(1)); + } + } + } + + private void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { + Thread[] thread = new Thread[randomIntBetween(3, 5)]; + CountDownLatch startGun = new CountDownLatch(thread.length); + AtomicInteger offset = new AtomicInteger(-1); + for (int i = 0; i < thread.length; i++) { + thread[i] = new Thread(() -> { + startGun.countDown(); + try { + startGun.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + int docOffset; + while ((docOffset = offset.incrementAndGet()) < ops.size()) { + try { + final Engine.Operation op = ops.get(docOffset); + if (op instanceof Engine.Index) { + engine.index((Engine.Index)op); + } else { + engine.delete((Engine.Delete)op); + } + if ((docOffset + 1) % 4 == 0) { + engine.refresh("test"); + } + } catch (IOException e) { + throw new AssertionError(e); + } + } + }); + thread[i].start(); + } + for (int i = 0; i < thread.length; i++) { + thread[i].join(); + } + } + public void testInternalVersioningOnPrimary() throws IOException { - final List ops = generateSingleDocHistory(false, false, false, 2); + final List ops = generateSingleDocHistory(false, false, false, 2, 2, 20); assertOpsOnPrimary(ops, Versions.NOT_FOUND, true, engine); } @@ -1698,7 +1758,7 @@ private int assertOpsOnPrimary(List ops, long currentOpVersion } public void testExternalVersioningOnPrimary() throws IOException { - final List ops = generateSingleDocHistory(false, true, false, 2); + final List ops = generateSingleDocHistory(false, true, false, 2, 2, 20); final Engine.Operation lastOp = ops.get(ops.size() - 1); final String lastFieldValue; if (lastOp instanceof Engine.Index) { @@ -1771,8 +1831,8 @@ public void testExternalVersioningOnPrimary() throws IOException { } public void testVersioningPromotedReplica() throws IOException { - final List replicaOps = generateSingleDocHistory(true, true, false, 1); - List primaryOps = generateSingleDocHistory(false, false, false, 2); + final List replicaOps = generateSingleDocHistory(true, true, false, 1, 2, 20); + List primaryOps = generateSingleDocHistory(false, false, false, 2, 2, 20); Engine.Operation lastReplicaOp = replicaOps.get(replicaOps.size() - 1); final boolean deletedOnReplica = lastReplicaOp instanceof Engine.Delete; final long finalReplicaVersion = lastReplicaOp.version(); @@ -1790,7 +1850,31 @@ public void testVersioningPromotedReplica() throws IOException { } } - // simulate a new replica getting data from an old priamry + public void testConcurrentExternalVersioningOnPrimary() throws IOException, InterruptedException { + final List ops = generateSingleDocHistory(false, true, false, 2, 100, 300); + 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; + } + shuffle(ops, random()); + concurrentlyApplyOps(ops, engine); + + assertVisibleCount(engine, lastFieldValue == null ? 0 : 1); + if (lastFieldValue != null) { + try (Searcher searcher = engine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new TermQuery(new Term("value", lastFieldValue)), collector); + assertThat(collector.getTotalHits(), equalTo(1)); + } + } + } + + // simulate a new replica getting data from an old primary public void testVersioningReplicaConflict1() throws IOException { IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", Settings.builder() .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), "1h") // make sure this doesn't kick in on us From ca138f1aa41a0c9ce9336e7c639061a104c2a713 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 11 Mar 2017 07:00:56 -0800 Subject: [PATCH 13/14] clean duplicate tests --- .../index/engine/InternalEngineTests.java | 577 +----------------- 1 file changed, 12 insertions(+), 565 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 2c3c5b1cdd9c8..6cc44917a2c1f 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1192,97 +1192,6 @@ public void testVersioningNewIndex() throws IOException { assertThat(indexResult.getVersion(), equalTo(1L)); } - public void testExternalVersioningNewIndex() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - Engine.IndexResult indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(12L)); - - index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - indexResult = replicaEngine.index(index); - assertThat(indexResult.getVersion(), equalTo(12L)); - } - - public void testVersioningIndexConflict() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index index = indexForDoc(doc); - Engine.IndexResult indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(1L)); - - index = indexForDoc(doc); - indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(2L)); - - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1L, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - - // future versions should not work as well - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - } - - public void testExternalVersioningIndexConflict() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - Engine.IndexResult indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(12L)); - - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 14, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(14L)); - - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - } - - public void testVersioningIndexConflictWithFlush() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index index = indexForDoc(doc); - Engine.IndexResult indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(1L)); - - index = indexForDoc(doc); - indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(2L)); - - engine.flush(); - - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - - // future versions should not work as well - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - } - - public void testExternalVersioningIndexConflictWithFlush() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - Engine.IndexResult indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(12L)); - - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 14, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(14L)); - - engine.flush(); - - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - } - public void testForceMerge() throws IOException { try (Store store = createStore(); Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), @@ -1380,78 +1289,6 @@ public void run() { } - public void testVersioningDeleteConflict() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index index = indexForDoc(doc); - Engine.IndexResult indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(1L)); - - index = indexForDoc(doc); - indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(2L)); - - Engine.Delete delete = new Engine.Delete("test", "1", newUid(doc), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1L, VersionType.INTERNAL, PRIMARY, 0); - Engine.DeleteResult result = engine.delete(delete); - assertTrue(result.hasFailure()); - assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); - - // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid(doc), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 3L, VersionType.INTERNAL, PRIMARY, 0); - result = engine.delete(delete); - assertTrue(result.hasFailure()); - assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); - - // now actually delete - delete = new Engine.Delete("test", "1", newUid(doc), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2L, VersionType.INTERNAL, PRIMARY, 0); - result = engine.delete(delete); - assertThat(result.getVersion(), equalTo(3L)); - - // now check if we can index to a delete doc with version - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - } - - public void testVersioningDeleteConflictWithFlush() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index index = indexForDoc(doc); - Engine.IndexResult indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(1L)); - - index = indexForDoc(doc); - indexResult = engine.index(index); - assertThat(indexResult.getVersion(), equalTo(2L)); - - engine.flush(); - - Engine.Delete delete = new Engine.Delete("test", "1", newUid(doc), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1L, VersionType.INTERNAL, PRIMARY, 0); - Engine.DeleteResult deleteResult = engine.delete(delete); - assertTrue(deleteResult.hasFailure()); - assertThat(deleteResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - - // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid(doc), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 3L, VersionType.INTERNAL, PRIMARY, 0); - deleteResult = engine.delete(delete); - assertTrue(deleteResult.hasFailure()); - assertThat(deleteResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - - engine.flush(); - - // now actually delete - delete = new Engine.Delete("test", "1", newUid(doc), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2L, VersionType.INTERNAL, PRIMARY, 0); - deleteResult = engine.delete(delete); - assertThat(deleteResult.getVersion(), equalTo(3L)); - - engine.flush(); - - // now check if we can index to a delete doc with version - index = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(index); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - } - public void testVersioningCreateExistsException() throws IOException { ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); Engine.Index create = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); @@ -1464,21 +1301,6 @@ public void testVersioningCreateExistsException() throws IOException { assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); } - public void testVersioningCreateExistsExceptionWithFlush() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index create = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - Engine.IndexResult indexResult = engine.index(create); - assertThat(indexResult.getVersion(), equalTo(1L)); - - engine.flush(); - - create = new Engine.Index(newUid(doc), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); - indexResult = engine.index(create); - assertTrue(indexResult.hasFailure()); - assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); - } - - protected List generateSingleDocHistory(boolean forReplica, boolean externalVersioning, boolean partialOldPrimary, long primaryTerm, int minOpCount, int maxOpCount) { final int numOfOps = randomIntBetween(minOpCount, maxOpCount); @@ -1679,19 +1501,25 @@ private int assertOpsOnPrimary(List ops, long currentOpVersion for (Engine.Operation op : ops) { final boolean versionConflict = rarely(); final boolean versionedOp = versionConflict || randomBoolean(); - logger.info("performing [{}]{}", - op.operationType().name().charAt(0), versionConflict ? " (conflict)" : versionedOp ? " (versioned)" : ""); + final long conflictingVersion = docDeleted || randomBoolean() ? + lastOpVersion + (randomBoolean() ? 1 : -1) : + Versions.MATCH_DELETED; + final long correctVersion = docDeleted && randomBoolean() ? Versions.MATCH_DELETED : lastOpVersion; + logger.info("performing [{}]{}{}", + op.operationType().name().charAt(0), + versionConflict ? " (conflict " + conflictingVersion +")" : "", + versionedOp ? " (versioned " + correctVersion + ")" : ""); if (op instanceof Engine.Index) { final Engine.Index index = (Engine.Index) op; if (versionConflict) { // generate a conflict - Engine.IndexResult result = engine.index(indexWithVersion.apply(lastOpVersion - 1, index)); + Engine.IndexResult result = engine.index(indexWithVersion.apply(conflictingVersion, index)); assertThat(result.isCreated(), equalTo(false)); assertThat(result.getVersion(), equalTo(lastOpVersion)); assertThat(result.hasFailure(), equalTo(true)); assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); } else { - Engine.IndexResult result = engine.index(versionedOp ? indexWithVersion.apply(lastOpVersion, index) : index); + Engine.IndexResult result = engine.index(versionedOp ? indexWithVersion.apply(correctVersion, index) : index); assertThat(result.isCreated(), equalTo(docDeleted)); assertThat(result.getVersion(), equalTo(Math.max(lastOpVersion + 1, 1))); assertThat(result.hasFailure(), equalTo(false)); @@ -1705,13 +1533,13 @@ private int assertOpsOnPrimary(List ops, long currentOpVersion final Engine.Delete delete = (Engine.Delete) op; if (versionConflict) { // generate a conflict - Engine.DeleteResult result = engine.delete(delWithVersion.apply(lastOpVersion - 1, delete)); + Engine.DeleteResult result = engine.delete(delWithVersion.apply(conflictingVersion, delete)); assertThat(result.isFound(), equalTo(docDeleted == false)); assertThat(result.getVersion(), equalTo(lastOpVersion)); assertThat(result.hasFailure(), equalTo(true)); assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); } else { - Engine.DeleteResult result = engine.delete(versionedOp ? delWithVersion.apply(lastOpVersion, delete) : delete); + Engine.DeleteResult result = engine.delete(versionedOp ? delWithVersion.apply(correctVersion, delete) : delete); assertThat(result.isFound(), equalTo(docDeleted == false)); assertThat(result.getVersion(), equalTo(Math.max(lastOpVersion + 1, 1))); assertThat(result.hasFailure(), equalTo(false)); @@ -1874,372 +1702,6 @@ public void testConcurrentExternalVersioningOnPrimary() throws IOException, Inte } } - // simulate a new replica getting data from an old primary - public void testVersioningReplicaConflict1() throws IOException { - IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", 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.V_5_4_0_UNRELEASED) - .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), - between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) - .build()); - - try (Store oldStore = createStore(); Store oldReplicaStore = createStore(); - InternalEngine engine = createEngine(oldSettings, oldStore, createTempDir("translog-old"), newMergePolicy()); - InternalEngine replicaEngine = - createEngine(oldSettings, oldReplicaStore, createTempDir("translog-old-replica"), newMergePolicy())) { - - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField("v_1") , B_1, null); - final Engine.Index v1Index = indexForDoc(doc); - final Engine.IndexResult v1Result = engine.index(v1Index); - assertThat(v1Result.getVersion(), equalTo(1L)); - - final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2") , B_1, null); - final Engine.Index v2Index = indexForDoc(docV2); - final Engine.IndexResult v2Result = engine.index(v2Index); - assertThat(v2Result.getVersion(), equalTo(2L)); - - // apply the second index to the replica, should work fine - final Engine.Index replicaV2Index = new Engine.Index( - newUid(docV2), - docV2, - SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup - v2Index.primaryTerm(), - v2Result.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0, - -1, - false); - final Engine.IndexResult replicaV2Result = replicaEngine.index(replicaV2Index); - assertThat(replicaV2Result.getVersion(), equalTo(2L)); - - // now, the old one should produce an indexing result - final Engine.Index replicaV1Index = new Engine.Index( - newUid(doc), - doc, - SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup - v1Index.primaryTerm(), - v1Result.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0, - -1, - false); - final Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); - assertFalse(replicaV1Result.hasFailure()); - assertFalse(replicaV1Result.isCreated()); - assertThat(replicaV1Result.getVersion(), equalTo(1L)); - - // second version on replica should fail as well - final Engine.IndexResult replicaV2ReplayResult = replicaEngine.index(replicaV2Index); - assertFalse(replicaV2Result.hasFailure()); - assertFalse(replicaV1Result.isCreated()); - assertThat(replicaV2ReplayResult.getVersion(), equalTo(2L)); - - replicaEngine.refresh("test"); - try (Searcher searchResult = replicaEngine.acquireSearcher("test")) { - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits( - new TermQuery(new Term("value", "v_2")), 1)); - } - } - } - - public void testVersioningReplicaConflict2() throws IOException { - IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", 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.V_5_4_0_UNRELEASED) - .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), - between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) - .build()); - - try (Store oldStore = createStore(); Store oldReplicaStore = createStore(); - InternalEngine engine = createEngine(oldSettings, oldStore, createTempDir("translog-old"), newMergePolicy()); - InternalEngine replicaEngine = - createEngine(oldSettings, oldReplicaStore, createTempDir("translog-old-replica"), newMergePolicy())) { - - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField("v_1") , B_1, null); - final Engine.Index v1Index = indexForDoc(doc); - final Engine.IndexResult v1Result = engine.index(v1Index); - assertThat(v1Result.getVersion(), equalTo(1L)); - - // apply the first index to the replica, should work fine - final Engine.Index replicaV1Index = new Engine.Index( - newUid(doc), - doc, - SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup - v1Index.primaryTerm(), - v1Result.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0, - -1, - false); - Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); - assertThat(replicaV1Result.getVersion(), equalTo(1L)); - - // index it again - final Engine.Index v2Index = indexForDoc(doc); - final Engine.IndexResult v2Result = engine.index(v2Index); - assertThat(v2Result.getVersion(), equalTo(2L)); - - // now delete it - final Engine.Delete delete = new Engine.Delete("test", "1", newUid(doc)); - final Engine.DeleteResult deleteResult = engine.delete(delete); - assertThat(deleteResult.getVersion(), equalTo(3L)); - - // apply the delete on the replica (skipping the second index) - final Engine.Delete replicaDelete = new Engine.Delete( - "test", - "1", - newUid(doc), - SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup - delete.primaryTerm(), - deleteResult.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0); - final Engine.DeleteResult replicaDeleteResult = replicaEngine.delete(replicaDelete); - assertThat(replicaDeleteResult.getVersion(), equalTo(3L)); - - // second time delete with same version should just produce the same version - final Engine.DeleteResult deleteReplayResult = replicaEngine.delete(replicaDelete); - assertFalse(deleteReplayResult.hasFailure()); - assertTrue(deleteReplayResult.isFound()); - assertThat(deleteReplayResult.getVersion(), equalTo(3L)); - - // now do the second index on the replica, it should not change things - final Engine.Index replicaV2Index = new Engine.Index( - newUid(doc), - doc, - SequenceNumbersService.UNASSIGNED_SEQ_NO, // make sure to fall back to version based dedup - v2Index.primaryTerm(), - v2Result.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0, - -1, - false); - final Engine.IndexResult replicaV2Result = replicaEngine.index(replicaV2Index); - assertFalse(replicaV2Result.hasFailure()); - assertFalse(replicaV2Result.isCreated()); - assertThat(replicaV2Result.getVersion(), equalTo(2L)); - - assertVisibleCount(replicaEngine, 0); - } - } - - // This models a new primary and a new replica, running on old data but generating seq# for the new data - public void testVersioningReplicaSomeDocsWithSeqNoSomeWithout() throws IOException { - IndexSettings oldSettings = IndexSettingsModule.newIndexSettings("testOld", 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.V_5_4_0_UNRELEASED) - .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.getKey(), - between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) - .build()); - - try (Store oldReplicaStore = createStore(); - InternalEngine replicaEngine = - createEngine(oldSettings, oldReplicaStore, createTempDir("translog-old-replica"), newMergePolicy())) { - - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocumentWithTextField("v_1"), B_1, null); - // insert the first operation into replica, simulating an old primary - final Engine.Index replicaV1Index = new Engine.Index(newUid(doc), doc, - SequenceNumbersService.UNASSIGNED_SEQ_NO, // old op, no seq no - 0, 1, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false); - final Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); - assertFalse(replicaV1Result.hasFailure()); - assertTrue(replicaV1Result.isCreated()); - assertThat(replicaV1Result.getVersion(), equalTo(1L)); - - switch (randomIntBetween(1, 3)) { - case 1: - replicaEngine.refresh("test"); - break; - case 2: - replicaEngine.flush(); - break; - default: - break; - } - - final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2"), B_1, null); - - final Engine.Index replicaV2Index = new Engine.Index(newUid(docV2), docV2, - 10, // a real sequence number - 1, 2, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, 0, -1, false); - - final Engine.IndexResult replicaV2Result = replicaEngine.index(replicaV2Index); - assertThat(replicaV2Result.getVersion(), equalTo(2L)); - - replicaEngine.refresh("test"); - try (Searcher searchResult = replicaEngine.acquireSearcher("test")) { - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits( - new TermQuery(new Term("value", "v_2")), 1)); - } - } - } - - public void testSeqNoReplicaConflict1() throws IOException { - - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - final Engine.Index v1Index = indexForDoc(doc); - final Engine.IndexResult v1Result = engine.index(v1Index); - assertThat(v1Result.getVersion(), equalTo(1L)); - assertThat(v1Result.getSeqNo(), equalTo(0L)); - - final ParsedDocument docV2 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_2") , B_1, null); - final Engine.Index v2Index = indexForDoc(docV2); - final Engine.IndexResult v2Result = engine.index(v2Index); - assertThat(v2Result.getVersion(), equalTo(2L)); - assertThat(v2Result.getSeqNo(), equalTo(1L)); - - // apply the second index to the replica, should work fine - final Engine.Index replicaV2Index = new Engine.Index( - newUid(docV2), - docV2, - v2Result.getSeqNo(), - v2Index.primaryTerm(), - v2Result.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0, - -1, - false); - final Engine.IndexResult replicaV2Result = replicaEngine.index(replicaV2Index); - assertThat(replicaV2Result.getVersion(), equalTo(2L)); - - // now, the old one should produce an indexing result - boolean rejectByTerm = randomBoolean(); - final Engine.Index replicaV1Index = new Engine.Index( - newUid(doc), - doc, - rejectByTerm ? v2Result.getSeqNo() : v1Result.getSeqNo(), - rejectByTerm || randomBoolean() ? v2Index.primaryTerm() - 1 : v1Index.primaryTerm(), - v1Result.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0, - -1, - false); - final Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); - assertFalse(replicaV1Result.hasFailure()); - assertFalse(replicaV1Result.isCreated()); - assertThat(replicaV1Result.getVersion(), equalTo(1L)); - - // second version on replica should fail as well - final Engine.IndexResult replicaV2ReplayResult = replicaEngine.index(replicaV2Index); - assertFalse(replicaV2ReplayResult.hasFailure()); - assertFalse(replicaV2ReplayResult.isCreated()); - assertThat(replicaV2ReplayResult.getVersion(), equalTo(2L)); - - replicaEngine.refresh("test"); - try (Searcher searchResult = replicaEngine.acquireSearcher("test")) { - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits( - new TermQuery(new Term("value", "v_2")), 1)); - } - - // now index, resolving by primaryTerm - final ParsedDocument docV3 = testParsedDocument(doc.id(), doc.type(), null, testDocumentWithTextField("v_3") , B_1, null); - final Engine.Index replicaV3Index = new Engine.Index( - newUid(docV3), - docV3, - v2Result.getSeqNo(), - v2Index.primaryTerm() + 1, - v2Result.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0, - -1, - false); - final Engine.IndexResult replicaV3Result = replicaEngine.index(replicaV3Index); - assertFalse(replicaV3Result.hasFailure()); - assertFalse(replicaV3Result.isCreated()); - assertThat(replicaV3Result.getVersion(), equalTo(v2Result.getVersion())); - replicaEngine.refresh("test"); - try (Searcher searchResult = replicaEngine.acquireSearcher("test")) { - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits( - new TermQuery(new Term("value", "v_3")), 1)); - } - } - - public void testSeqNoReplicaConflict2() throws IOException { - final ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - final Engine.Index v1Index = indexForDoc(doc); - final Engine.IndexResult v1Result = engine.index(v1Index); - assertThat(v1Result.getVersion(), equalTo(1L)); - - // apply the first index to the replica, should work fine - final Engine.Index replicaV1Index = new Engine.Index( - newUid(doc), - doc, - v1Result.getSeqNo(), - v1Index.primaryTerm(), - v1Result.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0, - -1, - false); - Engine.IndexResult replicaV1Result = replicaEngine.index(replicaV1Index); - assertThat(replicaV1Result.getVersion(), equalTo(1L)); - - // index it again - final Engine.Index v2Index = indexForDoc(doc); - final Engine.IndexResult v2Result = engine.index(v2Index); - assertThat(v2Result.getVersion(), equalTo(2L)); - - // now delete it - final Engine.Delete delete = new Engine.Delete("test", "1", newUid(doc)); - final Engine.DeleteResult deleteResult = engine.delete(delete); - assertThat(deleteResult.getVersion(), equalTo(3L)); - - // apply the delete on the replica (skipping the second index) - final Engine.Delete replicaDelete = new Engine.Delete( - "test", - "1", - newUid(doc), - deleteResult.getSeqNo(), - delete.primaryTerm(), - deleteResult.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0); - final Engine.DeleteResult replicaDeleteResult = replicaEngine.delete(replicaDelete); - assertThat(replicaDeleteResult.getVersion(), equalTo(3L)); - - // second time delete with same version should just produce the same version - final Engine.DeleteResult deleteReplayResult = replicaEngine.delete(replicaDelete); - assertFalse(deleteReplayResult.hasFailure()); - assertTrue(deleteReplayResult.isFound()); - assertThat(deleteReplayResult.getVersion(), equalTo(3L)); - - // now do the second index on the replica, it should not change things - final Engine.Index replicaV2Index = new Engine.Index( - newUid(doc), - doc, - v2Result.getSeqNo(), - v2Index.primaryTerm(), - v2Result.getVersion(), - VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), - REPLICA, - 0, - -1, - false); - final Engine.IndexResult replicaV2Result = replicaEngine.index(replicaV2Index); - assertFalse(replicaV2Result.hasFailure()); - assertFalse(replicaV2Result.isCreated()); - assertThat(replicaV2Result.getVersion(), equalTo(2L)); - - assertVisibleCount(replicaEngine, 0); - } public void testBasicCreatedFlag() throws IOException { ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); @@ -2258,21 +1720,6 @@ public void testBasicCreatedFlag() throws IOException { assertTrue(indexResult.isCreated()); } - public void testCreatedFlagAfterFlush() throws IOException { - ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null); - Engine.Index index = indexForDoc(doc); - Engine.IndexResult indexResult = engine.index(index); - assertTrue(indexResult.isCreated()); - - engine.delete(new Engine.Delete(null, "1", newUid(doc))); - - engine.flush(); - - index = indexForDoc(doc); - indexResult = engine.index(index); - assertTrue(indexResult.isCreated()); - } - private static class MockAppender extends AbstractAppender { public boolean sawIndexWriterMessage; From c0343ea927b88e477fcf5bbfd355d03e02c82406 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 11 Mar 2017 08:02:56 -0800 Subject: [PATCH 14/14] added testConcurrentGetAndSetOnPrimary --- .../index/engine/InternalEngineTests.java | 67 ++++++++++++++++++- 1 file changed, 66 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 6cc44917a2c1f..ea6527bfaebe5 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -75,6 +75,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; @@ -87,6 +88,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; @@ -97,6 +99,7 @@ import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.Engine.Searcher; +import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapperForType; @@ -294,6 +297,8 @@ private static ParsedDocument testParsedDocument(String id, String type, String document.add(seqID.seqNo); document.add(seqID.seqNoDocValue); document.add(seqID.primaryTerm); + BytesRef ref = source.toBytesRef(); + document.add(new StoredField(SourceFieldMapper.NAME, ref.bytes, ref.offset, ref.length)); return new ParsedDocument(versionField, seqID, id, type, routing, Arrays.asList(document), source, XContentType.JSON, mappingUpdate); } @@ -419,7 +424,11 @@ public void onFailedEngine(String reason, @Nullable Exception e) { private static final BytesReference B_1 = new BytesArray(new byte[]{1}); private static final BytesReference B_2 = new BytesArray(new byte[]{2}); private static final BytesReference B_3 = new BytesArray(new byte[]{3}); - private static final BytesArray SOURCE = new BytesArray("{}".getBytes(Charset.defaultCharset())); + private static final BytesArray SOURCE = bytesArray("{}"); + + private static BytesArray bytesArray(String string) { + return new BytesArray(string.getBytes(Charset.defaultCharset())); + } public void testSegments() throws Exception { try (Store store = createStore(); @@ -1702,6 +1711,62 @@ public void testConcurrentExternalVersioningOnPrimary() throws IOException, Inte } } + public void testConcurrentGetAndSetOnPrimary() throws IOException, InterruptedException { + Thread[] thread = new Thread[randomIntBetween(3, 5)]; + CountDownLatch startGun = new CountDownLatch(thread.length); + final int opsPerThread = randomIntBetween(10, 20); + final Set currentValues = ConcurrentCollections.newConcurrentSet(); + final AtomicInteger idGenerator = new AtomicInteger(); + ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), bytesArray(""), null); + final Term uidTerm = newUid(doc); + engine.index(indexForDoc(doc)); + for (int i = 0; i < thread.length; i++) { + thread[i] = new Thread(() -> { + startGun.countDown(); + try { + startGun.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + for (int op = 0; op < opsPerThread; op++) { + try (Engine.GetResult get = engine.get(new Engine.Get(true, uidTerm))) { + FieldsVisitor visitor = new FieldsVisitor(true); + get.docIdAndVersion().context.reader().document(get.docIdAndVersion().docId, visitor); + List values = new ArrayList<>(Strings.commaDelimitedListToSet(visitor.source().utf8ToString())); + String removed = op % 3 == 0 && values.size() > 0 ? values.remove(0) : null; + String added = "v_" + idGenerator.incrementAndGet(); + values.add(added); + Engine.Index index = new Engine.Index(uidTerm, + testParsedDocument("1", "test", null, testDocument(), + bytesArray(Strings.collectionToCommaDelimitedString(values)), null), + SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, + get.version(), VersionType.INTERNAL, + PRIMARY, System.currentTimeMillis(), -1, false); + Engine.IndexResult indexResult = engine.index(index); + if (indexResult.hasFailure() == false) { + boolean exists = removed == null ? true : currentValues.remove(removed); + assertTrue(removed + " should exist", exists); + exists = currentValues.add(added); + assertTrue(added + " should not exist", exists); + } + + } catch (IOException e) { + throw new AssertionError(e); + } + } + }); + thread[i].start(); + } + for (int i = 0; i < thread.length; i++) { + thread[i].join(); + } + try (Engine.GetResult get = engine.get(new Engine.Get(true, uidTerm))) { + FieldsVisitor visitor = new FieldsVisitor(true); + get.docIdAndVersion().context.reader().document(get.docIdAndVersion().docId, visitor); + List values = Arrays.asList(Strings.commaDelimitedListToStringArray(visitor.source().utf8ToString())); + assertThat(currentValues, equalTo(new HashSet<>(values))); + } + } public void testBasicCreatedFlag() throws IOException { ParsedDocument doc = testParsedDocument("1", "test", null, testDocument(), B_1, null);