From fed3cb927c5da0935ccd3e27e2901dc2b1beb7e5 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 5 Oct 2018 14:26:03 -0400 Subject: [PATCH 01/33] initial version --- .../AnnotatedTextFieldMapperTests.java | 9 +- .../action/bulk/TransportShardBulkAction.java | 5 +- .../action/delete/DeleteRequest.java | 45 ++++++++ .../action/index/IndexRequest.java | 53 ++++++++++ .../uid/PerThreadIDVersionAndSeqNoLookup.java | 24 ++++- .../lucene/uid/VersionsAndSeqNoResolver.java | 23 ++-- .../elasticsearch/index/engine/Engine.java | 40 +++++-- .../index/engine/InternalEngine.java | 53 +++++++--- .../VersionConflictEngineException.java | 11 ++ .../elasticsearch/index/shard/IndexShard.java | 54 ++++++---- .../bulk/TransportShardBulkActionTests.java | 20 ++-- .../common/lucene/uid/VersionLookupTests.java | 12 +-- .../common/lucene/uid/VersionsTests.java | 32 +++--- .../index/engine/InternalEngineTests.java | 100 +++++++++--------- .../index/engine/ReadOnlyEngineTests.java | 4 +- .../index/mapper/TextFieldMapperTests.java | 5 +- .../RecoveryDuringReplicationTests.java | 3 +- .../index/shard/IndexShardIT.java | 5 +- .../shard/PrimaryReplicaSyncerTests.java | 4 +- .../index/translog/TranslogTests.java | 4 +- .../elasticsearch/indices/flush/FlushIT.java | 3 +- .../indices/recovery/RecoveryTests.java | 2 +- .../index/engine/EngineTestCase.java | 10 +- .../index/engine/TranslogHandler.java | 5 +- .../index/shard/IndexShardTestCase.java | 7 +- .../index/engine/FollowingEngineTests.java | 4 +- .../SourceOnlySnapshotShardTests.java | 3 +- 27 files changed, 359 insertions(+), 181 deletions(-) diff --git a/plugins/mapper-annotated-text/src/test/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java b/plugins/mapper-annotated-text/src/test/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java index 06f4b728c8c1f..4156fd206722d 100644 --- a/plugins/mapper-annotated-text/src/test/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java +++ b/plugins/mapper-annotated-text/src/test/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java @@ -51,6 +51,7 @@ import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.TextFieldMapper; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.termvectors.TermVectorsService; import org.elasticsearch.indices.IndicesService; @@ -130,7 +131,7 @@ public void testAnnotationInjection() throws IOException { IndexShard shard = indexService.getShard(0); shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, - sourceToParse, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + sourceToParse, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); @@ -185,7 +186,7 @@ public void testToleranceForBadAnnotationMarkup() throws IOException { IndexShard shard = indexService.getShard(0); shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, - sourceToParse, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + sourceToParse, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); @@ -384,7 +385,7 @@ public void testDefaultPositionIncrementGap() throws IOException { IndexShard shard = indexService.getShard(0); shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, - sourceToParse, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + sourceToParse, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); @@ -426,7 +427,7 @@ public void testPositionIncrementGap() throws IOException { IndexShard shard = indexService.getShard(0); shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, - sourceToParse, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + sourceToParse, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 9c134ba4012da..e56cdc1440a0c 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -460,7 +460,7 @@ private static void executeIndexRequestOnPrimary(BulkPrimaryExecutionContext con executeOnPrimaryWhileHandlingMappingUpdates(context, () -> primary.applyIndexOperationOnPrimary(request.version(), request.versionType(), sourceToParse, - request.getAutoGeneratedTimestamp(), request.isRetry()), + request.compareAndWriteSeqNo(), request.compareAndWriteTerm(), request.getAutoGeneratedTimestamp(), request.isRetry()), e -> primary.getFailedIndexResult(e, request.version()), context::markOperationAsExecuted, mapping -> mappingUpdater.updateMappings(mapping, primary.shardId(), request.type())); @@ -471,7 +471,8 @@ private static void executeDeleteRequestOnPrimary(BulkPrimaryExecutionContext co final DeleteRequest request = context.getRequestToExecute(); final IndexShard primary = context.getPrimary(); executeOnPrimaryWhileHandlingMappingUpdates(context, - () -> primary.applyDeleteOperationOnPrimary(request.version(), request.type(), request.id(), request.versionType()), + () -> primary.applyDeleteOperationOnPrimary(request.version(), request.type(), request.id(), request.versionType(), + request.compareAndWriteSeqNo(), request.compareAndWriteTerm()), e -> primary.getFailedDeleteResult(e, request.version()), context::markOperationAsExecuted, mapping -> mappingUpdater.updateMappings(mapping, primary.shardId(), request.type())); diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java index 879e8e665cd44..a22b60f4733b7 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import java.io.IOException; @@ -54,6 +55,8 @@ public class DeleteRequest extends ReplicatedWriteRequest impleme private String routing; private long version = Versions.MATCH_ANY; private VersionType versionType = VersionType.INTERNAL; + private long compareAndWriteSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + private long compareAndWriteTerm = 0; public DeleteRequest() { } @@ -94,6 +97,12 @@ public ActionRequestValidationException validate() { if (versionType == VersionType.FORCE) { validationException = addValidationError("version type [force] may no longer be used", validationException); } + + if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO && ( + versionType != VersionType.INTERNAL || version != Versions.MATCH_ANY + )) { + validationException = addValidationError("compare and write operations can not use versioning", validationException); + } return validationException; } @@ -169,6 +178,26 @@ public DeleteRequest versionType(VersionType versionType) { return this; } + public long compareAndWriteSeqNo() { + return compareAndWriteSeqNo; + } + + public long compareAndWriteTerm() { + return compareAndWriteTerm; + } + + public DeleteRequest compareAndWrite(long seqNo, long term) { + if (seqNo < 0) { + throw new IllegalArgumentException("sequence numbers must be non negative. got [" + seqNo + "]."); + } + if (term <= 0) { + throw new IllegalArgumentException("primary term must be positive. got [" + term + "]"); + } + compareAndWriteSeqNo = seqNo; + compareAndWriteTerm = term; + return this; + } + @Override public VersionType versionType() { return this.versionType; @@ -190,6 +219,13 @@ public void readFrom(StreamInput in) throws IOException { } version = in.readLong(); versionType = VersionType.fromValue(in.readByte()); + if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + compareAndWriteSeqNo = in.readZLong(); + compareAndWriteTerm = in.readVLong(); + } else { + compareAndWriteSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + compareAndWriteTerm = 0; + } } @Override @@ -203,6 +239,15 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeLong(version); out.writeByte(versionType.getValue()); + if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + out.writeZLong(compareAndWriteSeqNo); + out.writeVLong(compareAndWriteTerm); + } else if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || compareAndWriteTerm != 0) { + assert false : "compareAndWrite [" + compareAndWriteSeqNo + "], currentDocTem [" + compareAndWriteTerm + "]"; + throw new IllegalStateException( + "sequence number based compare and write is not supported until all nodes are on version 7.0 or higher. " + + "Stream version [" + out.getVersion() + "]"); + } } @Override diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java index 339880dad44bf..fd757b516f483 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -43,6 +43,7 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import java.io.IOException; @@ -104,6 +105,8 @@ public class IndexRequest extends ReplicatedWriteRequest implement private long autoGeneratedTimestamp = UNSET_AUTO_GENERATED_TIMESTAMP; private boolean isRetry = false; + private long compareAndWriteSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + private long compareAndWriteTerm = 0; public IndexRequest() { @@ -162,6 +165,12 @@ public ActionRequestValidationException validate() { validationException = addValidationError("create operations do not support explicit versions. use index instead", validationException); return validationException; } + + if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || compareAndWriteTerm != 0) { + validationException = addValidationError("create operations do not support compare and set. use index instead", + validationException); + return validationException; + } } if (opType() != OpType.INDEX && id == null) { @@ -189,6 +198,12 @@ public ActionRequestValidationException validate() { validationException = addValidationError("pipeline cannot be an empty string", validationException); } + if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO && ( + versionType != VersionType.INTERNAL || version != Versions.MATCH_ANY + )) { + validationException = addValidationError("compare and write operations can not use versioning", validationException); + } + return validationException; } @@ -466,6 +481,26 @@ public IndexRequest versionType(VersionType versionType) { return this; } + public IndexRequest compareAndWrite(long seqNo, long term) { + if (seqNo < 0) { + throw new IllegalArgumentException("sequence numbers must be non negative. got [" + seqNo + "]."); + } + if (term <= 0) { + throw new IllegalArgumentException("primary term must be positive. got [" + term + "]"); + } + compareAndWriteSeqNo = seqNo; + compareAndWriteTerm = term; + return this; + } + + public long compareAndWriteSeqNo() { + return compareAndWriteSeqNo; + } + + public long compareAndWriteTerm() { + return compareAndWriteTerm; + } + @Override public VersionType versionType() { return this.versionType; @@ -487,6 +522,8 @@ public void process(Version indexCreatedVersion, @Nullable MappingMetaData mappi // generate id if not already provided if (id == null) { assert autoGeneratedTimestamp == -1 : "timestamp has already been generated!"; + assert compareAndWriteSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO; + assert compareAndWriteTerm == 0; autoGeneratedTimestamp = Math.max(0, System.currentTimeMillis()); // extra paranoia String uid; if (indexCreatedVersion.onOrAfter(Version.V_6_0_0_beta1)) { @@ -528,6 +565,13 @@ public void readFrom(StreamInput in) throws IOException { } else { contentType = null; } + if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + compareAndWriteSeqNo = in.readZLong(); + compareAndWriteTerm = in.readVLong(); + } else { + compareAndWriteSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + compareAndWriteTerm = SequenceNumbers.UNASSIGNED_SEQ_NO; + } } @Override @@ -559,6 +603,15 @@ public void writeTo(StreamOutput out) throws IOException { } else { out.writeBoolean(false); } + if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + out.writeZLong(compareAndWriteSeqNo); + out.writeVLong(compareAndWriteTerm); + } else if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || compareAndWriteTerm != 0) { + assert false : "compareAndWrite [" + compareAndWriteSeqNo + "], currentDocTem [" + compareAndWriteTerm + "]"; + throw new IllegalStateException( + "sequence number based compare and write is not supported until all nodes are on version 7.0 or higher. " + + "Stream version [" + out.getVersion() + "]"); + } } @Override diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java index 3a037bed62b7f..8b79c7f7ab13d 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java @@ -94,7 +94,7 @@ final class PerThreadIDVersionAndSeqNoLookup { * using the same cache key. Otherwise we'd have to disable caching * entirely for these readers. */ - public DocIdAndVersion lookupVersion(BytesRef id, LeafReaderContext context) + public DocIdAndVersion lookupVersion(BytesRef id, boolean loadSeqNo, LeafReaderContext context) throws IOException { assert context.reader().getCoreCacheHelper().getKey().equals(readerKey) : "context's reader is not the same as the reader class was initialized on."; @@ -108,7 +108,27 @@ public DocIdAndVersion lookupVersion(BytesRef id, LeafReaderContext context) if (versions.advanceExact(docID) == false) { throw new IllegalArgumentException("Document [" + docID + "] misses the [" + VersionFieldMapper.NAME + "] field"); } - return new DocIdAndVersion(docID, versions.longValue(), context.reader(), context.docBase); + final long seqNo; + final long term; + if (loadSeqNo) { + NumericDocValues seqNos = context.reader().getNumericDocValues(SeqNoFieldMapper.NAME); + if (seqNos != null && seqNos.advanceExact(docID)) { + seqNo = seqNos.longValue(); + } else { + seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + } + NumericDocValues terms = context.reader().getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME); + if (terms != null && terms.advanceExact(docID)) { + term = terms.longValue(); + } else { + term = 0; + } + + } else { + seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + term = 0; + } + return new DocIdAndVersion(docID, versions.longValue(), seqNo, term, context.reader(), context.docBase); } else { return null; } diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java index 9db7e3716d51a..4f47e33b1dcc4 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java @@ -33,8 +33,6 @@ import java.util.Objects; 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 final class VersionsAndSeqNoResolver { @@ -98,12 +96,16 @@ private VersionsAndSeqNoResolver() { public static class DocIdAndVersion { public final int docId; public final long version; + public final long seqNo; + public final long primaryTerm; public final LeafReader reader; public final int docBase; - public DocIdAndVersion(int docId, long version, LeafReader reader, int docBase) { + public DocIdAndVersion(int docId, long version, long seqNo, long primaryTerm, LeafReader reader, int docBase) { this.docId = docId; this.version = version; + this.seqNo = seqNo; + this.primaryTerm = primaryTerm; this.reader = reader; this.docBase = docBase; } @@ -129,7 +131,7 @@ public static class DocIdAndSeqNo { *
  • a doc ID and a version otherwise * */ - public static DocIdAndVersion loadDocIdAndVersion(IndexReader reader, Term term) throws IOException { + public static DocIdAndVersion loadDocIdAndVersion(IndexReader reader, Term term, boolean loadSeqNo) throws IOException { PerThreadIDVersionAndSeqNoLookup[] lookups = getLookupState(reader, term.field()); List leaves = reader.leaves(); // iterate backwards to optimize for the frequently updated documents @@ -137,7 +139,7 @@ public static DocIdAndVersion loadDocIdAndVersion(IndexReader reader, Term term) for (int i = leaves.size() - 1; i >= 0; i--) { final LeafReaderContext leaf = leaves.get(i); PerThreadIDVersionAndSeqNoLookup lookup = lookups[leaf.ord]; - DocIdAndVersion result = lookup.lookupVersion(term.bytes(), leaf); + DocIdAndVersion result = lookup.lookupVersion(term.bytes(), loadSeqNo, leaf); if (result != null) { return result; } @@ -182,15 +184,4 @@ public static long loadPrimaryTerm(DocIdAndSeqNo docIdAndSeqNo, String uidField) + " 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/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index 460501c8b5238..f3d6dc83f037e 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -604,7 +604,7 @@ protected final GetResult getFromSearcher(Get get, BiFunction getGcDeletesInMillis()) { @@ -932,7 +937,8 @@ protected final IndexingStrategy planIndexingAsPrimary(Index index) throws IOExc } else { versionMap.enforceSafeAccess(); // resolves incoming version - final VersionValue versionValue = resolveDocVersion(index); + final VersionValue versionValue = + resolveDocVersion(index, index.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO); final long currentVersion; final boolean currentNotFoundOrDeleted; if (versionValue == null) { @@ -942,7 +948,17 @@ protected final IndexingStrategy planIndexingAsPrimary(Index index) throws IOExc currentVersion = versionValue.version; currentNotFoundOrDeleted = versionValue.isDelete(); } - if (index.versionType().isVersionConflictForWrites( + if (index.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && versionValue == null) { + final VersionConflictEngineException e = new VersionConflictEngineException(shardId, index.type(), index.id(), + index.getCompareAndWriteSeqNo(), index.getCompareAndWriteTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + plan = IndexingStrategy.skipDueToVersionConflict(e, currentNotFoundOrDeleted, currentVersion, getPrimaryTerm()); + } else if (index.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && ( + versionValue.seqNo != index.getCompareAndWriteSeqNo() || versionValue.term != index.getCompareAndWriteTerm() + )) { + final VersionConflictEngineException e = new VersionConflictEngineException(shardId, index.type(), index.id(), + index.getCompareAndWriteSeqNo(), index.getCompareAndWriteTerm(), versionValue.seqNo, versionValue.term); + plan = IndexingStrategy.skipDueToVersionConflict(e, currentNotFoundOrDeleted, currentVersion, getPrimaryTerm()); + } else if (index.versionType().isVersionConflictForWrites( currentVersion, index.version(), currentNotFoundOrDeleted)) { final VersionConflictEngineException e = new VersionConflictEngineException(shardId, index, currentVersion, currentNotFoundOrDeleted); @@ -1252,7 +1268,7 @@ protected final DeletionStrategy planDeletionAsPrimary(Delete delete) throws IOE assert delete.origin() == Operation.Origin.PRIMARY : "planing as primary but got " + delete.origin(); assert getMaxSeqNoOfUpdatesOrDeletes() != SequenceNumbers.UNASSIGNED_SEQ_NO : "max_seq_no_of_updates is not initialized"; // resolve operation from external to internal - final VersionValue versionValue = resolveDocVersion(delete); + final VersionValue versionValue = resolveDocVersion(delete, delete.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO); assert incrementVersionLookup(); final long currentVersion; final boolean currentlyDeleted; @@ -1264,7 +1280,17 @@ protected final DeletionStrategy planDeletionAsPrimary(Delete delete) throws IOE currentlyDeleted = versionValue.isDelete(); } final DeletionStrategy plan; - if (delete.versionType().isVersionConflictForWrites(currentVersion, delete.version(), currentlyDeleted)) { + if (delete.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && versionValue == null) { + final VersionConflictEngineException e = new VersionConflictEngineException(shardId, delete.type(), delete.id(), + delete.getCompareAndWriteSeqNo(), delete.getCompareAndWriteTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + plan = DeletionStrategy.skipDueToVersionConflict(e, currentVersion, getPrimaryTerm(), currentlyDeleted); + } else if (delete.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && ( + versionValue.seqNo != delete.getCompareAndWriteSeqNo() || versionValue.term != delete.getCompareAndWriteTerm() + )) { + final VersionConflictEngineException e = new VersionConflictEngineException(shardId, delete.type(), delete.id(), + delete.getCompareAndWriteSeqNo(), delete.getCompareAndWriteTerm(), versionValue.seqNo, versionValue.term); + plan = DeletionStrategy.skipDueToVersionConflict(e, currentVersion, getPrimaryTerm(), currentlyDeleted); + } else if (delete.versionType().isVersionConflictForWrites(currentVersion, delete.version(), currentlyDeleted)) { final VersionConflictEngineException e = new VersionConflictEngineException(shardId, delete, currentVersion, currentlyDeleted); plan = DeletionStrategy.skipDueToVersionConflict(e, currentVersion, getPrimaryTerm(), currentlyDeleted); } else { @@ -2003,13 +2029,6 @@ protected final ReferenceManager getReferenceManager(SearcherScop } } - private long loadCurrentVersionFromIndex(Term uid) throws IOException { - assert incrementIndexVersionLookup(); - try (Searcher searcher = acquireSearcher("load_version", SearcherScope.INTERNAL)) { - return VersionsAndSeqNoResolver.loadVersion(searcher.reader(), uid); - } - } - private IndexWriter createWriter() throws IOException { try { final IndexWriterConfig iwc = getIndexWriterConfig(); diff --git a/server/src/main/java/org/elasticsearch/index/engine/VersionConflictEngineException.java b/server/src/main/java/org/elasticsearch/index/engine/VersionConflictEngineException.java index f829e35af8912..357c9c107836e 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/VersionConflictEngineException.java +++ b/server/src/main/java/org/elasticsearch/index/engine/VersionConflictEngineException.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.engine; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; @@ -30,6 +31,16 @@ public VersionConflictEngineException(ShardId shardId, Engine.Operation op, long this(shardId, op.type(), op.id(), op.versionType().explainConflictForWrites(currentVersion, op.version(), deleted)); } + public VersionConflictEngineException(ShardId shardId, String type, String id, + long compareAndWriteSeqNo, long compareAndWriteTerm, + long currentSeqNo, long currentTerm) { + this(shardId, type, id, "required seqNo [" + compareAndWriteSeqNo + "], primary term [" + compareAndWriteTerm +"]." + + (currentSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO ? + " but no document was found" : + " current document has seqNo [" + currentSeqNo + "] and primary term ["+ currentTerm + "]" + )); + } + public VersionConflictEngineException(ShardId shardId, String type, String id, String explanation) { this(shardId, null, type, id, explanation); } diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 11d8f44bef133..9bc70a472b1ee 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -501,7 +501,7 @@ public void updateShardState(final ShardRouting newRouting, * the reverted operations on this shard by replaying the translog to avoid losing acknowledged writes. */ final Engine engine = getEngine(); - if (getMaxSeqNoOfUpdatesOrDeletes() == SequenceNumbers.UNASSIGNED_SEQ_NO) { + if (getMaxSeqNoOfUpdatesOrDeletes() == UNASSIGNED_SEQ_NO) { // If the old primary was on an old version that did not replicate the msu, // we need to bootstrap it manually from its local history. assert indexSettings.getIndexVersionCreated().before(Version.V_6_5_0); @@ -674,20 +674,21 @@ private IndexShardState changeState(IndexShardState newState, String reason) { } public Engine.IndexResult applyIndexOperationOnPrimary(long version, VersionType versionType, SourceToParse sourceToParse, - long autoGeneratedTimestamp, boolean isRetry) throws IOException { + long compareAndWriteSeqNo, long compareAndWriteTerm, long autoGeneratedTimestamp, boolean isRetry) throws IOException { assert versionType.validateVersionForWrites(version); - return applyIndexOperation(UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, autoGeneratedTimestamp, - isRetry, Engine.Operation.Origin.PRIMARY, sourceToParse); + return applyIndexOperation(UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, compareAndWriteSeqNo, compareAndWriteTerm, + autoGeneratedTimestamp, isRetry, Engine.Operation.Origin.PRIMARY, sourceToParse); } public Engine.IndexResult applyIndexOperationOnReplica(long seqNo, long version, long autoGeneratedTimeStamp, boolean isRetry, SourceToParse sourceToParse) throws IOException { - return applyIndexOperation(seqNo, operationPrimaryTerm, version, null, autoGeneratedTimeStamp, isRetry, - Engine.Operation.Origin.REPLICA, sourceToParse); + return applyIndexOperation(seqNo, operationPrimaryTerm, version, null, UNASSIGNED_SEQ_NO, 0, + autoGeneratedTimeStamp, isRetry, Engine.Operation.Origin.REPLICA, sourceToParse); } private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, long version, @Nullable VersionType versionType, + long compareAndWriteSeqNo, long compareAndWriteTerm, long autoGeneratedTimeStamp, boolean isRetry, Engine.Operation.Origin origin, SourceToParse sourceToParse) throws IOException { assert opPrimaryTerm <= this.operationPrimaryTerm: "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm @@ -696,8 +697,8 @@ private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, l Engine.Index operation; try { operation = prepareIndex(docMapper(sourceToParse.type()), indexSettings.getIndexVersionCreated(), sourceToParse, seqNo, - opPrimaryTerm, version, versionType, origin, - autoGeneratedTimeStamp, isRetry); + opPrimaryTerm, version, versionType, origin, autoGeneratedTimeStamp, isRetry, + compareAndWriteSeqNo, compareAndWriteTerm); Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); if (update != null) { return new Engine.IndexResult(update); @@ -715,15 +716,17 @@ private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, l } public static Engine.Index prepareIndex(DocumentMapperForType docMapper, Version indexCreatedVersion, SourceToParse source, long seqNo, - long primaryTerm, long version, VersionType versionType, Engine.Operation.Origin origin, long autoGeneratedIdTimestamp, - boolean isRetry) { + long primaryTerm, long version, VersionType versionType, Engine.Operation.Origin origin, + long autoGeneratedIdTimestamp, boolean isRetry, + long compareAndWriteSeqNo, long compareAndWriteTerm) { long startTime = System.nanoTime(); ParsedDocument doc = docMapper.getDocumentMapper().parse(source); if (docMapper.getMapping() != null) { doc.addDynamicMappingsUpdate(docMapper.getMapping()); } Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(doc.id())); - return new Engine.Index(uid, doc, seqNo, primaryTerm, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry); + return new Engine.Index(uid, doc, seqNo, primaryTerm, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry, + compareAndWriteSeqNo, compareAndWriteTerm); } private Engine.IndexResult index(Engine engine, Engine.Index index) throws IOException { @@ -774,19 +777,22 @@ public Engine.DeleteResult getFailedDeleteResult(Exception e, long version) { return new Engine.DeleteResult(e, version, operationPrimaryTerm); } - public Engine.DeleteResult applyDeleteOperationOnPrimary(long version, String type, String id, VersionType versionType) + public Engine.DeleteResult applyDeleteOperationOnPrimary(long version, String type, String id, VersionType versionType, + long compareAndWriteSeqNo, long compareAndWriteTerm) throws IOException { assert versionType.validateVersionForWrites(version); return applyDeleteOperation(UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, type, id, versionType, - Engine.Operation.Origin.PRIMARY); + compareAndWriteSeqNo, compareAndWriteTerm, Engine.Operation.Origin.PRIMARY); } public Engine.DeleteResult applyDeleteOperationOnReplica(long seqNo, long version, String type, String id) throws IOException { - return applyDeleteOperation(seqNo, operationPrimaryTerm, version, type, id, null, Engine.Operation.Origin.REPLICA); + return applyDeleteOperation( + seqNo, operationPrimaryTerm, version, type, id, null, UNASSIGNED_SEQ_NO, 0, Engine.Operation.Origin.REPLICA); } private Engine.DeleteResult applyDeleteOperation(long seqNo, long opPrimaryTerm, long version, String type, String id, - @Nullable VersionType versionType, Engine.Operation.Origin origin) throws IOException { + @Nullable VersionType versionType, long compareAndWriteSeqNo, long compareAndWriteTerm, + Engine.Operation.Origin origin) throws IOException { assert opPrimaryTerm <= this.operationPrimaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm + "]"; ensureWriteAllowed(origin); @@ -808,14 +814,16 @@ private Engine.DeleteResult applyDeleteOperation(long seqNo, long opPrimaryTerm, } final Term uid = extractUidForDelete(type, id); final Engine.Delete delete = prepareDelete(type, id, uid, seqNo, opPrimaryTerm, version, - versionType, origin); + versionType, origin, compareAndWriteSeqNo, compareAndWriteTerm); return delete(getEngine(), delete); } private static Engine.Delete prepareDelete(String type, String id, Term uid, long seqNo, long primaryTerm, long version, - VersionType versionType, Engine.Operation.Origin origin) { + VersionType versionType, Engine.Operation.Origin origin, + long compareAndWriteSeqNo, long compareAndWriteTerm) { long startTime = System.nanoTime(); - return new Engine.Delete(type, id, uid, seqNo, primaryTerm, version, versionType, origin, startTime); + return new Engine.Delete(type, id, uid, seqNo, primaryTerm, version, versionType, origin, startTime, + compareAndWriteSeqNo, compareAndWriteTerm); } private Term extractUidForDelete(String type, String id) { @@ -1257,14 +1265,14 @@ public Engine.Result applyTranslogOperation(Translog.Operation operation, Engine // we set canHaveDuplicates to true all the time such that we de-optimze the translog case and ensure that all // autoGeneratedID docs that are coming from the primary are updated correctly. result = applyIndexOperation(index.seqNo(), index.primaryTerm(), index.version(), - versionType, index.getAutoGeneratedIdTimestamp(), true, origin, + versionType, UNASSIGNED_SEQ_NO, 0, index.getAutoGeneratedIdTimestamp(), true, origin, source(shardId.getIndexName(), index.type(), index.id(), index.source(), XContentHelper.xContentType(index.source())).routing(index.routing())); break; case DELETE: final Translog.Delete delete = (Translog.Delete) operation; result = applyDeleteOperation(delete.seqNo(), delete.primaryTerm(), delete.version(), delete.type(), delete.id(), - versionType, origin); + versionType, UNASSIGNED_SEQ_NO, 0, origin); break; case NO_OP: final Translog.NoOp noOp = (Translog.NoOp) operation; @@ -1954,7 +1962,7 @@ public void activateWithPrimaryContext(final ReplicationTracker.PrimaryContext p getLocalCheckpoint() == primaryContext.getCheckpointStates().get(routingEntry().allocationId().getId()).getLocalCheckpoint(); synchronized (mutex) { replicationTracker.activateWithPrimaryContext(primaryContext); // make changes to primaryMode flag only under mutex - if (getMaxSeqNoOfUpdatesOrDeletes() == SequenceNumbers.UNASSIGNED_SEQ_NO) { + if (getMaxSeqNoOfUpdatesOrDeletes() == UNASSIGNED_SEQ_NO) { // If the old primary was on an old version that did not replicate the msu, // we need to bootstrap it manually from its local history. assert indexSettings.getIndexVersionCreated().before(Version.V_6_5_0); @@ -2767,8 +2775,8 @@ public long getMaxSeqNoOfUpdatesOrDeletes() { * @see org.elasticsearch.indices.recovery.RecoveryTarget#indexTranslogOperations(List, int, long, long) */ public void advanceMaxSeqNoOfUpdatesOrDeletes(long seqNo) { - assert seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO - || getMaxSeqNoOfUpdatesOrDeletes() == SequenceNumbers.UNASSIGNED_SEQ_NO : + assert seqNo != UNASSIGNED_SEQ_NO + || getMaxSeqNoOfUpdatesOrDeletes() == UNASSIGNED_SEQ_NO : "replica has max_seq_no_of_updates=" + getMaxSeqNoOfUpdatesOrDeletes() + " but primary does not"; getEngine().advanceMaxSeqNoOfUpdatesOrDeletes(seqNo); assert seqNo <= getMaxSeqNoOfUpdatesOrDeletes() : getMaxSeqNoOfUpdatesOrDeletes() + " < " + seqNo; diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java index ca19dcc250948..78861596b7851 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java @@ -268,7 +268,7 @@ public void testExecuteBulkIndexRequestWithMappingUpdates() throws Exception { Engine.IndexResult success = new FakeIndexResult(1, 1, 13, true, resultLocation); IndexShard shard = mock(IndexShard.class); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyBoolean())).thenReturn(mappingUpdate); + when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())).thenReturn(mappingUpdate); // Pretend the mappings haven't made it to the node yet BulkPrimaryExecutionContext context = new BulkPrimaryExecutionContext(bulkShardRequest, shard); @@ -285,9 +285,9 @@ public void testExecuteBulkIndexRequestWithMappingUpdates() throws Exception { assertThat("mappings were \"updated\" once", updateCalled.get(), equalTo(1)); // Verify that the shard "executed" the operation twice - verify(shard, times(2)).applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyBoolean()); + verify(shard, times(2)).applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean()); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyBoolean())).thenReturn(success); + when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())).thenReturn(success); TransportShardBulkAction.executeBulkItemRequest(context, null, threadPool::absoluteTimeInMillis, (update, shardId, type) -> fail("should not have had to update the mappings"), () -> {}); @@ -295,7 +295,7 @@ public void testExecuteBulkIndexRequestWithMappingUpdates() throws Exception { // Verify that the shard "executed" the operation only once (2 for previous invocations plus // 1 for this execution) - verify(shard, times(3)).applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyBoolean()); + verify(shard, times(3)).applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean()); BulkItemResponse primaryResponse = bulkShardRequest.items()[0].getPrimaryResponse(); @@ -488,7 +488,7 @@ public void testUpdateRequestWithFailure() throws Exception { Exception err = new ElasticsearchException("I'm dead <(x.x)>"); Engine.IndexResult indexResult = new Engine.IndexResult(err, 0, 0, 0); IndexShard shard = mock(IndexShard.class); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyBoolean())).thenReturn(indexResult); + when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())).thenReturn(indexResult); when(shard.indexSettings()).thenReturn(indexSettings); UpdateHelper updateHelper = mock(UpdateHelper.class); @@ -536,7 +536,8 @@ public void testUpdateRequestWithConflictFailure() throws Exception { "I'm conflicted <(;_;)>"); Engine.IndexResult indexResult = new Engine.IndexResult(err, 0, 0, 0); IndexShard shard = mock(IndexShard.class); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyBoolean())).thenReturn(indexResult); + when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())) + .thenReturn(indexResult); when(shard.indexSettings()).thenReturn(indexSettings); UpdateHelper updateHelper = mock(UpdateHelper.class); @@ -581,7 +582,8 @@ public void testUpdateRequestWithSuccess() throws Exception { Translog.Location resultLocation = new Translog.Location(42, 42, 42); Engine.IndexResult indexResult = new FakeIndexResult(1, 1, 13, created, resultLocation); IndexShard shard = mock(IndexShard.class); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyBoolean())).thenReturn(indexResult); + when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())) + .thenReturn(indexResult); when(shard.indexSettings()).thenReturn(indexSettings); UpdateHelper updateHelper = mock(UpdateHelper.class); @@ -626,7 +628,7 @@ public void testUpdateWithDelete() throws Exception { final long resultSeqNo = 13; Engine.DeleteResult deleteResult = new FakeDeleteResult(1, 1, resultSeqNo, found, resultLocation); IndexShard shard = mock(IndexShard.class); - when(shard.applyDeleteOperationOnPrimary(anyLong(), any(), any(), any())).thenReturn(deleteResult); + when(shard.applyDeleteOperationOnPrimary(anyLong(), any(), any(), any(), anyLong(), anyLong())).thenReturn(deleteResult); when(shard.indexSettings()).thenReturn(indexSettings); UpdateHelper updateHelper = mock(UpdateHelper.class); @@ -769,7 +771,7 @@ public void testRetries() throws Exception { Engine.IndexResult success = new FakeIndexResult(1, 1, 13, true, resultLocation); IndexShard shard = mock(IndexShard.class); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyBoolean())).thenAnswer(ir -> { + when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())).thenAnswer(ir -> { if (randomBoolean()) { return conflictedResult; } diff --git a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java index e1ca8379972af..28b02de80f63c 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java @@ -58,19 +58,19 @@ public void testSimple() throws Exception { LeafReaderContext segment = reader.leaves().get(0); PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME); // found doc - DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), segment); + DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment); assertNotNull(result); assertEquals(87, result.version); assertEquals(0, result.docId); // not found doc - assertNull(lookup.lookupVersion(new BytesRef("7"), segment)); + assertNull(lookup.lookupVersion(new BytesRef("7"), randomBoolean(), segment)); // deleted doc writer.deleteDocuments(new Term(IdFieldMapper.NAME, "6")); reader.close(); reader = DirectoryReader.open(writer); segment = reader.leaves().get(0); lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME); - assertNull(lookup.lookupVersion(new BytesRef("6"), segment)); + assertNull(lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment)); reader.close(); writer.close(); dir.close(); @@ -93,7 +93,7 @@ public void testTwoDocuments() throws Exception { LeafReaderContext segment = reader.leaves().get(0); PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME); // return the last doc when there are duplicates - DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), segment); + DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment); assertNotNull(result); assertEquals(87, result.version); assertEquals(1, result.docId); @@ -103,7 +103,7 @@ public void testTwoDocuments() throws Exception { reader = DirectoryReader.open(writer); segment = reader.leaves().get(0); lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME); - result = lookup.lookupVersion(new BytesRef("6"), segment); + result = lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment); assertNotNull(result); assertEquals(87, result.version); assertEquals(1, result.docId); @@ -113,7 +113,7 @@ public void testTwoDocuments() throws Exception { reader = DirectoryReader.open(writer); segment = reader.leaves().get(0); lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME); - assertNull(lookup.lookupVersion(new BytesRef("6"), segment)); + assertNull(lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment)); reader.close(); writer.close(); dir.close(); diff --git a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java index 115cfcdf26f9c..fd648570a96e8 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java @@ -39,7 +39,6 @@ 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; @@ -63,15 +62,14 @@ 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(loadVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), equalTo(Versions.NOT_FOUND)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()), equalTo(Versions.NOT_FOUND)); Document doc = new Document(); doc.add(new Field(IdFieldMapper.NAME, "1", IdFieldMapper.Defaults.FIELD_TYPE)); doc.add(new NumericDocValuesField(VersionFieldMapper.NAME, 1)); writer.updateDocument(new Term(IdFieldMapper.NAME, "1"), doc); directoryReader = reopen(directoryReader); - assertThat(loadVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), equalTo(1L)); - assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")).version, equalTo(1L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(1L)); doc = new Document(); Field uid = new Field(IdFieldMapper.NAME, "1", IdFieldMapper.Defaults.FIELD_TYPE); @@ -80,8 +78,7 @@ public void testVersions() throws Exception { doc.add(version); writer.updateDocument(new Term(IdFieldMapper.NAME, "1"), doc); directoryReader = reopen(directoryReader); - assertThat(loadVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), equalTo(2L)); - assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")).version, equalTo(2L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(2L)); // test reuse of uid field doc = new Document(); @@ -91,13 +88,11 @@ public void testVersions() throws Exception { writer.updateDocument(new Term(IdFieldMapper.NAME, "1"), doc); directoryReader = reopen(directoryReader); - assertThat(loadVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), equalTo(3L)); - assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")).version, equalTo(3L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(3L)); writer.deleteDocuments(new Term(IdFieldMapper.NAME, "1")); directoryReader = reopen(directoryReader); - assertThat(loadVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), equalTo(Versions.NOT_FOUND)); - assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), nullValue()); + assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()), nullValue()); directoryReader.close(); writer.close(); dir.close(); @@ -123,21 +118,18 @@ public void testNestedDocuments() throws IOException { writer.updateDocuments(new Term(IdFieldMapper.NAME, "1"), docs); DirectoryReader directoryReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "_na_", 1)); - assertThat(loadVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), equalTo(5L)); - assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")).version, equalTo(5L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(5L)); version.setLongValue(6L); writer.updateDocuments(new Term(IdFieldMapper.NAME, "1"), docs); version.setLongValue(7L); writer.updateDocuments(new Term(IdFieldMapper.NAME, "1"), docs); directoryReader = reopen(directoryReader); - assertThat(loadVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), equalTo(7L)); - assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")).version, equalTo(7L)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(7L)); writer.deleteDocuments(new Term(IdFieldMapper.NAME, "1")); directoryReader = reopen(directoryReader); - assertThat(loadVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), equalTo(Versions.NOT_FOUND)); - assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1")), nullValue()); + assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()), nullValue()); directoryReader.close(); writer.close(); dir.close(); @@ -155,10 +147,10 @@ public void testCache() throws Exception { writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); // should increase cache size by 1 - assertEquals(87, loadVersion(reader, new Term(IdFieldMapper.NAME, "6"))); + assertEquals(87, loadDocIdAndVersion(reader, new Term(IdFieldMapper.NAME, "6"), randomBoolean()).version); assertEquals(size+1, VersionsAndSeqNoResolver.lookupStates.size()); // should be cache hit - assertEquals(87, loadVersion(reader, new Term(IdFieldMapper.NAME, "6"))); + assertEquals(87, loadDocIdAndVersion(reader, new Term(IdFieldMapper.NAME, "6"), randomBoolean()).version); assertEquals(size+1, VersionsAndSeqNoResolver.lookupStates.size()); reader.close(); @@ -179,11 +171,11 @@ public void testCacheFilterReader() throws Exception { doc.add(new NumericDocValuesField(VersionFieldMapper.NAME, 87)); writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); - assertEquals(87, loadVersion(reader, new Term(IdFieldMapper.NAME, "6"))); + assertEquals(87, loadDocIdAndVersion(reader, new Term(IdFieldMapper.NAME, "6"), randomBoolean()).version); assertEquals(size+1, VersionsAndSeqNoResolver.lookupStates.size()); // now wrap the reader DirectoryReader wrapped = ElasticsearchDirectoryReader.wrap(reader, new ShardId("bogus", "_na_", 5)); - assertEquals(87, loadVersion(wrapped, new Term(IdFieldMapper.NAME, "6"))); + assertEquals(87, loadDocIdAndVersion(wrapped, new Term(IdFieldMapper.NAME, "6"), randomBoolean()).version); // same size map: core cache key is shared assertEquals(size+1, VersionsAndSeqNoResolver.lookupStates.size()); diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 26c2453a27142..6b7e481c81423 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -698,11 +698,11 @@ public void testTranslogMultipleOperationsSameDocument() throws IOException { for (int i = 0; i < ops; i++) { final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null); if (randomBoolean()) { - final Engine.Index operation = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false); + final Engine.Index operation = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); operations.add(operation); initialEngine.index(operation); } else { - final Engine.Delete operation = new Engine.Delete("test", "1", newUid(doc), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime()); + final Engine.Delete operation = new Engine.Delete("test", "1", newUid(doc), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); operations.add(operation); initialEngine.delete(operation); } @@ -1143,7 +1143,7 @@ public void testRenewSyncFlush() throws Exception { final boolean forceMergeFlushes = randomBoolean(); final ParsedDocument parsedDoc3 = testParsedDocument("3", null, testDocumentWithTextField(), B_1, null); if (forceMergeFlushes) { - engine.index(new Engine.Index(newUid(parsedDoc3), parsedDoc3, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos(), -1, false)); + engine.index(new Engine.Index(newUid(parsedDoc3), parsedDoc3, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); } else { engine.index(indexForDoc(parsedDoc3)); } @@ -1243,7 +1243,7 @@ public void testVersioningNewCreate() throws IOException { assertThat(indexResult.getVersion(), equalTo(1L)); create = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), create.primaryTerm(), indexResult.getVersion(), - null, REPLICA, 0, -1, false); + null, REPLICA, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(create); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -1257,7 +1257,7 @@ public void testReplicatedVersioningWithFlush() throws IOException { create = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), create.primaryTerm(), indexResult.getVersion(), - null, REPLICA, 0, -1, false); + null, REPLICA, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(create); assertThat(indexResult.getVersion(), equalTo(1L)); assertTrue(indexResult.isCreated()); @@ -1276,7 +1276,7 @@ public void testReplicatedVersioningWithFlush() throws IOException { update = new Engine.Index(newUid(doc), doc, updateResult.getSeqNo(), update.primaryTerm(), updateResult.getVersion(), - null, REPLICA, 0, -1, false); + null, REPLICA, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); updateResult = replicaEngine.index(update); assertThat(updateResult.getVersion(), equalTo(2L)); assertFalse(updateResult.isCreated()); @@ -1329,7 +1329,7 @@ public void testVersioningNewIndex() throws IOException { Engine.IndexResult indexResult = engine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); - index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, 0, -1, false); + index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -1608,11 +1608,11 @@ public void run() { public void testVersioningCreateExistsException() throws IOException { ParsedDocument doc = testParsedDocument("1", null, testDocument(), B_1, null); - Engine.Index create = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); + Engine.Index create = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(create); assertThat(indexResult.getVersion(), equalTo(1L)); - create = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false); + create = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); indexResult = engine.index(create); assertThat(indexResult.getResultType(), equalTo(Engine.Result.Type.FAILURE)); assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); @@ -1655,11 +1655,11 @@ public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, Interrup Document doc = testDocumentWithTextField(index.docs().get(0).get("value")); ParsedDocument parsedDocument = testParsedDocument(index.id(), index.routing(), doc, index.source(), null); return new Engine.Index(index.uid(), parsedDocument, newSeqNo, index.primaryTerm(), index.version(), - index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry()); + index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } else { Engine.Delete delete = (Engine.Delete) operation; return new Engine.Delete(delete.type(), delete.id(), delete.uid(), newSeqNo, delete.primaryTerm(), - delete.version(), delete.versionType(), delete.origin(), delete.startTime()); + delete.version(), delete.versionType(), delete.origin(), delete.startTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } }; final List allOps = new ArrayList<>(); @@ -1733,9 +1733,9 @@ private int assertOpsOnPrimary(List ops, long currentOpVersion 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()); + index.getAutoGeneratedIdTimestamp(), index.isRetry(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); BiFunction delWithVersion = (version, delete) -> new Engine.Delete(delete.type(), delete.id(), - delete.uid(), delete.seqNo(), delete.primaryTerm(), version, delete.versionType(), delete.origin(), delete.startTime()); + delete.uid(), delete.seqNo(), delete.primaryTerm(), version, delete.versionType(), delete.origin(), delete.startTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); for (Engine.Operation op : ops) { final boolean versionConflict = rarely(); final boolean versionedOp = versionConflict || randomBoolean(); @@ -1994,7 +1994,7 @@ class OpAndVersion { bytesArray(Strings.collectionToCommaDelimitedString(values)), null), SequenceNumbers.UNASSIGNED_SEQ_NO, 2, get.version(), VersionType.INTERNAL, - PRIMARY, System.currentTimeMillis(), -1, false); + PRIMARY, System.currentTimeMillis(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(index); if (indexResult.getResultType() == Engine.Result.Type.SUCCESS) { history.add(new OpAndVersion(indexResult.getVersion(), removed, added)); @@ -2138,7 +2138,7 @@ public void testSeqNoAndCheckpoints() throws IOException { id = randomFrom(indexedIds); final Engine.Delete delete = new Engine.Delete( "test", id, newUid(id), SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm.get(), - rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); + rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); final Engine.DeleteResult result = initialEngine.delete(delete); if (result.getResultType() == Engine.Result.Type.SUCCESS) { assertThat(result.getSeqNo(), equalTo(primarySeqNo + 1)); @@ -2156,7 +2156,7 @@ public void testSeqNoAndCheckpoints() throws IOException { final Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm.get(), rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, - PRIMARY, 0, -1, false); + PRIMARY, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); final Engine.IndexResult result = initialEngine.index(index); if (result.getResultType() == Engine.Result.Type.SUCCESS) { assertThat(result.getSeqNo(), equalTo(primarySeqNo + 1)); @@ -2407,10 +2407,10 @@ public void testEnableGcDeletes() throws Exception { document.add(new TextField("value", "test1", Field.Store.YES)); ParsedDocument doc = testParsedDocument("1", null, document, B_2, null); - engine.index(new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false)); + engine.index(new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); // Delete document we just added: - engine.delete(new Engine.Delete("test", "1", newUid(doc), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.delete(new Engine.Delete("test", "1", newUid(doc), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); // Get should not find the document Engine.GetResult getResult = engine.get(newGet(true, doc), searcherFactory); @@ -2424,14 +2424,14 @@ public void testEnableGcDeletes() throws Exception { } // Delete non-existent document - engine.delete(new Engine.Delete("test", "2", newUid("2"), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.delete(new Engine.Delete("test", "2", newUid("2"), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); // Get should not find the document (we never indexed uid=2): getResult = engine.get(new Engine.Get(true, false, "type", "2", newUid("2")), searcherFactory); assertThat(getResult.exists(), equalTo(false)); // Try to index uid=1 with a too-old version, should fail: - Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false); + Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(index); assertThat(indexResult.getResultType(), equalTo(Engine.Result.Type.FAILURE)); assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); @@ -2441,7 +2441,7 @@ public void testEnableGcDeletes() throws Exception { assertThat(getResult.exists(), equalTo(false)); // Try to index uid=2 with a too-old version, should fail: - Engine.Index index1 = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false); + Engine.Index index1 = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); indexResult = engine.index(index1); assertThat(indexResult.getResultType(), equalTo(Engine.Result.Type.FAILURE)); assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); @@ -2525,7 +2525,7 @@ public void testCurrentTranslogIDisCommitted() throws IOException { store.associateIndexWithNewTranslog(translogUUID); ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); try (InternalEngine engine = createEngine(config)) { engine.index(firstIndexRequest); @@ -2627,7 +2627,7 @@ public void testTranslogReplayWithFailure() throws IOException { for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -2721,7 +2721,7 @@ public void testSkipTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -2761,7 +2761,7 @@ public void testTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -2793,7 +2793,7 @@ public void testTranslogReplay() throws IOException { final boolean flush = randomBoolean(); int randomId = randomIntBetween(numDocs + 1, numDocs + 10); ParsedDocument doc = testParsedDocument(Integer.toString(randomId), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); if (flush) { @@ -2802,7 +2802,7 @@ public void testTranslogReplay() throws IOException { } doc = testParsedDocument(Integer.toString(randomId), null, testDocument(), new BytesArray("{}"), null); - Engine.Index idxRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false); + Engine.Index idxRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult result = engine.index(idxRequest); engine.refresh("test"); assertThat(result.getVersion(), equalTo(2L)); @@ -2836,7 +2836,7 @@ public void testRecoverFromForeignTranslog() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult index = engine.index(firstIndexRequest); assertThat(index.getVersion(), equalTo(1L)); } @@ -3156,7 +3156,7 @@ public void testDoubleDeliveryReplicaAppendingAndDeleteOnly() throws IOException Engine.Index retry = appendOnlyReplica(doc, true, 1, randomIntBetween(0, 5)); Engine.Delete delete = new Engine.Delete(operation.type(), operation.id(), operation.uid(), Math.max(retry.seqNo(), operation.seqNo())+1, operation.primaryTerm(), operation.version()+1, operation.versionType(), - REPLICA, operation.startTime()+1); + REPLICA, operation.startTime()+1, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); // operations with a seq# equal or lower to the local checkpoint are not indexed to lucene // and the version lookup is skipped final boolean belowLckp = operation.seqNo() == 0 && retry.seqNo() == 0; @@ -3312,16 +3312,16 @@ public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOExcep boolean isRetry = false; long autoGeneratedIdTimestamp = 0; - Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); - index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); isRetry = true; - index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); indexResult = engine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); engine.refresh("test"); @@ -3330,7 +3330,7 @@ public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOExcep assertEquals(1, topDocs.totalHits.value); } - index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(index); assertThat(indexResult.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); replicaEngine.refresh("test"); @@ -3346,16 +3346,16 @@ public void testRetryWithAutogeneratedIdsAndWrongOrderWorksAndNoDuplicateDocs() boolean isRetry = true; long autoGeneratedIdTimestamp = 0; - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult result = engine.index(firstIndexRequest); assertThat(result.getVersion(), equalTo(1L)); - Engine.Index firstIndexRequestReplica = new Engine.Index(newUid(doc), doc, result.getSeqNo(), firstIndexRequest.primaryTerm(), result.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + Engine.Index firstIndexRequestReplica = new Engine.Index(newUid(doc), doc, result.getSeqNo(), firstIndexRequest.primaryTerm(), result.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexReplicaResult = replicaEngine.index(firstIndexRequestReplica); assertThat(indexReplicaResult.getVersion(), equalTo(1L)); isRetry = false; - Engine.Index secondIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + Engine.Index secondIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(secondIndexRequest); assertTrue(indexResult.isCreated()); engine.refresh("test"); @@ -3364,7 +3364,7 @@ public void testRetryWithAutogeneratedIdsAndWrongOrderWorksAndNoDuplicateDocs() assertEquals(1, topDocs.totalHits.value); } - Engine.Index secondIndexRequestReplica = new Engine.Index(newUid(doc), doc, result.getSeqNo(), secondIndexRequest.primaryTerm(), result.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry); + Engine.Index secondIndexRequestReplica = new Engine.Index(newUid(doc), doc, result.getSeqNo(), secondIndexRequest.primaryTerm(), result.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); replicaEngine.index(secondIndexRequestReplica); replicaEngine.refresh("test"); try (Engine.Searcher searcher = replicaEngine.acquireSearcher("test")) { @@ -3383,12 +3383,12 @@ public Engine.Index randomAppendOnly(ParsedDocument doc, boolean retry, final lo public Engine.Index appendOnlyPrimary(ParsedDocument doc, boolean retry, final long autoGeneratedIdTimestamp) { return new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, - VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, retry); + VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, retry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } public Engine.Index appendOnlyReplica(ParsedDocument doc, boolean retry, final long autoGeneratedIdTimestamp, final long seqNo) { return new Engine.Index(newUid(doc), doc, seqNo, 2, 1, null, - Engine.Operation.Origin.REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, retry); + Engine.Operation.Origin.REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, retry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } public void testRetryConcurrently() throws InterruptedException, IOException { @@ -3666,7 +3666,7 @@ public void testSequenceIDs() throws Exception { doc = testParsedDocument("1", null, document, B_1, null); engine.index(new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 3, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, - System.nanoTime(), -1, false)); + System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); engine.refresh("test"); seqID = getSequenceID(engine, newGet(false, doc)); @@ -3800,7 +3800,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio origin, System.nanoTime(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, - false); + false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); operations.add(index); } else { final Engine.Delete delete = new Engine.Delete( @@ -3812,7 +3812,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio i, origin == PRIMARY ? VersionType.EXTERNAL : null, origin, - System.nanoTime()); + System.nanoTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); operations.add(delete); } } @@ -4332,7 +4332,7 @@ public void testSeqNoGenerator() throws IOException { Engine.Operation.Origin.PRIMARY, System.currentTimeMillis(), System.currentTimeMillis(), - randomBoolean()); + randomBoolean(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); final Engine.IndexResult indexResult = e.index(index); assertThat(indexResult.getSeqNo(), equalTo(seqNo)); assertThat(seqNoGenerator.get(), equalTo(seqNo + 1)); @@ -4346,7 +4346,7 @@ public void testSeqNoGenerator() throws IOException { Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, - System.currentTimeMillis()); + System.currentTimeMillis(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); final Engine.DeleteResult deleteResult = e.delete(delete); assertThat(deleteResult.getSeqNo(), equalTo(seqNo + 1)); assertThat(seqNoGenerator.get(), equalTo(seqNo + 2)); @@ -4663,7 +4663,7 @@ public void testStressUpdateSameDocWhileGettingIt() throws IOException, Interrup engine.onSettingsChanged(); ParsedDocument document = testParsedDocument(Integer.toString(0), null, testDocumentWithTextField(), SOURCE, null); final Engine.Index doc = new Engine.Index(newUid(document), document, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false); + Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); // first index an append only document and then delete it. such that we have it in the tombstones engine.index(doc); engine.delete(new Engine.Delete(doc.type(), doc.id(), doc.uid(), primaryTerm.get())); @@ -4671,15 +4671,15 @@ public void testStressUpdateSameDocWhileGettingIt() throws IOException, Interrup // now index more append only docs and refresh so we re-enabel the optimization for unsafe version map ParsedDocument document1 = testParsedDocument(Integer.toString(1), null, testDocumentWithTextField(), SOURCE, null); engine.index(new Engine.Index(newUid(document1), document1, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false)); + Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); engine.refresh("test"); ParsedDocument document2 = testParsedDocument(Integer.toString(2), null, testDocumentWithTextField(), SOURCE, null); engine.index(new Engine.Index(newUid(document2), document2, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false)); + Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); engine.refresh("test"); ParsedDocument document3 = testParsedDocument(Integer.toString(3), null, testDocumentWithTextField(), SOURCE, null); final Engine.Index doc3 = new Engine.Index(newUid(document3), document3, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false); + Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); engine.index(doc3); engine.engineConfig.setEnableGcDeletes(true); // once we are here the version map is unsafe again and we need to do a refresh inside the get calls to ensure we @@ -4795,7 +4795,7 @@ public void testTrackMaxSeqNoOfNonAppendOnlyOperations() throws Exception { engine.index(doc); } else { engine.delete(new Engine.Delete(doc.type(), doc.id(), doc.uid(), seqno, doc.primaryTerm(), - doc.version(), doc.versionType(), doc.origin(), threadPool.relativeTimeInMillis())); + doc.version(), doc.versionType(), doc.origin(), threadPool.relativeTimeInMillis(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); } maxSeqNoOfNonAppendOnly = seqno; } else { // On primary - do not update max_seqno for non-append-only operations @@ -4863,7 +4863,7 @@ public void testTrimUnsafeCommits() throws Exception { for (int i = 0; i < seqNos.size(); i++) { ParsedDocument doc = testParsedDocument(Long.toString(seqNos.get(i)), null, testDocument(), new BytesArray("{}"), null); Engine.Index index = new Engine.Index(newUid(doc), doc, seqNos.get(i), 0, - 1, null, REPLICA, System.nanoTime(), -1, false); + 1, null, REPLICA, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); engine.index(index); if (randomBoolean()) { engine.flush(); diff --git a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java index 90469d71944ef..e2a3accc4b251 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java @@ -52,7 +52,7 @@ public void testReadOnlyEngine() throws Exception { } ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, - System.nanoTime(), -1, false)); + System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); if (get == null || rarely()) { get = newGet(randomBoolean(), doc); } @@ -121,7 +121,7 @@ public void testFlushes() throws IOException { } ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, - System.nanoTime(), -1, false)); + System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); if (rarely()) { engine.flush(); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java index f4856d51a2d4e..56caa94466c56 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java @@ -57,6 +57,7 @@ import org.elasticsearch.index.query.MatchPhraseQueryBuilder; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.search.MatchQuery; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -251,7 +252,7 @@ public void testDefaultPositionIncrementGap() throws IOException { IndexShard shard = indexService.getShard(0); shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, - sourceToParse, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + sourceToParse, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); @@ -293,7 +294,7 @@ public void testPositionIncrementGap() throws IOException { IndexShard shard = indexService.getShard(0); shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, - sourceToParse, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + sourceToParse, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); shard.refresh("test"); try (Engine.Searcher searcher = shard.acquireSearcher("test")) { LeafReader leaf = searcher.getDirectoryReader().leaves().get(0).reader(); diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index e32161af7fe0c..56263791b8e67 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.engine.InternalEngineTests; import org.elasticsearch.index.mapper.SourceToParse; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.PrimaryReplicaSyncer; @@ -202,7 +203,7 @@ public void testRecoveryToReplicaThatReceivedExtraDocument() throws Exception { Versions.MATCH_ANY, VersionType.INTERNAL, SourceToParse.source("index", "type", "primary", new BytesArray("{}"), XContentType.JSON), - randomNonNegativeLong(), + SequenceNumbers.UNASSIGNED_SEQ_NO, 0, randomNonNegativeLong(), false); } final IndexShard recoveredReplica = diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index 56a14da845fff..3042f2c538e37 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -62,6 +62,7 @@ import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.flush.FlushStats; import org.elasticsearch.index.mapper.SourceToParse; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.CircuitBreakerService; @@ -350,7 +351,7 @@ public void testMaybeFlush() throws Exception { assertFalse(shard.shouldPeriodicallyFlush()); shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, SourceToParse.source("test", "test", "1", new BytesArray("{}"), XContentType.JSON), - IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); assertTrue(shard.shouldPeriodicallyFlush()); final Translog translog = getTranslog(shard); assertEquals(2, translog.stats().getUncommittedOperations()); @@ -400,7 +401,7 @@ public void testMaybeRollTranslogGeneration() throws Exception { assertThat(translog.currentFileGeneration(), equalTo(generation + rolls)); final Engine.IndexResult result = shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, SourceToParse.source("test", "test", "1", new BytesArray("{}"), XContentType.JSON), - IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); final Translog.Location location = result.getTranslogLocation(); shard.afterWriteOperation(); if (location.translogLocation + location.size > generationThreshold) { diff --git a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java index 28e625b34dfd6..31fa2d4872731 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -76,7 +76,7 @@ public void testSyncerSendsOffCorrectDocuments() throws Exception { // Index doc but not advance local checkpoint. shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, SourceToParse.source(shard.shardId().getIndexName(), "_doc", Integer.toString(i), new BytesArray("{}"), XContentType.JSON), - randomBoolean() ? IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP : randomNonNegativeLong(), true); + SequenceNumbers.UNASSIGNED_SEQ_NO, 0, randomBoolean() ? IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP : randomNonNegativeLong(), true); } long globalCheckPoint = numDocs > 0 ? randomIntBetween(0, numDocs - 1) : 0; @@ -145,7 +145,7 @@ public void testSyncerOnClosingShard() throws Exception { // Index doc but not advance local checkpoint. shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, SourceToParse.source(shard.shardId().getIndexName(), "_doc", Integer.toString(i), new BytesArray("{}"), XContentType.JSON), - IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); } String allocationId = shard.routingEntry().allocationId().getId(); diff --git a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index 9d18845a05e33..97bf5ba30e8e1 100644 --- a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -2723,7 +2723,7 @@ public void testTranslogOpSerialization() throws Exception { null); Engine.Index eIndex = new Engine.Index(newUid(doc), doc, randomSeqNum, randomPrimaryTerm, - 1, VersionType.INTERNAL, Origin.PRIMARY, 0, 0, false); + 1, VersionType.INTERNAL, Origin.PRIMARY, 0, 0, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.IndexResult eIndexResult = new Engine.IndexResult(1, randomPrimaryTerm, randomSeqNum, true); Translog.Index index = new Translog.Index(eIndex, eIndexResult); @@ -2734,7 +2734,7 @@ public void testTranslogOpSerialization() throws Exception { assertEquals(index, serializedIndex); Engine.Delete eDelete = new Engine.Delete(doc.type(), doc.id(), newUid(doc), randomSeqNum, randomPrimaryTerm, - 2, VersionType.INTERNAL, Origin.PRIMARY, 0); + 2, VersionType.INTERNAL, Origin.PRIMARY, 0, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); Engine.DeleteResult eDeleteResult = new Engine.DeleteResult(2, randomPrimaryTerm, randomSeqNum, true); Translog.Delete delete = new Translog.Delete(eDelete, eDeleteResult); diff --git a/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java b/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java index ea23ae6308e47..edc18fbec6fd2 100644 --- a/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java +++ b/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java @@ -51,6 +51,7 @@ import org.elasticsearch.index.engine.InternalEngineTests; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.ShardId; @@ -240,7 +241,7 @@ public void testUnallocatedShardsDoesNotHang() throws InterruptedException { private void indexDoc(Engine engine, String id) throws IOException { final ParsedDocument doc = InternalEngineTests.createParsedDoc(id, null); final Engine.IndexResult indexResult = engine.index(new Engine.Index(new Term("_id", Uid.encodeId(doc.id())), doc, - engine.getLocalCheckpoint() + 1, 1L, 1L, null, Engine.Operation.Origin.REPLICA, randomLong(), -1L, false)); + engine.getLocalCheckpoint() + 1, 1L, 1L, null, Engine.Operation.Origin.REPLICA, randomLong(), -1L, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); assertThat(indexResult.getFailure(), nullValue()); } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index 2a53c79448d15..694032bd9887a 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -314,7 +314,7 @@ public void testPeerRecoverySendSafeCommitInFileBased() throws Exception { Engine.IndexResult result = primaryShard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, SourceToParse.source(primaryShard.shardId().getIndexName(), "_doc", Integer.toString(i), new BytesArray("{}"), XContentType.JSON), - IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); if (randomBoolean()) { globalCheckpoint = randomLongBetween(globalCheckpoint, i); diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 12f0d645d8a87..5ed0b58c445c4 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -594,11 +594,11 @@ protected Engine.Index indexForDoc(ParsedDocument doc) { protected Engine.Index replicaIndexForDoc(ParsedDocument doc, long version, long seqNo, boolean isRetry) { return new Engine.Index(newUid(doc), doc, seqNo, primaryTerm.get(), version, null, Engine.Operation.Origin.REPLICA, - System.nanoTime(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, isRetry); + System.nanoTime(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, long startTime) { - return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, null, Engine.Operation.Origin.REPLICA, startTime); + return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, null, Engine.Operation.Origin.REPLICA, startTime, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } protected static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { assertVisibleCount(engine, numDocs, true); @@ -649,8 +649,8 @@ public static List generateSingleDocHistory(boolean forReplica version, forReplica ? null : versionType, forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis(), -1, false - ); + System.currentTimeMillis(), -1, false, + SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } else { op = new Engine.Delete("test", docId, id, forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, @@ -658,7 +658,7 @@ public static List generateSingleDocHistory(boolean forReplica version, forReplica ? null : versionType, forReplica ? REPLICA : PRIMARY, - System.currentTimeMillis()); + System.currentTimeMillis(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } ops.add(op); } diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java b/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java index 12785841ef2d0..830fcec3726a0 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/TranslogHandler.java @@ -31,6 +31,7 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.RootObjectMapper; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.translog.Translog; @@ -124,12 +125,12 @@ private Engine.Operation convertToEngineOp(Translog.Operation operation, Engine. source(indexName, index.type(), index.id(), index.source(), XContentHelper.xContentType(index.source())) .routing(index.routing()), index.seqNo(), index.primaryTerm(), - index.version(), null, origin, index.getAutoGeneratedIdTimestamp(), true); + index.version(), null, origin, index.getAutoGeneratedIdTimestamp(), true, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); return engineIndex; case DELETE: final Translog.Delete delete = (Translog.Delete) operation; final Engine.Delete engineDelete = new Engine.Delete(delete.type(), delete.id(), delete.uid(), delete.seqNo(), - delete.primaryTerm(), delete.version(), null, origin, System.nanoTime()); + delete.primaryTerm(), delete.version(), null, origin, System.nanoTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); return engineDelete; case NO_OP: final Translog.NoOp noOp = (Translog.NoOp) operation; diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index c9ef79720a29e..8dd8f59c4672e 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -689,12 +689,12 @@ protected Engine.IndexResult indexDoc(IndexShard shard, String type, String id, Engine.IndexResult result; if (shard.routingEntry().primary()) { result = shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, sourceToParse, - IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); if (result.getResultType() == Engine.Result.Type.MAPPING_UPDATE_REQUIRED) { updateMappings(shard, IndexMetaData.builder(shard.indexSettings().getIndexMetaData()) .putMapping(type, result.getRequiredMappingUpdate().toString()).build()); result = shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, sourceToParse, - IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); + SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false); } shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(), shard.getLocalCheckpoint()); @@ -718,7 +718,8 @@ protected void updateMappings(IndexShard shard, IndexMetaData indexMetadata) { protected Engine.DeleteResult deleteDoc(IndexShard shard, String type, String id) throws IOException { final Engine.DeleteResult result; if (shard.routingEntry().primary()) { - result = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, type, id, VersionType.INTERNAL); + result = shard.applyDeleteOperationOnPrimary( + Versions.MATCH_ANY, type, id, VersionType.INTERNAL, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(), shard.getEngine().getLocalCheckpoint()); } else { final long seqNo = shard.seqNoStats().getMaxSeqNo() + 1; diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java index ce67cfe2d4484..c48d75330058a 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java @@ -196,7 +196,7 @@ public void runDeleteTest( randomNonNegativeLong(), VersionType.EXTERNAL, origin, - System.currentTimeMillis()); + System.currentTimeMillis(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); consumer.accept(followingEngine, delete); } @@ -285,7 +285,7 @@ private Engine.Index indexForFollowing(String id, long seqNo, Engine.Operation.O final long version = randomBoolean() ? 1 : randomNonNegativeLong(); final ParsedDocument parsedDocument = EngineTestCase.createParsedDoc(id, null); return new Engine.Index(EngineTestCase.newUid(parsedDocument), parsedDocument, seqNo, primaryTerm.get(), version, - VersionType.EXTERNAL, origin, System.currentTimeMillis(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, randomBoolean()); + VersionType.EXTERNAL, origin, System.currentTimeMillis(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, randomBoolean(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } private Engine.Index indexForPrimary(String id) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 7058724ecf0bc..645883fee3b41 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -47,6 +47,7 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.SeqNoStats; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -292,7 +293,7 @@ public IndexShard reindex(DirectoryReader reader, MappingMetaData mapping) throw assert source != null : "_source is null but should have been filtered out at snapshot time"; Engine.Result result = targetShard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, source (index, uid.type(), uid.id(), source, XContentHelper.xContentType(source)) - .routing(rootFieldsVisitor.routing()), 1, false); + .routing(rootFieldsVisitor.routing()), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 1, false); if (result.getResultType() != Engine.Result.Type.SUCCESS) { throw new IllegalStateException("failed applying post restore operation result: " + result .getResultType(), result.getFailure()); From c6fca4ce22b58de1d238e0edcb2d125d07371b7e Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 5 Oct 2018 14:30:07 -0400 Subject: [PATCH 02/33] lint --- .../action/bulk/TransportShardBulkActionTests.java | 9 ++++++--- .../elasticsearch/common/lucene/uid/VersionsTests.java | 1 - .../index/shard/PrimaryReplicaSyncerTests.java | 3 ++- .../org/elasticsearch/index/engine/EngineTestCase.java | 3 ++- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java index 78861596b7851..37e82884c5133 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportShardBulkActionTests.java @@ -268,7 +268,8 @@ public void testExecuteBulkIndexRequestWithMappingUpdates() throws Exception { Engine.IndexResult success = new FakeIndexResult(1, 1, 13, true, resultLocation); IndexShard shard = mock(IndexShard.class); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())).thenReturn(mappingUpdate); + when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())) + .thenReturn(mappingUpdate); // Pretend the mappings haven't made it to the node yet BulkPrimaryExecutionContext context = new BulkPrimaryExecutionContext(bulkShardRequest, shard); @@ -287,7 +288,8 @@ public void testExecuteBulkIndexRequestWithMappingUpdates() throws Exception { // Verify that the shard "executed" the operation twice verify(shard, times(2)).applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean()); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())).thenReturn(success); + when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())) + .thenReturn(success); TransportShardBulkAction.executeBulkItemRequest(context, null, threadPool::absoluteTimeInMillis, (update, shardId, type) -> fail("should not have had to update the mappings"), () -> {}); @@ -488,7 +490,8 @@ public void testUpdateRequestWithFailure() throws Exception { Exception err = new ElasticsearchException("I'm dead <(x.x)>"); Engine.IndexResult indexResult = new Engine.IndexResult(err, 0, 0, 0); IndexShard shard = mock(IndexShard.class); - when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())).thenReturn(indexResult); + when(shard.applyIndexOperationOnPrimary(anyLong(), any(), any(), anyLong(), anyLong(), anyLong(), anyBoolean())) + .thenReturn(indexResult); when(shard.indexSettings()).thenReturn(indexSettings); UpdateHelper updateHelper = mock(UpdateHelper.class); diff --git a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java index fd648570a96e8..5efa15f143c48 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; -import org.hamcrest.MatcherAssert; import java.io.IOException; import java.util.ArrayList; diff --git a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java index 31fa2d4872731..213a1fb2ca4a8 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -76,7 +76,8 @@ public void testSyncerSendsOffCorrectDocuments() throws Exception { // Index doc but not advance local checkpoint. shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, SourceToParse.source(shard.shardId().getIndexName(), "_doc", Integer.toString(i), new BytesArray("{}"), XContentType.JSON), - SequenceNumbers.UNASSIGNED_SEQ_NO, 0, randomBoolean() ? IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP : randomNonNegativeLong(), true); + SequenceNumbers.UNASSIGNED_SEQ_NO, 0, + randomBoolean() ? IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP : randomNonNegativeLong(), true); } long globalCheckPoint = numDocs > 0 ? randomIntBetween(0, numDocs - 1) : 0; diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 5ed0b58c445c4..4d748bf642606 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -598,7 +598,8 @@ protected Engine.Index replicaIndexForDoc(ParsedDocument doc, long version, long } protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, long startTime) { - return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, null, Engine.Operation.Origin.REPLICA, startTime, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, null, Engine.Operation.Origin.REPLICA, startTime, + SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } protected static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { assertVisibleCount(engine, numDocs, true); From a91d18f3a11845f615f8f022f85df37e615eaafc Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 8 Oct 2018 07:24:19 +0100 Subject: [PATCH 03/33] adapted test --- .../index/engine/InternalEngine.java | 6 +- .../index/engine/InternalEngineTests.java | 212 +++++++++++------- 2 files changed, 134 insertions(+), 84 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index ab2cc1961a887..e55b038e77014 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -999,7 +999,7 @@ private IndexResult indexIntoLucene(Index index, IndexingStrategy plan) assert assertDocDoesNotExist(index, canOptimizeAddDocument(index) == false); addDocs(index.docs(), indexWriter); } - return new IndexResult(plan.versionForIndexing, getPrimaryTerm(), plan.seqNoForIndexing, plan.currentNotFoundOrDeleted); + return new IndexResult(plan.versionForIndexing, index.primaryTerm(), plan.seqNoForIndexing, plan.currentNotFoundOrDeleted); } catch (Exception ex) { if (indexWriter.getTragicException() == null) { /* There is no tragic event recorded so this must be a document failure. @@ -1015,7 +1015,7 @@ private IndexResult indexIntoLucene(Index index, IndexingStrategy plan) * we return a `MATCH_ANY` version to indicate no document was index. The value is * not used anyway */ - return new IndexResult(ex, Versions.MATCH_ANY, getPrimaryTerm(), plan.seqNoForIndexing); + return new IndexResult(ex, Versions.MATCH_ANY, index.primaryTerm(), plan.seqNoForIndexing); } else { throw ex; } @@ -1337,7 +1337,7 @@ private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) throws if (indexWriter.getTragicException() == null) { // there is no tragic event and such it must be a document level failure return new DeleteResult( - ex, plan.versionOfDeletion, getPrimaryTerm(), plan.seqNoOfDeletion, plan.currentlyDeleted == false); + ex, plan.versionOfDeletion, delete.primaryTerm(), plan.seqNoOfDeletion, plan.currentlyDeleted == false); } else { throw ex; } diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 6b7e481c81423..7c32343e3c2b7 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -20,6 +20,8 @@ package org.elasticsearch.index.engine; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import com.carrotsearch.randomizedtesting.annotations.Repeat; +import com.carrotsearch.randomizedtesting.annotations.Seed; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import org.apache.logging.log4j.Level; @@ -79,6 +81,7 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -169,6 +172,7 @@ import static org.elasticsearch.index.engine.Engine.Operation.Origin.PEER_RECOVERY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.sameInstance; @@ -582,7 +586,7 @@ public void testSegmentsStatsIncludingFileSizes() throws Exception { public void testCommitStats() throws IOException { final AtomicLong maxSeqNo = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final AtomicLong localCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final AtomicLong globalCheckpoint = new AtomicLong(UNASSIGNED_SEQ_NO); try ( Store store = createStore(); InternalEngine engine = createEngine(store, createTempDir(), (maxSeq, localCP) -> new LocalCheckpointTracker( @@ -618,7 +622,7 @@ public long getCheckpoint() { rarely() || maxSeqNo.get() == SequenceNumbers.NO_OPS_PERFORMED ? SequenceNumbers.NO_OPS_PERFORMED : randomIntBetween(0, 1024)); globalCheckpoint.set(rarely() || localCheckpoint.get() == SequenceNumbers.NO_OPS_PERFORMED ? - SequenceNumbers.UNASSIGNED_SEQ_NO : randomIntBetween(0, (int) localCheckpoint.get())); + UNASSIGNED_SEQ_NO : randomIntBetween(0, (int) localCheckpoint.get())); final Engine.CommitId commitId = engine.flush(true, true); @@ -698,11 +702,11 @@ public void testTranslogMultipleOperationsSameDocument() throws IOException { for (int i = 0; i < ops; i++) { final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null); if (randomBoolean()) { - final Engine.Index operation = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + final Engine.Index operation = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); operations.add(operation); initialEngine.index(operation); } else { - final Engine.Delete operation = new Engine.Delete("test", "1", newUid(doc), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + final Engine.Delete operation = new Engine.Delete("test", "1", newUid(doc), UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), UNASSIGNED_SEQ_NO, 0); operations.add(operation); initialEngine.delete(operation); } @@ -1143,7 +1147,7 @@ public void testRenewSyncFlush() throws Exception { final boolean forceMergeFlushes = randomBoolean(); final ParsedDocument parsedDoc3 = testParsedDocument("3", null, testDocumentWithTextField(), B_1, null); if (forceMergeFlushes) { - engine.index(new Engine.Index(newUid(parsedDoc3), parsedDoc3, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); + engine.index(new Engine.Index(newUid(parsedDoc3), parsedDoc3, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos(), -1, false, UNASSIGNED_SEQ_NO, 0)); } else { engine.index(indexForDoc(parsedDoc3)); } @@ -1206,7 +1210,7 @@ public void testSyncedFlushSurvivesEngineRestart() throws IOException { } if (randomBoolean()) { final String translogUUID = Translog.createEmptyTranslog(config.getTranslogConfig().getTranslogPath(), - SequenceNumbers.UNASSIGNED_SEQ_NO, shardId, primaryTerm.get()); + UNASSIGNED_SEQ_NO, shardId, primaryTerm.get()); store.associateIndexWithNewTranslog(translogUUID); } trimUnsafeCommits(config); @@ -1243,7 +1247,7 @@ public void testVersioningNewCreate() throws IOException { assertThat(indexResult.getVersion(), equalTo(1L)); create = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), create.primaryTerm(), indexResult.getVersion(), - null, REPLICA, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + null, REPLICA, 0, -1, false, UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(create); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -1257,7 +1261,7 @@ public void testReplicatedVersioningWithFlush() throws IOException { create = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), create.primaryTerm(), indexResult.getVersion(), - null, REPLICA, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + null, REPLICA, 0, -1, false, UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(create); assertThat(indexResult.getVersion(), equalTo(1L)); assertTrue(indexResult.isCreated()); @@ -1276,7 +1280,7 @@ public void testReplicatedVersioningWithFlush() throws IOException { update = new Engine.Index(newUid(doc), doc, updateResult.getSeqNo(), update.primaryTerm(), updateResult.getVersion(), - null, REPLICA, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + null, REPLICA, 0, -1, false, UNASSIGNED_SEQ_NO, 0); updateResult = replicaEngine.index(update); assertThat(updateResult.getVersion(), equalTo(2L)); assertFalse(updateResult.isCreated()); @@ -1329,7 +1333,7 @@ public void testVersioningNewIndex() throws IOException { Engine.IndexResult indexResult = engine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); - index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, 0, -1, false, UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -1608,11 +1612,11 @@ public void run() { public void testVersioningCreateExistsException() throws IOException { ParsedDocument doc = testParsedDocument("1", null, testDocument(), B_1, null); - Engine.Index create = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index create = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(create); assertThat(indexResult.getVersion(), equalTo(1L)); - create = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + create = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false, UNASSIGNED_SEQ_NO, 0); indexResult = engine.index(create); assertThat(indexResult.getResultType(), equalTo(Engine.Result.Type.FAILURE)); assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); @@ -1655,11 +1659,11 @@ public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, Interrup Document doc = testDocumentWithTextField(index.docs().get(0).get("value")); ParsedDocument parsedDocument = testParsedDocument(index.id(), index.routing(), doc, index.source(), null); return new Engine.Index(index.uid(), parsedDocument, newSeqNo, index.primaryTerm(), index.version(), - index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry(), UNASSIGNED_SEQ_NO, 0); } else { Engine.Delete delete = (Engine.Delete) operation; return new Engine.Delete(delete.type(), delete.id(), delete.uid(), newSeqNo, delete.primaryTerm(), - delete.version(), delete.versionType(), delete.origin(), delete.startTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + delete.version(), delete.versionType(), delete.origin(), delete.startTime(), UNASSIGNED_SEQ_NO, 0); } }; final List allOps = new ArrayList<>(); @@ -1731,33 +1735,63 @@ private int assertOpsOnPrimary(List ops, long currentOpVersion String lastFieldValue = null; int opsPerformed = 0; long lastOpVersion = currentOpVersion; + long lastOpSeqNo = UNASSIGNED_SEQ_NO; + long lastOpTerm = 0; + final AtomicLong currentTerm = new AtomicLong(1); 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(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + UNASSIGNED_SEQ_NO, currentTerm.get(), version, index.versionType(), index.origin(), index.startTime(), + index.getAutoGeneratedIdTimestamp(), index.isRetry(), UNASSIGNED_SEQ_NO, 0); BiFunction delWithVersion = (version, delete) -> new Engine.Delete(delete.type(), delete.id(), - delete.uid(), delete.seqNo(), delete.primaryTerm(), version, delete.versionType(), delete.origin(), delete.startTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + delete.uid(), UNASSIGNED_SEQ_NO, currentTerm.get(), version, delete.versionType(), delete.origin(), delete.startTime(), + UNASSIGNED_SEQ_NO, 0); + TriFunction indexWithSeq = (seqNo, term, index) -> new Engine.Index(index.uid(), index.parsedDoc(), + UNASSIGNED_SEQ_NO, currentTerm.get(), index.version(), index.versionType(), index.origin(), index.startTime(), + index.getAutoGeneratedIdTimestamp(), index.isRetry(), seqNo, term); + TriFunction delWitSeq = (seqNo, term, delete) -> new Engine.Delete(delete.type(), delete.id(), + delete.uid(), UNASSIGNED_SEQ_NO, currentTerm.get(), delete.version(), delete.versionType(), delete.origin(), delete.startTime(), + seqNo, term); for (Engine.Operation op : ops) { final boolean versionConflict = rarely(); final boolean versionedOp = versionConflict || randomBoolean(); final long conflictingVersion = docDeleted || randomBoolean() ? lastOpVersion + (randomBoolean() ? 1 : -1) : Versions.MATCH_DELETED; + final long conflictingSeqNo = randomBoolean() ? lastOpSeqNo : lastOpSeqNo + 1; + final long conflictingTerm = conflictingSeqNo == lastOpSeqNo || randomBoolean() ? lastOpTerm + 1 : lastOpTerm; + if (rarely()) { + currentTerm.incrementAndGet(); + } final long correctVersion = docDeleted && randomBoolean() ? Versions.MATCH_DELETED : lastOpVersion; logger.info("performing [{}]{}{}", op.operationType().name().charAt(0), versionConflict ? " (conflict " + conflictingVersion + ")" : "", - versionedOp ? " (versioned " + correctVersion + ")" : ""); + versionedOp ? " (versioned " + correctVersion + ", seqNo " + lastOpSeqNo + ", term " + lastOpTerm + " )" : ""); if (op instanceof Engine.Index) { final Engine.Index index = (Engine.Index) op; if (versionConflict) { // generate a conflict - Engine.IndexResult result = engine.index(indexWithVersion.apply(conflictingVersion, index)); + final Engine.IndexResult result; + if (randomBoolean()) { + result = engine.index(indexWithSeq.apply(conflictingSeqNo, conflictingTerm, index)); + } else { + result = engine.index(indexWithVersion.apply(conflictingVersion, index)); + } assertThat(result.isCreated(), equalTo(false)); assertThat(result.getVersion(), equalTo(lastOpVersion)); assertThat(result.getResultType(), equalTo(Engine.Result.Type.FAILURE)); assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); } else { - Engine.IndexResult result = engine.index(versionedOp ? indexWithVersion.apply(correctVersion, index) : index); + final Engine.IndexResult result; + if (versionedOp) { + // TODO: add support for non-existing docs + if (randomBoolean() && lastOpSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { + result = engine.index(indexWithSeq.apply(lastOpSeqNo, lastOpTerm, index)); + } else { + result = engine.index(indexWithVersion.apply(correctVersion, index)); + } + } else { + result = engine.index(index); + } assertThat(result.isCreated(), equalTo(docDeleted)); assertThat(result.getVersion(), equalTo(Math.max(lastOpVersion + 1, 1))); assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); @@ -1765,25 +1799,41 @@ private int assertOpsOnPrimary(List ops, long currentOpVersion lastFieldValue = index.docs().get(0).get("value"); docDeleted = false; lastOpVersion = result.getVersion(); + lastOpSeqNo = result.getSeqNo(); + lastOpTerm = result.getTerm(); opsPerformed++; } } else { final Engine.Delete delete = (Engine.Delete) op; if (versionConflict) { // generate a conflict - Engine.DeleteResult result = engine.delete(delWithVersion.apply(conflictingVersion, delete)); + Engine.DeleteResult result; + if (randomBoolean()) { + result = engine.delete(delWitSeq.apply(conflictingSeqNo, conflictingTerm, delete)); + } else { + result = engine.delete(delWithVersion.apply(conflictingVersion, delete)); + } assertThat(result.isFound(), equalTo(docDeleted == false)); assertThat(result.getVersion(), equalTo(lastOpVersion)); assertThat(result.getResultType(), equalTo(Engine.Result.Type.FAILURE)); assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class)); } else { - Engine.DeleteResult result = engine.delete(versionedOp ? delWithVersion.apply(correctVersion, delete) : delete); + final Engine.DeleteResult result; + if (versionedOp && lastOpSeqNo != UNASSIGNED_SEQ_NO && randomBoolean()) { + result = engine.delete(delWitSeq.apply(lastOpSeqNo, lastOpTerm, delete)); + } else if (versionedOp) { + result = engine.delete(delWithVersion.apply(correctVersion, delete)); + } else { + result = engine.delete(delete); + } assertThat(result.isFound(), equalTo(docDeleted == false)); assertThat(result.getVersion(), equalTo(Math.max(lastOpVersion + 1, 1))); assertThat(result.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); assertThat(result.getFailure(), nullValue()); docDeleted = true; lastOpVersion = result.getVersion(); + lastOpSeqNo = UNASSIGNED_SEQ_NO; + lastOpTerm = 0; opsPerformed++; } } @@ -1992,9 +2042,9 @@ class OpAndVersion { Engine.Index index = new Engine.Index(uidTerm, testParsedDocument("1", null, testDocument(), bytesArray(Strings.collectionToCommaDelimitedString(values)), null), - SequenceNumbers.UNASSIGNED_SEQ_NO, 2, + UNASSIGNED_SEQ_NO, 2, get.version(), VersionType.INTERNAL, - PRIMARY, System.currentTimeMillis(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + PRIMARY, System.currentTimeMillis(), -1, false, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(index); if (indexResult.getResultType() == Engine.Result.Type.SUCCESS) { history.add(new OpAndVersion(indexResult.getVersion(), removed, added)); @@ -2137,8 +2187,8 @@ public void testSeqNoAndCheckpoints() throws IOException { // we have some docs indexed, so delete one of them id = randomFrom(indexedIds); final Engine.Delete delete = new Engine.Delete( - "test", id, newUid(id), SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm.get(), - rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + "test", id, newUid(id), UNASSIGNED_SEQ_NO, primaryTerm.get(), + rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0, UNASSIGNED_SEQ_NO, 0); final Engine.DeleteResult result = initialEngine.delete(delete); if (result.getResultType() == Engine.Result.Type.SUCCESS) { assertThat(result.getSeqNo(), equalTo(primarySeqNo + 1)); @@ -2146,7 +2196,7 @@ public void testSeqNoAndCheckpoints() throws IOException { indexedIds.remove(id); primarySeqNo++; } else { - assertThat(result.getSeqNo(), equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO)); + assertThat(result.getSeqNo(), equalTo(UNASSIGNED_SEQ_NO)); assertThat(initialEngine.getSeqNoStats(-1).getMaxSeqNo(), equalTo(primarySeqNo)); } } else { @@ -2154,9 +2204,9 @@ public void testSeqNoAndCheckpoints() throws IOException { id = randomFrom(ids); ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null); final Engine.Index index = new Engine.Index(newUid(doc), doc, - SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm.get(), + UNASSIGNED_SEQ_NO, primaryTerm.get(), rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, - PRIMARY, 0, -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + PRIMARY, 0, -1, false, UNASSIGNED_SEQ_NO, 0); final Engine.IndexResult result = initialEngine.index(index); if (result.getResultType() == Engine.Result.Type.SUCCESS) { assertThat(result.getSeqNo(), equalTo(primarySeqNo + 1)); @@ -2164,7 +2214,7 @@ public void testSeqNoAndCheckpoints() throws IOException { indexedIds.add(id); primarySeqNo++; } else { - assertThat(result.getSeqNo(), equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO)); + assertThat(result.getSeqNo(), equalTo(UNASSIGNED_SEQ_NO)); assertThat(initialEngine.getSeqNoStats(-1).getMaxSeqNo(), equalTo(primarySeqNo)); } } @@ -2294,7 +2344,7 @@ public void testConcurrentWritesAndCommits() throws Exception { SequenceNumbers.NO_OPS_PERFORMED; long maxSeqNo = userData.containsKey(SequenceNumbers.MAX_SEQ_NO) ? Long.parseLong(userData.get(SequenceNumbers.MAX_SEQ_NO)) : - SequenceNumbers.UNASSIGNED_SEQ_NO; + UNASSIGNED_SEQ_NO; // local checkpoint and max seq no shouldn't go backwards assertThat(localCheckpoint, greaterThanOrEqualTo(prevLocalCheckpoint)); assertThat(maxSeqNo, greaterThanOrEqualTo(prevMaxSeqNo)); @@ -2407,10 +2457,10 @@ public void testEnableGcDeletes() throws Exception { document.add(new TextField("value", "test1", Field.Store.YES)); ParsedDocument doc = testParsedDocument("1", null, document, B_2, null); - engine.index(new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); + engine.index(new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0)); // Delete document we just added: - engine.delete(new Engine.Delete("test", "1", newUid(doc), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); + engine.delete(new Engine.Delete("test", "1", newUid(doc), UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), UNASSIGNED_SEQ_NO, 0)); // Get should not find the document Engine.GetResult getResult = engine.get(newGet(true, doc), searcherFactory); @@ -2424,14 +2474,14 @@ public void testEnableGcDeletes() throws Exception { } // Delete non-existent document - engine.delete(new Engine.Delete("test", "2", newUid("2"), SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); + engine.delete(new Engine.Delete("test", "2", newUid("2"), UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), UNASSIGNED_SEQ_NO, 0)); // Get should not find the document (we never indexed uid=2): getResult = engine.get(new Engine.Get(true, false, "type", "2", newUid("2")), searcherFactory); assertThat(getResult.exists(), equalTo(false)); // Try to index uid=1 with a too-old version, should fail: - Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index index = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(index); assertThat(indexResult.getResultType(), equalTo(Engine.Result.Type.FAILURE)); assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); @@ -2441,7 +2491,7 @@ public void testEnableGcDeletes() throws Exception { assertThat(getResult.exists(), equalTo(false)); // Try to index uid=2 with a too-old version, should fail: - Engine.Index index1 = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index index1 = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); indexResult = engine.index(index1); assertThat(indexResult.getResultType(), equalTo(Engine.Result.Type.FAILURE)); assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class)); @@ -2524,8 +2574,8 @@ public void testCurrentTranslogIDisCommitted() throws IOException { Translog.createEmptyTranslog(config.getTranslogConfig().getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get()); store.associateIndexWithNewTranslog(translogUUID); ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); try (InternalEngine engine = createEngine(config)) { engine.index(firstIndexRequest); @@ -2612,7 +2662,7 @@ public void testMissingTranslog() throws IOException { // expected } // when a new translog is created it should be ok - final String translogUUID = Translog.createEmptyTranslog(primaryTranslogDir, SequenceNumbers.UNASSIGNED_SEQ_NO, shardId, primaryTerm); + final String translogUUID = Translog.createEmptyTranslog(primaryTranslogDir, UNASSIGNED_SEQ_NO, shardId, primaryTerm); store.associateIndexWithNewTranslog(translogUUID); EngineConfig config = config(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null); engine = new InternalEngine(config); @@ -2626,8 +2676,8 @@ public void testTranslogReplayWithFailure() throws IOException { try (InternalEngine engine = createEngine(store, translogPath)) { for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -2721,7 +2771,7 @@ public void testSkipTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -2761,7 +2811,7 @@ public void testTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); } @@ -2793,7 +2843,7 @@ public void testTranslogReplay() throws IOException { final boolean flush = randomBoolean(); int randomId = randomIntBetween(numDocs + 1, numDocs + 10); ParsedDocument doc = testParsedDocument(Integer.toString(randomId), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(firstIndexRequest); assertThat(indexResult.getVersion(), equalTo(1L)); if (flush) { @@ -2802,7 +2852,7 @@ public void testTranslogReplay() throws IOException { } doc = testParsedDocument(Integer.toString(randomId), null, testDocument(), new BytesArray("{}"), null); - Engine.Index idxRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index idxRequest = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult result = engine.index(idxRequest); engine.refresh("test"); assertThat(result.getVersion(), equalTo(2L)); @@ -2836,7 +2886,7 @@ public void testRecoverFromForeignTranslog() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult index = engine.index(firstIndexRequest); assertThat(index.getVersion(), equalTo(1L)); } @@ -2863,7 +2913,7 @@ public void testRecoverFromForeignTranslog() throws IOException { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, - new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm::get, tombstoneDocSupplier()); + new NoneCircuitBreakerService(), () -> UNASSIGNED_SEQ_NO, primaryTerm::get, tombstoneDocSupplier()); expectThrows(EngineCreationFailureException.class, () -> new InternalEngine(brokenConfig)); engine = createEngine(store, primaryTranslogDir); // and recover again! @@ -3156,7 +3206,7 @@ public void testDoubleDeliveryReplicaAppendingAndDeleteOnly() throws IOException Engine.Index retry = appendOnlyReplica(doc, true, 1, randomIntBetween(0, 5)); Engine.Delete delete = new Engine.Delete(operation.type(), operation.id(), operation.uid(), Math.max(retry.seqNo(), operation.seqNo())+1, operation.primaryTerm(), operation.version()+1, operation.versionType(), - REPLICA, operation.startTime()+1, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + REPLICA, operation.startTime()+1, UNASSIGNED_SEQ_NO, 0); // operations with a seq# equal or lower to the local checkpoint are not indexed to lucene // and the version lookup is skipped final boolean belowLckp = operation.seqNo() == 0 && retry.seqNo() == 0; @@ -3312,16 +3362,16 @@ public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOExcep boolean isRetry = false; long autoGeneratedIdTimestamp = 0; - Engine.Index index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index index = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); - index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); isRetry = true; - index = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + index = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, UNASSIGNED_SEQ_NO, 0); indexResult = engine.index(index); assertThat(indexResult.getVersion(), equalTo(1L)); engine.refresh("test"); @@ -3330,7 +3380,7 @@ public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOExcep assertEquals(1, topDocs.totalHits.value); } - index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + index = new Engine.Index(newUid(doc), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, UNASSIGNED_SEQ_NO, 0); indexResult = replicaEngine.index(index); assertThat(indexResult.getResultType(), equalTo(Engine.Result.Type.SUCCESS)); replicaEngine.refresh("test"); @@ -3346,16 +3396,16 @@ public void testRetryWithAutogeneratedIdsAndWrongOrderWorksAndNoDuplicateDocs() boolean isRetry = true; long autoGeneratedIdTimestamp = 0; - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult result = engine.index(firstIndexRequest); assertThat(result.getVersion(), equalTo(1L)); - Engine.Index firstIndexRequestReplica = new Engine.Index(newUid(doc), doc, result.getSeqNo(), firstIndexRequest.primaryTerm(), result.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index firstIndexRequestReplica = new Engine.Index(newUid(doc), doc, result.getSeqNo(), firstIndexRequest.primaryTerm(), result.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexReplicaResult = replicaEngine.index(firstIndexRequestReplica); assertThat(indexReplicaResult.getVersion(), equalTo(1L)); isRetry = false; - Engine.Index secondIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index secondIndexRequest = new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, UNASSIGNED_SEQ_NO, 0); Engine.IndexResult indexResult = engine.index(secondIndexRequest); assertTrue(indexResult.isCreated()); engine.refresh("test"); @@ -3364,7 +3414,7 @@ public void testRetryWithAutogeneratedIdsAndWrongOrderWorksAndNoDuplicateDocs() assertEquals(1, topDocs.totalHits.value); } - Engine.Index secondIndexRequestReplica = new Engine.Index(newUid(doc), doc, result.getSeqNo(), secondIndexRequest.primaryTerm(), result.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Index secondIndexRequestReplica = new Engine.Index(newUid(doc), doc, result.getSeqNo(), secondIndexRequest.primaryTerm(), result.getVersion(), null, REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry, UNASSIGNED_SEQ_NO, 0); replicaEngine.index(secondIndexRequestReplica); replicaEngine.refresh("test"); try (Engine.Searcher searcher = replicaEngine.acquireSearcher("test")) { @@ -3382,13 +3432,13 @@ public Engine.Index randomAppendOnly(ParsedDocument doc, boolean retry, final lo } public Engine.Index appendOnlyPrimary(ParsedDocument doc, boolean retry, final long autoGeneratedIdTimestamp) { - return new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, - VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, retry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + return new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, + VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, retry, UNASSIGNED_SEQ_NO, 0); } public Engine.Index appendOnlyReplica(ParsedDocument doc, boolean retry, final long autoGeneratedIdTimestamp, final long seqNo) { return new Engine.Index(newUid(doc), doc, seqNo, 2, 1, null, - Engine.Operation.Origin.REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, retry, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Engine.Operation.Origin.REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, retry, UNASSIGNED_SEQ_NO, 0); } public void testRetryConcurrently() throws InterruptedException, IOException { @@ -3633,7 +3683,7 @@ public void afterRefresh(boolean didRefresh) throws IOException { public void testSequenceIDs() throws Exception { Tuple seqID = getSequenceID(engine, new Engine.Get(false, false, "type", "2", newUid("1"))); // Non-existent doc returns no seqnum and no primary term - assertThat(seqID.v1(), equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO)); + assertThat(seqID.v1(), equalTo(UNASSIGNED_SEQ_NO)); assertThat(seqID.v2(), equalTo(0L)); // create a document @@ -3664,9 +3714,9 @@ public void testSequenceIDs() throws Exception { document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); doc = testParsedDocument("1", null, document, B_1, null); - engine.index(new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 3, + engine.index(new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 3, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, - System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); + System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0)); engine.refresh("test"); seqID = getSequenceID(engine, newGet(false, doc)); @@ -3780,7 +3830,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio final AtomicLong sequenceNumber = new AtomicLong(); final Engine.Operation.Origin origin = randomFrom(LOCAL_TRANSLOG_RECOVERY, PEER_RECOVERY, PRIMARY, REPLICA); final LongSupplier sequenceNumberSupplier = - origin == PRIMARY ? () -> SequenceNumbers.UNASSIGNED_SEQ_NO : sequenceNumber::getAndIncrement; + origin == PRIMARY ? () -> UNASSIGNED_SEQ_NO : sequenceNumber::getAndIncrement; final Supplier doc = () -> { final Document document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); @@ -3800,7 +3850,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio origin, System.nanoTime(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, - false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + false, UNASSIGNED_SEQ_NO, 0); operations.add(index); } else { final Engine.Delete delete = new Engine.Delete( @@ -3812,7 +3862,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio i, origin == PRIMARY ? VersionType.EXTERNAL : null, origin, - System.nanoTime(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + System.nanoTime(), UNASSIGNED_SEQ_NO, 0); operations.add(delete); } } @@ -4056,7 +4106,7 @@ private Tuple getSequenceID(Engine engine, Engine.Get get) throws En DocIdAndSeqNo docIdAndSeqNo = VersionsAndSeqNoResolver.loadDocIdAndSeqNo(searcher.reader(), get.uid()); if (docIdAndSeqNo == null) { primaryTerm = 0; - seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + seqNo = UNASSIGNED_SEQ_NO; } else { seqNo = docIdAndSeqNo.seqNo; primaryTerm = VersionsAndSeqNoResolver.loadPrimaryTerm(docIdAndSeqNo, get.uid().field()); @@ -4325,14 +4375,14 @@ public void testSeqNoGenerator() throws IOException { final Engine.Index index = new Engine.Index( new Term("_id", parsedDocument.id()), parsedDocument, - SequenceNumbers.UNASSIGNED_SEQ_NO, + UNASSIGNED_SEQ_NO, randomIntBetween(1, 8), Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.currentTimeMillis(), System.currentTimeMillis(), - randomBoolean(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + randomBoolean(), UNASSIGNED_SEQ_NO, 0); final Engine.IndexResult indexResult = e.index(index); assertThat(indexResult.getSeqNo(), equalTo(seqNo)); assertThat(seqNoGenerator.get(), equalTo(seqNo + 1)); @@ -4341,12 +4391,12 @@ public void testSeqNoGenerator() throws IOException { type, id, new Term("_id", parsedDocument.id()), - SequenceNumbers.UNASSIGNED_SEQ_NO, + UNASSIGNED_SEQ_NO, randomIntBetween(1, 8), Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, - System.currentTimeMillis(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + System.currentTimeMillis(), UNASSIGNED_SEQ_NO, 0); final Engine.DeleteResult deleteResult = e.delete(delete); assertThat(deleteResult.getSeqNo(), equalTo(seqNo + 1)); assertThat(seqNoGenerator.get(), equalTo(seqNo + 2)); @@ -4400,7 +4450,7 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s final List commits = DirectoryReader.listCommits(store.directory()); // Keep only one safe commit as the oldest commit. final IndexCommit safeCommit = commits.get(0); - if (lastSyncedGlobalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) { + if (lastSyncedGlobalCheckpoint == UNASSIGNED_SEQ_NO) { // If the global checkpoint is still unassigned, we keep an empty(eg. initial) commit as a safe commit. assertThat(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)), equalTo(SequenceNumbers.NO_OPS_PERFORMED)); @@ -4662,24 +4712,24 @@ public void testStressUpdateSameDocWhileGettingIt() throws IOException, Interrup engine.engineConfig.getIndexSettings().updateIndexMetaData(indexMetaData); engine.onSettingsChanged(); ParsedDocument document = testParsedDocument(Integer.toString(0), null, testDocumentWithTextField(), SOURCE, null); - final Engine.Index doc = new Engine.Index(newUid(document), document, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + final Engine.Index doc = new Engine.Index(newUid(document), document, UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, UNASSIGNED_SEQ_NO, 0); // first index an append only document and then delete it. such that we have it in the tombstones engine.index(doc); engine.delete(new Engine.Delete(doc.type(), doc.id(), doc.uid(), primaryTerm.get())); // now index more append only docs and refresh so we re-enabel the optimization for unsafe version map ParsedDocument document1 = testParsedDocument(Integer.toString(1), null, testDocumentWithTextField(), SOURCE, null); - engine.index(new Engine.Index(newUid(document1), document1, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); + engine.index(new Engine.Index(newUid(document1), document1, UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, UNASSIGNED_SEQ_NO, 0)); engine.refresh("test"); ParsedDocument document2 = testParsedDocument(Integer.toString(2), null, testDocumentWithTextField(), SOURCE, null); - engine.index(new Engine.Index(newUid(document2), document2, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); + engine.index(new Engine.Index(newUid(document2), document2, UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, UNASSIGNED_SEQ_NO, 0)); engine.refresh("test"); ParsedDocument document3 = testParsedDocument(Integer.toString(3), null, testDocumentWithTextField(), SOURCE, null); - final Engine.Index doc3 = new Engine.Index(newUid(document3), document3, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + final Engine.Index doc3 = new Engine.Index(newUid(document3), document3, UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), 0, false, UNASSIGNED_SEQ_NO, 0); engine.index(doc3); engine.engineConfig.setEnableGcDeletes(true); // once we are here the version map is unsafe again and we need to do a refresh inside the get calls to ensure we @@ -4795,7 +4845,7 @@ public void testTrackMaxSeqNoOfNonAppendOnlyOperations() throws Exception { engine.index(doc); } else { engine.delete(new Engine.Delete(doc.type(), doc.id(), doc.uid(), seqno, doc.primaryTerm(), - doc.version(), doc.versionType(), doc.origin(), threadPool.relativeTimeInMillis(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); + doc.version(), doc.versionType(), doc.origin(), threadPool.relativeTimeInMillis(), UNASSIGNED_SEQ_NO, 0)); } maxSeqNoOfNonAppendOnly = seqno; } else { // On primary - do not update max_seqno for non-append-only operations @@ -4863,7 +4913,7 @@ public void testTrimUnsafeCommits() throws Exception { for (int i = 0; i < seqNos.size(); i++) { ParsedDocument doc = testParsedDocument(Long.toString(seqNos.get(i)), null, testDocument(), new BytesArray("{}"), null); Engine.Index index = new Engine.Index(newUid(doc), doc, seqNos.get(i), 0, - 1, null, REPLICA, System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + 1, null, REPLICA, System.nanoTime(), -1, false, UNASSIGNED_SEQ_NO, 0); engine.index(index); if (randomBoolean()) { engine.flush(); From 60680aea370a1319750d8a126b369b9cfc966cfc Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 8 Oct 2018 07:24:56 +0100 Subject: [PATCH 04/33] lint --- .../org/elasticsearch/index/engine/InternalEngineTests.java | 3 --- .../xpack/ccr/index/engine/FollowingEngineTests.java | 3 ++- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 7c32343e3c2b7..b1ffe1d01e2f2 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -20,10 +20,7 @@ package org.elasticsearch.index.engine; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; -import com.carrotsearch.randomizedtesting.annotations.Repeat; -import com.carrotsearch.randomizedtesting.annotations.Seed; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; - import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java index c48d75330058a..67c334b879552 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java @@ -285,7 +285,8 @@ private Engine.Index indexForFollowing(String id, long seqNo, Engine.Operation.O final long version = randomBoolean() ? 1 : randomNonNegativeLong(); final ParsedDocument parsedDocument = EngineTestCase.createParsedDoc(id, null); return new Engine.Index(EngineTestCase.newUid(parsedDocument), parsedDocument, seqNo, primaryTerm.get(), version, - VersionType.EXTERNAL, origin, System.currentTimeMillis(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, randomBoolean(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + VersionType.EXTERNAL, origin, System.currentTimeMillis(), IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, randomBoolean(), + SequenceNumbers.UNASSIGNED_SEQ_NO, 0); } private Engine.Index indexForPrimary(String id) { From c177f2793b2c0414dfd7357098668c0d2baf86aa Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 8 Oct 2018 13:37:30 +0100 Subject: [PATCH 05/33] initial rest parsing --- .../rest-api-spec/test/index/30_cas.yml | 44 +++++++++++++++++ .../action/bulk/BulkRequest.java | 18 +++++-- .../action/bulk/TransportShardBulkAction.java | 6 +-- .../action/delete/DeleteRequest.java | 36 +++++++------- .../action/index/IndexRequest.java | 42 ++++++++-------- .../elasticsearch/index/engine/Engine.java | 48 +++++++++---------- .../index/engine/InternalEngine.java | 24 +++++----- .../elasticsearch/index/shard/IndexShard.java | 24 +++++----- .../action/document/RestDeleteAction.java | 3 ++ .../rest/action/document/RestIndexAction.java | 3 ++ 10 files changed, 154 insertions(+), 94 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml new file mode 100644 index 0000000000000..ded4ee1a5d912 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml @@ -0,0 +1,44 @@ +--- +"Internal version": + + - skip: + version: " - 6.99.99" + reason: cas ops are introduced in 7.0.0 + + - do: + index: + index: test_1 + id: 1 + body: { foo: bar } + - match: { _version: 1} + - set: { _seq_no: $seqno } + - set: { _primary_term: $primary_term } + + - do: + catch: conflict + index: + index: test_1 + id: 1 + cas_seq_no: 10000 + cas_primary_term: $primary_term + body: { foo: bar2 } + - catch: conflict + + - do: + catch: conflict + index: + index: test_1 + id: 1 + cas_seq_no: $seqno + cas_primary_term: 1 + body: { foo: bar2 } + + - do: + index: + index: test_1 + id: 1 + cas_seq_no: $seqno + cas_primary_term: $primary_term + body: { foo: bar2 } + + - match: { _version: 2 } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index 6698aa4b62ab5..9e87519dfc31e 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -44,6 +44,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import java.io.IOException; @@ -77,6 +78,8 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private static final ParseField RETRY_ON_CONFLICT = new ParseField("retry_on_conflict"); private static final ParseField PIPELINE = new ParseField("pipeline"); private static final ParseField SOURCE = new ParseField("_source"); + private static final ParseField CAS_SEQ_NO = new ParseField("cas_seq_no"); + private static final ParseField CAS_TERM = new ParseField("cas_term"); /** * Requests that are part of this request. It is only possible to add things that are both {@link ActionRequest}s and @@ -332,6 +335,8 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null String opType = null; long version = Versions.MATCH_ANY; VersionType versionType = VersionType.INTERNAL; + long casSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + long casPrimaryTerm = 0; int retryOnConflict = 0; String pipeline = defaultPipeline; @@ -362,6 +367,10 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null version = parser.longValue(); } else if (VERSION_TYPE.match(currentFieldName, parser.getDeprecationHandler())) { versionType = VersionType.fromString(parser.text()); + } else if (CAS_SEQ_NO.match(currentFieldName, parser.getDeprecationHandler())) { + casSeqNo = parser.longValue(); + } else if (CAS_TERM.match(currentFieldName, parser.getDeprecationHandler())) { + casPrimaryTerm = parser.longValue(); } else if (RETRY_ON_CONFLICT.match(currentFieldName, parser.getDeprecationHandler())) { retryOnConflict = parser.intValue(); } else if (PIPELINE.match(currentFieldName, parser.getDeprecationHandler())) { @@ -386,7 +395,8 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null } if ("delete".equals(action)) { - add(new DeleteRequest(index, type, id).routing(routing).version(version).versionType(versionType), payload); + add(new DeleteRequest(index, type, id).routing(routing) + .version(version).versionType(versionType).compareAndSet(casSeqNo, casPrimaryTerm), payload); } else { nextMarker = findNextMarker(marker, from, data, length); if (nextMarker == -1) { @@ -399,16 +409,16 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null if ("index".equals(action)) { if (opType == null) { internalAdd(new IndexRequest(index, type, id).routing(routing).version(version).versionType(versionType) - .setPipeline(pipeline) + .setPipeline(pipeline).compareAndSet(casSeqNo, casPrimaryTerm) .source(sliceTrimmingCarriageReturn(data, from, nextMarker,xContentType), xContentType), payload); } else { internalAdd(new IndexRequest(index, type, id).routing(routing).version(version).versionType(versionType) - .create("create".equals(opType)).setPipeline(pipeline) + .create("create".equals(opType)).setPipeline(pipeline).compareAndSet(casSeqNo, casPrimaryTerm) .source(sliceTrimmingCarriageReturn(data, from, nextMarker, xContentType), xContentType), payload); } } else if ("create".equals(action)) { internalAdd(new IndexRequest(index, type, id).routing(routing).version(version).versionType(versionType) - .create(true).setPipeline(pipeline) + .create(true).setPipeline(pipeline).compareAndSet(casSeqNo, casPrimaryTerm) .source(sliceTrimmingCarriageReturn(data, from, nextMarker, xContentType), xContentType), payload); } else if ("update".equals(action)) { UpdateRequest updateRequest = new UpdateRequest(index, type, id).routing(routing).retryOnConflict(retryOnConflict) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index cbe5eddf56a54..f3aa91d8a1620 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -19,8 +19,8 @@ package org.elasticsearch.action.bulk; -import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.DocWriteRequest; @@ -460,7 +460,7 @@ private static void executeIndexRequestOnPrimary(BulkPrimaryExecutionContext con executeOnPrimaryWhileHandlingMappingUpdates(context, () -> primary.applyIndexOperationOnPrimary(request.version(), request.versionType(), sourceToParse, - request.compareAndWriteSeqNo(), request.compareAndWriteTerm(), request.getAutoGeneratedTimestamp(), request.isRetry()), + request.casSeqNo(), request.casPrimaryTerm(), request.getAutoGeneratedTimestamp(), request.isRetry()), e -> primary.getFailedIndexResult(e, request.version()), context::markOperationAsExecuted, mapping -> mappingUpdater.updateMappings(mapping, primary.shardId(), request.type())); @@ -472,7 +472,7 @@ private static void executeDeleteRequestOnPrimary(BulkPrimaryExecutionContext co final IndexShard primary = context.getPrimary(); executeOnPrimaryWhileHandlingMappingUpdates(context, () -> primary.applyDeleteOperationOnPrimary(request.version(), request.type(), request.id(), request.versionType(), - request.compareAndWriteSeqNo(), request.compareAndWriteTerm()), + request.casSeqNo(), request.casPrimaryTerm()), e -> primary.getFailedDeleteResult(e, request.version()), context::markOperationAsExecuted, mapping -> mappingUpdater.updateMappings(mapping, primary.shardId(), request.type())); diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java index a22b60f4733b7..c66310f2d26d4 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java @@ -55,8 +55,8 @@ public class DeleteRequest extends ReplicatedWriteRequest impleme private String routing; private long version = Versions.MATCH_ANY; private VersionType versionType = VersionType.INTERNAL; - private long compareAndWriteSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - private long compareAndWriteTerm = 0; + private long casSeqNp = SequenceNumbers.UNASSIGNED_SEQ_NO; + private long casPrimaryTerm = 0; public DeleteRequest() { } @@ -98,7 +98,7 @@ public ActionRequestValidationException validate() { validationException = addValidationError("version type [force] may no longer be used", validationException); } - if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO && ( + if (casSeqNp != SequenceNumbers.UNASSIGNED_SEQ_NO && ( versionType != VersionType.INTERNAL || version != Versions.MATCH_ANY )) { validationException = addValidationError("compare and write operations can not use versioning", validationException); @@ -178,23 +178,23 @@ public DeleteRequest versionType(VersionType versionType) { return this; } - public long compareAndWriteSeqNo() { - return compareAndWriteSeqNo; + public long casSeqNo() { + return casSeqNp; } - public long compareAndWriteTerm() { - return compareAndWriteTerm; + public long casPrimaryTerm() { + return casPrimaryTerm; } - public DeleteRequest compareAndWrite(long seqNo, long term) { + public DeleteRequest compareAndSet(long seqNo, long term) { if (seqNo < 0) { throw new IllegalArgumentException("sequence numbers must be non negative. got [" + seqNo + "]."); } if (term <= 0) { throw new IllegalArgumentException("primary term must be positive. got [" + term + "]"); } - compareAndWriteSeqNo = seqNo; - compareAndWriteTerm = term; + casSeqNp = seqNo; + casPrimaryTerm = term; return this; } @@ -220,11 +220,11 @@ public void readFrom(StreamInput in) throws IOException { version = in.readLong(); versionType = VersionType.fromValue(in.readByte()); if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { - compareAndWriteSeqNo = in.readZLong(); - compareAndWriteTerm = in.readVLong(); + casSeqNp = in.readZLong(); + casPrimaryTerm = in.readVLong(); } else { - compareAndWriteSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - compareAndWriteTerm = 0; + casSeqNp = SequenceNumbers.UNASSIGNED_SEQ_NO; + casPrimaryTerm = 0; } } @@ -240,10 +240,10 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); out.writeByte(versionType.getValue()); if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { - out.writeZLong(compareAndWriteSeqNo); - out.writeVLong(compareAndWriteTerm); - } else if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || compareAndWriteTerm != 0) { - assert false : "compareAndWrite [" + compareAndWriteSeqNo + "], currentDocTem [" + compareAndWriteTerm + "]"; + out.writeZLong(casSeqNp); + out.writeVLong(casPrimaryTerm); + } else if (casSeqNp != SequenceNumbers.UNASSIGNED_SEQ_NO || casPrimaryTerm != 0) { + assert false : "compareAndSet [" + casSeqNp + "], currentDocTem [" + casPrimaryTerm + "]"; throw new IllegalStateException( "sequence number based compare and write is not supported until all nodes are on version 7.0 or higher. " + "Stream version [" + out.getVersion() + "]"); diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java index fd757b516f483..c0eadbbd26654 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -105,8 +105,8 @@ public class IndexRequest extends ReplicatedWriteRequest implement private long autoGeneratedTimestamp = UNSET_AUTO_GENERATED_TIMESTAMP; private boolean isRetry = false; - private long compareAndWriteSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - private long compareAndWriteTerm = 0; + private long casSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + private long casPrimaryTerm = 0; public IndexRequest() { @@ -166,7 +166,7 @@ public ActionRequestValidationException validate() { return validationException; } - if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || compareAndWriteTerm != 0) { + if (casSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || casPrimaryTerm != 0) { validationException = addValidationError("create operations do not support compare and set. use index instead", validationException); return validationException; @@ -198,7 +198,7 @@ public ActionRequestValidationException validate() { validationException = addValidationError("pipeline cannot be an empty string", validationException); } - if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO && ( + if (casSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO && ( versionType != VersionType.INTERNAL || version != Versions.MATCH_ANY )) { validationException = addValidationError("compare and write operations can not use versioning", validationException); @@ -481,24 +481,24 @@ public IndexRequest versionType(VersionType versionType) { return this; } - public IndexRequest compareAndWrite(long seqNo, long term) { + public IndexRequest compareAndSet(long seqNo, long term) { if (seqNo < 0) { throw new IllegalArgumentException("sequence numbers must be non negative. got [" + seqNo + "]."); } if (term <= 0) { throw new IllegalArgumentException("primary term must be positive. got [" + term + "]"); } - compareAndWriteSeqNo = seqNo; - compareAndWriteTerm = term; + casSeqNo = seqNo; + casPrimaryTerm = term; return this; } - public long compareAndWriteSeqNo() { - return compareAndWriteSeqNo; + public long casSeqNo() { + return casSeqNo; } - public long compareAndWriteTerm() { - return compareAndWriteTerm; + public long casPrimaryTerm() { + return casPrimaryTerm; } @Override @@ -522,8 +522,8 @@ public void process(Version indexCreatedVersion, @Nullable MappingMetaData mappi // generate id if not already provided if (id == null) { assert autoGeneratedTimestamp == -1 : "timestamp has already been generated!"; - assert compareAndWriteSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO; - assert compareAndWriteTerm == 0; + assert casSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO; + assert casPrimaryTerm == 0; autoGeneratedTimestamp = Math.max(0, System.currentTimeMillis()); // extra paranoia String uid; if (indexCreatedVersion.onOrAfter(Version.V_6_0_0_beta1)) { @@ -566,11 +566,11 @@ public void readFrom(StreamInput in) throws IOException { contentType = null; } if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { - compareAndWriteSeqNo = in.readZLong(); - compareAndWriteTerm = in.readVLong(); + casSeqNo = in.readZLong(); + casPrimaryTerm = in.readVLong(); } else { - compareAndWriteSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - compareAndWriteTerm = SequenceNumbers.UNASSIGNED_SEQ_NO; + casSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + casPrimaryTerm = SequenceNumbers.UNASSIGNED_SEQ_NO; } } @@ -604,10 +604,10 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(false); } if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { - out.writeZLong(compareAndWriteSeqNo); - out.writeVLong(compareAndWriteTerm); - } else if (compareAndWriteSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || compareAndWriteTerm != 0) { - assert false : "compareAndWrite [" + compareAndWriteSeqNo + "], currentDocTem [" + compareAndWriteTerm + "]"; + out.writeZLong(casSeqNo); + out.writeVLong(casPrimaryTerm); + } else if (casSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || casPrimaryTerm != 0) { + assert false : "compareAndSet [" + casSeqNo + "], currentDocTem [" + casPrimaryTerm + "]"; throw new IllegalStateException( "sequence number based compare and write is not supported until all nodes are on version 7.0 or higher. " + "Stream version [" + out.getVersion() + "]"); diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index f3d6dc83f037e..2a327ffeddec3 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -1336,21 +1336,21 @@ public static class Index extends Operation { private final ParsedDocument doc; private final long autoGeneratedIdTimestamp; private final boolean isRetry; - private final long compareAndWriteSeqNo; - private final long compareAndWriteTerm; + private final long casSeqNo; + private final long casPrimaryTerm; public Index(Term uid, ParsedDocument doc, long seqNo, long primaryTerm, long version, VersionType versionType, Origin origin, - long startTime, long autoGeneratedIdTimestamp, boolean isRetry, long compareAndWriteSeqNo, long compareAndWriteTerm) { + long startTime, long autoGeneratedIdTimestamp, boolean isRetry, long casSeqNo, long casPrimaryTerm) { super(uid, seqNo, primaryTerm, version, versionType, origin, startTime); assert (origin == Origin.PRIMARY) == (versionType != null) : "invalid version_type=" + versionType + " for origin=" + origin; - assert compareAndWriteTerm >= 0; - assert compareAndWriteSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO || compareAndWriteSeqNo >=0; - assert (origin == Origin.PRIMARY) || (compareAndWriteSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO && compareAndWriteTerm == 0); + assert casPrimaryTerm >= 0; + assert casSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO || casSeqNo >=0; + assert (origin == Origin.PRIMARY) || (casSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO && casPrimaryTerm == 0); this.doc = doc; this.isRetry = isRetry; this.autoGeneratedIdTimestamp = autoGeneratedIdTimestamp; - this.compareAndWriteSeqNo = compareAndWriteSeqNo; - this.compareAndWriteTerm = compareAndWriteTerm; + this.casSeqNo = casSeqNo; + this.casPrimaryTerm = casPrimaryTerm; } public Index(Term uid, long primaryTerm, ParsedDocument doc) { @@ -1415,12 +1415,12 @@ public boolean isRetry() { return isRetry; } - public long getCompareAndWriteSeqNo() { - return compareAndWriteSeqNo; + public long getCasSeqNo() { + return casSeqNo; } - public long getCompareAndWriteTerm() { - return compareAndWriteTerm; + public long getCasPrimaryTerm() { + return casPrimaryTerm; } } @@ -1428,20 +1428,20 @@ public static class Delete extends Operation { private final String type; private final String id; - private final long compareAndWriteSeqNo; - private final long compareAndWriteTerm; + private final long casSeqNo; + private final long casPrimaryTerm; public Delete(String type, String id, Term uid, long seqNo, long primaryTerm, long version, VersionType versionType, - Origin origin, long startTime, long compareAndWriteSeqNo, long compareAndWriteTerm) { + Origin origin, long startTime, long casSeqNo, long casPrimaryTerm) { super(uid, seqNo, primaryTerm, version, versionType, origin, startTime); assert (origin == Origin.PRIMARY) == (versionType != null) : "invalid version_type=" + versionType + " for origin=" + origin; - assert compareAndWriteTerm >= 0; - assert compareAndWriteSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO || compareAndWriteSeqNo >=0; - assert (origin == Origin.PRIMARY) || (compareAndWriteSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO && compareAndWriteTerm == 0); + assert casPrimaryTerm >= 0; + assert casSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO || casSeqNo >=0; + assert (origin == Origin.PRIMARY) || (casSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO && casPrimaryTerm == 0); this.type = Objects.requireNonNull(type); this.id = Objects.requireNonNull(id); - this.compareAndWriteSeqNo = compareAndWriteSeqNo; - this.compareAndWriteTerm = compareAndWriteTerm; + this.casSeqNo = casSeqNo; + this.casPrimaryTerm = casPrimaryTerm; } public Delete(String type, String id, Term uid, long primaryTerm) { @@ -1473,12 +1473,12 @@ public int estimatedSizeInBytes() { return (uid().field().length() + uid().text().length()) * 2 + 20; } - public long getCompareAndWriteSeqNo() { - return compareAndWriteSeqNo; + public long getCasSeqNo() { + return casSeqNo; } - public long getCompareAndWriteTerm() { - return compareAndWriteTerm; + public long getCasPrimaryTerm() { + return casPrimaryTerm; } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index e55b038e77014..d4b9b696231c8 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -938,7 +938,7 @@ protected final IndexingStrategy planIndexingAsPrimary(Index index) throws IOExc versionMap.enforceSafeAccess(); // resolves incoming version final VersionValue versionValue = - resolveDocVersion(index, index.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO); + resolveDocVersion(index, index.getCasSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO); final long currentVersion; final boolean currentNotFoundOrDeleted; if (versionValue == null) { @@ -948,15 +948,15 @@ protected final IndexingStrategy planIndexingAsPrimary(Index index) throws IOExc currentVersion = versionValue.version; currentNotFoundOrDeleted = versionValue.isDelete(); } - if (index.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && versionValue == null) { + if (index.getCasSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && versionValue == null) { final VersionConflictEngineException e = new VersionConflictEngineException(shardId, index.type(), index.id(), - index.getCompareAndWriteSeqNo(), index.getCompareAndWriteTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + index.getCasSeqNo(), index.getCasPrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); plan = IndexingStrategy.skipDueToVersionConflict(e, currentNotFoundOrDeleted, currentVersion, getPrimaryTerm()); - } else if (index.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && ( - versionValue.seqNo != index.getCompareAndWriteSeqNo() || versionValue.term != index.getCompareAndWriteTerm() + } else if (index.getCasSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && ( + versionValue.seqNo != index.getCasSeqNo() || versionValue.term != index.getCasPrimaryTerm() )) { final VersionConflictEngineException e = new VersionConflictEngineException(shardId, index.type(), index.id(), - index.getCompareAndWriteSeqNo(), index.getCompareAndWriteTerm(), versionValue.seqNo, versionValue.term); + index.getCasSeqNo(), index.getCasPrimaryTerm(), versionValue.seqNo, versionValue.term); plan = IndexingStrategy.skipDueToVersionConflict(e, currentNotFoundOrDeleted, currentVersion, getPrimaryTerm()); } else if (index.versionType().isVersionConflictForWrites( currentVersion, index.version(), currentNotFoundOrDeleted)) { @@ -1268,7 +1268,7 @@ protected final DeletionStrategy planDeletionAsPrimary(Delete delete) throws IOE assert delete.origin() == Operation.Origin.PRIMARY : "planing as primary but got " + delete.origin(); assert getMaxSeqNoOfUpdatesOrDeletes() != SequenceNumbers.UNASSIGNED_SEQ_NO : "max_seq_no_of_updates is not initialized"; // resolve operation from external to internal - final VersionValue versionValue = resolveDocVersion(delete, delete.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO); + final VersionValue versionValue = resolveDocVersion(delete, delete.getCasSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO); assert incrementVersionLookup(); final long currentVersion; final boolean currentlyDeleted; @@ -1280,15 +1280,15 @@ protected final DeletionStrategy planDeletionAsPrimary(Delete delete) throws IOE currentlyDeleted = versionValue.isDelete(); } final DeletionStrategy plan; - if (delete.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && versionValue == null) { + if (delete.getCasSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && versionValue == null) { final VersionConflictEngineException e = new VersionConflictEngineException(shardId, delete.type(), delete.id(), - delete.getCompareAndWriteSeqNo(), delete.getCompareAndWriteTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + delete.getCasSeqNo(), delete.getCasPrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, 0); plan = DeletionStrategy.skipDueToVersionConflict(e, currentVersion, getPrimaryTerm(), currentlyDeleted); - } else if (delete.getCompareAndWriteSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && ( - versionValue.seqNo != delete.getCompareAndWriteSeqNo() || versionValue.term != delete.getCompareAndWriteTerm() + } else if (delete.getCasSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && ( + versionValue.seqNo != delete.getCasSeqNo() || versionValue.term != delete.getCasPrimaryTerm() )) { final VersionConflictEngineException e = new VersionConflictEngineException(shardId, delete.type(), delete.id(), - delete.getCompareAndWriteSeqNo(), delete.getCompareAndWriteTerm(), versionValue.seqNo, versionValue.term); + delete.getCasSeqNo(), delete.getCasPrimaryTerm(), versionValue.seqNo, versionValue.term); plan = DeletionStrategy.skipDueToVersionConflict(e, currentVersion, getPrimaryTerm(), currentlyDeleted); } else if (delete.versionType().isVersionConflictForWrites(currentVersion, delete.version(), currentlyDeleted)) { final VersionConflictEngineException e = new VersionConflictEngineException(shardId, delete, currentVersion, currentlyDeleted); diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 0e6a11f480ded..0cee5cff375b6 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -674,9 +674,9 @@ private IndexShardState changeState(IndexShardState newState, String reason) { } public Engine.IndexResult applyIndexOperationOnPrimary(long version, VersionType versionType, SourceToParse sourceToParse, - long compareAndWriteSeqNo, long compareAndWriteTerm, long autoGeneratedTimestamp, boolean isRetry) throws IOException { + long casSeqNo, long casPrimaryTerm, long autoGeneratedTimestamp, boolean isRetry) throws IOException { assert versionType.validateVersionForWrites(version); - return applyIndexOperation(UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, compareAndWriteSeqNo, compareAndWriteTerm, + return applyIndexOperation(UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, casSeqNo, casPrimaryTerm, autoGeneratedTimestamp, isRetry, Engine.Operation.Origin.PRIMARY, sourceToParse); } @@ -688,7 +688,7 @@ public Engine.IndexResult applyIndexOperationOnReplica(long seqNo, long version, } private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, long version, @Nullable VersionType versionType, - long compareAndWriteSeqNo, long compareAndWriteTerm, + long casSeqNo, long casPrimaryTerm, long autoGeneratedTimeStamp, boolean isRetry, Engine.Operation.Origin origin, SourceToParse sourceToParse) throws IOException { assert opPrimaryTerm <= this.operationPrimaryTerm: "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm @@ -698,7 +698,7 @@ private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, l try { operation = prepareIndex(docMapper(sourceToParse.type()), indexSettings.getIndexVersionCreated(), sourceToParse, seqNo, opPrimaryTerm, version, versionType, origin, autoGeneratedTimeStamp, isRetry, - compareAndWriteSeqNo, compareAndWriteTerm); + casSeqNo, casPrimaryTerm); Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); if (update != null) { return new Engine.IndexResult(update); @@ -718,7 +718,7 @@ private Engine.IndexResult applyIndexOperation(long seqNo, long opPrimaryTerm, l public static Engine.Index prepareIndex(DocumentMapperForType docMapper, Version indexCreatedVersion, SourceToParse source, long seqNo, long primaryTerm, long version, VersionType versionType, Engine.Operation.Origin origin, long autoGeneratedIdTimestamp, boolean isRetry, - long compareAndWriteSeqNo, long compareAndWriteTerm) { + long casSeqNo, long casPrimaryTerm) { long startTime = System.nanoTime(); ParsedDocument doc = docMapper.getDocumentMapper().parse(source); if (docMapper.getMapping() != null) { @@ -726,7 +726,7 @@ public static Engine.Index prepareIndex(DocumentMapperForType docMapper, Version } Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(doc.id())); return new Engine.Index(uid, doc, seqNo, primaryTerm, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry, - compareAndWriteSeqNo, compareAndWriteTerm); + casSeqNo, casPrimaryTerm); } private Engine.IndexResult index(Engine engine, Engine.Index index) throws IOException { @@ -778,11 +778,11 @@ public Engine.DeleteResult getFailedDeleteResult(Exception e, long version) { } public Engine.DeleteResult applyDeleteOperationOnPrimary(long version, String type, String id, VersionType versionType, - long compareAndWriteSeqNo, long compareAndWriteTerm) + long casSeqNo, long casPrimaryTerm) throws IOException { assert versionType.validateVersionForWrites(version); return applyDeleteOperation(UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, type, id, versionType, - compareAndWriteSeqNo, compareAndWriteTerm, Engine.Operation.Origin.PRIMARY); + casSeqNo, casPrimaryTerm, Engine.Operation.Origin.PRIMARY); } public Engine.DeleteResult applyDeleteOperationOnReplica(long seqNo, long version, String type, String id) throws IOException { @@ -791,7 +791,7 @@ public Engine.DeleteResult applyDeleteOperationOnReplica(long seqNo, long versio } private Engine.DeleteResult applyDeleteOperation(long seqNo, long opPrimaryTerm, long version, String type, String id, - @Nullable VersionType versionType, long compareAndWriteSeqNo, long compareAndWriteTerm, + @Nullable VersionType versionType, long casSeqNo, long casPrimaryTerm, Engine.Operation.Origin origin) throws IOException { assert opPrimaryTerm <= this.operationPrimaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm + "]"; @@ -814,16 +814,16 @@ private Engine.DeleteResult applyDeleteOperation(long seqNo, long opPrimaryTerm, } final Term uid = extractUidForDelete(type, id); final Engine.Delete delete = prepareDelete(type, id, uid, seqNo, opPrimaryTerm, version, - versionType, origin, compareAndWriteSeqNo, compareAndWriteTerm); + versionType, origin, casSeqNo, casPrimaryTerm); return delete(getEngine(), delete); } private static Engine.Delete prepareDelete(String type, String id, Term uid, long seqNo, long primaryTerm, long version, VersionType versionType, Engine.Operation.Origin origin, - long compareAndWriteSeqNo, long compareAndWriteTerm) { + long casSeqNo, long casPrimaryTerm) { long startTime = System.nanoTime(); return new Engine.Delete(type, id, uid, seqNo, primaryTerm, version, versionType, origin, startTime, - compareAndWriteSeqNo, compareAndWriteTerm); + casSeqNo, casPrimaryTerm); } private Term extractUidForDelete(String type, String id) { diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java index 05b60d3d7cbb2..1c358e7e77fc8 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -60,6 +61,8 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC deleteRequest.setRefreshPolicy(request.param("refresh")); deleteRequest.version(RestActions.parseVersion(request)); deleteRequest.versionType(VersionType.fromString(request.param("version_type"), deleteRequest.versionType())); + deleteRequest.compareAndSet( + request.paramAsLong("cas_seq_no", SequenceNumbers.NO_OPS_PERFORMED), request.paramAsLong("cas_primary_term", 0)); String waitForActiveShards = request.param("wait_for_active_shards"); if (waitForActiveShards != null) { diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java index 619fd811e6a7c..8538af8048cc0 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -93,6 +94,8 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC indexRequest.setRefreshPolicy(request.param("refresh")); indexRequest.version(RestActions.parseVersion(request)); indexRequest.versionType(VersionType.fromString(request.param("version_type"), indexRequest.versionType())); + indexRequest.compareAndSet( + request.paramAsLong("cas_seq_no", SequenceNumbers.NO_OPS_PERFORMED), request.paramAsLong("cas_primary_term", 0)); String sOpType = request.param("op_type"); String waitForActiveShards = request.param("wait_for_active_shards"); if (waitForActiveShards != null) { From f8645ae1bb6e21705e97a3d7b68e951cc065730b Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 8 Oct 2018 23:18:07 +0100 Subject: [PATCH 06/33] fix version test --- .../java/org/elasticsearch/common/lucene/uid/VersionsTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java index 5efa15f143c48..b5596a7362ff3 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java @@ -61,7 +61,7 @@ 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)); - assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()), equalTo(Versions.NOT_FOUND)); + assertThat(loadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()), nullValue()); Document doc = new Document(); doc.add(new Field(IdFieldMapper.NAME, "1", IdFieldMapper.Defaults.FIELD_TYPE)); From 17d4c5a570065e7a41701154ef54493e2b192cab Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 8 Oct 2018 23:43:22 +0100 Subject: [PATCH 07/33] fix validation --- .../elasticsearch/action/delete/DeleteRequest.java | 12 +++++++++--- .../elasticsearch/action/index/IndexRequest.java | 13 ++++++++++--- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java index c66310f2d26d4..4b7d36ef1208c 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java @@ -187,11 +187,17 @@ public long casPrimaryTerm() { } public DeleteRequest compareAndSet(long seqNo, long term) { - if (seqNo < 0) { + if (term == 0 && seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { + throw new IllegalArgumentException("seqNo is set, but primary term is [0]"); + } + if (term != 0 && seqNo == SequenceNumbers.UNASSIGNED_SEQ_NO) { + throw new IllegalArgumentException("seqNo is unassigned, but primary term is [" + term + "]"); + } + if (seqNo < 0 && seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { throw new IllegalArgumentException("sequence numbers must be non negative. got [" + seqNo + "]."); } - if (term <= 0) { - throw new IllegalArgumentException("primary term must be positive. got [" + term + "]"); + if (term < 0) { + throw new IllegalArgumentException("primary term must be non negative. got [" + term + "]"); } casSeqNp = seqNo; casPrimaryTerm = term; diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java index c0eadbbd26654..e68929b658508 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -482,11 +482,18 @@ public IndexRequest versionType(VersionType versionType) { } public IndexRequest compareAndSet(long seqNo, long term) { - if (seqNo < 0) { + if (term == 0 && seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { + throw new IllegalArgumentException("seqNo is set, but primary term is [0]"); + } + + if (term != 0 && seqNo == SequenceNumbers.UNASSIGNED_SEQ_NO) { + throw new IllegalArgumentException("seqNo is unassigned, but primary term is [" + term + "]"); + } + if (seqNo < 0 && seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { throw new IllegalArgumentException("sequence numbers must be non negative. got [" + seqNo + "]."); } - if (term <= 0) { - throw new IllegalArgumentException("primary term must be positive. got [" + term + "]"); + if (term < 0) { + throw new IllegalArgumentException("primary term must be non negative. got [" + term + "]"); } casSeqNo = seqNo; casPrimaryTerm = term; From 24fbaebf592accf31fe737d93778ead3db26b58e Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 9 Oct 2018 07:56:02 +0100 Subject: [PATCH 08/33] magic numbers are tricky --- .../org/elasticsearch/index/engine/InternalEngineTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index b1ffe1d01e2f2..b591992a89bd2 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1753,7 +1753,7 @@ private int assertOpsOnPrimary(List ops, long currentOpVersion final long conflictingVersion = docDeleted || randomBoolean() ? lastOpVersion + (randomBoolean() ? 1 : -1) : Versions.MATCH_DELETED; - final long conflictingSeqNo = randomBoolean() ? lastOpSeqNo : lastOpSeqNo + 1; + final long conflictingSeqNo = randomBoolean() ? lastOpSeqNo : lastOpSeqNo + 5; // use 5 to go above 0 for magic numbers final long conflictingTerm = conflictingSeqNo == lastOpSeqNo || randomBoolean() ? lastOpTerm + 1 : lastOpTerm; if (rarely()) { currentTerm.incrementAndGet(); From d7ffbe2498dbcbda5daf94b7c97639b44ffadbb6 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 10 Oct 2018 10:21:47 +0100 Subject: [PATCH 09/33] seq# on delete fix --- .../org/elasticsearch/index/engine/InternalEngineTests.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index b591992a89bd2..6acfc64d09d5a 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1753,7 +1753,9 @@ private int assertOpsOnPrimary(List ops, long currentOpVersion final long conflictingVersion = docDeleted || randomBoolean() ? lastOpVersion + (randomBoolean() ? 1 : -1) : Versions.MATCH_DELETED; - final long conflictingSeqNo = randomBoolean() ? lastOpSeqNo : lastOpSeqNo + 5; // use 5 to go above 0 for magic numbers + final long conflictingSeqNo = lastOpSeqNo == UNASSIGNED_SEQ_NO || randomBoolean() ? + lastOpSeqNo + 5 : // use 5 to go above 0 for magic numbers + lastOpSeqNo; final long conflictingTerm = conflictingSeqNo == lastOpSeqNo || randomBoolean() ? lastOpTerm + 1 : lastOpTerm; if (rarely()) { currentTerm.incrementAndGet(); From d3badd7e1f7ad19aa79592886125ad5d147edacc Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 11 Oct 2018 17:09:28 +0100 Subject: [PATCH 10/33] fix rest params --- .../elasticsearch/rest/action/document/RestDeleteAction.java | 2 +- .../org/elasticsearch/rest/action/document/RestIndexAction.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java index 1c358e7e77fc8..d232127d540ff 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java @@ -62,7 +62,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC deleteRequest.version(RestActions.parseVersion(request)); deleteRequest.versionType(VersionType.fromString(request.param("version_type"), deleteRequest.versionType())); deleteRequest.compareAndSet( - request.paramAsLong("cas_seq_no", SequenceNumbers.NO_OPS_PERFORMED), request.paramAsLong("cas_primary_term", 0)); + request.paramAsLong("cas_seq_no", SequenceNumbers.UNASSIGNED_SEQ_NO), request.paramAsLong("cas_primary_term", 0)); String waitForActiveShards = request.param("wait_for_active_shards"); if (waitForActiveShards != null) { diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java index 8538af8048cc0..0c96f3a23018e 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java @@ -95,7 +95,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC indexRequest.version(RestActions.parseVersion(request)); indexRequest.versionType(VersionType.fromString(request.param("version_type"), indexRequest.versionType())); indexRequest.compareAndSet( - request.paramAsLong("cas_seq_no", SequenceNumbers.NO_OPS_PERFORMED), request.paramAsLong("cas_primary_term", 0)); + request.paramAsLong("cas_seq_no", SequenceNumbers.UNASSIGNED_SEQ_NO), request.paramAsLong("cas_primary_term", 0)); String sOpType = request.param("op_type"); String waitForActiveShards = request.param("wait_for_active_shards"); if (waitForActiveShards != null) { From c7917c1ad12998acf4626a2b017c4c0816b4dc17 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 13 Oct 2018 08:11:36 +0100 Subject: [PATCH 11/33] rest tests --- .../src/main/resources/rest-api-spec/test/index/30_cas.yml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml index ded4ee1a5d912..6ef596c484d22 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml @@ -1,5 +1,5 @@ --- -"Internal version": +"Compare And Swap Sequence Numbers": - skip: version: " - 6.99.99" @@ -15,14 +15,13 @@ - set: { _primary_term: $primary_term } - do: - catch: conflict + catch: conflict index: index: test_1 id: 1 cas_seq_no: 10000 cas_primary_term: $primary_term body: { foo: bar2 } - - catch: conflict - do: catch: conflict From 93bb4374515872bd81545aa35e1441e6788d9874 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 13 Oct 2018 08:50:24 +0100 Subject: [PATCH 12/33] compile --- .../xpack/ccr/index/engine/FollowingEngineTests.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java index 7d89f63eb36c5..46a6b5ae195e6 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java @@ -308,11 +308,12 @@ private Engine.Result applyOperation(Engine engine, Engine.Operation op, if (op instanceof Engine.Index) { Engine.Index index = (Engine.Index) op; result = engine.index(new Engine.Index(index.uid(), index.parsedDoc(), index.seqNo(), primaryTerm, index.version(), - versionType, origin, index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry())); + versionType, origin, index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry(), + index.getCasSeqNo(), index.getCasPrimaryTerm())); } else if (op instanceof Engine.Delete) { Engine.Delete delete = (Engine.Delete) op; result = engine.delete(new Engine.Delete(delete.type(), delete.id(), delete.uid(), delete.seqNo(), primaryTerm, - delete.version(), versionType, origin, delete.startTime())); + delete.version(), versionType, origin, delete.startTime(), delete.getCasSeqNo(), delete.getCasPrimaryTerm())); } else { Engine.NoOp noOp = (Engine.NoOp) op; result = engine.noOp(new Engine.NoOp(noOp.seqNo(), primaryTerm, origin, noOp.startTime(), noOp.reason())); @@ -565,10 +566,12 @@ public void testProcessOnceOnPrimary() throws Exception { ParsedDocument doc = EngineTestCase.createParsedDoc(Integer.toString(between(1, 100)), null); if (randomBoolean()) { operations.add(new Engine.Index(EngineTestCase.newUid(doc), doc, i, primaryTerm.get(), 1L, - VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, threadPool.relativeTimeInMillis(), -1, true)); + VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, threadPool.relativeTimeInMillis(), -1, true, + SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); } else { operations.add(new Engine.Delete(doc.type(), doc.id(), EngineTestCase.newUid(doc), i, primaryTerm.get(), 1L, - VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, threadPool.relativeTimeInMillis())); + VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, threadPool.relativeTimeInMillis(), + SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); } } Randomness.shuffle(operations); From 96fb99ec67ae6a338d005e5ccc751079c4412a6d Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 14 Oct 2018 21:40:46 +0200 Subject: [PATCH 13/33] fix rest --- .../src/main/resources/rest-api-spec/test/index/30_cas.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml index 6ef596c484d22..2bded9dafff74 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml @@ -11,8 +11,8 @@ id: 1 body: { foo: bar } - match: { _version: 1} - - set: { _seq_no: $seqno } - - set: { _primary_term: $primary_term } + - set: { _seq_no: seqno } + - set: { _primary_term: primary_term } - do: catch: conflict From 12316f007631911e608358cb74aaa02a875098e9 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 14 Oct 2018 22:36:17 +0200 Subject: [PATCH 14/33] rest doc --- .../src/main/resources/rest-api-spec/api/delete.json | 8 ++++++++ .../src/main/resources/rest-api-spec/api/index.json | 8 ++++++++ 2 files changed, 16 insertions(+) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json index b146c34b441ea..64864637704bd 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json @@ -47,6 +47,14 @@ "type" : "time", "description" : "Explicit operation timeout" }, + "cas_seq_no" : { + "type" : "number", + "description" : "only perform the delete operation if the last operation to have changed the document has the specified sequence number" + }, + "cas_primary_term" : { + "type" : "number", + "description" : "only perform the delete operation if the last operation to have changed the document has the specified primary term" + }, "version" : { "type" : "number", "description" : "Explicit version number for concurrency control" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json index 3e07ff7acfa37..7653a1d25ee8c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json @@ -61,6 +61,14 @@ "options" : ["internal", "external", "external_gte", "force"], "description" : "Specific version type" }, + "cas_seq_no" : { + "type" : "number", + "description" : "only perform the index operation if the last operation to have changed the document has the specified sequence number" + }, + "cas_primary_term" : { + "type" : "number", + "description" : "only perform the index operation if the last operation to have changed the document has the specified primary term" + }, "pipeline" : { "type" : "string", "description" : "The pipeline id to preprocess incoming documents with" From 005c477e6118f911b5844b0b49a8dc195ff2424b Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 14 Oct 2018 23:02:34 +0200 Subject: [PATCH 15/33] fix rest --- .../src/main/resources/rest-api-spec/test/index/30_cas.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml index 2bded9dafff74..7b09abf47305b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml @@ -29,7 +29,7 @@ index: test_1 id: 1 cas_seq_no: $seqno - cas_primary_term: 1 + cas_primary_term: 1000 body: { foo: bar2 } - do: From d59712d0149b26875bad7b4d8d58047681a022b6 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 12 Nov 2018 19:37:41 +0100 Subject: [PATCH 16/33] fix compilation issues --- .../elasticsearch/action/delete/DeleteRequest.java | 4 ++-- .../elasticsearch/action/index/IndexRequest.java | 4 ++-- .../index/engine/InternalEngineTests.java | 14 +++++++++----- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java index 47bde6fac47ff..ba5a53125ab6f 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java @@ -229,7 +229,7 @@ public void readFrom(StreamInput in) throws IOException { } version = in.readLong(); versionType = VersionType.fromValue(in.readByte()); - if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + if (in.getVersion().onOrAfter(Version.V_7_0_0)) { casSeqNp = in.readZLong(); casPrimaryTerm = in.readVLong(); } else { @@ -249,7 +249,7 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeLong(version); out.writeByte(versionType.getValue()); - if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + if (out.getVersion().onOrAfter(Version.V_7_0_0)) { out.writeZLong(casSeqNp); out.writeVLong(casPrimaryTerm); } else if (casSeqNp != SequenceNumbers.UNASSIGNED_SEQ_NO || casPrimaryTerm != 0) { diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java index 41ebf26600581..ee5ec2b4b6c74 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -577,7 +577,7 @@ public void readFrom(StreamInput in) throws IOException { } else { contentType = null; } - if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + if (in.getVersion().onOrAfter(Version.V_7_0_0)) { casSeqNo = in.readZLong(); casPrimaryTerm = in.readVLong(); } else { @@ -615,7 +615,7 @@ public void writeTo(StreamOutput out) throws IOException { } else { out.writeBoolean(false); } - if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + if (out.getVersion().onOrAfter(Version.V_7_0_0)) { out.writeZLong(casSeqNo); out.writeVLong(casPrimaryTerm); } else if (casSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || casPrimaryTerm != 0) { diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 2fa10aa36f479..9afe4c8d29691 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -3745,10 +3745,12 @@ public void testLookupSeqNoByIdInLucene() throws Exception { final ParsedDocument doc = EngineTestCase.createParsedDoc(id, null); if (isIndexing) { operations.add(new Engine.Index(EngineTestCase.newUid(doc), doc, seqNo, primaryTerm.get(), - i, null, Engine.Operation.Origin.REPLICA, threadPool.relativeTimeInMillis(), -1, true)); + i, null, Engine.Operation.Origin.REPLICA, threadPool.relativeTimeInMillis(), -1, true, + UNASSIGNED_SEQ_NO, 0L)); } else { operations.add(new Engine.Delete(doc.type(), doc.id(), EngineTestCase.newUid(doc), seqNo, primaryTerm.get(), - i, null, Engine.Operation.Origin.REPLICA, threadPool.relativeTimeInMillis())); + i, null, Engine.Operation.Origin.REPLICA, threadPool.relativeTimeInMillis(), + UNASSIGNED_SEQ_NO, 0L)); } } seqNo++; @@ -3774,7 +3776,7 @@ public void testLookupSeqNoByIdInLucene() throws Exception { assertThat(msg, docIdAndSeqNo.isLive, equalTo(latestOps.get(id).operationType() == Engine.Operation.TYPE.INDEX)); } assertThat(VersionsAndSeqNoResolver.loadDocIdAndVersion( - searcher.reader(), newUid("any-" + between(1, 10))), nullValue()); + searcher.reader(), newUid("any-" + between(1, 10)), randomBoolean()), nullValue()); Map liveOps = latestOps.entrySet().stream() .filter(e -> e.getValue().operationType() == Engine.Operation.TYPE.INDEX) .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue().seqNo())); @@ -5281,10 +5283,12 @@ public void testRebuildLocalCheckpointTracker() throws Exception { final ParsedDocument doc = EngineTestCase.createParsedDoc(Integer.toString(between(1, 100)), null); if (randomBoolean()) { operations.add(new Engine.Index(EngineTestCase.newUid(doc), doc, seqNo, primaryTerm.get(), - i, null, Engine.Operation.Origin.REPLICA, threadPool.relativeTimeInMillis(), -1, true)); + i, null, Engine.Operation.Origin.REPLICA, threadPool.relativeTimeInMillis(), -1, true, + UNASSIGNED_SEQ_NO, 0L)); } else if (randomBoolean()) { operations.add(new Engine.Delete(doc.type(), doc.id(), EngineTestCase.newUid(doc), seqNo, primaryTerm.get(), - i, null, Engine.Operation.Origin.REPLICA, threadPool.relativeTimeInMillis())); + i, null, Engine.Operation.Origin.REPLICA, threadPool.relativeTimeInMillis(), + UNASSIGNED_SEQ_NO, 0L)); } else { operations.add(new Engine.NoOp(seqNo, primaryTerm.get(), Engine.Operation.Origin.REPLICA, threadPool.relativeTimeInMillis(), "test-" + i)); From c139f7d57500dc6696e16624d715cc3afc18fd4f Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 14 Nov 2018 14:40:15 +0100 Subject: [PATCH 17/33] iter --- docs/reference/docs/bulk.asciidoc | 2 +- .../PercolateQueryBuilderTests.java | 4 +- .../rest-api-spec/test/index/30_cas.yml | 7 ++ .../action/delete/DeleteRequestBuilder.java | 10 +++ .../elasticsearch/action/get/GetResponse.java | 14 ++++ .../action/index/IndexRequestBuilder.java | 9 +++ .../action/update/UpdateHelper.java | 2 +- .../action/update/UpdateResponse.java | 2 +- .../elasticsearch/index/get/GetResult.java | 55 ++++++++++++++- .../index/get/ShardGetService.java | 8 ++- .../action/explain/ExplainResponseTests.java | 4 +- .../action/get/GetResponseTests.java | 20 +++--- .../action/get/MultiGetResponseTests.java | 2 +- .../action/update/UpdateRequestTests.java | 22 +++--- .../action/update/UpdateResponseTests.java | 2 +- .../index/get/GetResultTests.java | 63 ++++++++++------- .../query/GeoShapeQueryBuilderTests.java | 2 +- .../index/query/TermsQueryBuilderTests.java | 2 +- .../versioning/SimpleVersioningIT.java | 68 +++++++++++++++++++ .../authc/esnative/NativeUsersStoreTests.java | 6 +- .../store/NativePrivilegeStoreTests.java | 4 +- .../execution/ExecutionServiceTests.java | 6 +- .../ack/TransportAckWatchActionTests.java | 2 +- 23 files changed, 248 insertions(+), 68 deletions(-) diff --git a/docs/reference/docs/bulk.asciidoc b/docs/reference/docs/bulk.asciidoc index 4c6e55af318f5..e6dd55653f833 100644 --- a/docs/reference/docs/bulk.asciidoc +++ b/docs/reference/docs/bulk.asciidoc @@ -38,7 +38,7 @@ optional_source\n *NOTE*: the final line of data must end with a newline character `\n`. Each newline character may be preceded by a carriage return `\r`. When sending requests to this endpoint the -`Content-Type` header should be set to `application/x-ndjson`. +`Content-Type` header should be set to `application/x- ndjson`. The possible actions are `index`, `create`, `delete` and `update`. `index` and `create` expect a source on the next diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java index be9c3f83f3f4b..aef9c36a802a3 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java @@ -152,12 +152,12 @@ protected GetResponse executeGet(GetRequest getRequest) { assertThat(getRequest.version(), Matchers.equalTo(indexedDocumentVersion)); if (indexedDocumentExists) { return new GetResponse( - new GetResult(indexedDocumentIndex, indexedDocumentType, indexedDocumentId, 0L, true, + new GetResult(indexedDocumentIndex, indexedDocumentType, indexedDocumentId, 0, 1, 0L, true, documentSource.iterator().next(), Collections.emptyMap()) ); } else { return new GetResponse( - new GetResult(indexedDocumentIndex, indexedDocumentType, indexedDocumentId, -1, false, null, Collections.emptyMap()) + new GetResult(indexedDocumentIndex, indexedDocumentType, indexedDocumentId, 0, 1, -1, false, null, Collections.emptyMap()) ); } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml index 7b09abf47305b..37a2b2bdd2dea 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml @@ -14,6 +14,13 @@ - set: { _seq_no: seqno } - set: { _primary_term: primary_term } + - do: + get: + index: test_1 + id: 1 + - match: { _seq_no: $seqno } + - match: { _primary_term: $primary_term } + - do: catch: conflict index: diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequestBuilder.java index 9060af8e17c8c..53867f0fddffc 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequestBuilder.java @@ -80,4 +80,14 @@ public DeleteRequestBuilder setVersionType(VersionType versionType) { request.versionType(versionType); return this; } + + /** + * only performs this delete request if the document was last modification was assigned the given + * sequence number and primary term + */ + public DeleteRequestBuilder setCompareAndSet(long seqNo, long term) { + request.compareAndSet(seqNo, term); + return this; + } + } diff --git a/server/src/main/java/org/elasticsearch/action/get/GetResponse.java b/server/src/main/java/org/elasticsearch/action/get/GetResponse.java index 455aab7f6e3d6..e96fb04c0b443 100644 --- a/server/src/main/java/org/elasticsearch/action/get/GetResponse.java +++ b/server/src/main/java/org/elasticsearch/action/get/GetResponse.java @@ -90,6 +90,20 @@ public long getVersion() { return getResult.getVersion(); } + /** + * The sequence number assigned to the last operation to have changed this document, if found. + */ + public long getSeqNo() { + return getResult.getSeqNo(); + } + + /** + * The primary term of the last primary that have changed this document, if found. + */ + public long getPrimaryTerm() { + return getResult.getPrimaryTerm(); + } + /** * The source of the document if exists. */ diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java index b81d09abda3ab..8c6ec7a275892 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java @@ -199,6 +199,15 @@ public IndexRequestBuilder setVersionType(VersionType versionType) { return this; } + /** + * only performs this indexing request if the document was last modification was assigned the given + * sequence number and primary term + */ + public IndexRequestBuilder setCompareAndSet(long seqNo, long term) { + request.compareAndSet(seqNo, term); + return this; + } + /** * Sets the ingest pipeline to be executed before indexing the document */ diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index 6538c0b4b5e59..94efdffd404d0 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -316,7 +316,7 @@ public static GetResult extractGetResult(final UpdateRequest request, String con } // TODO when using delete/none, we can still return the source as bytes by generating it (using the sourceContentType) - return new GetResult(concreteIndex, request.type(), request.id(), version, true, sourceFilteredAsBytes, Collections.emptyMap()); + return new GetResult(concreteIndex, request.type(), request.id(), 0, 1, version, true, sourceFilteredAsBytes, Collections.emptyMap()); } public static class Result { diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateResponse.java b/server/src/main/java/org/elasticsearch/action/update/UpdateResponse.java index 9e33e62622a0e..9c9bd0455e27b 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateResponse.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateResponse.java @@ -162,7 +162,7 @@ public UpdateResponse build() { update = new UpdateResponse(shardId, type, id, version, result); } if (getResult != null) { - update.setGetResult(new GetResult(update.getIndex(), update.getType(), update.getId(), update.getVersion(), + update.setGetResult(new GetResult(update.getIndex(), update.getType(), update.getId(), 0, 1, update.getVersion(), getResult.isExists(),getResult.internalSourceRef(), getResult.getFields())); } update.setForcedRefresh(forcedRefresh); diff --git a/server/src/main/java/org/elasticsearch/index/get/GetResult.java b/server/src/main/java/org/elasticsearch/index/get/GetResult.java index ba70c7035506b..bdaf1e5f6eb23 100644 --- a/server/src/main/java/org/elasticsearch/index/get/GetResult.java +++ b/server/src/main/java/org/elasticsearch/index/get/GetResult.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.get; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.Version; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressorFactory; @@ -33,6 +34,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.mapper.IgnoredFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.search.lookup.SourceLookup; import java.io.IOException; @@ -53,6 +55,8 @@ public class GetResult implements Streamable, Iterable, ToXConten public static final String _TYPE = "_type"; public static final String _ID = "_id"; private static final String _VERSION = "_version"; + private static final String _SEQ_NO = "_seq_no"; + private static final String _PRIMARY_TERM = "_primary_term"; private static final String FOUND = "found"; private static final String FIELDS = "fields"; @@ -60,6 +64,8 @@ public class GetResult implements Streamable, Iterable, ToXConten private String type; private String id; private long version; + private long seqNo; + private long primaryTerm; private boolean exists; private Map fields; private Map sourceAsMap; @@ -69,11 +75,17 @@ public class GetResult implements Streamable, Iterable, ToXConten GetResult() { } - public GetResult(String index, String type, String id, long version, boolean exists, BytesReference source, - Map fields) { + public GetResult(String index, String type, String id, long seqNo, long primaryTerm, long version, boolean exists, + BytesReference source, Map fields) { this.index = index; this.type = type; this.id = id; + this.seqNo = seqNo; + this.primaryTerm = primaryTerm; + assert (seqNo == SequenceNumbers.UNASSIGNED_SEQ_NO && primaryTerm == 0) || + (seqNo >= 0 && primaryTerm >=1) : "seqNo: " + seqNo + " primaryTerm: " + primaryTerm; + assert exists || (seqNo == SequenceNumbers.UNASSIGNED_SEQ_NO && primaryTerm == 0) : + "doc not found but seqNo/primaryTerm are set"; this.version = version; this.exists = exists; this.source = source; @@ -118,6 +130,20 @@ public long getVersion() { return version; } + /** + * The sequence number assigned to the last operation to have changed this document, if found. + */ + public long getSeqNo() { + return seqNo; + } + + /** + * The primary term of the last primary that have changed this document, if found. + */ + public long getPrimaryTerm() { + return primaryTerm; + } + /** * The source of the document if exists. */ @@ -259,6 +285,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(_TYPE, type); builder.field(_ID, id); if (isExists()) { + if (seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { // seqNo may not be assigned if read from an old node + builder.field(_SEQ_NO, seqNo); + builder.field(_PRIMARY_TERM, primaryTerm); + } if (version != -1) { builder.field(_VERSION, version); } @@ -282,6 +312,8 @@ public static GetResult fromXContentEmbedded(XContentParser parser, String index String currentFieldName = parser.currentName(); long version = -1; + long seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + long primaryTerm = 0; Boolean found = null; BytesReference source = null; Map fields = new HashMap<>(); @@ -297,6 +329,10 @@ public static GetResult fromXContentEmbedded(XContentParser parser, String index id = parser.text(); } else if (_VERSION.equals(currentFieldName)) { version = parser.longValue(); + } else if (_SEQ_NO.equals(currentFieldName)) { + seqNo = parser.longValue(); + } else if (_PRIMARY_TERM.equals(currentFieldName)) { + primaryTerm = parser.longValue(); } else if (FOUND.equals(currentFieldName)) { found = parser.booleanValue(); } else { @@ -326,7 +362,7 @@ public static GetResult fromXContentEmbedded(XContentParser parser, String index } } } - return new GetResult(index, type, id, version, found, source, fields); + return new GetResult(index, type, id, seqNo, primaryTerm, version, found, source, fields); } public static GetResult fromXContent(XContentParser parser) throws IOException { @@ -347,6 +383,13 @@ public void readFrom(StreamInput in) throws IOException { index = in.readString(); type = in.readOptionalString(); id = in.readString(); + if (in.getVersion().onOrAfter(Version.V_7_0_0)) { + seqNo = in.readZLong(); + primaryTerm = in.readVLong(); + } else { + seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + primaryTerm = 0L; + } version = in.readLong(); exists = in.readBoolean(); if (exists) { @@ -372,6 +415,10 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(index); out.writeOptionalString(type); out.writeString(id); + if (out.getVersion().onOrAfter(Version.V_7_0_0)) { + out.writeZLong(seqNo); + out.writeVLong(primaryTerm); + } out.writeLong(version); out.writeBoolean(exists); if (exists) { @@ -397,6 +444,8 @@ public boolean equals(Object o) { } GetResult getResult = (GetResult) o; return version == getResult.version && + seqNo == getResult.seqNo && + primaryTerm == getResult.primaryTerm && exists == getResult.exists && Objects.equals(index, getResult.index) && Objects.equals(type, getResult.type) && diff --git a/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java b/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java index 5c5554cddcfc7..d1b510c043552 100644 --- a/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java +++ b/server/src/main/java/org/elasticsearch/index/get/ShardGetService.java @@ -43,6 +43,7 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.RoutingFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; @@ -107,7 +108,7 @@ public GetResult getForUpdate(String type, String id, long version, VersionType */ public GetResult get(Engine.GetResult engineGetResult, String id, String type, String[] fields, FetchSourceContext fetchSourceContext) { if (!engineGetResult.exists()) { - return new GetResult(shardId.getIndexName(), type, id, -1, false, null, null); + return new GetResult(shardId.getIndexName(), type, id, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, -1, false, null, null); } currentMetric.inc(); @@ -165,7 +166,7 @@ private GetResult innerGet(String type, String id, String[] gFields, boolean rea } if (get == null || get.exists() == false) { - return new GetResult(shardId.getIndexName(), type, id, -1, false, null, null); + return new GetResult(shardId.getIndexName(), type, id, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, -1, false, null, null); } try { @@ -229,7 +230,8 @@ private GetResult innerGetLoadFromStoredFields(String type, String id, String[] } } - return new GetResult(shardId.getIndexName(), type, id, get.version(), get.exists(), source, fields); + return new GetResult(shardId.getIndexName(), type, id, get.docIdAndVersion().seqNo, get.docIdAndVersion().primaryTerm, + get.version(), get.exists(), source, fields); } private static FieldsVisitor buildFieldsVisitors(String[] fields, FetchSourceContext fetchSourceContext) { diff --git a/server/src/test/java/org/elasticsearch/action/explain/ExplainResponseTests.java b/server/src/test/java/org/elasticsearch/action/explain/ExplainResponseTests.java index ca5c35ccab3ed..361955342287f 100644 --- a/server/src/test/java/org/elasticsearch/action/explain/ExplainResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/explain/ExplainResponseTests.java @@ -65,7 +65,7 @@ protected ExplainResponse createTestInstance() { GetResult getResult = new GetResult(randomAlphaOfLengthBetween(3, 10), randomAlphaOfLengthBetween(3, 10), randomAlphaOfLengthBetween(3, 10), - randomNonNegativeLong(), + 0, 1, randomNonNegativeLong(), true, RandomObjects.randomSource(random()), singletonMap(fieldName, new DocumentField(fieldName, values))); @@ -83,7 +83,7 @@ public void testToXContent() throws IOException { String id = "1"; boolean exist = true; Explanation explanation = Explanation.match(1.0f, "description", Collections.emptySet()); - GetResult getResult = new GetResult(null, null, null, -1, true, new BytesArray("{ \"field1\" : " + + GetResult getResult = new GetResult(null, null, null, 0, 1, -1, true, new BytesArray("{ \"field1\" : " + "\"value1\", \"field2\":\"value2\"}"), singletonMap("field1", new DocumentField("field1", singletonList("value1")))); ExplainResponse response = new ExplainResponse(index, type, id, exist, explanation, getResult); diff --git a/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java b/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java index ee4be1a5396ce..146cc55da8630 100644 --- a/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java @@ -39,6 +39,7 @@ import static org.elasticsearch.index.get.GetResultTests.copyGetResult; import static org.elasticsearch.index.get.GetResultTests.mutateGetResult; import static org.elasticsearch.index.get.GetResultTests.randomGetResult; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; import static org.elasticsearch.test.XContentTestUtils.insertRandomFields; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent; @@ -91,15 +92,16 @@ private void doFromXContentTestWithRandomFields(boolean addRandomFields) throws public void testToXContent() { { - GetResponse getResponse = new GetResponse(new GetResult("index", "type", "id", 1, true, new BytesArray("{ \"field1\" : " + + GetResponse getResponse = new GetResponse(new GetResult("index", "type", "id", 0, 1, 1, true, new BytesArray("{ \"field1\" : " + "\"value1\", \"field2\":\"value2\"}"), Collections.singletonMap("field1", new DocumentField("field1", Collections.singletonList("value1"))))); String output = Strings.toString(getResponse); - assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_version\":1,\"found\":true,\"_source\":{ \"field1\" " + - ": \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", output); + assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_seq_no\":0,\"_primary_term\":1,\"_version\":1," + + "\"found\":true,\"_source\":{ \"field1\": \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", + output); } { - GetResponse getResponse = new GetResponse(new GetResult("index", "type", "id", 1, false, null, null)); + GetResponse getResponse = new GetResponse(new GetResult("index", "type", "id", 0, 1, 1, false, null, null)); String output = Strings.toString(getResponse); assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"found\":false}", output); } @@ -107,10 +109,11 @@ public void testToXContent() { public void testToString() { GetResponse getResponse = new GetResponse( - new GetResult("index", "type", "id", 1, true, new BytesArray("{ \"field1\" : " + "\"value1\", \"field2\":\"value2\"}"), + new GetResult("index", "type", "id", 0, 1, 1, true, new BytesArray("{ \"field1\" : " + "\"value1\", \"field2\":\"value2\"}"), Collections.singletonMap("field1", new DocumentField("field1", Collections.singletonList("value1"))))); - assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_version\":1,\"found\":true,\"_source\":{ \"field1\" " - + ": \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", getResponse.toString()); + assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_seq_no\":0,\"_primary_term\":1,\"_version\":1," + + "\"found\":true,\"_source\":{ \"field1\": \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", + getResponse.toString()); } public void testEqualsAndHashcode() { @@ -119,7 +122,8 @@ public void testEqualsAndHashcode() { } public void testFromXContentThrowsParsingException() throws IOException { - GetResponse getResponse = new GetResponse(new GetResult(null, null, null, randomIntBetween(1, 5), randomBoolean(), null, null)); + GetResponse getResponse = + new GetResponse(new GetResult(null, null, null, UNASSIGNED_SEQ_NO, 0, randomIntBetween(1, 5), randomBoolean(), null, null)); XContentType xContentType = randomFrom(XContentType.values()); BytesReference originalBytes = toShuffledXContent(getResponse, xContentType, ToXContent.EMPTY_PARAMS, randomBoolean()); diff --git a/server/src/test/java/org/elasticsearch/action/get/MultiGetResponseTests.java b/server/src/test/java/org/elasticsearch/action/get/MultiGetResponseTests.java index 6331d5ef31dff..101313f3001c6 100644 --- a/server/src/test/java/org/elasticsearch/action/get/MultiGetResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/get/MultiGetResponseTests.java @@ -70,7 +70,7 @@ private static MultiGetResponse createTestInstance() { for (int i = 0; i < items.length; i++) { if (randomBoolean()) { items[i] = new MultiGetItemResponse(new GetResponse(new GetResult( - randomAlphaOfLength(4), randomAlphaOfLength(4), randomAlphaOfLength(4), randomNonNegativeLong(), + randomAlphaOfLength(4), randomAlphaOfLength(4), randomAlphaOfLength(4), 0, 1, randomNonNegativeLong(), true, null, null )), null); } else { diff --git a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java index 26c1b4dac79d7..a53fa804f5f17 100644 --- a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java @@ -359,7 +359,7 @@ public void testNowInScript() throws IOException { .scriptedUpsert(true); long nowInMillis = randomNonNegativeLong(); // We simulate that the document is not existing yet - GetResult getResult = new GetResult("test", "type1", "2", 0, false, null, null); + GetResult getResult = new GetResult("test", "type1", "2", 0, 1, 0, false, null, null); UpdateHelper.Result result = updateHelper.prepare(new ShardId("test", "_na_", 0), updateRequest, getResult, () -> nowInMillis); Streamable action = result.action(); assertThat(action, instanceOf(IndexRequest.class)); @@ -372,7 +372,7 @@ public void testNowInScript() throws IOException { .script(mockInlineScript("ctx._timestamp = ctx._now")) .scriptedUpsert(true); // We simulate that the document is not existing yet - GetResult getResult = new GetResult("test", "type1", "2", 0, true, new BytesArray("{}"), null); + GetResult getResult = new GetResult("test", "type1", "2", 0, 1, 0, true, new BytesArray("{}"), null); UpdateHelper.Result result = updateHelper.prepare(new ShardId("test", "_na_", 0), updateRequest, getResult, () -> 42L); Streamable action = result.action(); assertThat(action, instanceOf(IndexRequest.class)); @@ -381,7 +381,7 @@ public void testNowInScript() throws IOException { public void testIndexTimeout() { final GetResult getResult = - new GetResult("test", "type", "1", 0, true, new BytesArray("{\"f\":\"v\"}"), null); + new GetResult("test", "type", "1", 0, 1, 0, true, new BytesArray("{\"f\":\"v\"}"), null); final UpdateRequest updateRequest = new UpdateRequest("test", "type", "1") .script(mockInlineScript("return")) @@ -391,7 +391,7 @@ public void testIndexTimeout() { public void testDeleteTimeout() { final GetResult getResult = - new GetResult("test", "type", "1", 0, true, new BytesArray("{\"f\":\"v\"}"), null); + new GetResult("test", "type", "1", 0, 1, 0, true, new BytesArray("{\"f\":\"v\"}"), null); final UpdateRequest updateRequest = new UpdateRequest("test", "type", "1") .script(mockInlineScript("ctx.op = delete")) @@ -402,7 +402,7 @@ public void testDeleteTimeout() { public void testUpsertTimeout() throws IOException { final boolean exists = randomBoolean(); final BytesReference source = exists ? new BytesArray("{\"f\":\"v\"}") : null; - final GetResult getResult = new GetResult("test", "type", "1", 0, exists, source, null); + final GetResult getResult = new GetResult("test", "type", "1", 0, 1, 0, exists, source, null); final XContentBuilder sourceBuilder = jsonBuilder(); sourceBuilder.startObject(); { @@ -535,7 +535,7 @@ public void testValidate() { } public void testRoutingExtraction() throws Exception { - GetResult getResult = new GetResult("test", "type", "1", 0, false, null, null); + GetResult getResult = new GetResult("test", "type", "1", 0, 1, 0, false, null, null); IndexRequest indexRequest = new IndexRequest("test", "type", "1"); // There is no routing and parent because the document doesn't exist @@ -545,7 +545,7 @@ public void testRoutingExtraction() throws Exception { assertNull(UpdateHelper.calculateRouting(getResult, indexRequest)); // Doc exists but has no source or fields - getResult = new GetResult("test", "type", "1", 0, true, null, null); + getResult = new GetResult("test", "type", "1", 0, 1, 0, true, null, null); // There is no routing and parent on either request assertNull(UpdateHelper.calculateRouting(getResult, indexRequest)); @@ -554,7 +554,7 @@ public void testRoutingExtraction() throws Exception { fields.put("_routing", new DocumentField("_routing", Collections.singletonList("routing1"))); // Doc exists and has the parent and routing fields - getResult = new GetResult("test", "type", "1", 0, true, null, fields); + getResult = new GetResult("test", "type", "1", 0, 1, 0, true, null, fields); // Use the get result parent and routing assertThat(UpdateHelper.calculateRouting(getResult, indexRequest), equalTo("routing1")); @@ -563,7 +563,7 @@ public void testRoutingExtraction() throws Exception { @SuppressWarnings("deprecated") // VersionType.FORCE is deprecated public void testCalculateUpdateVersion() throws Exception { long randomVersion = randomIntBetween(0, 100); - GetResult getResult = new GetResult("test", "type", "1", randomVersion, true, new BytesArray("{}"), null); + GetResult getResult = new GetResult("test", "type", "1", 0, 1, randomVersion, true, new BytesArray("{}"), null); UpdateRequest request = new UpdateRequest("test", "type1", "1"); long version = UpdateHelper.calculateUpdateVersion(request, getResult); @@ -580,7 +580,7 @@ public void testCalculateUpdateVersion() throws Exception { public void testNoopDetection() throws Exception { ShardId shardId = new ShardId("test", "", 0); - GetResult getResult = new GetResult("test", "type", "1", 0, true, + GetResult getResult = new GetResult("test", "type", "1", 0, 1, 0, true, new BytesArray("{\"body\": \"foo\"}"), null); @@ -611,7 +611,7 @@ public void testNoopDetection() throws Exception { public void testUpdateScript() throws Exception { ShardId shardId = new ShardId("test", "", 0); - GetResult getResult = new GetResult("test", "type", "1", 0, true, + GetResult getResult = new GetResult("test", "type", "1", 0, 1, 0, true, new BytesArray("{\"body\": \"bar\"}"), null); diff --git a/server/src/test/java/org/elasticsearch/action/update/UpdateResponseTests.java b/server/src/test/java/org/elasticsearch/action/update/UpdateResponseTests.java index c8d63f737329b..afcbb74772e97 100644 --- a/server/src/test/java/org/elasticsearch/action/update/UpdateResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/update/UpdateResponseTests.java @@ -74,7 +74,7 @@ public void testToXContent() throws IOException { UpdateResponse updateResponse = new UpdateResponse(new ReplicationResponse.ShardInfo(3, 2), new ShardId("books", "books_uuid", 2), "book", "1", 7, 17, 2, UPDATED); - updateResponse.setGetResult(new GetResult("books", "book", "1", 2, true, source, fields)); + updateResponse.setGetResult(new GetResult("books", "book", "1", 0, 1, 2, true, source, fields)); String output = Strings.toString(updateResponse); assertEquals("{\"_index\":\"books\",\"_type\":\"book\",\"_id\":\"1\",\"_version\":2,\"result\":\"updated\"," + diff --git a/server/src/test/java/org/elasticsearch/index/get/GetResultTests.java b/server/src/test/java/org/elasticsearch/index/get/GetResultTests.java index 1cc2612041f46..da25c3f49d3a8 100644 --- a/server/src/test/java/org/elasticsearch/index/get/GetResultTests.java +++ b/server/src/test/java/org/elasticsearch/index/get/GetResultTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.RandomObjects; @@ -72,15 +73,16 @@ public void testToAndFromXContent() throws Exception { public void testToXContent() throws IOException { { - GetResult getResult = new GetResult("index", "type", "id", 1, true, new BytesArray("{ \"field1\" : " + + GetResult getResult = new GetResult("index", "type", "id", 0, 1, 1, true, new BytesArray("{ \"field1\" : " + "\"value1\", \"field2\":\"value2\"}"), singletonMap("field1", new DocumentField("field1", singletonList("value1")))); String output = Strings.toString(getResult); - assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_version\":1,\"found\":true,\"_source\":{ \"field1\" " + - ": \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", output); + assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\", \"_seq_no\":0,\"_primary_Term\": 1,\"_version\":1," + + "\"found\":true,\"_source\":{ \"field1\": \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", + output); } { - GetResult getResult = new GetResult("index", "type", "id", 1, false, null, null); + GetResult getResult = new GetResult("index", "type", "id", 0, 1, 1, false, null, null); String output = Strings.toString(getResult); assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"found\":false}", output); } @@ -92,7 +94,7 @@ public void testToAndFromXContentEmbedded() throws Exception { GetResult getResult = tuple.v1(); // We don't expect to retrieve the index/type/id of the GetResult because they are not rendered // by the toXContentEmbedded method. - GetResult expectedGetResult = new GetResult(null, null, null, -1, + GetResult expectedGetResult = new GetResult(null, null, null, 0, 1, -1, tuple.v2().isExists(), tuple.v2().sourceRef(), tuple.v2().getFields()); boolean humanReadable = randomBoolean(); @@ -118,7 +120,7 @@ public void testToXContentEmbedded() throws IOException { fields.put("foo", new DocumentField("foo", singletonList("bar"))); fields.put("baz", new DocumentField("baz", Arrays.asList("baz_0", "baz_1"))); - GetResult getResult = new GetResult("index", "type", "id", 2, true, + GetResult getResult = new GetResult("index", "type", "id", 0, 1, 2, true, new BytesArray("{\"foo\":\"bar\",\"baz\":[\"baz_0\",\"baz_1\"]}"), fields); BytesReference originalBytes = toXContentEmbedded(getResult, XContentType.JSON, false); @@ -127,7 +129,7 @@ public void testToXContentEmbedded() throws IOException { } public void testToXContentEmbeddedNotFound() throws IOException { - GetResult getResult = new GetResult("index", "type", "id", 1, false, null, null); + GetResult getResult = new GetResult("index", "type", "id", 0, 1, 1, false, null, null); BytesReference originalBytes = toXContentEmbedded(getResult, XContentType.JSON, false); assertEquals("{\"found\":false}", originalBytes.utf8ToString()); @@ -149,25 +151,34 @@ public void testEqualsAndHashcode() { } public static GetResult copyGetResult(GetResult getResult) { - return new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), getResult.getVersion(), - getResult.isExists(), getResult.internalSourceRef(), getResult.getFields()); + return new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), + getResult.getSeqNo(), getResult.getPrimaryTerm(), getResult.getVersion(), + getResult.isExists(), getResult.internalSourceRef(), getResult.getFields()); } public static GetResult mutateGetResult(GetResult getResult) { List> mutations = new ArrayList<>(); - mutations.add(() -> new GetResult(randomUnicodeOfLength(15), getResult.getType(), getResult.getId(), getResult.getVersion(), + mutations.add(() -> new GetResult(randomUnicodeOfLength(15), getResult.getType(), getResult.getId(), + getResult.getSeqNo(), getResult.getPrimaryTerm(), getResult.getVersion(), getResult.isExists(), getResult.internalSourceRef(), getResult.getFields())); - mutations.add(() -> new GetResult(getResult.getIndex(), randomUnicodeOfLength(15), getResult.getId(), getResult.getVersion(), - getResult.isExists(), getResult.internalSourceRef(), getResult.getFields())); - mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), randomUnicodeOfLength(15), getResult.getVersion(), - getResult.isExists(), getResult.internalSourceRef(), getResult.getFields())); - mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), randomNonNegativeLong(), - getResult.isExists(), getResult.internalSourceRef(), getResult.getFields())); - mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), getResult.getVersion(), - getResult.isExists() == false, getResult.internalSourceRef(), getResult.getFields())); - mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), getResult.getVersion(), - getResult.isExists(), RandomObjects.randomSource(random()), getResult.getFields())); - mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), getResult.getVersion(), + mutations.add(() -> new GetResult(getResult.getIndex(), randomUnicodeOfLength(15), getResult.getId(), + getResult.getSeqNo(), getResult.getPrimaryTerm(), getResult.getVersion(), + getResult.isExists(), getResult.internalSourceRef(), getResult.getFields())); + mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), randomUnicodeOfLength(15), + getResult.getSeqNo(), getResult.getPrimaryTerm(), getResult.getVersion(), + getResult.isExists(), getResult.internalSourceRef(), getResult.getFields())); + mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), + getResult.getSeqNo(), getResult.getPrimaryTerm(), randomNonNegativeLong(), + getResult.isExists(), getResult.internalSourceRef(), getResult.getFields())); + mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), + getResult.isExists() ? SequenceNumbers.UNASSIGNED_SEQ_NO : getResult.getSeqNo(), + getResult.isExists() ? 0 : getResult.getPrimaryTerm(), + getResult.getVersion(), getResult.isExists() == false, getResult.internalSourceRef(), getResult.getFields())); + mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), + getResult.getSeqNo(), getResult.getPrimaryTerm(), getResult.getVersion(), getResult.isExists(), + RandomObjects.randomSource(random()), getResult.getFields())); + mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), + getResult.getSeqNo(), getResult.getPrimaryTerm(), getResult.getVersion(), getResult.isExists(), getResult.internalSourceRef(), randomDocumentFields(XContentType.JSON).v1())); return randomFrom(mutations).get(); } @@ -177,12 +188,16 @@ public static Tuple randomGetResult(XContentType xContentT final String type = randomAlphaOfLengthBetween(3, 10); final String id = randomAlphaOfLengthBetween(3, 10); final long version; + final long seqNo; + final long primaryTerm; final boolean exists; BytesReference source = null; Map fields = null; Map expectedFields = null; if (frequently()) { version = randomNonNegativeLong(); + seqNo = randomNonNegativeLong(); + primaryTerm = randomLongBetween(1, 100); exists = true; if (frequently()) { source = RandomObjects.randomSource(random()); @@ -193,11 +208,13 @@ public static Tuple randomGetResult(XContentType xContentT expectedFields = tuple.v2(); } } else { + seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + primaryTerm = 0; version = -1; exists = false; } - GetResult getResult = new GetResult(index, type, id, version, exists, source, fields); - GetResult expectedGetResult = new GetResult(index, type, id, version, exists, source, expectedFields); + GetResult getResult = new GetResult(index, type, id, seqNo, primaryTerm, version, exists, source, fields); + GetResult expectedGetResult = new GetResult(index, type, id, seqNo, primaryTerm, version, exists, source, expectedFields); return Tuple.tuple(getResult, expectedGetResult); } diff --git a/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java index ca9a21973aa39..9d7d3a192a55f 100644 --- a/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java @@ -131,7 +131,7 @@ protected GetResponse executeGet(GetRequest getRequest) { } catch (IOException ex) { throw new ElasticsearchException("boom", ex); } - return new GetResponse(new GetResult(indexedShapeIndex, indexedShapeType, indexedShapeId, 0, true, new BytesArray(json), null)); + return new GetResponse(new GetResult(indexedShapeIndex, indexedShapeType, indexedShapeId, 0, 1, 0, true, new BytesArray(json), null)); } @After diff --git a/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java index 9ed303316d9cb..7c2b40cebc3c9 100644 --- a/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java @@ -201,7 +201,7 @@ public GetResponse executeGet(GetRequest getRequest) { } catch (IOException ex) { throw new ElasticsearchException("boom", ex); } - return new GetResponse(new GetResult(getRequest.index(), getRequest.type(), getRequest.id(), 0, true, new BytesArray(json), null)); + return new GetResponse(new GetResult(getRequest.index(), getRequest.type(), getRequest.id(), 0, 1, 0, true, new BytesArray(json), null)); } public void testNumeric() throws IOException { diff --git a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java index e8d9dd0fc2cc9..6ce875e23f28c 100644 --- a/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java +++ b/server/src/test/java/org/elasticsearch/versioning/SimpleVersioningIT.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteResponse; +import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -262,6 +263,73 @@ public void testInternalVersioning() throws Exception { assertThat(deleteResponse.getVersion(), equalTo(4L)); } + public void testCompareAndSet() { + createIndex("test"); + ensureGreen(); + + IndexResponse indexResponse = client().prepareIndex("test", "type", "1").setSource("field1", "value1_1").execute().actionGet(); + assertThat(indexResponse.getSeqNo(), equalTo(0L)); + assertThat(indexResponse.getPrimaryTerm(), equalTo(1L)); + + indexResponse = client().prepareIndex("test", "type", "1").setSource("field1", "value1_2").setVersion(1).execute().actionGet(); + assertThat(indexResponse.getSeqNo(), equalTo(1L)); + assertThat(indexResponse.getPrimaryTerm(), equalTo(1L)); + + assertThrows( + client().prepareIndex("test", "type", "1").setSource("field1", "value1_1").setCompareAndSet(10, 1).execute(), + VersionConflictEngineException.class); + + assertThrows( + client().prepareIndex("test", "type", "1").setSource("field1", "value1_1").setCompareAndSet(10, 2).execute(), + VersionConflictEngineException.class); + + assertThrows( + client().prepareIndex("test", "type", "1").setSource("field1", "value1_1").setCompareAndSet(1, 2).execute(), + VersionConflictEngineException.class); + + + assertThrows(client().prepareDelete("test", "type", "1").setCompareAndSet(10, 1).execute(), VersionConflictEngineException.class); + assertThrows(client().prepareDelete("test", "type", "1").setCompareAndSet(10, 2).execute(), VersionConflictEngineException.class); + assertThrows(client().prepareDelete("test", "type", "1").setCompareAndSet(1, 2).execute(), VersionConflictEngineException.class); + + client().admin().indices().prepareRefresh().execute().actionGet(); + for (int i = 0; i < 10; i++) { + final GetResponse response = client().prepareGet("test", "type", "1").get(); + assertThat(response.getSeqNo(), equalTo(1L)); + assertThat(response.getPrimaryTerm(), equalTo(1L)); + } + + // search with versioning + for (int i = 0; i < 10; i++) { + // TODO: ADD SEQ NO! + SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setVersion(true).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).getVersion(), equalTo(2L)); + } + + // search without versioning + for (int i = 0; i < 10; i++) { + SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).getVersion(), equalTo(Versions.NOT_FOUND)); + } + + DeleteResponse deleteResponse = client().prepareDelete("test", "type", "1").setCompareAndSet(1, 1).execute().actionGet(); + assertEquals(DocWriteResponse.Result.DELETED, deleteResponse.getResult()); + assertThat(deleteResponse.getSeqNo(), equalTo(2L)); + assertThat(deleteResponse.getPrimaryTerm(), equalTo(1L)); + + assertThrows(client().prepareDelete("test", "type", "1").setCompareAndSet(1, 1).execute(), VersionConflictEngineException.class); + assertThrows(client().prepareDelete("test", "type", "1").setCompareAndSet(3, 2).execute(), VersionConflictEngineException.class); + assertThrows(client().prepareDelete("test", "type", "1").setCompareAndSet(1, 2).execute(), VersionConflictEngineException.class); + + + // This is intricate - the object was deleted but a delete transaction was with the right version. We add another one + // and thus the transaction is increased. + deleteResponse = client().prepareDelete("test", "type", "1").setCompareAndSet(2, 1).execute().actionGet(); + assertEquals(DocWriteResponse.Result.NOT_FOUND, deleteResponse.getResult()); + assertThat(deleteResponse.getSeqNo(), equalTo(3L)); + assertThat(deleteResponse.getPrimaryTerm(), equalTo(1L)); + } + public void testSimpleVersioningWithFlush() throws Exception { createIndex("test"); ensureGreen(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java index 41bd8bfc6e620..922c35e2f0c70 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeUsersStoreTests.java @@ -112,7 +112,7 @@ public void testBlankPasswordInIndexImpliesDefaultPassword() throws Exception { SecurityIndexManager.SECURITY_INDEX_NAME, NativeUsersStore.INDEX_TYPE, NativeUsersStore.getIdForUser(NativeUsersStore.RESERVED_USER_TYPE, randomAlphaOfLength(12)), - 1L, + 0, 1, 1L, true, BytesReference.bytes(jsonBuilder().map(values)), Collections.emptyMap()); @@ -181,7 +181,7 @@ public void testVerifyNonExistentUser() throws Exception { SecurityIndexManager.SECURITY_INDEX_NAME, NativeUsersStore.INDEX_TYPE, NativeUsersStore.getIdForUser(NativeUsersStore.USER_DOC_TYPE, username), - 1L, + 0, 1, 1L, false, null, Collections.emptyMap()); @@ -223,7 +223,7 @@ private void respondToGetUserRequest(String username, SecureString password, Str SecurityIndexManager.SECURITY_INDEX_NAME, NativeUsersStore.INDEX_TYPE, NativeUsersStore.getIdForUser(NativeUsersStore.USER_DOC_TYPE, username), - 1L, + 0, 1, 1L, true, source, Collections.emptyMap()); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java index 2c5bd4a321e33..87b1385e3dc68 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java @@ -128,7 +128,7 @@ public void testGetSinglePrivilegeByName() throws Exception { final String docSource = Strings.toString(sourcePrivilege); listener.get().onResponse(new GetResponse( - new GetResult(request.index(), request.type(), request.id(), 1L, true, new BytesArray(docSource), emptyMap()) + new GetResult(request.index(), request.type(), request.id(), 0, 1, 1L, true, new BytesArray(docSource), emptyMap()) )); final ApplicationPrivilegeDescriptor getPrivilege = future.get(1, TimeUnit.SECONDS); assertThat(getPrivilege, equalTo(sourcePrivilege)); @@ -145,7 +145,7 @@ public void testGetMissingPrivilege() throws Exception { assertThat(request.id(), equalTo("application-privilege_myapp:admin")); listener.get().onResponse(new GetResponse( - new GetResult(request.index(), request.type(), request.id(), -1, false, null, emptyMap()) + new GetResult(request.index(), request.type(), request.id(), 0, 1, -1, false, null, emptyMap()) )); final ApplicationPrivilegeDescriptor getPrivilege = future.get(1, TimeUnit.SECONDS); assertThat(getPrivilege, Matchers.nullValue()); diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/ExecutionServiceTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/ExecutionServiceTests.java index 13761948adccf..a4cb1a0383005 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/ExecutionServiceTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/ExecutionServiceTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.ObjectPath; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; @@ -53,7 +54,6 @@ import org.elasticsearch.xpack.core.watcher.history.WatchRecord; import org.elasticsearch.xpack.core.watcher.input.ExecutableInput; import org.elasticsearch.xpack.core.watcher.input.Input; -import org.elasticsearch.common.xcontent.ObjectPath; import org.elasticsearch.xpack.core.watcher.transform.ExecutableTransform; import org.elasticsearch.xpack.core.watcher.transform.Transform; import org.elasticsearch.xpack.core.watcher.trigger.TriggerEvent; @@ -1148,7 +1148,7 @@ private void mockGetWatchResponse(Client client, String id, GetResponse response if (request.id().equals(id)) { listener.onResponse(response); } else { - GetResult notFoundResult = new GetResult(request.index(), request.type(), request.id(), -1, false, null, null); + GetResult notFoundResult = new GetResult(request.index(), request.type(), request.id(), 0, 1, -1, false, null, null); listener.onResponse(new GetResponse(notFoundResult)); } return null; @@ -1162,7 +1162,7 @@ private void mockGetWatchException(Client client, String id, Exception e) { if (request.id().equals(id)) { listener.onFailure(e); } else { - GetResult notFoundResult = new GetResult(request.index(), request.type(), request.id(), -1, false, null, null); + GetResult notFoundResult = new GetResult(request.index(), request.type(), request.id(), 0, 1, -1, false, null, null); listener.onResponse(new GetResponse(notFoundResult)); } return null; diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/transport/actions/ack/TransportAckWatchActionTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/transport/actions/ack/TransportAckWatchActionTests.java index 0b57c856744f0..f1425101b67c1 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/transport/actions/ack/TransportAckWatchActionTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/transport/actions/ack/TransportAckWatchActionTests.java @@ -67,7 +67,7 @@ public void testWatchNotFound() { String watchId = "my_watch_id"; doAnswer(invocation -> { ActionListener listener = (ActionListener) invocation.getArguments()[1]; - listener.onResponse(new GetResponse(new GetResult(Watch.INDEX, Watch.DOC_TYPE, watchId, -1, false, + listener.onResponse(new GetResponse(new GetResult(Watch.INDEX, Watch.DOC_TYPE, watchId, 0, 1, -1, false, BytesArray.EMPTY, Collections.emptyMap()))); return null; }).when(client).get(anyObject(), anyObject()); From d2b6b6d4d5d4e4c49093fb880e4a8e1d7c3bdeb4 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 15 Nov 2018 15:46:15 +0100 Subject: [PATCH 18/33] fix get response tests --- .../java/org/elasticsearch/action/get/GetResponseTests.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java b/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java index 146cc55da8630..9b24a9bd80ee4 100644 --- a/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java @@ -97,11 +97,11 @@ public void testToXContent() { Collections.singletonList("value1"))))); String output = Strings.toString(getResponse); assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_seq_no\":0,\"_primary_term\":1,\"_version\":1," + - "\"found\":true,\"_source\":{ \"field1\": \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", + "\"found\":true,\"_source\":{ \"field1\" : \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", output); } { - GetResponse getResponse = new GetResponse(new GetResult("index", "type", "id", 0, 1, 1, false, null, null)); + GetResponse getResponse = new GetResponse(new GetResult("index", "type", "id", UNASSIGNED_SEQ_NO, 0, 1, false, null, null)); String output = Strings.toString(getResponse); assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"found\":false}", output); } @@ -112,7 +112,7 @@ public void testToString() { new GetResult("index", "type", "id", 0, 1, 1, true, new BytesArray("{ \"field1\" : " + "\"value1\", \"field2\":\"value2\"}"), Collections.singletonMap("field1", new DocumentField("field1", Collections.singletonList("value1"))))); assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_seq_no\":0,\"_primary_term\":1,\"_version\":1," + - "\"found\":true,\"_source\":{ \"field1\": \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", + "\"found\":true,\"_source\":{ \"field1\" : \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", getResponse.toString()); } From 1cb3faabd4a057f5af84bc060a92dd56b1b25b2f Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 15 Nov 2018 15:53:32 +0100 Subject: [PATCH 19/33] updates should use the right seq no info --- .../action/bulk/TransportShardBulkAction.java | 4 +++- .../action/update/TransportUpdateAction.java | 8 ++++---- .../elasticsearch/action/update/UpdateHelper.java | 14 +++++++------- 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 2cdea0ff68612..f27c265042e45 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -307,6 +307,7 @@ static BulkItemResponse processUpdateResponse(final UpdateRequest updateRequest, final Tuple> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true, updateIndexRequest.getContentType()); updateResponse.setGetResult(UpdateHelper.extractGetResult(updateRequest, concreteIndex, + indexResponse.getSeqNo(), indexResponse.getPrimaryTerm(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); } } else if (translatedResult == DocWriteResponse.Result.DELETED) { @@ -315,7 +316,8 @@ static BulkItemResponse processUpdateResponse(final UpdateRequest updateRequest, deleteResponse.getType(), deleteResponse.getId(), deleteResponse.getSeqNo(), deleteResponse.getPrimaryTerm(), deleteResponse.getVersion(), deleteResponse.getResult()); - final GetResult getResult = UpdateHelper.extractGetResult(updateRequest, concreteIndex, deleteResponse.getVersion(), + final GetResult getResult = UpdateHelper.extractGetResult(updateRequest, concreteIndex, + deleteResponse.getPrimaryTerm(), deleteResponse.getSeqNo(), deleteResponse.getVersion(), translate.updatedSourceAsMap(), translate.updateSourceContentType(), null); updateResponse.setGetResult(getResult); diff --git a/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java b/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java index 9cf85c1c7732f..95bae6f518f48 100644 --- a/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java +++ b/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.update; +import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.RoutingMissingException; @@ -50,7 +51,6 @@ import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; @@ -181,7 +181,7 @@ protected void shardOperation(final UpdateRequest request, final ActionListener< if (request.fetchSource() != null && request.fetchSource().fetchSource()) { Tuple> sourceAndContent = XContentHelper.convertToMap(upsertSourceBytes, true, upsertRequest.getContentType()); - update.setGetResult(UpdateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), upsertSourceBytes)); + update.setGetResult(UpdateHelper.extractGetResult(request, request.concreteIndex(), response.getPrimaryTerm(), response.getSeqNo(), response.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), upsertSourceBytes)); } else { update.setGetResult(null); } @@ -198,7 +198,7 @@ protected void shardOperation(final UpdateRequest request, final ActionListener< client.bulk(toSingleItemBulkRequest(indexRequest), wrapBulkResponse( ActionListener.wrap(response -> { UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getSeqNo(), response.getPrimaryTerm(), response.getVersion(), response.getResult()); - update.setGetResult(UpdateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes)); + update.setGetResult(UpdateHelper.extractGetResult(request, request.concreteIndex(), response.getPrimaryTerm(), response.getSeqNo(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes)); update.setForcedRefresh(response.forcedRefresh()); listener.onResponse(update); }, exception -> handleUpdateFailureWithRetry(listener, request, exception, retryCount))) @@ -209,7 +209,7 @@ protected void shardOperation(final UpdateRequest request, final ActionListener< client.bulk(toSingleItemBulkRequest(deleteRequest), wrapBulkResponse( ActionListener.wrap(response -> { UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getSeqNo(), response.getPrimaryTerm(), response.getVersion(), response.getResult()); - update.setGetResult(UpdateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null)); + update.setGetResult(UpdateHelper.extractGetResult(request, request.concreteIndex(), response.getPrimaryTerm(), response.getSeqNo(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null)); update.setForcedRefresh(response.forcedRefresh()); listener.onResponse(update); }, exception -> handleUpdateFailureWithRetry(listener, request, exception, retryCount))) diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index 94efdffd404d0..473d6fbb33973 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -207,8 +207,8 @@ Result prepareUpdateIndexRequest(ShardId shardId, UpdateRequest request, GetResu if (detectNoop && noop) { UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), DocWriteResponse.Result.NOOP); - update.setGetResult(extractGetResult(request, request.index(), getResult.getVersion(), updatedSourceAsMap, - updateSourceContentType, getResult.internalSourceRef())); + update.setGetResult(extractGetResult(request, request.index(), getResult.getPrimaryTerm(), getResult.getSeqNo(), + getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef())); return new Result(update, DocWriteResponse.Result.NOOP, updatedSourceAsMap, updateSourceContentType); } else { final IndexRequest finalIndexRequest = Requests.indexRequest(request.index()) @@ -268,10 +268,9 @@ Result prepareUpdateScriptRequest(ShardId shardId, UpdateRequest request, GetRes // If it was neither an INDEX or DELETE operation, treat it as a noop UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), DocWriteResponse.Result.NOOP); - update.setGetResult(extractGetResult(request, request.index(), getResult.getVersion(), updatedSourceAsMap, - updateSourceContentType, getResult.internalSourceRef())); + update.setGetResult(extractGetResult(request, request.index(), getResult.getPrimaryTerm(), getResult.getSeqNo(), + getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef())); return new Result(update, DocWriteResponse.Result.NOOP, updatedSourceAsMap, updateSourceContentType); - } } @@ -291,7 +290,7 @@ private Map executeScript(Script script, Map ctx /** * Applies {@link UpdateRequest#fetchSource()} to the _source of the updated document to be returned in a update response. */ - public static GetResult extractGetResult(final UpdateRequest request, String concreteIndex, long version, + public static GetResult extractGetResult(final UpdateRequest request, String concreteIndex, long primaryTerm, long seqNo, long version, final Map source, XContentType sourceContentType, @Nullable final BytesReference sourceAsBytes) { if (request.fetchSource() == null || request.fetchSource().fetchSource() == false) { @@ -316,7 +315,8 @@ public static GetResult extractGetResult(final UpdateRequest request, String con } // TODO when using delete/none, we can still return the source as bytes by generating it (using the sourceContentType) - return new GetResult(concreteIndex, request.type(), request.id(), 0, 1, version, true, sourceFilteredAsBytes, Collections.emptyMap()); + return new GetResult(concreteIndex, request.type(), request.id(), primaryTerm, seqNo, version, true, sourceFilteredAsBytes, + Collections.emptyMap()); } public static class Result { From c87530d03e2372faf1603c73bd07aee0817e05da Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 15 Nov 2018 16:01:51 +0100 Subject: [PATCH 20/33] line length --- .../elasticsearch/percolator/PercolateQueryBuilderTests.java | 4 +++- .../java/org/elasticsearch/action/get/GetResponseTests.java | 4 ++-- .../elasticsearch/index/query/GeoShapeQueryBuilderTests.java | 3 ++- .../org/elasticsearch/index/query/TermsQueryBuilderTests.java | 3 ++- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java index aef9c36a802a3..d4fe0fe1dddd4 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolateQueryBuilderTests.java @@ -54,6 +54,7 @@ import java.util.Map; import java.util.Set; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.equalTo; public class PercolateQueryBuilderTests extends AbstractQueryTestCase { @@ -157,7 +158,8 @@ protected GetResponse executeGet(GetRequest getRequest) { ); } else { return new GetResponse( - new GetResult(indexedDocumentIndex, indexedDocumentType, indexedDocumentId, 0, 1, -1, false, null, Collections.emptyMap()) + new GetResult(indexedDocumentIndex, indexedDocumentType, indexedDocumentId, UNASSIGNED_SEQ_NO, 0, -1, + false, null, Collections.emptyMap()) ); } } diff --git a/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java b/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java index 9b24a9bd80ee4..7db2e65c858c2 100644 --- a/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java @@ -108,8 +108,8 @@ public void testToXContent() { } public void testToString() { - GetResponse getResponse = new GetResponse( - new GetResult("index", "type", "id", 0, 1, 1, true, new BytesArray("{ \"field1\" : " + "\"value1\", \"field2\":\"value2\"}"), + GetResponse getResponse = new GetResponse(new GetResult("index", "type", "id", 0, 1, 1, true, + new BytesArray("{ \"field1\" : " + "\"value1\", \"field2\":\"value2\"}"), Collections.singletonMap("field1", new DocumentField("field1", Collections.singletonList("value1"))))); assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_seq_no\":0,\"_primary_term\":1,\"_version\":1," + "\"found\":true,\"_source\":{ \"field1\" : \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", diff --git a/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java index 9d7d3a192a55f..bcd2b4ef14440 100644 --- a/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java @@ -131,7 +131,8 @@ protected GetResponse executeGet(GetRequest getRequest) { } catch (IOException ex) { throw new ElasticsearchException("boom", ex); } - return new GetResponse(new GetResult(indexedShapeIndex, indexedShapeType, indexedShapeId, 0, 1, 0, true, new BytesArray(json), null)); + return new GetResponse(new GetResult(indexedShapeIndex, indexedShapeType, indexedShapeId, 0, 1, 0, true, new BytesArray(json), + null)); } @After diff --git a/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java index 7c2b40cebc3c9..c954be5403f49 100644 --- a/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java @@ -201,7 +201,8 @@ public GetResponse executeGet(GetRequest getRequest) { } catch (IOException ex) { throw new ElasticsearchException("boom", ex); } - return new GetResponse(new GetResult(getRequest.index(), getRequest.type(), getRequest.id(), 0, 1, 0, true, new BytesArray(json), null)); + return new GetResponse(new GetResult(getRequest.index(), getRequest.type(), getRequest.id(), 0, 1, 0, true, + new BytesArray(json), null)); } public void testNumeric() throws IOException { From 4e20e2b8bcfae96651a848535db2445620d86c80 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 17 Nov 2018 12:04:05 +0100 Subject: [PATCH 21/33] compile --- .../java/org/elasticsearch/index/engine/FrozenEngineTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenEngineTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenEngineTests.java index c876c4cf98df1..689a6e3db7ed5 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenEngineTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenEngineTests.java @@ -174,7 +174,7 @@ private int addDocuments(AtomicLong globalCheckpoint, InternalEngine engine) thr numDocsAdded++; ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, - System.nanoTime(), -1, false)); + System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); if (rarely()) { engine.flush(); } From 95d6c6a727d6d6622accb291022659ca28b50087 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 29 Nov 2018 16:51:36 +0100 Subject: [PATCH 22/33] lint --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index c952bb09852f3..fa248db38f1cd 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -683,7 +683,8 @@ private IndexShardState changeState(IndexShardState newState, String reason) { } public Engine.IndexResult applyIndexOperationOnPrimary(long version, VersionType versionType, SourceToParse sourceToParse, - long casSeqNo, long casPrimaryTerm, long autoGeneratedTimestamp, boolean isRetry) throws IOException { + long casSeqNo, long casPrimaryTerm, long autoGeneratedTimestamp, boolean isRetry) + throws IOException { assert versionType.validateVersionForWrites(version); return applyIndexOperation(UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, casSeqNo, casPrimaryTerm, autoGeneratedTimestamp, isRetry, Engine.Operation.Origin.PRIMARY, sourceToParse); From 1c0f81f271c63b919033b8d22985e539e2e7552a Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 29 Nov 2018 17:22:24 +0100 Subject: [PATCH 23/33] compilation --- .../rest/action/document/RestGetSourceActionTests.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestGetSourceActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestGetSourceActionTests.java index b9fd724fb656d..a4a0b81c76d32 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestGetSourceActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestGetSourceActionTests.java @@ -32,6 +32,7 @@ import org.junit.AfterClass; import static java.util.Collections.emptyMap; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.rest.RestStatus.OK; import static org.elasticsearch.rest.action.document.RestGetSourceAction.RestGetSourceResponseListener; import static org.hamcrest.Matchers.equalTo; @@ -51,7 +52,8 @@ public static void cleanupReferences() { public void testRestGetSourceAction() throws Exception { final BytesReference source = new BytesArray("{\"foo\": \"bar\"}"); - final GetResponse response = new GetResponse(new GetResult("index1", "_doc", "1", -1, true, source, emptyMap())); + final GetResponse response = + new GetResponse(new GetResult("index1", "_doc", "1", UNASSIGNED_SEQ_NO, 0, -1, true, source, emptyMap())); final RestResponse restResponse = listener.buildResponse(response); @@ -61,7 +63,8 @@ public void testRestGetSourceAction() throws Exception { } public void testRestGetSourceActionWithMissingDocument() { - final GetResponse response = new GetResponse(new GetResult("index1", "_doc", "1", -1, false, null, emptyMap())); + final GetResponse response = + new GetResponse(new GetResult("index1", "_doc", "1", -1, UNASSIGNED_SEQ_NO, 0, false, null, emptyMap())); final ResourceNotFoundException exception = expectThrows(ResourceNotFoundException.class, () -> listener.buildResponse(response)); @@ -69,7 +72,8 @@ public void testRestGetSourceActionWithMissingDocument() { } public void testRestGetSourceActionWithMissingDocumentSource() { - final GetResponse response = new GetResponse(new GetResult("index1", "_doc", "1", -1, true, null, emptyMap())); + final GetResponse response = + new GetResponse(new GetResult("index1", "_doc", "1", UNASSIGNED_SEQ_NO, 0, -1, true, null, emptyMap())); final ResourceNotFoundException exception = expectThrows(ResourceNotFoundException.class, () -> listener.buildResponse(response)); From 97db90a54f5d868468cd6c93a9276dd1f630de91 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 29 Nov 2018 21:26:51 +0100 Subject: [PATCH 24/33] lint --- .../index/engine/InternalEngineTests.java | 20 ++++++++-------- .../elasticsearch/indices/flush/FlushIT.java | 23 ++++++++++--------- 2 files changed, 22 insertions(+), 21 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 57a571cf349a2..d2e11078c39ba 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -67,7 +67,6 @@ import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.FixedBitSet; -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; @@ -1721,7 +1720,8 @@ public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, Interrup Document doc = testDocumentWithTextField(index.docs().get(0).get("value")); ParsedDocument parsedDocument = testParsedDocument(index.id(), index.routing(), doc, index.source(), null); return new Engine.Index(index.uid(), parsedDocument, newSeqNo, index.primaryTerm(), index.version(), - index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry(), UNASSIGNED_SEQ_NO, 0); + index.versionType(), index.origin(), index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry(), + UNASSIGNED_SEQ_NO, 0); } else { Engine.Delete delete = (Engine.Delete) operation; return new Engine.Delete(delete.type(), delete.id(), delete.uid(), newSeqNo, delete.primaryTerm(), @@ -1808,12 +1808,12 @@ private int assertOpsOnPrimary(List ops, long currentOpVersion BiFunction delWithVersion = (version, delete) -> new Engine.Delete(delete.type(), delete.id(), delete.uid(), UNASSIGNED_SEQ_NO, currentTerm.get(), version, delete.versionType(), delete.origin(), delete.startTime(), UNASSIGNED_SEQ_NO, 0); - TriFunction indexWithSeq = (seqNo, term, index) -> new Engine.Index(index.uid(), index.parsedDoc(), - UNASSIGNED_SEQ_NO, currentTerm.get(), index.version(), index.versionType(), index.origin(), index.startTime(), - index.getAutoGeneratedIdTimestamp(), index.isRetry(), seqNo, term); - TriFunction delWitSeq = (seqNo, term, delete) -> new Engine.Delete(delete.type(), delete.id(), - delete.uid(), UNASSIGNED_SEQ_NO, currentTerm.get(), delete.version(), delete.versionType(), delete.origin(), delete.startTime(), - seqNo, term); + TriFunction indexWithSeq = (seqNo, term, index) -> new Engine.Index(index.uid(), + index.parsedDoc(), UNASSIGNED_SEQ_NO, currentTerm.get(), index.version(), index.versionType(), index.origin(), + index.startTime(), index.getAutoGeneratedIdTimestamp(), index.isRetry(), seqNo, term); + TriFunction delWitSeq = (seqNo, term, delete) -> new Engine.Delete(delete.type(), + delete.id(), delete.uid(), UNASSIGNED_SEQ_NO, currentTerm.get(), delete.version(), delete.versionType(), delete.origin(), + delete.startTime(), seqNo, term); for (Engine.Operation op : ops) { final boolean versionConflict = rarely(); final boolean versionedOp = versionConflict || randomBoolean(); @@ -3542,8 +3542,8 @@ public Engine.Index randomAppendOnly(ParsedDocument doc, boolean retry, final lo } public Engine.Index appendOnlyPrimary(ParsedDocument doc, boolean retry, final long autoGeneratedIdTimestamp) { - return new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, - VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, retry, UNASSIGNED_SEQ_NO, 0); + return new Engine.Index(newUid(doc), doc, UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, + Engine.Operation.Origin.PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, retry, UNASSIGNED_SEQ_NO, 0); } public Engine.Index appendOnlyReplica(ParsedDocument doc, boolean retry, final long autoGeneratedIdTimestamp, final long seqNo) { diff --git a/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java b/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java index 67b24e756de5c..e190285ebef83 100644 --- a/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java +++ b/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java @@ -18,16 +18,6 @@ */ package org.elasticsearch.indices.flush; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; - import org.apache.lucene.index.Term; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; @@ -58,6 +48,16 @@ import org.elasticsearch.indices.IndicesService; import org.elasticsearch.test.ESIntegTestCase; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; @@ -254,7 +254,8 @@ public void testUnallocatedShardsDoesNotHang() throws InterruptedException { private void indexDoc(Engine engine, String id) throws IOException { final ParsedDocument doc = InternalEngineTests.createParsedDoc(id, null); final Engine.IndexResult indexResult = engine.index(new Engine.Index(new Term("_id", Uid.encodeId(doc.id())), doc, - engine.getLocalCheckpoint() + 1, 1L, 1L, null, Engine.Operation.Origin.REPLICA, randomLong(), -1L, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); + engine.getLocalCheckpoint() + 1, 1L, 1L, null, Engine.Operation.Origin.REPLICA, randomLong(), -1L, false, + SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); assertThat(indexResult.getFailure(), nullValue()); } From a6d9ef607b5a2b2891c191bfe22c467cdcd82ca2 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 29 Nov 2018 21:53:38 +0100 Subject: [PATCH 25/33] fix UpdateRequestTests --- .../elasticsearch/action/update/UpdateRequestTests.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java index a53fa804f5f17..32bbadae13483 100644 --- a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java @@ -60,6 +60,7 @@ import static java.util.Collections.emptyMap; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentHelper.toXContent; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.script.MockScriptEngine.mockInlineScript; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent; import static org.hamcrest.CoreMatchers.hasItems; @@ -359,7 +360,7 @@ public void testNowInScript() throws IOException { .scriptedUpsert(true); long nowInMillis = randomNonNegativeLong(); // We simulate that the document is not existing yet - GetResult getResult = new GetResult("test", "type1", "2", 0, 1, 0, false, null, null); + GetResult getResult = new GetResult("test", "type1", "2", UNASSIGNED_SEQ_NO, 0, 0, false, null, null); UpdateHelper.Result result = updateHelper.prepare(new ShardId("test", "_na_", 0), updateRequest, getResult, () -> nowInMillis); Streamable action = result.action(); assertThat(action, instanceOf(IndexRequest.class)); @@ -402,7 +403,7 @@ public void testDeleteTimeout() { public void testUpsertTimeout() throws IOException { final boolean exists = randomBoolean(); final BytesReference source = exists ? new BytesArray("{\"f\":\"v\"}") : null; - final GetResult getResult = new GetResult("test", "type", "1", 0, 1, 0, exists, source, null); + final GetResult getResult = new GetResult("test", "type", "1", UNASSIGNED_SEQ_NO, 0, 0, exists, source, null); final XContentBuilder sourceBuilder = jsonBuilder(); sourceBuilder.startObject(); { @@ -535,7 +536,7 @@ public void testValidate() { } public void testRoutingExtraction() throws Exception { - GetResult getResult = new GetResult("test", "type", "1", 0, 1, 0, false, null, null); + GetResult getResult = new GetResult("test", "type", "1", UNASSIGNED_SEQ_NO, 0, 0, false, null, null); IndexRequest indexRequest = new IndexRequest("test", "type", "1"); // There is no routing and parent because the document doesn't exist From 7f2d0d5cb41b041285ed2d735572e10896ddd051 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 30 Nov 2018 10:29:38 +0100 Subject: [PATCH 26/33] render seq# with get results of updates. --- .../action/update/UpdateResponse.java | 5 +++-- .../org/elasticsearch/index/get/GetResult.java | 9 +++++---- .../action/explain/ExplainResponseTests.java | 2 ++ .../action/get/GetResponseTests.java | 4 ++-- .../action/update/UpdateResponseTests.java | 5 +++-- .../index/get/GetResultTests.java | 18 +++++++++--------- 6 files changed, 24 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateResponse.java b/server/src/main/java/org/elasticsearch/action/update/UpdateResponse.java index 9c9bd0455e27b..45649f50a0a60 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateResponse.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateResponse.java @@ -162,8 +162,9 @@ public UpdateResponse build() { update = new UpdateResponse(shardId, type, id, version, result); } if (getResult != null) { - update.setGetResult(new GetResult(update.getIndex(), update.getType(), update.getId(), 0, 1, update.getVersion(), - getResult.isExists(),getResult.internalSourceRef(), getResult.getFields())); + update.setGetResult(new GetResult(update.getIndex(), update.getType(), update.getId(), + getResult.getSeqNo(), getResult.getPrimaryTerm(), update.getVersion(), + getResult.isExists(),getResult.internalSourceRef(), getResult.getFields())); } update.setForcedRefresh(forcedRefresh); return update; diff --git a/server/src/main/java/org/elasticsearch/index/get/GetResult.java b/server/src/main/java/org/elasticsearch/index/get/GetResult.java index bdaf1e5f6eb23..19e5de98ba832 100644 --- a/server/src/main/java/org/elasticsearch/index/get/GetResult.java +++ b/server/src/main/java/org/elasticsearch/index/get/GetResult.java @@ -239,6 +239,11 @@ public Iterator iterator() { } public XContentBuilder toXContentEmbedded(XContentBuilder builder, Params params) throws IOException { + if (seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { // seqNo may not be assigned if read from an old node + builder.field(_SEQ_NO, seqNo); + builder.field(_PRIMARY_TERM, primaryTerm); + } + List metaFields = new ArrayList<>(); List otherFields = new ArrayList<>(); if (fields != null && !fields.isEmpty()) { @@ -285,10 +290,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(_TYPE, type); builder.field(_ID, id); if (isExists()) { - if (seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { // seqNo may not be assigned if read from an old node - builder.field(_SEQ_NO, seqNo); - builder.field(_PRIMARY_TERM, primaryTerm); - } if (version != -1) { builder.field(_VERSION, version); } diff --git a/server/src/test/java/org/elasticsearch/action/explain/ExplainResponseTests.java b/server/src/test/java/org/elasticsearch/action/explain/ExplainResponseTests.java index 361955342287f..2a04a97667722 100644 --- a/server/src/test/java/org/elasticsearch/action/explain/ExplainResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/explain/ExplainResponseTests.java @@ -105,6 +105,8 @@ public void testToXContent() throws IOException { " \"details\":[]\n" + " },\n" + " \"get\":{\n" + + " \"_seq_no\":0," + + " \"_primary_term\":1," + " \"found\":true,\n" + " \"_source\":{\n" + " \"field1\":\"value1\",\n" + diff --git a/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java b/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java index 7db2e65c858c2..a215a47b89466 100644 --- a/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/get/GetResponseTests.java @@ -96,7 +96,7 @@ public void testToXContent() { "\"value1\", \"field2\":\"value2\"}"), Collections.singletonMap("field1", new DocumentField("field1", Collections.singletonList("value1"))))); String output = Strings.toString(getResponse); - assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_seq_no\":0,\"_primary_term\":1,\"_version\":1," + + assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_version\":1,\"_seq_no\":0,\"_primary_term\":1," + "\"found\":true,\"_source\":{ \"field1\" : \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", output); } @@ -111,7 +111,7 @@ public void testToString() { GetResponse getResponse = new GetResponse(new GetResult("index", "type", "id", 0, 1, 1, true, new BytesArray("{ \"field1\" : " + "\"value1\", \"field2\":\"value2\"}"), Collections.singletonMap("field1", new DocumentField("field1", Collections.singletonList("value1"))))); - assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_seq_no\":0,\"_primary_term\":1,\"_version\":1," + + assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_version\":1,\"_seq_no\":0,\"_primary_term\":1," + "\"found\":true,\"_source\":{ \"field1\" : \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", getResponse.toString()); } diff --git a/server/src/test/java/org/elasticsearch/action/update/UpdateResponseTests.java b/server/src/test/java/org/elasticsearch/action/update/UpdateResponseTests.java index afcbb74772e97..8ec0423b40699 100644 --- a/server/src/test/java/org/elasticsearch/action/update/UpdateResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/update/UpdateResponseTests.java @@ -74,11 +74,12 @@ public void testToXContent() throws IOException { UpdateResponse updateResponse = new UpdateResponse(new ReplicationResponse.ShardInfo(3, 2), new ShardId("books", "books_uuid", 2), "book", "1", 7, 17, 2, UPDATED); - updateResponse.setGetResult(new GetResult("books", "book", "1", 0, 1, 2, true, source, fields)); + updateResponse.setGetResult(new GetResult("books", "book", "1",0, 1, 2, true, source, fields)); String output = Strings.toString(updateResponse); assertEquals("{\"_index\":\"books\",\"_type\":\"book\",\"_id\":\"1\",\"_version\":2,\"result\":\"updated\"," + - "\"_shards\":{\"total\":3,\"successful\":2,\"failed\":0},\"_seq_no\":7,\"_primary_term\":17,\"get\":{\"found\":true," + + "\"_shards\":{\"total\":3,\"successful\":2,\"failed\":0},\"_seq_no\":7,\"_primary_term\":17,\"get\":{" + + "\"_seq_no\":0,\"_primary_term\":1,\"found\":true," + "\"_source\":{\"title\":\"Book title\",\"isbn\":\"ABC-123\"},\"fields\":{\"isbn\":[\"ABC-123\"],\"title\":[\"Book " + "title\"]}}}", output); } diff --git a/server/src/test/java/org/elasticsearch/index/get/GetResultTests.java b/server/src/test/java/org/elasticsearch/index/get/GetResultTests.java index da25c3f49d3a8..0dc6b2573ea6e 100644 --- a/server/src/test/java/org/elasticsearch/index/get/GetResultTests.java +++ b/server/src/test/java/org/elasticsearch/index/get/GetResultTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.RandomObjects; @@ -45,6 +44,7 @@ import static org.elasticsearch.common.xcontent.XContentHelper.toXContent; import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; import static org.elasticsearch.index.get.DocumentFieldTests.randomDocumentField; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent; @@ -77,12 +77,12 @@ public void testToXContent() throws IOException { "\"value1\", \"field2\":\"value2\"}"), singletonMap("field1", new DocumentField("field1", singletonList("value1")))); String output = Strings.toString(getResult); - assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\", \"_seq_no\":0,\"_primary_Term\": 1,\"_version\":1," + - "\"found\":true,\"_source\":{ \"field1\": \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", + assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"_version\":1,\"_seq_no\":0,\"_primary_term\":1," + + "\"found\":true,\"_source\":{ \"field1\" : \"value1\", \"field2\":\"value2\"},\"fields\":{\"field1\":[\"value1\"]}}", output); } { - GetResult getResult = new GetResult("index", "type", "id", 0, 1, 1, false, null, null); + GetResult getResult = new GetResult("index", "type", "id", UNASSIGNED_SEQ_NO, 0, 1, false, null, null); String output = Strings.toString(getResult); assertEquals("{\"_index\":\"index\",\"_type\":\"type\",\"_id\":\"id\",\"found\":false}", output); } @@ -94,7 +94,7 @@ public void testToAndFromXContentEmbedded() throws Exception { GetResult getResult = tuple.v1(); // We don't expect to retrieve the index/type/id of the GetResult because they are not rendered // by the toXContentEmbedded method. - GetResult expectedGetResult = new GetResult(null, null, null, 0, 1, -1, + GetResult expectedGetResult = new GetResult(null, null, null, tuple.v2().getSeqNo(), tuple.v2().getPrimaryTerm(), -1, tuple.v2().isExists(), tuple.v2().sourceRef(), tuple.v2().getFields()); boolean humanReadable = randomBoolean(); @@ -124,12 +124,12 @@ public void testToXContentEmbedded() throws IOException { new BytesArray("{\"foo\":\"bar\",\"baz\":[\"baz_0\",\"baz_1\"]}"), fields); BytesReference originalBytes = toXContentEmbedded(getResult, XContentType.JSON, false); - assertEquals("{\"found\":true,\"_source\":{\"foo\":\"bar\",\"baz\":[\"baz_0\",\"baz_1\"]}," + + assertEquals("{\"_seq_no\":0,\"_primary_term\":1,\"found\":true,\"_source\":{\"foo\":\"bar\",\"baz\":[\"baz_0\",\"baz_1\"]}," + "\"fields\":{\"foo\":[\"bar\"],\"baz\":[\"baz_0\",\"baz_1\"]}}", originalBytes.utf8ToString()); } public void testToXContentEmbeddedNotFound() throws IOException { - GetResult getResult = new GetResult("index", "type", "id", 0, 1, 1, false, null, null); + GetResult getResult = new GetResult("index", "type", "id", UNASSIGNED_SEQ_NO, 0, 1, false, null, null); BytesReference originalBytes = toXContentEmbedded(getResult, XContentType.JSON, false); assertEquals("{\"found\":false}", originalBytes.utf8ToString()); @@ -171,7 +171,7 @@ public static GetResult mutateGetResult(GetResult getResult) { getResult.getSeqNo(), getResult.getPrimaryTerm(), randomNonNegativeLong(), getResult.isExists(), getResult.internalSourceRef(), getResult.getFields())); mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), - getResult.isExists() ? SequenceNumbers.UNASSIGNED_SEQ_NO : getResult.getSeqNo(), + getResult.isExists() ? UNASSIGNED_SEQ_NO : getResult.getSeqNo(), getResult.isExists() ? 0 : getResult.getPrimaryTerm(), getResult.getVersion(), getResult.isExists() == false, getResult.internalSourceRef(), getResult.getFields())); mutations.add(() -> new GetResult(getResult.getIndex(), getResult.getType(), getResult.getId(), @@ -208,7 +208,7 @@ public static Tuple randomGetResult(XContentType xContentT expectedFields = tuple.v2(); } } else { - seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; + seqNo = UNASSIGNED_SEQ_NO; primaryTerm = 0; version = -1; exists = false; From 97ba1fb4f65aae5e0a9954cba10155090d3bf62f Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 30 Nov 2018 11:40:57 +0100 Subject: [PATCH 27/33] fix RestGetSourceActionTests --- .../rest/action/document/RestGetSourceActionTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestGetSourceActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestGetSourceActionTests.java index a4a0b81c76d32..e8f573dc57ddc 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestGetSourceActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestGetSourceActionTests.java @@ -64,7 +64,7 @@ public void testRestGetSourceAction() throws Exception { public void testRestGetSourceActionWithMissingDocument() { final GetResponse response = - new GetResponse(new GetResult("index1", "_doc", "1", -1, UNASSIGNED_SEQ_NO, 0, false, null, emptyMap())); + new GetResponse(new GetResult("index1", "_doc", "1", UNASSIGNED_SEQ_NO, 0, -1, false, null, emptyMap())); final ResourceNotFoundException exception = expectThrows(ResourceNotFoundException.class, () -> listener.buildResponse(response)); From f673e83eea7d491c3593223dab6396ae729f43c7 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 30 Nov 2018 12:04:52 +0100 Subject: [PATCH 28/33] woops --- .../main/java/org/elasticsearch/action/update/UpdateHelper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index f0b3d2d1d701a..2e7c2ce8ebe6d 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -317,7 +317,7 @@ public static GetResult extractGetResult(final UpdateRequest request, String con } // TODO when using delete/none, we can still return the source as bytes by generating it (using the sourceContentType) - return new GetResult(concreteIndex, request.type(), request.id(), primaryTerm, seqNo, version, true, sourceFilteredAsBytes, + return new GetResult(concreteIndex, request.type(), request.id(), seqNo, primaryTerm, version, true, sourceFilteredAsBytes, Collections.emptyMap()); } From 593ddf278c9f10e19c26f278bd1a0682bafb5615 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 30 Nov 2018 13:35:04 +0100 Subject: [PATCH 29/33] fix wrong ordering --- .../elasticsearch/action/bulk/TransportShardBulkAction.java | 2 +- .../elasticsearch/action/update/TransportUpdateAction.java | 6 +++--- .../java/org/elasticsearch/action/update/UpdateHelper.java | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index f27c265042e45..7c7cef0afdd99 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -317,7 +317,7 @@ static BulkItemResponse processUpdateResponse(final UpdateRequest updateRequest, deleteResponse.getVersion(), deleteResponse.getResult()); final GetResult getResult = UpdateHelper.extractGetResult(updateRequest, concreteIndex, - deleteResponse.getPrimaryTerm(), deleteResponse.getSeqNo(), deleteResponse.getVersion(), + deleteResponse.getSeqNo(), deleteResponse.getPrimaryTerm(), deleteResponse.getVersion(), translate.updatedSourceAsMap(), translate.updateSourceContentType(), null); updateResponse.setGetResult(getResult); diff --git a/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java b/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java index a7a05b27d2ca2..3bde6ba5d6638 100644 --- a/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java +++ b/server/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java @@ -186,7 +186,7 @@ protected void shardOperation(final UpdateRequest request, final ActionListener< Tuple> sourceAndContent = XContentHelper.convertToMap(upsertSourceBytes, true, upsertRequest.getContentType()); update.setGetResult(UpdateHelper.extractGetResult(request, request.concreteIndex(), - response.getPrimaryTerm(), response.getSeqNo(), response.getVersion(), sourceAndContent.v2(), + response.getSeqNo(), response.getPrimaryTerm(), response.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), upsertSourceBytes)); } else { update.setGetResult(null); @@ -207,7 +207,7 @@ protected void shardOperation(final UpdateRequest request, final ActionListener< response.getType(), response.getId(), response.getSeqNo(), response.getPrimaryTerm(), response.getVersion(), response.getResult()); update.setGetResult(UpdateHelper.extractGetResult(request, request.concreteIndex(), - response.getPrimaryTerm(), response.getSeqNo(), response.getVersion(), + response.getSeqNo(), response.getPrimaryTerm(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes)); update.setForcedRefresh(response.forcedRefresh()); listener.onResponse(update); @@ -222,7 +222,7 @@ protected void shardOperation(final UpdateRequest request, final ActionListener< response.getId(), response.getSeqNo(), response.getPrimaryTerm(), response.getVersion(), response.getResult()); update.setGetResult(UpdateHelper.extractGetResult(request, request.concreteIndex(), - response.getPrimaryTerm(), response.getSeqNo(), response.getVersion(), + response.getSeqNo(), response.getPrimaryTerm(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null)); update.setForcedRefresh(response.forcedRefresh()); listener.onResponse(update); diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index 2e7c2ce8ebe6d..255161c8f3200 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -209,7 +209,7 @@ Result prepareUpdateIndexRequest(ShardId shardId, UpdateRequest request, GetResu if (detectNoop && noop) { UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), DocWriteResponse.Result.NOOP); - update.setGetResult(extractGetResult(request, request.index(), getResult.getPrimaryTerm(), getResult.getSeqNo(), + update.setGetResult(extractGetResult(request, request.index(), getResult.getSeqNo(), getResult.getPrimaryTerm(), getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef())); return new Result(update, DocWriteResponse.Result.NOOP, updatedSourceAsMap, updateSourceContentType); } else { @@ -270,7 +270,7 @@ Result prepareUpdateScriptRequest(ShardId shardId, UpdateRequest request, GetRes // If it was neither an INDEX or DELETE operation, treat it as a noop UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), DocWriteResponse.Result.NOOP); - update.setGetResult(extractGetResult(request, request.index(), getResult.getPrimaryTerm(), getResult.getSeqNo(), + update.setGetResult(extractGetResult(request, request.index(), getResult.getSeqNo(), getResult.getPrimaryTerm(), getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef())); return new Result(update, DocWriteResponse.Result.NOOP, updatedSourceAsMap, updateSourceContentType); } @@ -292,7 +292,7 @@ private Map executeScript(Script script, Map ctx /** * Applies {@link UpdateRequest#fetchSource()} to the _source of the updated document to be returned in a update response. */ - public static GetResult extractGetResult(final UpdateRequest request, String concreteIndex, long primaryTerm, long seqNo, long version, + public static GetResult extractGetResult(final UpdateRequest request, String concreteIndex, long seqNo, long primaryTerm, long version, final Map source, XContentType sourceContentType, @Nullable final BytesReference sourceAsBytes) { if (request.fetchSource() == null || request.fetchSource().fetchSource() == false) { From ed47866244e46d27050e10da5ffe53355cf1c157 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 30 Nov 2018 16:53:03 +0100 Subject: [PATCH 30/33] doc fixes --- docs/plugins/ingest-attachment.asciidoc | 16 +++++++++++---- docs/plugins/ingest-geoip.asciidoc | 12 ++++++++--- docs/plugins/ingest-user-agent.asciidoc | 4 +++- docs/reference/docs/get.asciidoc | 12 ++++++++--- docs/reference/docs/reindex.asciidoc | 4 +++- docs/reference/getting-started.asciidoc | 6 ++++-- docs/reference/ingest/ingest-node.asciidoc | 24 ++++++++++++++-------- 7 files changed, 55 insertions(+), 23 deletions(-) diff --git a/docs/plugins/ingest-attachment.asciidoc b/docs/plugins/ingest-attachment.asciidoc index 2f9564294d0b8..a3d716ff2d9eb 100644 --- a/docs/plugins/ingest-attachment.asciidoc +++ b/docs/plugins/ingest-attachment.asciidoc @@ -63,6 +63,8 @@ Returns this: "_type": "_doc", "_id": "my_id", "_version": 1, + "_seq_no": 22, + "_primary_term": 1, "_source": { "data": "e1xydGYxXGFuc2kNCkxvcmVtIGlwc3VtIGRvbG9yIHNpdCBhbWV0DQpccGFyIH0=", "attachment": { @@ -74,7 +76,7 @@ Returns this: } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] To specify only some fields to be extracted: @@ -146,6 +148,8 @@ Returns this: "_type": "_doc", "_id": "my_id", "_version": 1, + "_seq_no": 35, + "_primary_term": 1, "_source": { "data": "e1xydGYxXGFuc2kNCkxvcmVtIGlwc3VtIGRvbG9yIHNpdCBhbWV0DQpccGFyIH0=", "attachment": { @@ -157,7 +161,7 @@ Returns this: } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] [source,js] @@ -194,6 +198,8 @@ Returns this: "_type": "_doc", "_id": "my_id_2", "_version": 1, + "_seq_no": 40, + "_primary_term": 1, "_source": { "data": "e1xydGYxXGFuc2kNCkxvcmVtIGlwc3VtIGRvbG9yIHNpdCBhbWV0DQpccGFyIH0=", "max_size": 5, @@ -206,7 +212,7 @@ Returns this: } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] [[ingest-attachment-with-arrays]] @@ -285,6 +291,8 @@ Returns this: "_type" : "_doc", "_id" : "my_id", "_version" : 1, + "_seq_no" : 50, + "_primary_term" : 1, "found" : true, "_source" : { "attachments" : [ @@ -312,7 +320,7 @@ Returns this: } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] Note that the `target_field` needs to be set, otherwise the diff --git a/docs/plugins/ingest-geoip.asciidoc b/docs/plugins/ingest-geoip.asciidoc index a255d0b51217a..dd109ca95e1b0 100644 --- a/docs/plugins/ingest-geoip.asciidoc +++ b/docs/plugins/ingest-geoip.asciidoc @@ -75,6 +75,8 @@ Which returns: "_type": "_doc", "_id": "my_id", "_version": 1, + "_seq_no": 55, + "_primary_term": 1, "_source": { "ip": "8.8.8.8", "geoip": { @@ -85,7 +87,7 @@ Which returns: } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term":1/"_primary_term" : $body._primary_term/] Here is an example that uses the default country database and adds the geographical information to the `geo` field based on the `ip` field`. Note that @@ -124,6 +126,8 @@ returns this: "_type": "_doc", "_id": "my_id", "_version": 1, + "_seq_no": 65, + "_primary_term": 1, "_source": { "ip": "8.8.8.8", "geo": { @@ -133,7 +137,7 @@ returns this: } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] Not all IP addresses find geo information from the database, When this @@ -174,13 +178,15 @@ Which returns: "_type" : "_doc", "_id" : "my_id", "_version" : 1, + "_seq_no" : 71, + "_primary_term": 1, "found" : true, "_source" : { "ip" : "80.231.5.0" } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] [[ingest-geoip-mappings-note]] ===== Recognizing Location as a Geopoint diff --git a/docs/plugins/ingest-user-agent.asciidoc b/docs/plugins/ingest-user-agent.asciidoc index 57594eab57342..a0e6d3257f301 100644 --- a/docs/plugins/ingest-user-agent.asciidoc +++ b/docs/plugins/ingest-user-agent.asciidoc @@ -57,6 +57,8 @@ Which returns "_type": "_doc", "_id": "my_id", "_version": 1, + "_seq_no": 22, + "_primary_term": 1, "_source": { "agent": "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_10_5) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/51.0.2704.103 Safari/537.36", "user_agent": { @@ -73,7 +75,7 @@ Which returns } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term": 1/"_primary_term" : $body._primary_term/] ===== Using a custom regex file To use a custom regex file for parsing the user agents, that file has to be put into the `config/ingest-user-agent` directory and diff --git a/docs/reference/docs/get.asciidoc b/docs/reference/docs/get.asciidoc index ec6ef28534fd6..3ab5fa11fa179 100644 --- a/docs/reference/docs/get.asciidoc +++ b/docs/reference/docs/get.asciidoc @@ -21,6 +21,8 @@ The result of the above get operation is: "_type" : "_doc", "_id" : "0", "_version" : 1, + "_seq_no" : 10, + "_primary_term" : 1, "found": true, "_source" : { "user" : "kimchy", @@ -30,7 +32,7 @@ The result of the above get operation is: } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] The above result includes the `_index`, `_type`, `_id` and `_version` of the document we wish to retrieve, including the actual `_source` @@ -156,6 +158,8 @@ The result of the above get operation is: "_type": "_doc", "_id": "1", "_version": 1, + "_seq_no" : 22, + "_primary_term" : 1, "found": true, "fields": { "tags": [ @@ -164,7 +168,7 @@ The result of the above get operation is: } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] Field values fetched from the document itself are always returned as an array. @@ -199,6 +203,8 @@ The result of the above get operation is: "_type": "_doc", "_id": "2", "_version": 1, + "_seq_no" : 13, + "_primary_term" : 1, "_routing": "user1", "found": true, "fields": { @@ -208,7 +214,7 @@ The result of the above get operation is: } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] Also only leaf fields can be returned via the `stored_field` option. So object fields can't be returned and such requests will fail. diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index 5a5004f17aea4..7f0ae962b3a30 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -870,13 +870,15 @@ which will return: "_index": "test2", "_type": "_doc", "_version": 1, + "_seq_no": 44, + "_primary_term": 1, "_source": { "text": "words words", "tag": "foo" } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term": 1/"_primary_term" : $body._primary_term/] [float] [[docs-reindex-slice]] diff --git a/docs/reference/getting-started.asciidoc b/docs/reference/getting-started.asciidoc index 1a60d4d83e298..043fb806eb9c5 100755 --- a/docs/reference/getting-started.asciidoc +++ b/docs/reference/getting-started.asciidoc @@ -414,7 +414,7 @@ And the response: "_primary_term" : 1 } -------------------------------------------------- -// TESTRESPONSE[s/"_seq_no" : 0/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] From the above, we can see that a new customer document was successfully created inside the customer index. The document also has an internal id of 1 which we specified at index time. @@ -438,11 +438,13 @@ And the response: "_type" : "_doc", "_id" : "1", "_version" : 1, + "_seq_no" : 25, + "_primary_term" : 1, "found" : true, "_source" : { "name": "John Doe" } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] Nothing out of the ordinary here other than a field, `found`, stating that we found a document with the requested ID 1 and another field, `_source`, which returns the full JSON document that we indexed from the previous step. diff --git a/docs/reference/ingest/ingest-node.asciidoc b/docs/reference/ingest/ingest-node.asciidoc index 584a2c773151f..2f422a1f4f70c 100644 --- a/docs/reference/ingest/ingest-node.asciidoc +++ b/docs/reference/ingest/ingest-node.asciidoc @@ -730,13 +730,15 @@ GET test/_doc/2 "_type": "_doc", "_id": "2", "_version": 1, + "_seq_no": 22, + "_primary_term": 1, "found": true, "_source": { "foo": "bar" } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term": 1/"_primary_term" : $body._primary_term/] //// The source document can also use dot delimited fields to represent nested fields. @@ -967,6 +969,8 @@ GET test/_doc/2 "_type": "_doc", "_id": "2", "_version": 1, + "_seq_no": 34, + "_primary_term": 1, "found": true, "_source": { "tags": [ @@ -976,7 +980,7 @@ GET test/_doc/2 } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] //// @@ -1088,6 +1092,8 @@ GET test/_doc/1 "_type": "_doc", "_id": "1", "_version": 1, + "_seq_no": 60, + "_primary_term": 1, "found": true, "_source": { "href": { @@ -1097,7 +1103,7 @@ GET test/_doc/1 } } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] Regular expressions can be expensive and should be avoided if viable @@ -1548,11 +1554,11 @@ PUT /myindex/_doc/1?pipeline=monthlyindex "successful" : 1, "failed" : 0 }, - "_seq_no" : 0, + "_seq_no" : 55, "_primary_term" : 1 } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] The above request will not index this document into the `myindex` index, but into the `myindex-2016-04-01` index because @@ -2787,11 +2793,11 @@ Response from the index request: "successful": 1, "failed": 0 }, - "_seq_no": 0, + "_seq_no": 66, "_primary_term": 1, } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] Indexed document: @@ -2963,11 +2969,11 @@ The response from the above index request: "successful": 1, "failed": 0 }, - "_seq_no": 0, + "_seq_no": 89, "_primary_term": 1, } -------------------------------------------------- -// TESTRESPONSE +// TESTRESPONSE[s/"_seq_no": \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] In the above response, you can see that our document was actually indexed into `my_index` instead of `any_index`. This type of manipulation is often convenient in pipelines that have various branches of transformation, From 5d688fbbd46c7fd2bca300acf588480db914b8d4 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 13 Dec 2018 22:36:14 +0100 Subject: [PATCH 31/33] move to if_*_match --- .../resources/rest-api-spec/api/delete.json | 4 +- .../resources/rest-api-spec/api/index.json | 4 +- .../rest-api-spec/test/index/30_cas.yml | 12 +++--- .../action/bulk/BulkRequest.java | 20 +++++----- .../action/bulk/TransportShardBulkAction.java | 4 +- .../action/delete/DeleteRequest.java | 22 +++++----- .../action/index/IndexRequest.java | 40 +++++++++---------- .../elasticsearch/index/shard/IndexShard.java | 24 +++++------ .../action/document/RestDeleteAction.java | 4 +- .../rest/action/document/RestIndexAction.java | 2 +- 10 files changed, 68 insertions(+), 68 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json index b9c13a6927ef5..b731c3fe6d10c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json @@ -43,11 +43,11 @@ "type" : "time", "description" : "Explicit operation timeout" }, - "cas_seq_no" : { + "if_seq_no_match" : { "type" : "number", "description" : "only perform the delete operation if the last operation to have changed the document has the specified sequence number" }, - "cas_primary_term" : { + "if_primary_term_match" : { "type" : "number", "description" : "only perform the delete operation if the last operation to have changed the document has the specified primary term" }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json index 093086f663990..f0c58ae6ade45 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json @@ -57,11 +57,11 @@ "options" : ["internal", "external", "external_gte", "force"], "description" : "Specific version type" }, - "cas_seq_no" : { + "if_seq_no_match" : { "type" : "number", "description" : "only perform the index operation if the last operation to have changed the document has the specified sequence number" }, - "cas_primary_term" : { + "if_primary_term_match" : { "type" : "number", "description" : "only perform the index operation if the last operation to have changed the document has the specified primary term" }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml index 37a2b2bdd2dea..b8c60e5a7cf8b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/30_cas.yml @@ -26,8 +26,8 @@ index: index: test_1 id: 1 - cas_seq_no: 10000 - cas_primary_term: $primary_term + if_seq_no_match: 10000 + if_primary_term_match: $primary_term body: { foo: bar2 } - do: @@ -35,16 +35,16 @@ index: index: test_1 id: 1 - cas_seq_no: $seqno - cas_primary_term: 1000 + if_seq_no_match: $seqno + if_primary_term_match: 1000 body: { foo: bar2 } - do: index: index: test_1 id: 1 - cas_seq_no: $seqno - cas_primary_term: $primary_term + if_seq_no_match: $seqno + if_primary_term_match: $primary_term body: { foo: bar2 } - match: { _version: 2 } diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index 9e6b19335272e..f75ebf036a110 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -78,7 +78,7 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private static final ParseField RETRY_ON_CONFLICT = new ParseField("retry_on_conflict"); private static final ParseField PIPELINE = new ParseField("pipeline"); private static final ParseField SOURCE = new ParseField("_source"); - private static final ParseField CAS_SEQ_NO = new ParseField("cas_seq_no"); + private static final ParseField if_seq_no_match = new ParseField("if_seq_no_match"); private static final ParseField CAS_TERM = new ParseField("cas_term"); /** @@ -350,8 +350,8 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null String opType = null; long version = Versions.MATCH_ANY; VersionType versionType = VersionType.INTERNAL; - long casSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - long casPrimaryTerm = 0; + long ifSeqNoMatch = SequenceNumbers.UNASSIGNED_SEQ_NO; + long ifPrimaryTermMatch = 0; int retryOnConflict = 0; String pipeline = valueOrDefault(defaultPipeline, globalPipeline); @@ -382,10 +382,10 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null version = parser.longValue(); } else if (VERSION_TYPE.match(currentFieldName, parser.getDeprecationHandler())) { versionType = VersionType.fromString(parser.text()); - } else if (CAS_SEQ_NO.match(currentFieldName, parser.getDeprecationHandler())) { - casSeqNo = parser.longValue(); + } else if (if_seq_no_match.match(currentFieldName, parser.getDeprecationHandler())) { + ifSeqNoMatch = parser.longValue(); } else if (CAS_TERM.match(currentFieldName, parser.getDeprecationHandler())) { - casPrimaryTerm = parser.longValue(); + ifPrimaryTermMatch = parser.longValue(); } else if (RETRY_ON_CONFLICT.match(currentFieldName, parser.getDeprecationHandler())) { retryOnConflict = parser.intValue(); } else if (PIPELINE.match(currentFieldName, parser.getDeprecationHandler())) { @@ -414,7 +414,7 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null if ("delete".equals(action)) { add(new DeleteRequest(index, type, id).routing(routing) - .version(version).versionType(versionType).compareAndSet(casSeqNo, casPrimaryTerm), payload); + .version(version).versionType(versionType).compareAndSet(ifSeqNoMatch, ifPrimaryTermMatch), payload); } else { nextMarker = findNextMarker(marker, from, data, length); if (nextMarker == -1) { @@ -427,16 +427,16 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null if ("index".equals(action)) { if (opType == null) { internalAdd(new IndexRequest(index, type, id).routing(routing).version(version).versionType(versionType) - .setPipeline(pipeline).compareAndSet(casSeqNo, casPrimaryTerm) + .setPipeline(pipeline).compareAndSet(ifSeqNoMatch, ifPrimaryTermMatch) .source(sliceTrimmingCarriageReturn(data, from, nextMarker,xContentType), xContentType), payload); } else { internalAdd(new IndexRequest(index, type, id).routing(routing).version(version).versionType(versionType) - .create("create".equals(opType)).setPipeline(pipeline).compareAndSet(casSeqNo, casPrimaryTerm) + .create("create".equals(opType)).setPipeline(pipeline).compareAndSet(ifSeqNoMatch, ifPrimaryTermMatch) .source(sliceTrimmingCarriageReturn(data, from, nextMarker, xContentType), xContentType), payload); } } else if ("create".equals(action)) { internalAdd(new IndexRequest(index, type, id).routing(routing).version(version).versionType(versionType) - .create(true).setPipeline(pipeline).compareAndSet(casSeqNo, casPrimaryTerm) + .create(true).setPipeline(pipeline).compareAndSet(ifSeqNoMatch, ifPrimaryTermMatch) .source(sliceTrimmingCarriageReturn(data, from, nextMarker, xContentType), xContentType), payload); } else if ("update".equals(action)) { UpdateRequest updateRequest = new UpdateRequest(index, type, id).routing(routing).retryOnConflict(retryOnConflict) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 7c7cef0afdd99..b2d6fb518f88a 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -462,7 +462,7 @@ private static void executeIndexRequestOnPrimary(BulkPrimaryExecutionContext con executeOnPrimaryWhileHandlingMappingUpdates(context, () -> primary.applyIndexOperationOnPrimary(request.version(), request.versionType(), sourceToParse, - request.casSeqNo(), request.casPrimaryTerm(), request.getAutoGeneratedTimestamp(), request.isRetry()), + request.ifSeqNoMatch(), request.ifPrimaryTermMatch(), request.getAutoGeneratedTimestamp(), request.isRetry()), e -> primary.getFailedIndexResult(e, request.version()), context::markOperationAsExecuted, mapping -> mappingUpdater.updateMappings(mapping, primary.shardId(), request.type())); @@ -474,7 +474,7 @@ private static void executeDeleteRequestOnPrimary(BulkPrimaryExecutionContext co final IndexShard primary = context.getPrimary(); executeOnPrimaryWhileHandlingMappingUpdates(context, () -> primary.applyDeleteOperationOnPrimary(request.version(), request.type(), request.id(), request.versionType(), - request.casSeqNo(), request.casPrimaryTerm()), + request.ifSeqNoMatch(), request.ifPrimaryTermMatch()), e -> primary.getFailedDeleteResult(e, request.version()), context::markOperationAsExecuted, mapping -> mappingUpdater.updateMappings(mapping, primary.shardId(), request.type())); diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java index a1b70a2820ef2..11cf82fbd703f 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java @@ -30,8 +30,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import java.io.IOException; @@ -59,7 +59,7 @@ public class DeleteRequest extends ReplicatedWriteRequest private long version = Versions.MATCH_ANY; private VersionType versionType = VersionType.INTERNAL; private long casSeqNp = SequenceNumbers.UNASSIGNED_SEQ_NO; - private long casPrimaryTerm = 0; + private long ifPrimaryTermMatch = 0; public DeleteRequest() { } @@ -203,12 +203,12 @@ public DeleteRequest versionType(VersionType versionType) { return this; } - public long casSeqNo() { + public long ifSeqNoMatch() { return casSeqNp; } - public long casPrimaryTerm() { - return casPrimaryTerm; + public long ifPrimaryTermMatch() { + return ifPrimaryTermMatch; } public DeleteRequest compareAndSet(long seqNo, long term) { @@ -225,7 +225,7 @@ public DeleteRequest compareAndSet(long seqNo, long term) { throw new IllegalArgumentException("primary term must be non negative. got [" + term + "]"); } casSeqNp = seqNo; - casPrimaryTerm = term; + ifPrimaryTermMatch = term; return this; } @@ -252,10 +252,10 @@ public void readFrom(StreamInput in) throws IOException { versionType = VersionType.fromValue(in.readByte()); if (in.getVersion().onOrAfter(Version.V_7_0_0)) { casSeqNp = in.readZLong(); - casPrimaryTerm = in.readVLong(); + ifPrimaryTermMatch = in.readVLong(); } else { casSeqNp = SequenceNumbers.UNASSIGNED_SEQ_NO; - casPrimaryTerm = 0; + ifPrimaryTermMatch = 0; } } @@ -272,9 +272,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeByte(versionType.getValue()); if (out.getVersion().onOrAfter(Version.V_7_0_0)) { out.writeZLong(casSeqNp); - out.writeVLong(casPrimaryTerm); - } else if (casSeqNp != SequenceNumbers.UNASSIGNED_SEQ_NO || casPrimaryTerm != 0) { - assert false : "compareAndSet [" + casSeqNp + "], currentDocTem [" + casPrimaryTerm + "]"; + out.writeVLong(ifPrimaryTermMatch); + } else if (casSeqNp != SequenceNumbers.UNASSIGNED_SEQ_NO || ifPrimaryTermMatch != 0) { + assert false : "compareAndSet [" + casSeqNp + "], currentDocTem [" + ifPrimaryTermMatch + "]"; throw new IllegalStateException( "sequence number based compare and write is not supported until all nodes are on version 7.0 or higher. " + "Stream version [" + out.getVersion() + "]"); diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java index ee5ec2b4b6c74..80068efc844a3 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -105,8 +105,8 @@ public class IndexRequest extends ReplicatedWriteRequest implement private long autoGeneratedTimestamp = UNSET_AUTO_GENERATED_TIMESTAMP; private boolean isRetry = false; - private long casSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - private long casPrimaryTerm = 0; + private long ifSeqNoMatch = SequenceNumbers.UNASSIGNED_SEQ_NO; + private long ifPrimaryTermMatch = 0; public IndexRequest() { @@ -168,7 +168,7 @@ public ActionRequestValidationException validate() { return validationException; } - if (casSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || casPrimaryTerm != 0) { + if (ifSeqNoMatch != SequenceNumbers.UNASSIGNED_SEQ_NO || ifPrimaryTermMatch != 0) { validationException = addValidationError("create operations do not support compare and set. use index instead", validationException); return validationException; @@ -201,7 +201,7 @@ public ActionRequestValidationException validate() { validationException = addValidationError("pipeline cannot be an empty string", validationException); } - if (casSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO && ( + if (ifSeqNoMatch != SequenceNumbers.UNASSIGNED_SEQ_NO && ( versionType != VersionType.INTERNAL || version != Versions.MATCH_ANY )) { validationException = addValidationError("compare and write operations can not use versioning", validationException); @@ -500,17 +500,17 @@ public IndexRequest compareAndSet(long seqNo, long term) { if (term < 0) { throw new IllegalArgumentException("primary term must be non negative. got [" + term + "]"); } - casSeqNo = seqNo; - casPrimaryTerm = term; + ifSeqNoMatch = seqNo; + ifPrimaryTermMatch = term; return this; } - public long casSeqNo() { - return casSeqNo; + public long ifSeqNoMatch() { + return ifSeqNoMatch; } - public long casPrimaryTerm() { - return casPrimaryTerm; + public long ifPrimaryTermMatch() { + return ifPrimaryTermMatch; } @Override @@ -534,8 +534,8 @@ public void process(Version indexCreatedVersion, @Nullable MappingMetaData mappi // generate id if not already provided if (id == null) { assert autoGeneratedTimestamp == -1 : "timestamp has already been generated!"; - assert casSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO; - assert casPrimaryTerm == 0; + assert ifSeqNoMatch == SequenceNumbers.UNASSIGNED_SEQ_NO; + assert ifPrimaryTermMatch == 0; autoGeneratedTimestamp = Math.max(0, System.currentTimeMillis()); // extra paranoia String uid; if (indexCreatedVersion.onOrAfter(Version.V_6_0_0_beta1)) { @@ -578,11 +578,11 @@ public void readFrom(StreamInput in) throws IOException { contentType = null; } if (in.getVersion().onOrAfter(Version.V_7_0_0)) { - casSeqNo = in.readZLong(); - casPrimaryTerm = in.readVLong(); + ifSeqNoMatch = in.readZLong(); + ifPrimaryTermMatch = in.readVLong(); } else { - casSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - casPrimaryTerm = SequenceNumbers.UNASSIGNED_SEQ_NO; + ifSeqNoMatch = SequenceNumbers.UNASSIGNED_SEQ_NO; + ifPrimaryTermMatch = SequenceNumbers.UNASSIGNED_SEQ_NO; } } @@ -616,10 +616,10 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(false); } if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - out.writeZLong(casSeqNo); - out.writeVLong(casPrimaryTerm); - } else if (casSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO || casPrimaryTerm != 0) { - assert false : "compareAndSet [" + casSeqNo + "], currentDocTem [" + casPrimaryTerm + "]"; + out.writeZLong(ifSeqNoMatch); + out.writeVLong(ifPrimaryTermMatch); + } else if (ifSeqNoMatch != SequenceNumbers.UNASSIGNED_SEQ_NO || ifPrimaryTermMatch != 0) { + assert false : "compareAndSet [" + ifSeqNoMatch + "], currentDocTem [" + ifPrimaryTermMatch + "]"; throw new IllegalStateException( "sequence number based compare and write is not supported until all nodes are on version 7.0 or higher. " + "Stream version [" + out.getVersion() + "]"); diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index bff984faf1546..0b494fb187251 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -686,10 +686,10 @@ private IndexShardState changeState(IndexShardState newState, String reason) { } public Engine.IndexResult applyIndexOperationOnPrimary(long version, VersionType versionType, SourceToParse sourceToParse, - long casSeqNo, long casPrimaryTerm, long autoGeneratedTimestamp, boolean isRetry) + long ifSeqNoMatch, long ifPrimaryTermMatch, long autoGeneratedTimestamp, boolean isRetry) throws IOException { assert versionType.validateVersionForWrites(version); - return applyIndexOperation(getEngine(), UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, casSeqNo, casPrimaryTerm, + return applyIndexOperation(getEngine(), UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, ifSeqNoMatch, ifPrimaryTermMatch, autoGeneratedTimestamp, isRetry, Engine.Operation.Origin.PRIMARY, sourceToParse); } @@ -701,7 +701,7 @@ public Engine.IndexResult applyIndexOperationOnReplica(long seqNo, long version, } private Engine.IndexResult applyIndexOperation(Engine engine, long seqNo, long opPrimaryTerm, long version, - @Nullable VersionType versionType, long casSeqNo, long casPrimaryTerm, + @Nullable VersionType versionType, long ifSeqNoMatch, long ifPrimaryTermMatch, long autoGeneratedTimeStamp, boolean isRetry, Engine.Operation.Origin origin, SourceToParse sourceToParse) throws IOException { assert opPrimaryTerm <= this.operationPrimaryTerm: "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm @@ -711,7 +711,7 @@ private Engine.IndexResult applyIndexOperation(Engine engine, long seqNo, long o try { operation = prepareIndex(docMapper(sourceToParse.type()), indexSettings.getIndexVersionCreated(), sourceToParse, seqNo, opPrimaryTerm, version, versionType, origin, autoGeneratedTimeStamp, isRetry, - casSeqNo, casPrimaryTerm); + ifSeqNoMatch, ifPrimaryTermMatch); Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); if (update != null) { return new Engine.IndexResult(update); @@ -731,7 +731,7 @@ private Engine.IndexResult applyIndexOperation(Engine engine, long seqNo, long o public static Engine.Index prepareIndex(DocumentMapperForType docMapper, Version indexCreatedVersion, SourceToParse source, long seqNo, long primaryTerm, long version, VersionType versionType, Engine.Operation.Origin origin, long autoGeneratedIdTimestamp, boolean isRetry, - long casSeqNo, long casPrimaryTerm) { + long ifSeqNoMatch, long ifPrimaryTermMatch) { long startTime = System.nanoTime(); ParsedDocument doc = docMapper.getDocumentMapper().parse(source); if (docMapper.getMapping() != null) { @@ -739,7 +739,7 @@ public static Engine.Index prepareIndex(DocumentMapperForType docMapper, Version } Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(doc.id())); return new Engine.Index(uid, doc, seqNo, primaryTerm, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry, - casSeqNo, casPrimaryTerm); + ifSeqNoMatch, ifPrimaryTermMatch); } private Engine.IndexResult index(Engine engine, Engine.Index index) throws IOException { @@ -791,11 +791,11 @@ public Engine.DeleteResult getFailedDeleteResult(Exception e, long version) { } public Engine.DeleteResult applyDeleteOperationOnPrimary(long version, String type, String id, VersionType versionType, - long casSeqNo, long casPrimaryTerm) + long ifSeqNoMatch, long ifPrimaryTermMatch) throws IOException { assert versionType.validateVersionForWrites(version); return applyDeleteOperation(getEngine(), UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, type, id, versionType, - casSeqNo, casPrimaryTerm, Engine.Operation.Origin.PRIMARY); + ifSeqNoMatch, ifPrimaryTermMatch, Engine.Operation.Origin.PRIMARY); } public Engine.DeleteResult applyDeleteOperationOnReplica(long seqNo, long version, String type, String id) throws IOException { @@ -804,7 +804,7 @@ public Engine.DeleteResult applyDeleteOperationOnReplica(long seqNo, long versio } private Engine.DeleteResult applyDeleteOperation(Engine engine, long seqNo, long opPrimaryTerm, long version, String type, String id, - @Nullable VersionType versionType, long casSeqNo, long casPrimaryTerm, + @Nullable VersionType versionType, long ifSeqNoMatch, long ifPrimaryTermMatch, Engine.Operation.Origin origin) throws IOException { assert opPrimaryTerm <= this.operationPrimaryTerm : "op term [ " + opPrimaryTerm + " ] > shard term [" + this.operationPrimaryTerm + "]"; @@ -834,16 +834,16 @@ private Engine.DeleteResult applyDeleteOperation(Engine engine, long seqNo, long } final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); final Engine.Delete delete = prepareDelete(type, id, uid, seqNo, opPrimaryTerm, version, - versionType, origin, casSeqNo, casPrimaryTerm); + versionType, origin, ifSeqNoMatch, ifPrimaryTermMatch); return delete(engine, delete); } private Engine.Delete prepareDelete(String type, String id, Term uid, long seqNo, long primaryTerm, long version, VersionType versionType, Engine.Operation.Origin origin, - long casSeqNo, long casPrimaryTerm) { + long ifSeqNoMatch, long ifPrimaryTermMatch) { long startTime = System.nanoTime(); return new Engine.Delete(resolveType(type), id, uid, seqNo, primaryTerm, version, versionType, origin, startTime, - casSeqNo, casPrimaryTerm); + ifSeqNoMatch, ifPrimaryTermMatch); } private Engine.DeleteResult delete(Engine engine, Engine.Delete delete) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java index 4d190730cef50..18079896043ea 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestDeleteAction.java @@ -26,8 +26,8 @@ import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -68,7 +68,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC deleteRequest.version(RestActions.parseVersion(request)); deleteRequest.versionType(VersionType.fromString(request.param("version_type"), deleteRequest.versionType())); deleteRequest.compareAndSet( - request.paramAsLong("cas_seq_no", SequenceNumbers.UNASSIGNED_SEQ_NO), request.paramAsLong("cas_primary_term", 0)); + request.paramAsLong("if_seq_no_match", SequenceNumbers.UNASSIGNED_SEQ_NO), request.paramAsLong("if_primary_term_match", 0)); String waitForActiveShards = request.param("wait_for_active_shards"); if (waitForActiveShards != null) { diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java index 0c96f3a23018e..043551978e35e 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestIndexAction.java @@ -95,7 +95,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC indexRequest.version(RestActions.parseVersion(request)); indexRequest.versionType(VersionType.fromString(request.param("version_type"), indexRequest.versionType())); indexRequest.compareAndSet( - request.paramAsLong("cas_seq_no", SequenceNumbers.UNASSIGNED_SEQ_NO), request.paramAsLong("cas_primary_term", 0)); + request.paramAsLong("if_seq_no_match", SequenceNumbers.UNASSIGNED_SEQ_NO), request.paramAsLong("if_primary_term_match", 0)); String sOpType = request.param("op_type"); String waitForActiveShards = request.param("wait_for_active_shards"); if (waitForActiveShards != null) { From 352196ec11c2c8ef5e4596b104859f15031619b2 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 13 Dec 2018 22:41:32 +0100 Subject: [PATCH 32/33] fix compilation --- .../index/shard/IndexShardTests.java | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index fb77f1def5a94..4e4084456e8df 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -172,6 +172,7 @@ import static org.elasticsearch.common.lucene.Lucene.cleanLuceneIndex; import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.repositories.RepositoryData.EMPTY_REPO_GEN; import static org.elasticsearch.test.hamcrest.RegexMatcher.matches; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -323,10 +324,10 @@ public void testClosesPreventsNewOperations() throws Exception { expectThrows(IndexShardClosedException.class, () -> indexShard.acquireAllPrimaryOperationsPermits(null, TimeValue.timeValueSeconds(30L))); expectThrows(IndexShardClosedException.class, - () -> indexShard.acquireReplicaOperationPermit(indexShard.getPendingPrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, + () -> indexShard.acquireReplicaOperationPermit(indexShard.getPendingPrimaryTerm(), UNASSIGNED_SEQ_NO, randomNonNegativeLong(), null, ThreadPool.Names.WRITE, "")); expectThrows(IndexShardClosedException.class, - () -> indexShard.acquireAllReplicaOperationsPermits(indexShard.getPendingPrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, + () -> indexShard.acquireAllReplicaOperationsPermits(indexShard.getPendingPrimaryTerm(), UNASSIGNED_SEQ_NO, randomNonNegativeLong(), null, TimeValue.timeValueSeconds(30L))); } @@ -334,7 +335,7 @@ public void testRejectOperationPermitWithHigherTermWhenNotStarted() throws IOExc IndexShard indexShard = newShard(false); expectThrows(IndexShardNotStartedException.class, () -> randomReplicaOperationPermitAcquisition(indexShard, indexShard.getPendingPrimaryTerm() + randomIntBetween(1, 100), - SequenceNumbers.UNASSIGNED_SEQ_NO, randomNonNegativeLong(), null, "")); + UNASSIGNED_SEQ_NO, randomNonNegativeLong(), null, "")); closeShards(indexShard); } @@ -828,7 +829,7 @@ public void testOperationPermitOnReplicaShards() throws Exception { newGlobalCheckPoint = randomIntBetween((int) indexShard.getGlobalCheckpoint(), (int) localCheckPoint); } final long expectedLocalCheckpoint; - if (newGlobalCheckPoint == SequenceNumbers.UNASSIGNED_SEQ_NO) { + if (newGlobalCheckPoint == UNASSIGNED_SEQ_NO) { expectedLocalCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; } else { expectedLocalCheckpoint = newGlobalCheckPoint; @@ -1039,10 +1040,10 @@ public void testRestoreLocalHistoryFromTranslogOnPromotion() throws IOException, indexOnReplicaWithGaps(indexShard, operations, Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED)); final long maxSeqNo = indexShard.seqNoStats().getMaxSeqNo(); - final long globalCheckpointOnReplica = randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, indexShard.getLocalCheckpoint()); + final long globalCheckpointOnReplica = randomLongBetween(UNASSIGNED_SEQ_NO, indexShard.getLocalCheckpoint()); indexShard.updateGlobalCheckpointOnReplica(globalCheckpointOnReplica, "test"); - final long globalCheckpoint = randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, indexShard.getLocalCheckpoint()); + final long globalCheckpoint = randomLongBetween(UNASSIGNED_SEQ_NO, indexShard.getLocalCheckpoint()); final long currentMaxSeqNoOfUpdates = indexShard.getMaxSeqNoOfUpdatesOrDeletes(); final long maxSeqNoOfUpdatesOrDeletes = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, maxSeqNo); final Set docsBeforeRollback = getShardDocUIDs(indexShard); @@ -1104,9 +1105,9 @@ public void testRollbackReplicaEngineOnPromotion() throws IOException, Interrupt final int operations = 1024 - scaledRandomIntBetween(0, 1024); indexOnReplicaWithGaps(indexShard, operations, Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED)); - final long globalCheckpointOnReplica = randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, indexShard.getLocalCheckpoint()); + final long globalCheckpointOnReplica = randomLongBetween(UNASSIGNED_SEQ_NO, indexShard.getLocalCheckpoint()); indexShard.updateGlobalCheckpointOnReplica(globalCheckpointOnReplica, "test"); - final long globalCheckpoint = randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, indexShard.getLocalCheckpoint()); + final long globalCheckpoint = randomLongBetween(UNASSIGNED_SEQ_NO, indexShard.getLocalCheckpoint()); Set docsBelowGlobalCheckpoint = getShardDocUIDs(indexShard).stream() .filter(id -> Long.parseLong(id) <= Math.max(globalCheckpointOnReplica, globalCheckpoint)).collect(Collectors.toSet()); final CountDownLatch latch = new CountDownLatch(1); @@ -1132,7 +1133,7 @@ public void onFailure(final Exception e) { }, ""); latch.await(); - if (globalCheckpointOnReplica == SequenceNumbers.UNASSIGNED_SEQ_NO && globalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) { + if (globalCheckpointOnReplica == UNASSIGNED_SEQ_NO && globalCheckpoint == UNASSIGNED_SEQ_NO) { assertThat(indexShard.getLocalCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED)); } else { assertThat(indexShard.getLocalCheckpoint(), equalTo(Math.max(globalCheckpoint, globalCheckpointOnReplica))); @@ -3711,10 +3712,11 @@ public void testTypelessDelete() throws IOException { Engine.IndexResult indexResult = indexDoc(shard, "some_type", "id", "{}"); assertTrue(indexResult.isCreated()); - DeleteResult deleteResult = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, "some_other_type", "id", VersionType.INTERNAL); + DeleteResult deleteResult = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, "some_other_type", "id", VersionType.INTERNAL, + UNASSIGNED_SEQ_NO, 0); assertFalse(deleteResult.isFound()); - deleteResult = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, "_doc", "id", VersionType.INTERNAL); + deleteResult = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, "_doc", "id", VersionType.INTERNAL, UNASSIGNED_SEQ_NO, 0); assertTrue(deleteResult.isFound()); closeShards(shard); From bf62361008cf06af3da452dda093e5a8545c4f30 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Thu, 13 Dec 2018 22:50:11 +0100 Subject: [PATCH 33/33] lint --- .../java/org/elasticsearch/index/shard/IndexShard.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 0b494fb187251..217c2b70f8d77 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -686,11 +686,12 @@ private IndexShardState changeState(IndexShardState newState, String reason) { } public Engine.IndexResult applyIndexOperationOnPrimary(long version, VersionType versionType, SourceToParse sourceToParse, - long ifSeqNoMatch, long ifPrimaryTermMatch, long autoGeneratedTimestamp, boolean isRetry) + long ifSeqNoMatch, long ifPrimaryTermMatch, long autoGeneratedTimestamp, + boolean isRetry) throws IOException { assert versionType.validateVersionForWrites(version); - return applyIndexOperation(getEngine(), UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, ifSeqNoMatch, ifPrimaryTermMatch, - autoGeneratedTimestamp, isRetry, Engine.Operation.Origin.PRIMARY, sourceToParse); + return applyIndexOperation(getEngine(), UNASSIGNED_SEQ_NO, operationPrimaryTerm, version, versionType, ifSeqNoMatch, + ifPrimaryTermMatch, autoGeneratedTimestamp, isRetry, Engine.Operation.Origin.PRIMARY, sourceToParse); } public Engine.IndexResult applyIndexOperationOnReplica(long seqNo, long version, long autoGeneratedTimeStamp,