Add internal _primary_term doc values field, fix _seq_no indexing

This adds the `_primary_term` field internally to the mappings. This field is
populated with the current shard's primary term.

It is intended to be used for collision resolution when two document copies have
the same sequence id, therefore, doc_values for the field are stored but the
filed itself is not indexed.

This also fixes the `_seq_no` field so that doc_values are retrievable (they
were previously stored but irretrievable) and changes the `stats` implementation
to more efficiently use the points API to retrieve the min/max instead of
iterating on each doc_value value. Additionally, even though we intend to be
able to search on the field, it was previously not searchable. This commit makes
it searchable.

There is no user-visible `_primary_term` field. Instead, the fields are
updated by calling:

```java
index.parsedDoc().updateSeqID(seqNum, primaryTerm);
```

This includes example methods in `Versions` and `Engine` for retrieving the
sequence id values from the index (see `Engine.getSequenceID`) that are only
used in unit tests. These will be extended/replaced by actual implementations
once we make use of sequence numbers as a conflict resolution measure.

Relates to #10708
Supercedes #21480

P.S. As a side effect of this commit, `SlowCompositeReaderWrapper` cannot be
used for documents that contain `_seq_no` because it is a Point value and SCRW
cannot wrap documents with points, so the tests have been updated to loop
through the `LeafReaderContext`s now instead.
This commit is contained in:
Lee Hinman 2016-10-31 16:21:38 -06:00
parent 2087234d74
commit ee22a477df
30 changed files with 979 additions and 600 deletions

View File

@ -156,8 +156,8 @@ public class TransportDeleteAction extends TransportWriteAction<DeleteRequest, D
}
public static Engine.DeleteResult executeDeleteRequestOnReplica(DeleteRequest request, IndexShard replica) {
final Engine.Delete delete =
replica.prepareDeleteOnReplica(request.type(), request.id(), request.seqNo(), request.version(), request.versionType());
final Engine.Delete delete = replica.prepareDeleteOnReplica(request.type(), request.id(),
request.seqNo(), request.primaryTerm(), request.version(), request.versionType());
return replica.delete(delete);
}

View File

@ -228,4 +228,3 @@ public class TransportIndexAction extends TransportWriteAction<IndexRequest, Ind
}
}

View File

@ -19,14 +19,25 @@
package org.elasticsearch.common.lucene.uid;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReader.CoreClosedListener;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.seqno.SequenceNumbersService;
import java.io.IOException;
import java.util.List;
@ -143,4 +154,115 @@ public class Versions {
final DocIdAndVersion docIdAndVersion = loadDocIdAndVersion(reader, term);
return docIdAndVersion == null ? NOT_FOUND : docIdAndVersion.version;
}
/**
* Returns the sequence number for the given uid term, returning
* {@code SequenceNumbersService.UNASSIGNED_SEQ_NO} if none is found.
*/
public static long loadSeqNo(IndexReader reader, Term term) throws IOException {
assert term.field().equals(UidFieldMapper.NAME) : "can only load _seq_no by uid";
List<LeafReaderContext> leaves = reader.leaves();
if (leaves.isEmpty()) {
return SequenceNumbersService.UNASSIGNED_SEQ_NO;
}
// iterate backwards to optimize for the frequently updated documents
// which are likely to be in the last segments
for (int i = leaves.size() - 1; i >= 0; i--) {
LeafReader leaf = leaves.get(i).reader();
Bits liveDocs = leaf.getLiveDocs();
TermsEnum termsEnum = null;
SortedNumericDocValues dvField = null;
PostingsEnum docsEnum = null;
final Fields fields = leaf.fields();
if (fields != null) {
Terms terms = fields.terms(UidFieldMapper.NAME);
if (terms != null) {
termsEnum = terms.iterator();
assert termsEnum != null;
dvField = leaf.getSortedNumericDocValues(SeqNoFieldMapper.NAME);
assert dvField != null;
final BytesRef id = term.bytes();
if (termsEnum.seekExact(id)) {
// there may be more than one matching docID, in the
// case of nested docs, so we want the last one:
docsEnum = termsEnum.postings(docsEnum, 0);
int docID = DocIdSetIterator.NO_MORE_DOCS;
for (int d = docsEnum.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d = docsEnum.nextDoc()) {
if (liveDocs != null && liveDocs.get(d) == false) {
continue;
}
docID = d;
}
if (docID != DocIdSetIterator.NO_MORE_DOCS) {
dvField.setDocument(docID);
assert dvField.count() == 1 : "expected only a single value for _seq_no but got " +
dvField.count();
return dvField.valueAt(0);
}
}
}
}
}
return SequenceNumbersService.UNASSIGNED_SEQ_NO;
}
/**
* Returns the primary term for the given uid term, returning {@code 0} if none is found.
*/
public static long loadPrimaryTerm(IndexReader reader, Term term) throws IOException {
assert term.field().equals(UidFieldMapper.NAME) : "can only load _primary_term by uid";
List<LeafReaderContext> leaves = reader.leaves();
if (leaves.isEmpty()) {
return 0;
}
// iterate backwards to optimize for the frequently updated documents
// which are likely to be in the last segments
for (int i = leaves.size() - 1; i >= 0; i--) {
LeafReader leaf = leaves.get(i).reader();
Bits liveDocs = leaf.getLiveDocs();
TermsEnum termsEnum = null;
NumericDocValues dvField = null;
PostingsEnum docsEnum = null;
final Fields fields = leaf.fields();
if (fields != null) {
Terms terms = fields.terms(UidFieldMapper.NAME);
if (terms != null) {
termsEnum = terms.iterator();
assert termsEnum != null;
dvField = leaf.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME);
assert dvField != null;
final BytesRef id = term.bytes();
if (termsEnum.seekExact(id)) {
// there may be more than one matching docID, in the
// case of nested docs, so we want the last one:
docsEnum = termsEnum.postings(docsEnum, 0);
int docID = DocIdSetIterator.NO_MORE_DOCS;
for (int d = docsEnum.nextDoc(); d != DocIdSetIterator.NO_MORE_DOCS; d = docsEnum.nextDoc()) {
if (liveDocs != null && liveDocs.get(d) == false) {
continue;
}
docID = d;
}
if (docID != DocIdSetIterator.NO_MORE_DOCS) {
return dvField.get(docID);
}
}
}
}
}
return 0;
}
}

View File

@ -47,6 +47,7 @@ import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
@ -925,13 +926,15 @@ public abstract class Engine implements Closeable {
private final Term uid;
private final long version;
private final long seqNo;
private final long primaryTerm;
private final VersionType versionType;
private final Origin origin;
private final long startTime;
public Operation(Term uid, long seqNo, long version, VersionType versionType, Origin origin, long startTime) {
public Operation(Term uid, long seqNo, long primaryTerm, long version, VersionType versionType, Origin origin, long startTime) {
this.uid = uid;
this.seqNo = seqNo;
this.primaryTerm = primaryTerm;
this.version = version;
this.versionType = versionType;
this.origin = origin;
@ -965,6 +968,10 @@ public abstract class Engine implements Closeable {
return seqNo;
}
public long primaryTerm() {
return primaryTerm;
}
public abstract int estimatedSizeInBytes();
public VersionType versionType() {
@ -991,9 +998,9 @@ public abstract class Engine implements Closeable {
private final long autoGeneratedIdTimestamp;
private final boolean isRetry;
public Index(Term uid, ParsedDocument doc, long seqNo, long version, VersionType versionType, Origin origin, long startTime,
long autoGeneratedIdTimestamp, boolean isRetry) {
super(uid, seqNo, version, versionType, origin, startTime);
public Index(Term uid, ParsedDocument doc, long seqNo, long primaryTerm, long version, VersionType versionType, Origin origin,
long startTime, long autoGeneratedIdTimestamp, boolean isRetry) {
super(uid, seqNo, primaryTerm, version, versionType, origin, startTime);
this.doc = doc;
this.isRetry = isRetry;
this.autoGeneratedIdTimestamp = autoGeneratedIdTimestamp;
@ -1004,7 +1011,8 @@ public abstract class Engine implements Closeable {
} // TEST ONLY
Index(Term uid, ParsedDocument doc, long version) {
this(uid, doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime(), -1, false);
this(uid, doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, version, VersionType.INTERNAL,
Origin.PRIMARY, System.nanoTime(), -1, false);
}
public ParsedDocument parsedDoc() {
@ -1071,18 +1079,20 @@ public abstract class Engine implements Closeable {
private final String type;
private final String id;
public Delete(String type, String id, Term uid, long seqNo, long version, VersionType versionType, Origin origin, long startTime) {
super(uid, seqNo, version, versionType, origin, startTime);
public Delete(String type, String id, Term uid, long seqNo, long primaryTerm, long version, VersionType versionType,
Origin origin, long startTime) {
super(uid, seqNo, primaryTerm, version, versionType, origin, startTime);
this.type = type;
this.id = id;
}
public Delete(String type, String id, Term uid) {
this(type, id, uid, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime());
this(type, id, uid, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime());
}
public Delete(Delete template, VersionType versionType) {
this(template.type(), template.id(), template.uid(), template.seqNo(), template.version(), versionType, template.origin(), template.startTime());
this(template.type(), template.id(), template.uid(), template.seqNo(), template.primaryTerm(), template.version(),
versionType, template.origin(), template.startTime());
}
@Override

View File

@ -650,6 +650,16 @@ public class InternalEngine extends Engine {
}
updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion);
index.parsedDoc().version().setLongValue(updatedVersion);
// Update the document's sequence number and primary term, the
// sequence number here is derived here from either the sequence
// number service if this is on the primary, or the existing
// document's sequence number if this is on the replica. The
// primary term here has already been set, see
// IndexShard.prepareIndex where the Engine.Index operation is
// created
index.parsedDoc().updateSeqID(seqNo, index.primaryTerm());
if (currentVersion == Versions.NOT_FOUND && forceUpdateDocument == false) {
// document does not exists, we can optimize for create, but double check if assertions are running
assert assertDocDoesNotExist(index, canOptimizeAddDocument == false);

View File

@ -148,7 +148,7 @@ final class DocumentParser {
private static ParsedDocument parsedDocument(SourceToParse source, ParseContext.InternalParseContext context, Mapping update) {
return new ParsedDocument(
context.version(),
context.seqNo(),
context.seqID(),
context.sourceToParse().id(),
context.sourceToParse().type(),
source.routing(),

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.lucene.all.AllEntries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import java.util.ArrayList;
import java.util.Iterator;
@ -254,13 +255,13 @@ public abstract class ParseContext {
}
@Override
public Field seqNo() {
return in.seqNo();
public SeqNoFieldMapper.SequenceID seqID() {
return in.seqID();
}
@Override
public void seqNo(Field seqNo) {
in.seqNo(seqNo);
public void seqID(SeqNoFieldMapper.SequenceID seqID) {
in.seqID(seqID);
}
@Override
@ -310,7 +311,7 @@ public abstract class ParseContext {
private Field version;
private Field seqNo;
private SeqNoFieldMapper.SequenceID seqID;
private final AllEntries allEntries;
@ -404,16 +405,15 @@ public abstract class ParseContext {
}
@Override
public Field seqNo() {
return this.seqNo;
public SeqNoFieldMapper.SequenceID seqID() {
return this.seqID;
}
@Override
public void seqNo(Field seqNo) {
this.seqNo = seqNo;
public void seqID(SeqNoFieldMapper.SequenceID seqID) {
this.seqID = seqID;
}
@Override
public AllEntries allEntries() {
return this.allEntries;
@ -540,9 +540,9 @@ public abstract class ParseContext {
public abstract void version(Field version);
public abstract Field seqNo();
public abstract SeqNoFieldMapper.SequenceID seqID();
public abstract void seqNo(Field seqNo);
public abstract void seqID(SeqNoFieldMapper.SequenceID seqID);
public final boolean includeInAll(Boolean includeInAll, FieldMapper mapper) {
return includeInAll(includeInAll, mapper.fieldType().indexOptions() != IndexOptions.NONE);

View File

@ -23,6 +23,7 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import java.util.List;
@ -35,7 +36,7 @@ public class ParsedDocument {
private final String id, type;
private final BytesRef uid;
private final Field seqNo;
private final SeqNoFieldMapper.SequenceID seqID;
private final String routing;
@ -47,17 +48,16 @@ public class ParsedDocument {
private String parent;
public ParsedDocument(
Field version,
Field seqNo,
String id,
String type,
String routing,
List<Document> documents,
BytesReference source,
Mapping dynamicMappingsUpdate) {
public ParsedDocument(Field version,
SeqNoFieldMapper.SequenceID seqID,
String id,
String type,
String routing,
List<Document> documents,
BytesReference source,
Mapping dynamicMappingsUpdate) {
this.version = version;
this.seqNo = seqNo;
this.seqID = seqID;
this.id = id;
this.type = type;
this.uid = Uid.createUidAsBytes(type, id);
@ -83,8 +83,10 @@ public class ParsedDocument {
return version;
}
public Field seqNo() {
return seqNo;
public void updateSeqID(long sequenceNumber, long primaryTerm) {
this.seqID.seqNo.setLongValue(sequenceNumber);
this.seqID.seqNoDocValue.setLongValue(sequenceNumber);
this.seqID.primaryTerm.setLongValue(primaryTerm);
}
public String routing() {

View File

@ -0,0 +1,298 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.PointValues;
import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.index.seqno.SequenceNumbersService;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
* Mapper for the {@code _seq_no} field.
*
* We expect to use the seq# for sorting, during collision checking and for
* doing range searches. Therefore the {@code _seq_no} field is stored both
* as a numeric doc value and as numeric indexed field.
*
* This mapper also manages the primary term field, which has no ES named
* equivalent. The primary term is only used during collision after receiving
* identical seq# values for two document copies. The primary term is stored as
* a doc value field without being indexed, since it is only intended for use
* as a key-value lookup.
*/
public class SeqNoFieldMapper extends MetadataFieldMapper {
/**
* A sequence ID, which is made up of a sequence number (both the searchable
* and doc_value version of the field) and the primary term.
*/
public static class SequenceID {
public final Field seqNo;
public final Field seqNoDocValue;
public final Field primaryTerm;
public SequenceID(Field seqNo, Field seqNoDocValue, Field primaryTerm) {
Objects.requireNonNull(seqNo, "sequence number field cannot be null");
Objects.requireNonNull(seqNoDocValue, "sequence number dv field cannot be null");
Objects.requireNonNull(primaryTerm, "primary term field cannot be null");
this.seqNo = seqNo;
this.seqNoDocValue = seqNoDocValue;
this.primaryTerm = primaryTerm;
}
public static SequenceID emptySeqID() {
return new SequenceID(new LongPoint(NAME, SequenceNumbersService.UNASSIGNED_SEQ_NO),
new SortedNumericDocValuesField(NAME, SequenceNumbersService.UNASSIGNED_SEQ_NO),
new NumericDocValuesField(PRIMARY_TERM_NAME, 0));
}
}
public static final String NAME = "_seq_no";
public static final String CONTENT_TYPE = "_seq_no";
public static final String PRIMARY_TERM_NAME = "_primary_term";
public static class SeqNoDefaults {
public static final String NAME = SeqNoFieldMapper.NAME;
public static final MappedFieldType FIELD_TYPE = new SeqNoFieldType();
static {
FIELD_TYPE.setName(NAME);
FIELD_TYPE.setDocValuesType(DocValuesType.SORTED);
FIELD_TYPE.setHasDocValues(true);
FIELD_TYPE.freeze();
}
}
public static class Builder extends MetadataFieldMapper.Builder<Builder, SeqNoFieldMapper> {
public Builder() {
super(SeqNoDefaults.NAME, SeqNoDefaults.FIELD_TYPE, SeqNoDefaults.FIELD_TYPE);
}
@Override
public SeqNoFieldMapper build(BuilderContext context) {
return new SeqNoFieldMapper(context.indexSettings());
}
}
public static class TypeParser implements MetadataFieldMapper.TypeParser {
@Override
public MetadataFieldMapper.Builder<?, ?> parse(String name, Map<String, Object> node, ParserContext parserContext)
throws MapperParsingException {
throw new MapperParsingException(NAME + " is not configurable");
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
return new SeqNoFieldMapper(indexSettings);
}
}
static final class SeqNoFieldType extends MappedFieldType {
public SeqNoFieldType() {
}
protected SeqNoFieldType(SeqNoFieldType ref) {
super(ref);
}
@Override
public MappedFieldType clone() {
return new SeqNoFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
private long parse(Object value) {
if (value instanceof Number) {
double doubleValue = ((Number) value).doubleValue();
if (doubleValue < Long.MIN_VALUE || doubleValue > Long.MAX_VALUE) {
throw new IllegalArgumentException("Value [" + value + "] is out of range for a long");
}
if (doubleValue % 1 != 0) {
throw new IllegalArgumentException("Value [" + value + "] has a decimal part");
}
return ((Number) value).longValue();
}
if (value instanceof BytesRef) {
value = ((BytesRef) value).utf8ToString();
}
return Long.parseLong(value.toString());
}
@Override
public Query termQuery(Object value, @Nullable QueryShardContext context) {
long v = parse(value);
return LongPoint.newExactQuery(name(), v);
}
@Override
public Query termsQuery(List<?> values, @Nullable QueryShardContext context) {
long[] v = new long[values.size()];
for (int i = 0; i < values.size(); ++i) {
v[i] = parse(values.get(i));
}
return LongPoint.newSetQuery(name(), v);
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower,
boolean includeUpper, QueryShardContext context) {
long l = Long.MIN_VALUE;
long u = Long.MAX_VALUE;
if (lowerTerm != null) {
l = parse(lowerTerm);
if (includeLower == false) {
if (l == Long.MAX_VALUE) {
return new MatchNoDocsQuery();
}
++l;
}
}
if (upperTerm != null) {
u = parse(upperTerm);
if (includeUpper == false) {
if (u == Long.MIN_VALUE) {
return new MatchNoDocsQuery();
}
--u;
}
}
return LongPoint.newRangeQuery(name(), l, u);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.LONG);
}
@Override
public FieldStats stats(IndexReader reader) throws IOException {
String fieldName = name();
long size = PointValues.size(reader, fieldName);
if (size == 0) {
return null;
}
int docCount = PointValues.getDocCount(reader, fieldName);
byte[] min = PointValues.getMinPackedValue(reader, fieldName);
byte[] max = PointValues.getMaxPackedValue(reader, fieldName);
return new FieldStats.Long(reader.maxDoc(),docCount, -1L, size, true, false,
LongPoint.decodeDimension(min, 0), LongPoint.decodeDimension(max, 0));
}
}
public SeqNoFieldMapper(Settings indexSettings) {
super(NAME, SeqNoDefaults.FIELD_TYPE, SeqNoDefaults.FIELD_TYPE, indexSettings);
}
@Override
public void preParse(ParseContext context) throws IOException {
super.parse(context);
}
@Override
protected void parseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
// see InternalEngine.innerIndex to see where the real version value is set
// also see ParsedDocument.updateSeqID (called by innerIndex)
SequenceID seqID = SequenceID.emptySeqID();
context.seqID(seqID);
fields.add(seqID.seqNo);
fields.add(seqID.seqNoDocValue);
fields.add(seqID.primaryTerm);
}
@Override
public Mapper parse(ParseContext context) throws IOException {
// fields are added in parseCreateField
return null;
}
@Override
public void postParse(ParseContext context) throws IOException {
// In the case of nested docs, let's fill nested docs with seqNo=1 and
// primaryTerm=0 so that Lucene doesn't write a Bitset for documents
// that don't have the field. This is consistent with the default value
// for efficiency.
for (int i = 1; i < context.docs().size(); i++) {
final Document doc = context.docs().get(i);
doc.add(new LongPoint(NAME, 1));
doc.add(new SortedNumericDocValuesField(NAME, 1L));
doc.add(new NumericDocValuesField(PRIMARY_TERM_NAME, 0L));
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
return builder;
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
// nothing to do
}
}

View File

@ -1,197 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper.internal;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.Bits;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.index.seqno.SequenceNumbersService;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/** Mapper for the _seq_no field. */
public class SeqNoFieldMapper extends MetadataFieldMapper {
public static final String NAME = "_seq_no";
public static final String CONTENT_TYPE = "_seq_no";
public static class Defaults {
public static final String NAME = SeqNoFieldMapper.NAME;
public static final MappedFieldType FIELD_TYPE = new SeqNoFieldType();
static {
FIELD_TYPE.setName(NAME);
FIELD_TYPE.setDocValuesType(DocValuesType.NUMERIC);
FIELD_TYPE.setHasDocValues(true);
FIELD_TYPE.freeze();
}
}
public static class Builder extends MetadataFieldMapper.Builder<Builder, SeqNoFieldMapper> {
public Builder() {
super(Defaults.NAME, Defaults.FIELD_TYPE, Defaults.FIELD_TYPE);
}
@Override
public SeqNoFieldMapper build(BuilderContext context) {
return new SeqNoFieldMapper(context.indexSettings());
}
}
public static class TypeParser implements MetadataFieldMapper.TypeParser {
@Override
public MetadataFieldMapper.Builder<?, ?> parse(String name, Map<String, Object> node, ParserContext parserContext)
throws MapperParsingException {
throw new MapperParsingException(NAME + " is not configurable");
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
return new SeqNoFieldMapper(indexSettings);
}
}
static final class SeqNoFieldType extends MappedFieldType {
public SeqNoFieldType() {
}
protected SeqNoFieldType(SeqNoFieldType ref) {
super(ref);
}
@Override
public MappedFieldType clone() {
return new SeqNoFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
@Override
public Query termQuery(Object value, @Nullable QueryShardContext context) {
throw new QueryShardException(context, "SeqNoField field [" + name() + "] is not searchable");
}
@Override
public FieldStats stats(IndexReader reader) throws IOException {
// TODO: remove implementation when late-binding commits are possible
final List<LeafReaderContext> leaves = reader.leaves();
if (leaves.isEmpty()) {
return null;
}
long currentMin = Long.MAX_VALUE;
long currentMax = Long.MIN_VALUE;
boolean found = false;
for (int i = 0; i < leaves.size(); i++) {
final LeafReader leaf = leaves.get(i).reader();
final NumericDocValues values = leaf.getNumericDocValues(name());
if (values == null) continue;
final Bits bits = leaf.getLiveDocs();
for (int docID = 0; docID < leaf.maxDoc(); docID++) {
if (bits == null || bits.get(docID)) {
found = true;
final long value = values.get(docID);
currentMin = Math.min(currentMin, value);
currentMax = Math.max(currentMax, value);
}
}
}
return found ? new FieldStats.Long(reader.maxDoc(), 0, -1, -1, false, true, currentMin, currentMax) : null;
}
}
public SeqNoFieldMapper(Settings indexSettings) {
super(NAME, Defaults.FIELD_TYPE, Defaults.FIELD_TYPE, indexSettings);
}
@Override
public void preParse(ParseContext context) throws IOException {
super.parse(context);
}
@Override
protected void parseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
// see InternalEngine.updateVersion to see where the real version value is set
final Field seqNo = new NumericDocValuesField(NAME, SequenceNumbersService.UNASSIGNED_SEQ_NO);
context.seqNo(seqNo);
fields.add(seqNo);
}
@Override
public Mapper parse(ParseContext context) throws IOException {
// _seq_no added in pre-parse
return null;
}
@Override
public void postParse(ParseContext context) throws IOException {
// In the case of nested docs, let's fill nested docs with seqNo=1 so that Lucene doesn't write a Bitset for documents
// that don't have the field. This is consistent with the default value for efficiency.
for (int i = 1; i < context.docs().size(); i++) {
final Document doc = context.docs().get(i);
doc.add(new NumericDocValuesField(NAME, 1L));
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
return builder;
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
// nothing to do
}
}

View File

@ -510,7 +510,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
boolean isRetry) {
try {
verifyPrimary();
return prepareIndex(docMapper(source.type()), source, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, versionType,
return prepareIndex(docMapper(source.type()), source, SequenceNumbersService.UNASSIGNED_SEQ_NO, primaryTerm, version, versionType,
Engine.Operation.Origin.PRIMARY, autoGeneratedIdTimestamp, isRetry);
} catch (Exception e) {
verifyNotClosed(e);
@ -522,16 +522,17 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
boolean isRetry) {
try {
verifyReplicationTarget();
return prepareIndex(docMapper(source.type()), source, seqNo, version, versionType, Engine.Operation.Origin.REPLICA, autoGeneratedIdTimestamp,
isRetry);
return prepareIndex(docMapper(source.type()), source, seqNo, primaryTerm, version, versionType,
Engine.Operation.Origin.REPLICA, autoGeneratedIdTimestamp, isRetry);
} catch (Exception e) {
verifyNotClosed(e);
throw e;
}
}
static Engine.Index prepareIndex(DocumentMapperForType docMapper, SourceToParse source, long seqNo, long version, VersionType versionType,
Engine.Operation.Origin origin, long autoGeneratedIdTimestamp, boolean isRetry) {
static Engine.Index prepareIndex(DocumentMapperForType docMapper, SourceToParse source, long seqNo, long primaryTerm, long version,
VersionType versionType, Engine.Operation.Origin origin, long autoGeneratedIdTimestamp,
boolean isRetry) {
long startTime = System.nanoTime();
ParsedDocument doc = docMapper.getDocumentMapper().parse(source);
if (docMapper.getMapping() != null) {
@ -540,8 +541,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
MappedFieldType uidFieldType = docMapper.getDocumentMapper().uidMapper().fieldType();
Query uidQuery = uidFieldType.termQuery(doc.uid(), null);
Term uid = MappedFieldType.extractTerm(uidQuery);
doc.seqNo().setLongValue(seqNo);
return new Engine.Index(uid, doc, seqNo, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry);
return new Engine.Index(uid, doc, seqNo, primaryTerm, version, versionType, origin, startTime, autoGeneratedIdTimestamp, isRetry);
}
public Engine.IndexResult index(Engine.Index index) {
@ -573,21 +573,24 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
final MappedFieldType uidFieldType = documentMapper.uidMapper().fieldType();
final Query uidQuery = uidFieldType.termQuery(Uid.createUid(type, id), null);
final Term uid = MappedFieldType.extractTerm(uidQuery);
return prepareDelete(type, id, uid, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, versionType, Engine.Operation.Origin.PRIMARY);
return prepareDelete(type, id, uid, SequenceNumbersService.UNASSIGNED_SEQ_NO, primaryTerm, version,
versionType, Engine.Operation.Origin.PRIMARY);
}
public Engine.Delete prepareDeleteOnReplica(String type, String id, long seqNo, long version, VersionType versionType) {
public Engine.Delete prepareDeleteOnReplica(String type, String id, long seqNo, long primaryTerm,
long version, VersionType versionType) {
verifyReplicationTarget();
final DocumentMapper documentMapper = docMapper(type).getDocumentMapper();
final MappedFieldType uidFieldType = documentMapper.uidMapper().fieldType();
final Query uidQuery = uidFieldType.termQuery(Uid.createUid(type, id), null);
final Term uid = MappedFieldType.extractTerm(uidQuery);
return prepareDelete(type, id, uid, seqNo, version, versionType, Engine.Operation.Origin.REPLICA);
return prepareDelete(type, id, uid, seqNo, primaryTerm, version, versionType, Engine.Operation.Origin.REPLICA);
}
static Engine.Delete prepareDelete(String type, String id, Term uid, long seqNo, long version, VersionType versionType, Engine.Operation.Origin origin) {
static Engine.Delete prepareDelete(String type, String id, Term uid, long seqNo, long primaryTerm, long version,
VersionType versionType, Engine.Operation.Origin origin) {
long startTime = System.nanoTime();
return new Engine.Delete(type, id, uid, seqNo, version, versionType, origin, startTime);
return new Engine.Delete(type, id, uid, seqNo, primaryTerm, version, versionType, origin, startTime);
}
public Engine.DeleteResult delete(Engine.Delete delete) {

View File

@ -155,8 +155,8 @@ public class TranslogRecoveryPerformer {
// 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.
Engine.Index engineIndex = IndexShard.prepareIndex(docMapper(index.type()), source(shardId.getIndexName(), index.type(), index.id(), index.source())
.routing(index.routing()).parent(index.parent()), index.seqNo(),
index.version(), index.versionType().versionTypeForReplicationAndRecovery(), origin, index.getAutoGeneratedIdTimestamp(), true);
.routing(index.routing()).parent(index.parent()), index.seqNo(), index.primaryTerm(),
index.version(), index.versionType().versionTypeForReplicationAndRecovery(), origin, index.getAutoGeneratedIdTimestamp(), true);
maybeAddMappingUpdate(engineIndex.type(), engineIndex.parsedDoc().dynamicMappingsUpdate(), engineIndex.id(), allowMappingUpdates);
if (logger.isTraceEnabled()) {
logger.trace("[translog] recover [index] op of [{}][{}]", index.type(), index.id());
@ -170,7 +170,8 @@ public class TranslogRecoveryPerformer {
logger.trace("[translog] recover [delete] op of [{}][{}]", uid.type(), uid.id());
}
final Engine.Delete engineDelete = new Engine.Delete(uid.type(), uid.id(), delete.uid(), delete.seqNo(),
delete.version(), delete.versionType().versionTypeForReplicationAndRecovery(), origin, System.nanoTime());
delete.primaryTerm(), delete.version(), delete.versionType().versionTypeForReplicationAndRecovery(),
origin, System.nanoTime());
delete(engine, engineDelete);
break;
default:

View File

@ -825,6 +825,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
private final long autoGeneratedIdTimestamp;
private final String type;
private long seqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO;
private long primaryTerm = 0;
private final long version;
private final VersionType versionType;
private final BytesReference source;
@ -853,6 +854,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
}
if (format >= FORMAT_SEQ_NO) {
seqNo = in.readVLong();
primaryTerm = in.readVLong();
}
}
@ -863,6 +865,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
this.routing = index.routing();
this.parent = index.parent();
this.seqNo = indexResult.getSeqNo();
this.primaryTerm = index.primaryTerm();
this.version = indexResult.getVersion();
this.versionType = index.versionType();
this.autoGeneratedIdTimestamp = index.getAutoGeneratedIdTimestamp();
@ -914,6 +917,10 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
return seqNo;
}
public long primaryTerm() {
return primaryTerm;
}
public long version() {
return this.version;
}
@ -940,6 +947,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
out.writeByte(versionType.getValue());
out.writeLong(autoGeneratedIdTimestamp);
out.writeVLong(seqNo);
out.writeVLong(primaryTerm);
}
@Override
@ -955,6 +963,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
if (version != index.version ||
seqNo != index.seqNo ||
primaryTerm != index.primaryTerm ||
id.equals(index.id) == false ||
type.equals(index.type) == false ||
versionType != index.versionType ||
@ -974,6 +983,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
int result = id.hashCode();
result = 31 * result + type.hashCode();
result = 31 * result + Long.hashCode(seqNo);
result = 31 * result + Long.hashCode(primaryTerm);
result = 31 * result + Long.hashCode(version);
result = 31 * result + versionType.hashCode();
result = 31 * result + source.hashCode();
@ -1003,6 +1013,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
private Term uid;
private long seqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO;
private long primaryTerm = 0;
private long version = Versions.MATCH_ANY;
private VersionType versionType = VersionType.INTERNAL;
@ -1015,21 +1026,23 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
assert versionType.validateVersionForWrites(this.version);
if (format >= FORMAT_SEQ_NO) {
seqNo = in.readVLong();
primaryTerm = in.readVLong();
}
}
public Delete(Engine.Delete delete, Engine.DeleteResult deleteResult) {
this(delete.uid(), deleteResult.getSeqNo(), deleteResult.getVersion(), delete.versionType());
this(delete.uid(), deleteResult.getSeqNo(), delete.primaryTerm(), deleteResult.getVersion(), delete.versionType());
}
/** utility for testing */
public Delete(Term uid) {
this(uid, 0, Versions.MATCH_ANY, VersionType.INTERNAL);
this(uid, 0, 0, Versions.MATCH_ANY, VersionType.INTERNAL);
}
public Delete(Term uid, long seqNo, long version, VersionType versionType) {
public Delete(Term uid, long seqNo, long primaryTerm, long version, VersionType versionType) {
this.uid = uid;
this.seqNo = seqNo;
this.primaryTerm = primaryTerm;
this.version = version;
this.versionType = versionType;
}
@ -1052,6 +1065,10 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
return seqNo;
}
public long primaryTerm() {
return primaryTerm;
}
public long version() {
return this.version;
}
@ -1073,6 +1090,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
out.writeLong(version);
out.writeByte(versionType.getValue());
out.writeVLong(seqNo);
out.writeVLong(primaryTerm);
}
@Override
@ -1086,7 +1104,9 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
Delete delete = (Delete) o;
return version == delete.version && seqNo == delete.seqNo &&
return version == delete.version &&
seqNo == delete.seqNo &&
primaryTerm == delete.primaryTerm &&
uid.equals(delete.uid) &&
versionType == delete.versionType;
}
@ -1095,6 +1115,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
public int hashCode() {
int result = uid.hashCode();
result = 31 * result + Long.hashCode(seqNo);
result = 31 * result + Long.hashCode(primaryTerm);
result = 31 * result + Long.hashCode(version);
result = 31 * result + versionType.hashCode();
return result;

View File

@ -45,13 +45,13 @@ import org.elasticsearch.index.mapper.ParentFieldMapper;
import org.elasticsearch.index.mapper.RangeFieldMapper;
import org.elasticsearch.index.mapper.RoutingFieldMapper;
import org.elasticsearch.index.mapper.ScaledFloatFieldMapper;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.index.mapper.TokenCountFieldMapper;
import org.elasticsearch.index.mapper.TypeFieldMapper;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.mapper.VersionFieldMapper;
import org.elasticsearch.index.mapper.internal.SeqNoFieldMapper;
import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction;
import org.elasticsearch.indices.cluster.IndicesClusterStateService;
import org.elasticsearch.indices.flush.SyncedFlushService;

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.IndexingSlowLog.SlowLogParsedDocumentPrinter;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
@ -39,7 +40,7 @@ import static org.hamcrest.Matchers.startsWith;
public class IndexingSlowLogTests extends ESTestCase {
public void testSlowLogParsedDocumentPrinterSourceToLog() throws IOException {
BytesReference source = JsonXContent.contentBuilder().startObject().field("foo", "bar").endObject().bytes();
ParsedDocument pd = new ParsedDocument(new NumericDocValuesField("version", 1), new NumericDocValuesField("seqNo", 1), "id",
ParsedDocument pd = new ParsedDocument(new NumericDocValuesField("version", 1), SeqNoFieldMapper.SequenceID.emptySeqID(), "id",
"test", null, null, source, null);
Index index = new Index("foo", "123");
// Turning off document logging doesn't log source[]

View File

@ -31,6 +31,7 @@ import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.CorruptIndexException;
@ -75,7 +76,9 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.uid.Versions;
@ -100,9 +103,9 @@ import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.RootObjectMapper;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.mapper.internal.SeqNoFieldMapper;
import org.elasticsearch.index.seqno.SequenceNumbersService;
import org.elasticsearch.index.shard.DocsStats;
import org.elasticsearch.index.shard.IndexSearcherWrapper;
@ -268,11 +271,13 @@ public class InternalEngineTests extends ESTestCase {
private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, Document document, BytesReference source, Mapping mappingUpdate) {
Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE);
Field versionField = new NumericDocValuesField("_version", 0);
Field seqNoField = new NumericDocValuesField("_seq_no", 0);
SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID();
document.add(uidField);
document.add(versionField);
document.add(seqNoField);
return new ParsedDocument(versionField, seqNoField, id, type, routing, Arrays.asList(document), source, mappingUpdate);
document.add(seqID.seqNo);
document.add(seqID.seqNoDocValue);
document.add(seqID.primaryTerm);
return new ParsedDocument(versionField, seqID, id, type, routing, Arrays.asList(document), source, mappingUpdate);
}
protected Store createStore() throws IOException {
@ -708,11 +713,11 @@ public class InternalEngineTests extends ESTestCase {
for (int i = 0; i < ops; i++) {
final ParsedDocument doc = testParsedDocument("1", "1", "test", null, testDocumentWithTextField(), SOURCE, null);
if (randomBoolean()) {
final Engine.Index operation = new Engine.Index(newUid("test#1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false);
final Engine.Index operation = new Engine.Index(newUid("test#1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false);
operations.add(operation);
initialEngine.index(operation);
} else {
final Engine.Delete operation = new Engine.Delete("test", "1", newUid("test#1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime());
final Engine.Delete operation = new Engine.Delete("test", "1", newUid("test#1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, i, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime());
operations.add(operation);
initialEngine.delete(operation);
}
@ -1039,7 +1044,7 @@ public class InternalEngineTests extends ESTestCase {
engine.flush();
final boolean forceMergeFlushes = randomBoolean();
if (forceMergeFlushes) {
engine.index(new Engine.Index(newUid("3"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos(), -1, false));
engine.index(new Engine.Index(newUid("3"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos(), -1, false));
} else {
engine.index(new Engine.Index(newUid("3"), doc));
}
@ -1126,7 +1131,7 @@ public class InternalEngineTests extends ESTestCase {
Engine.IndexResult indexResult = engine.index(create);
assertThat(indexResult.getVersion(), equalTo(1L));
create = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), indexResult.getVersion(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
create = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), create.primaryTerm(), indexResult.getVersion(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
indexResult = replicaEngine.index(create);
assertThat(indexResult.getVersion(), equalTo(1L));
}
@ -1137,18 +1142,18 @@ public class InternalEngineTests extends ESTestCase {
Engine.IndexResult indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(1L));
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
indexResult = replicaEngine.index(index);
assertThat(indexResult.getVersion(), equalTo(1L));
}
public void testExternalVersioningNewIndex() {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, testDocument(), B_1, null);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
Engine.IndexResult indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(12L));
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
indexResult = replicaEngine.index(index);
assertThat(indexResult.getVersion(), equalTo(12L));
}
@ -1163,13 +1168,13 @@ public class InternalEngineTests extends ESTestCase {
indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(2L));
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1L, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1L, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
// future versions should not work as well
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1177,15 +1182,15 @@ public class InternalEngineTests extends ESTestCase {
public void testExternalVersioningIndexConflict() {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, testDocument(), B_1, null);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
Engine.IndexResult indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(12L));
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 14, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 14, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(14L));
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1193,7 +1198,7 @@ public class InternalEngineTests extends ESTestCase {
public void testForceVersioningNotAllowedExceptForOlderIndices() throws Exception {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, testDocument(), B_1, null);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 42, VersionType.FORCE, PRIMARY, 0, -1, false);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 42, VersionType.FORCE, PRIMARY, 0, -1, false);
Engine.IndexResult indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
@ -1205,13 +1210,13 @@ public class InternalEngineTests extends ESTestCase {
.build());
try (Store store = createStore();
Engine engine = createEngine(oldIndexSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) {
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 84, VersionType.FORCE, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 84, VersionType.FORCE, PRIMARY, 0, -1, false);
Engine.IndexResult result = engine.index(index);
assertTrue(result.hasFailure());
assertThat(result.getFailure(), instanceOf(IllegalArgumentException.class));
assertThat(result.getFailure().getMessage(), containsString("version type [FORCE] may not be used for non-translog operations"));
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 84, VersionType.FORCE,
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 84, VersionType.FORCE,
Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY, 0, -1, false);
result = engine.index(index);
assertThat(result.getVersion(), equalTo(84L));
@ -1230,13 +1235,13 @@ public class InternalEngineTests extends ESTestCase {
engine.flush();
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
// future versions should not work as well
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 3L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1244,17 +1249,17 @@ public class InternalEngineTests extends ESTestCase {
public void testExternalVersioningIndexConflictWithFlush() {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, testDocument(), B_1, null);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 12, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
Engine.IndexResult indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(12L));
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 14, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 14, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(14L));
engine.flush();
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 13, VersionType.EXTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1365,24 +1370,24 @@ public class InternalEngineTests extends ESTestCase {
indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(2L));
Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 1L, VersionType.INTERNAL, PRIMARY, 0);
Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1L, VersionType.INTERNAL, PRIMARY, 0);
Engine.DeleteResult result = engine.delete(delete);
assertTrue(result.hasFailure());
assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class));
// future versions should not work as well
delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 3L, VersionType.INTERNAL, PRIMARY, 0);
delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 3L, VersionType.INTERNAL, PRIMARY, 0);
result = engine.delete(delete);
assertTrue(result.hasFailure());
assertThat(result.getFailure(), instanceOf(VersionConflictEngineException.class));
// now actually delete
delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 2L, VersionType.INTERNAL, PRIMARY, 0);
delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2L, VersionType.INTERNAL, PRIMARY, 0);
result = engine.delete(delete);
assertThat(result.getVersion(), equalTo(3L));
// now check if we can index to a delete doc with version
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1400,13 +1405,13 @@ public class InternalEngineTests extends ESTestCase {
engine.flush();
Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 1L, VersionType.INTERNAL, PRIMARY, 0);
Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1L, VersionType.INTERNAL, PRIMARY, 0);
Engine.DeleteResult deleteResult = engine.delete(delete);
assertTrue(deleteResult.hasFailure());
assertThat(deleteResult.getFailure(), instanceOf(VersionConflictEngineException.class));
// future versions should not work as well
delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 3L, VersionType.INTERNAL, PRIMARY, 0);
delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 3L, VersionType.INTERNAL, PRIMARY, 0);
deleteResult = engine.delete(delete);
assertTrue(deleteResult.hasFailure());
assertThat(deleteResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1414,14 +1419,14 @@ public class InternalEngineTests extends ESTestCase {
engine.flush();
// now actually delete
delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 2L, VersionType.INTERNAL, PRIMARY, 0);
delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2L, VersionType.INTERNAL, PRIMARY, 0);
deleteResult = engine.delete(delete);
assertThat(deleteResult.getVersion(), equalTo(3L));
engine.flush();
// now check if we can index to a delete doc with version
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2L, VersionType.INTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1429,11 +1434,11 @@ public class InternalEngineTests extends ESTestCase {
public void testVersioningCreateExistsException() {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, testDocument(), B_1, null);
Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false);
Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false);
Engine.IndexResult indexResult = engine.index(create);
assertThat(indexResult.getVersion(), equalTo(1L));
create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false);
create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(create);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1441,13 +1446,13 @@ public class InternalEngineTests extends ESTestCase {
public void testVersioningCreateExistsExceptionWithFlush() {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, testDocument(), B_1, null);
Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false);
Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false);
Engine.IndexResult indexResult = engine.index(create);
assertThat(indexResult.getVersion(), equalTo(1L));
engine.flush();
create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false);
create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0, -1, false);
indexResult = engine.index(create);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1464,19 +1469,20 @@ public class InternalEngineTests extends ESTestCase {
assertThat(indexResult.getVersion(), equalTo(2L));
// apply the second index to the replica, should work fine
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), indexResult.getVersion(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
indexResult = replicaEngine.index(index);
assertThat(indexResult.getVersion(), equalTo(2L));
long seqNo = indexResult.getSeqNo();
long primaryTerm = index.primaryTerm();
// now, the old one should not work
index = new Engine.Index(newUid("1"), doc, seqNo, 1L, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, seqNo, primaryTerm, 1L, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
indexResult = replicaEngine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
// second version on replica should fail as well
index = new Engine.Index(newUid("1"), doc, seqNo, 2L
index = new Engine.Index(newUid("1"), doc, seqNo, primaryTerm, 2L
, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
indexResult = replicaEngine.index(index);
assertThat(indexResult.getVersion(), equalTo(2L));
@ -1490,8 +1496,8 @@ public class InternalEngineTests extends ESTestCase {
assertThat(indexResult.getVersion(), equalTo(1L));
// apply the first index to the replica, should work fine
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), 1L
, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), index.primaryTerm(), 1L,
VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
indexResult = replicaEngine.index(index);
assertThat(indexResult.getVersion(), equalTo(1L));
@ -1506,20 +1512,20 @@ public class InternalEngineTests extends ESTestCase {
assertThat(deleteResult.getVersion(), equalTo(3L));
// apply the delete on the replica (skipping the second index)
delete = new Engine.Delete("test", "1", newUid("1"), deleteResult.getSeqNo(), 3L
delete = new Engine.Delete("test", "1", newUid("1"), deleteResult.getSeqNo(), delete.primaryTerm(), 3L
, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0);
deleteResult = replicaEngine.delete(delete);
assertThat(deleteResult.getVersion(), equalTo(3L));
// second time delete with same version should fail
delete = new Engine.Delete("test", "1", newUid("1"), deleteResult.getSeqNo(), 3L
delete = new Engine.Delete("test", "1", newUid("1"), deleteResult.getSeqNo(), delete.primaryTerm(), 3L
, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0);
deleteResult = replicaEngine.delete(delete);
assertTrue(deleteResult.hasFailure());
assertThat(deleteResult.getFailure(), instanceOf(VersionConflictEngineException.class));
// now do the second index on the replica, it should fail
index = new Engine.Index(newUid("1"), doc, deleteResult.getSeqNo(), 2L, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
index = new Engine.Index(newUid("1"), doc, deleteResult.getSeqNo(), delete.primaryTerm(), 2L, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, -1, false);
indexResult = replicaEngine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1638,7 +1644,7 @@ public class InternalEngineTests extends ESTestCase {
// we have some docs indexed, so delete one of them
id = randomFrom(indexedIds);
final Engine.Delete delete = new Engine.Delete(
"test", id, newUid("test#" + id), SequenceNumbersService.UNASSIGNED_SEQ_NO,
"test", id, newUid("test#" + id), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0,
rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0);
final Engine.DeleteResult result = initialEngine.delete(delete);
if (!result.hasFailure()) {
@ -1655,7 +1661,7 @@ public class InternalEngineTests extends ESTestCase {
id = randomFrom(ids);
ParsedDocument doc = testParsedDocument("test#" + id, id, "test", null, testDocumentWithTextField(), SOURCE, null);
final Engine.Index index = new Engine.Index(newUid("test#" + id), doc,
SequenceNumbersService.UNASSIGNED_SEQ_NO,
SequenceNumbersService.UNASSIGNED_SEQ_NO, 0,
rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL,
PRIMARY, 0, -1, false);
final Engine.IndexResult result = initialEngine.index(index);
@ -1794,7 +1800,7 @@ public class InternalEngineTests extends ESTestCase {
assertThat(localCheckpoint, greaterThanOrEqualTo(prevLocalCheckpoint));
assertThat(maxSeqNo, greaterThanOrEqualTo(prevMaxSeqNo));
try (IndexReader reader = DirectoryReader.open(commit)) {
FieldStats stats = SeqNoFieldMapper.Defaults.FIELD_TYPE.stats(reader);
FieldStats stats = SeqNoFieldMapper.SeqNoDefaults.FIELD_TYPE.stats(reader);
final long highestSeqNo;
if (stats != null) {
highestSeqNo = (long) stats.getMaxValue();
@ -1887,10 +1893,10 @@ public class InternalEngineTests extends ESTestCase {
document.add(new TextField("value", "test1", Field.Store.YES));
ParsedDocument doc = testParsedDocument("1", "1", "test", null, document, B_2, null);
engine.index(new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false));
engine.index(new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false));
// Delete document we just added:
engine.delete(new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime()));
engine.delete(new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime()));
// Get should not find the document
Engine.GetResult getResult = engine.get(new Engine.Get(true, newUid("1")));
@ -1904,14 +1910,14 @@ public class InternalEngineTests extends ESTestCase {
}
// Delete non-existent document
engine.delete(new Engine.Delete("test", "2", newUid("2"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime()));
engine.delete(new Engine.Delete("test", "2", newUid("2"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime()));
// Get should not find the document (we never indexed uid=2):
getResult = engine.get(new Engine.Get(true, newUid("2")));
assertThat(getResult.exists(), equalTo(false));
// Try to index uid=1 with a too-old version, should fail:
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult indexResult = engine.index(index);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -1921,7 +1927,7 @@ public class InternalEngineTests extends ESTestCase {
assertThat(getResult.exists(), equalTo(false));
// Try to index uid=2 with a too-old version, should fail:
Engine.Index index1 = new Engine.Index(newUid("2"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false);
Engine.Index index1 = new Engine.Index(newUid("2"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), -1, false);
indexResult = engine.index(index1);
assertTrue(indexResult.hasFailure());
assertThat(indexResult.getFailure(), instanceOf(VersionConflictEngineException.class));
@ -2019,7 +2025,7 @@ public class InternalEngineTests extends ESTestCase {
final int numDocs = randomIntBetween(1, 10);
for (int i = 0; i < numDocs; i++) {
ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult indexResult = engine.index(firstIndexRequest);
assertThat(indexResult.getVersion(), equalTo(1L));
}
@ -2069,7 +2075,7 @@ public class InternalEngineTests extends ESTestCase {
final int numDocs = randomIntBetween(1, 10);
for (int i = 0; i < numDocs; i++) {
ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult indexResult = engine.index(firstIndexRequest);
assertThat(indexResult.getVersion(), equalTo(1L));
}
@ -2162,7 +2168,7 @@ public class InternalEngineTests extends ESTestCase {
final int numExtraDocs = randomIntBetween(1, 10);
for (int i = 0; i < numExtraDocs; i++) {
ParsedDocument doc = testParsedDocument("extra" + Integer.toString(i), "extra" + Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult indexResult = engine.index(firstIndexRequest);
assertThat(indexResult.getVersion(), equalTo(1L));
}
@ -2191,7 +2197,7 @@ public class InternalEngineTests extends ESTestCase {
final int numDocs = randomIntBetween(1, 10);
for (int i = 0; i < numDocs; i++) {
ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult indexResult = engine.index(firstIndexRequest);
assertThat(indexResult.getVersion(), equalTo(1L));
}
@ -2234,7 +2240,7 @@ public class InternalEngineTests extends ESTestCase {
int randomId = randomIntBetween(numDocs + 1, numDocs + 10);
String uuidValue = "test#" + Integer.toString(randomId);
ParsedDocument doc = testParsedDocument(uuidValue, Integer.toString(randomId), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(uuidValue), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(uuidValue), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult indexResult = engine.index(firstIndexRequest);
assertThat(indexResult.getVersion(), equalTo(1L));
if (flush) {
@ -2242,7 +2248,7 @@ public class InternalEngineTests extends ESTestCase {
}
doc = testParsedDocument(uuidValue, Integer.toString(randomId), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index idxRequest = new Engine.Index(newUid(uuidValue), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index idxRequest = new Engine.Index(newUid(uuidValue), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult result = engine.index(idxRequest);
engine.refresh("test");
assertThat(result.getVersion(), equalTo(2L));
@ -2308,7 +2314,7 @@ public class InternalEngineTests extends ESTestCase {
final int numDocs = randomIntBetween(1, 10);
for (int i = 0; i < numDocs; i++) {
ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult index = engine.index(firstIndexRequest);
assertThat(index.getVersion(), equalTo(1L));
}
@ -2396,7 +2402,7 @@ public class InternalEngineTests extends ESTestCase {
// create
{
ParsedDocument doc = testParsedDocument(Integer.toString(0), Integer.toString(0), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(0)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(0)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG))){
assertFalse(engine.isRecovering());
@ -2523,7 +2529,7 @@ public class InternalEngineTests extends ESTestCase {
final int numDocs = randomIntBetween(2, 10); // at least 2 documents otherwise we don't see any deletes below
for (int i = 0; i < numDocs; i++) {
ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult indexResult = engine.index(firstIndexRequest);
assertThat(indexResult.getVersion(), equalTo(1L));
}
@ -2533,7 +2539,7 @@ public class InternalEngineTests extends ESTestCase {
engine.forceMerge(randomBoolean(), 1, false, false, false);
ParsedDocument doc = testParsedDocument(Integer.toString(0), Integer.toString(0), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(0)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(0)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult index = engine.index(firstIndexRequest);
assertThat(index.getVersion(), equalTo(2L));
engine.flush(); // flush - buffered deletes are not counted
@ -2607,16 +2613,16 @@ public class InternalEngineTests extends ESTestCase {
boolean isRetry = false;
long autoGeneratedIdTimestamp = 0;
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
Engine.IndexResult indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(1L));
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
indexResult = replicaEngine.index(index);
assertThat(indexResult.getVersion(), equalTo(1L));
isRetry = true;
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
indexResult = engine.index(index);
assertThat(indexResult.getVersion(), equalTo(1L));
engine.refresh("test");
@ -2625,7 +2631,7 @@ public class InternalEngineTests extends ESTestCase {
assertEquals(1, topDocs.totalHits);
}
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
index = new Engine.Index(newUid("1"), doc, indexResult.getSeqNo(), index.primaryTerm(), indexResult.getVersion(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
indexResult = replicaEngine.index(index);
assertThat(indexResult.hasFailure(), equalTo(false));
replicaEngine.refresh("test");
@ -2641,16 +2647,16 @@ public class InternalEngineTests extends ESTestCase {
boolean isRetry = true;
long autoGeneratedIdTimestamp = 0;
Engine.Index firstIndexRequest = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
Engine.Index firstIndexRequest = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
Engine.IndexResult result = engine.index(firstIndexRequest);
assertThat(result.getVersion(), equalTo(1L));
Engine.Index firstIndexRequestReplica = new Engine.Index(newUid("1"), doc, result.getSeqNo(), result.getVersion(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
Engine.Index firstIndexRequestReplica = new Engine.Index(newUid("1"), doc, result.getSeqNo(), firstIndexRequest.primaryTerm(), result.getVersion(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
Engine.IndexResult indexReplicaResult = replicaEngine.index(firstIndexRequestReplica);
assertThat(indexReplicaResult.getVersion(), equalTo(1L));
isRetry = false;
Engine.Index secondIndexRequest = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
Engine.Index secondIndexRequest = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
Engine.IndexResult indexResult = engine.index(secondIndexRequest);
assertTrue(indexResult.isCreated());
engine.refresh("test");
@ -2659,7 +2665,7 @@ public class InternalEngineTests extends ESTestCase {
assertEquals(1, topDocs.totalHits);
}
Engine.Index secondIndexRequestReplica = new Engine.Index(newUid("1"), doc, result.getSeqNo(), result.getVersion(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
Engine.Index secondIndexRequestReplica = new Engine.Index(newUid("1"), doc, result.getSeqNo(), secondIndexRequest.primaryTerm(), result.getVersion(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), autoGeneratedIdTimestamp, isRetry);
replicaEngine.index(secondIndexRequestReplica);
replicaEngine.refresh("test");
try (Engine.Searcher searcher = replicaEngine.acquireSearcher("test")) {
@ -2670,9 +2676,9 @@ public class InternalEngineTests extends ESTestCase {
public Engine.Index randomAppendOnly(int docId, ParsedDocument doc, boolean retry) {
if (randomBoolean()) {
return new Engine.Index(newUid(Integer.toString(docId)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), docId, retry);
return new Engine.Index(newUid(Integer.toString(docId)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), docId, retry);
}
return new Engine.Index(newUid(Integer.toString(docId)), doc, 0, 1, VersionType.EXTERNAL, Engine.Operation.Origin.REPLICA, System.nanoTime(), docId, retry);
return new Engine.Index(newUid(Integer.toString(docId)), doc, 0, 0, 1, VersionType.EXTERNAL, Engine.Operation.Origin.REPLICA, System.nanoTime(), docId, retry);
}
public void testRetryConcurrently() throws InterruptedException, IOException {
@ -2857,37 +2863,74 @@ public class InternalEngineTests extends ESTestCase {
}
}
public void testTragicEventErrorBubblesUp() throws IOException {
engine.close();
final AtomicBoolean failWithFatalError = new AtomicBoolean(true);
final VirtualMachineError error = randomFrom(
new InternalError(),
new OutOfMemoryError(),
new StackOverflowError(),
new UnknownError());
engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName) {
return new TokenStreamComponents(new Tokenizer() {
@Override
public boolean incrementToken() throws IOException {
if (failWithFatalError.get()) {
throw error;
} else {
throw new AssertionError("should not get to this point");
}
}
});
}
}));
final Document document = testDocument();
document.add(new TextField("value", "test", Field.Store.YES));
final ParsedDocument doc = testParsedDocument("1", "1", "test", null, document, B_1, null);
final Engine.Index first = new Engine.Index(newUid("1"), doc);
expectThrows(error.getClass(), () -> engine.index(first));
failWithFatalError.set(false);
expectThrows(error.getClass(), () -> engine.index(first));
assertNull(engine.failedEngine.get());
public void testSequenceIDs() throws Exception {
Tuple<Long, Long> seqID = getSequenceID(engine, new Engine.Get(false, newUid("1")));
// Non-existent doc returns no seqnum and no primary term
assertThat(seqID.v1(), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO));
assertThat(seqID.v2(), equalTo(0L));
// create a document
Document document = testDocumentWithTextField();
document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE));
ParsedDocument doc = testParsedDocument("1", "1", "test", null, document, B_1, null);
engine.index(new Engine.Index(newUid("1"), doc));
engine.refresh("test");
seqID = getSequenceID(engine, new Engine.Get(false, newUid("1")));
logger.info("--> got seqID: {}", seqID);
assertThat(seqID.v1(), equalTo(0L));
assertThat(seqID.v2(), equalTo(0L));
// Index the same document again
document = testDocumentWithTextField();
document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE));
doc = testParsedDocument("1", "1", "test", null, document, B_1, null);
engine.index(new Engine.Index(newUid("1"), doc));
engine.refresh("test");
seqID = getSequenceID(engine, new Engine.Get(false, newUid("1")));
logger.info("--> got seqID: {}", seqID);
assertThat(seqID.v1(), equalTo(1L));
assertThat(seqID.v2(), equalTo(0L));
// Index the same document for the third time, this time changing the primary term
document = testDocumentWithTextField();
document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE));
doc = testParsedDocument("1", "1", "test", null, document, B_1, null);
engine.index(new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1,
Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY,
System.nanoTime(), -1, false));
engine.refresh("test");
seqID = getSequenceID(engine, new Engine.Get(false, newUid("1")));
logger.info("--> got seqID: {}", seqID);
assertThat(seqID.v1(), equalTo(2L));
assertThat(seqID.v2(), equalTo(1L));
// we can query by the _seq_no
Engine.Searcher searchResult = engine.acquireSearcher("test");
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1));
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(LongPoint.newExactQuery("_seq_no", 2), 1));
searchResult.close();
}
/**
* Return a tuple representing the sequence ID for the given {@code Get}
* operation. The first value in the tuple is the sequence number, the
* second is the primary term.
*/
private Tuple<Long, Long> getSequenceID(Engine engine, Engine.Get get) throws EngineException {
final Searcher searcher = engine.acquireSearcher("get");
try {
long seqNum = Versions.loadSeqNo(searcher.reader(), get.uid());
long primaryTerm = Versions.loadPrimaryTerm(searcher.reader(), get.uid());
return new Tuple(seqNum, primaryTerm);
} catch (Exception e) {
Releasables.closeWhileHandlingException(searcher);
throw new EngineException(shardId, "unable to retrieve sequence id", e);
} finally {
searcher.close();
}
}
}

View File

@ -54,6 +54,7 @@ import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.seqno.SequenceNumbersService;
@ -174,11 +175,14 @@ public class ShadowEngineTests extends ESTestCase {
private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, ParseContext.Document document, BytesReference source, Mapping mappingsUpdate) {
Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE);
Field versionField = new NumericDocValuesField("_version", 0);
Field seqNoField = new NumericDocValuesField("_seq_no", 0);
SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID();
document.add(uidField);
document.add(versionField);
document.add(seqID.seqNo);
document.add(seqID.seqNoDocValue);
document.add(seqID.primaryTerm);
document.add(new LongPoint("point_field", 42)); // so that points report memory/disk usage
return new ParsedDocument(versionField, seqNoField, id, type, routing, Arrays.asList(document), source, mappingsUpdate);
return new ParsedDocument(versionField, seqID, id, type, routing, Arrays.asList(document), source, mappingsUpdate);
}
protected Store createStore(Path p) throws IOException {
@ -984,7 +988,7 @@ public class ShadowEngineTests extends ESTestCase {
final int numDocs = randomIntBetween(2, 10); // at least 2 documents otherwise we don't see any deletes below
for (int i = 0; i < numDocs; i++) {
ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, testDocument(), new BytesArray("{}"), null);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
Engine.IndexResult indexResult = primaryEngine.index(firstIndexRequest);
assertThat(indexResult.getVersion(), equalTo(1L));
}

View File

@ -31,6 +31,7 @@ import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.Strings;
import org.elasticsearch.search.MultiValueMode;
import java.util.List;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ -71,57 +72,61 @@ public abstract class AbstractFieldDataImplTestCase extends AbstractFieldDataTes
public void testDeletedDocs() throws Exception {
add2SingleValuedDocumentsAndDeleteOneOfThem();
IndexFieldData indexFieldData = getForField("value");
LeafReaderContext readerContext = refreshReader();
AtomicFieldData fieldData = indexFieldData.load(readerContext);
SortedBinaryDocValues values = fieldData.getBytesValues();
for (int i = 0; i < readerContext.reader().maxDoc(); ++i) {
values.setDocument(i);
assertThat(values.count(), greaterThanOrEqualTo(1));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
SortedBinaryDocValues values = fieldData.getBytesValues();
for (int i = 0; i < readerContext.reader().maxDoc(); ++i) {
values.setDocument(i);
assertThat(values.count(), greaterThanOrEqualTo(1));
}
}
}
public void testSingleValueAllSet() throws Exception {
fillSingleValueAllSet();
IndexFieldData indexFieldData = getForField("value");
LeafReaderContext readerContext = refreshReader();
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
bytesValues.setDocument(0);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(two())));
bytesValues.setDocument(1);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(one())));
bytesValues.setDocument(2);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(three())));
bytesValues.setDocument(0);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(two())));
bytesValues.setDocument(1);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(one())));
bytesValues.setDocument(2);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(three())));
assertValues(bytesValues, 0, two());
assertValues(bytesValues, 1, one());
assertValues(bytesValues, 2, three());
assertValues(bytesValues, 0, two());
assertValues(bytesValues, 1, one());
assertValues(bytesValues, 2, three());
IndexSearcher searcher = new IndexSearcher(readerContext.reader());
TopFieldDocs topDocs;
IndexSearcher searcher = new IndexSearcher(readerContext.reader());
TopFieldDocs topDocs;
topDocs = searcher.search(new MatchAllDocsQuery(), 10,
new Sort(new SortField("value", indexFieldData.comparatorSource(null, MultiValueMode.MIN, null))));
assertThat(topDocs.totalHits, equalTo(3));
assertThat(topDocs.scoreDocs[0].doc, equalTo(1));
assertThat(toString(((FieldDoc) topDocs.scoreDocs[0]).fields[0]), equalTo(one()));
assertThat(topDocs.scoreDocs[1].doc, equalTo(0));
assertThat(toString(((FieldDoc) topDocs.scoreDocs[1]).fields[0]), equalTo(two()));
assertThat(topDocs.scoreDocs[2].doc, equalTo(2));
assertThat(toString(((FieldDoc) topDocs.scoreDocs[2]).fields[0]), equalTo(three()));
topDocs = searcher.search(new MatchAllDocsQuery(), 10,
new Sort(new SortField("value", indexFieldData.comparatorSource(null, MultiValueMode.MIN, null))));
assertThat(topDocs.totalHits, equalTo(3));
assertThat(topDocs.scoreDocs[0].doc, equalTo(1));
assertThat(toString(((FieldDoc) topDocs.scoreDocs[0]).fields[0]), equalTo(one()));
assertThat(topDocs.scoreDocs[1].doc, equalTo(0));
assertThat(toString(((FieldDoc) topDocs.scoreDocs[1]).fields[0]), equalTo(two()));
assertThat(topDocs.scoreDocs[2].doc, equalTo(2));
assertThat(toString(((FieldDoc) topDocs.scoreDocs[2]).fields[0]), equalTo(three()));
topDocs = searcher.search(new MatchAllDocsQuery(), 10,
new Sort(new SortField("value", indexFieldData.comparatorSource(null, MultiValueMode.MAX, null), true)));
assertThat(topDocs.totalHits, equalTo(3));
assertThat(topDocs.scoreDocs[0].doc, equalTo(2));
assertThat(topDocs.scoreDocs[1].doc, equalTo(0));
assertThat(topDocs.scoreDocs[2].doc, equalTo(1));
topDocs = searcher.search(new MatchAllDocsQuery(), 10,
new Sort(new SortField("value", indexFieldData.comparatorSource(null, MultiValueMode.MAX, null), true)));
assertThat(topDocs.totalHits, equalTo(3));
assertThat(topDocs.scoreDocs[0].doc, equalTo(2));
assertThat(topDocs.scoreDocs[1].doc, equalTo(0));
assertThat(topDocs.scoreDocs[2].doc, equalTo(1));
}
}
protected abstract void fillSingleValueWithMissing() throws Exception;
@ -145,45 +150,52 @@ public abstract class AbstractFieldDataImplTestCase extends AbstractFieldDataTes
public void testSingleValueWithMissing() throws Exception {
fillSingleValueWithMissing();
IndexFieldData indexFieldData = getForField("value");
AtomicFieldData fieldData = indexFieldData.load(refreshReader());
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
SortedBinaryDocValues bytesValues = fieldData
.getBytesValues();
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
assertValues(bytesValues, 0, two());
assertValues(bytesValues, 1, Strings.EMPTY_ARRAY);
assertValues(bytesValues, 2, three());
assertValues(bytesValues, 0, two());
assertValues(bytesValues, 1, Strings.EMPTY_ARRAY);
assertValues(bytesValues, 2, three());
}
}
protected abstract void fillMultiValueAllSet() throws Exception;
public void testMultiValueAllSet() throws Exception {
fillMultiValueAllSet();
// the segments are force merged to a single segment so that the sorted binary doc values can be asserted within a single segment.
// Previously we used the SlowCompositeReaderWrapper but this is an unideal solution so force merging is a better idea.
writer.forceMerge(1);
IndexFieldData indexFieldData = getForField("value");
AtomicFieldData fieldData = indexFieldData.load(refreshReader());
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
assertValues(bytesValues, 0, two(), four());
assertValues(bytesValues, 1, one());
assertValues(bytesValues, 2, three());
assertValues(bytesValues, 0, two(), four());
assertValues(bytesValues, 1, one());
assertValues(bytesValues, 2, three());
IndexSearcher searcher = new IndexSearcher(DirectoryReader.open(writer));
TopFieldDocs topDocs = searcher.search(new MatchAllDocsQuery(), 10, new Sort(new SortField("value", indexFieldData.comparatorSource(null, MultiValueMode.MIN, null))));
assertThat(topDocs.totalHits, equalTo(3));
assertThat(topDocs.scoreDocs.length, equalTo(3));
assertThat(topDocs.scoreDocs[0].doc, equalTo(1));
assertThat(topDocs.scoreDocs[1].doc, equalTo(0));
assertThat(topDocs.scoreDocs[2].doc, equalTo(2));
IndexSearcher searcher = new IndexSearcher(DirectoryReader.open(writer));
TopFieldDocs topDocs = searcher.search(new MatchAllDocsQuery(), 10, new Sort(new SortField("value", indexFieldData.comparatorSource(null, MultiValueMode.MIN, null))));
assertThat(topDocs.totalHits, equalTo(3));
assertThat(topDocs.scoreDocs.length, equalTo(3));
assertThat(topDocs.scoreDocs[0].doc, equalTo(1));
assertThat(topDocs.scoreDocs[1].doc, equalTo(0));
assertThat(topDocs.scoreDocs[2].doc, equalTo(2));
topDocs = searcher.search(new MatchAllDocsQuery(), 10, new Sort(new SortField("value", indexFieldData.comparatorSource(null, MultiValueMode.MAX, null), true)));
assertThat(topDocs.totalHits, equalTo(3));
assertThat(topDocs.scoreDocs.length, equalTo(3));
assertThat(topDocs.scoreDocs[0].doc, equalTo(0));
assertThat(topDocs.scoreDocs[1].doc, equalTo(2));
assertThat(topDocs.scoreDocs[2].doc, equalTo(1));
topDocs = searcher.search(new MatchAllDocsQuery(), 10, new Sort(new SortField("value", indexFieldData.comparatorSource(null, MultiValueMode.MAX, null), true)));
assertThat(topDocs.totalHits, equalTo(3));
assertThat(topDocs.scoreDocs.length, equalTo(3));
assertThat(topDocs.scoreDocs[0].doc, equalTo(0));
assertThat(topDocs.scoreDocs[1].doc, equalTo(2));
assertThat(topDocs.scoreDocs[2].doc, equalTo(1));
}
}
protected abstract void fillMultiValueWithMissing() throws Exception;
@ -191,33 +203,39 @@ public abstract class AbstractFieldDataImplTestCase extends AbstractFieldDataTes
public void testMultiValueWithMissing() throws Exception {
fillMultiValueWithMissing();
IndexFieldData indexFieldData = getForField("value");
AtomicFieldData fieldData = indexFieldData.load(refreshReader());
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
assertValues(bytesValues, 0, two(), four());
assertValues(bytesValues, 1, Strings.EMPTY_ARRAY);
assertValues(bytesValues, 2, three());
assertValues(bytesValues, 0, two(), four());
assertValues(bytesValues, 1, Strings.EMPTY_ARRAY);
assertValues(bytesValues, 2, three());
}
}
public void testMissingValueForAll() throws Exception {
fillAllMissing();
IndexFieldData indexFieldData = getForField("value");
AtomicFieldData fieldData = indexFieldData.load(refreshReader());
// Some impls (FST) return size 0 and some (PagedBytes) do take size in the case no actual data is loaded
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(0L));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
// Some impls (FST) return size 0 and some (PagedBytes) do take size in the case no actual data is loaded
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(0L));
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
assertValues(bytesValues, 0, Strings.EMPTY_ARRAY);
assertValues(bytesValues, 1, Strings.EMPTY_ARRAY);
assertValues(bytesValues, 2, Strings.EMPTY_ARRAY);
SortedBinaryDocValues hashedBytesValues = fieldData.getBytesValues();
assertValues(bytesValues, 0, Strings.EMPTY_ARRAY);
assertValues(bytesValues, 1, Strings.EMPTY_ARRAY);
assertValues(bytesValues, 2, Strings.EMPTY_ARRAY);
SortedBinaryDocValues hashedBytesValues = fieldData.getBytesValues();
assertValues(hashedBytesValues, 0, Strings.EMPTY_ARRAY);
assertValues(hashedBytesValues, 1, Strings.EMPTY_ARRAY);
assertValues(hashedBytesValues, 2, Strings.EMPTY_ARRAY);
assertValues(hashedBytesValues, 0, Strings.EMPTY_ARRAY);
assertValues(hashedBytesValues, 1, Strings.EMPTY_ARRAY);
assertValues(hashedBytesValues, 2, Strings.EMPTY_ARRAY);
}
}
protected abstract void fillAllMissing() throws Exception;

View File

@ -29,7 +29,6 @@ import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.LogByteSizeMergePolicy;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.RAMDirectory;
@ -61,6 +60,7 @@ import org.junit.Before;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import static org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import static org.hamcrest.Matchers.equalTo;
@ -73,8 +73,8 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
protected IndexFieldDataService ifdService;
protected MapperService mapperService;
protected IndexWriter writer;
protected LeafReaderContext readerContext;
protected DirectoryReader topLevelReader;
protected List<LeafReaderContext> readerContexts = null;
protected DirectoryReader topLevelReader = null;
protected IndicesFieldDataCache indicesFieldDataCache;
protected abstract String getFieldDataType();
@ -146,22 +146,21 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
writer = new IndexWriter(new RAMDirectory(), new IndexWriterConfig(new StandardAnalyzer()).setMergePolicy(new LogByteSizeMergePolicy()));
}
protected final LeafReaderContext refreshReader() throws Exception {
if (readerContext != null) {
readerContext.reader().close();
protected final List<LeafReaderContext> refreshReader() throws Exception {
if (readerContexts != null && topLevelReader != null) {
topLevelReader.close();
}
topLevelReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "_na_", 1));
LeafReader reader = SlowCompositeReaderWrapper.wrap(topLevelReader);
readerContext = reader.getContext();
return readerContext;
readerContexts = topLevelReader.leaves();
return readerContexts;
}
@Override
@After
public void tearDown() throws Exception {
super.tearDown();
if (readerContext != null) {
readerContext.reader().close();
if (topLevelReader != null) {
topLevelReader.close();
}
writer.close();
}
@ -179,14 +178,16 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
IndexFieldData fieldData = getForField("non_existing_field");
int max = randomInt(7);
AtomicFieldData previous = null;
for (int i = 0; i < max; i++) {
AtomicFieldData current = fieldData.load(readerContext);
assertThat(current.ramBytesUsed(), equalTo(0L));
if (previous != null) {
assertThat(current, not(sameInstance(previous)));
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData previous = null;
for (int i = 0; i < max; i++) {
AtomicFieldData current = fieldData.load(readerContext);
assertThat(current.ramBytesUsed(), equalTo(0L));
if (previous != null) {
assertThat(current, not(sameInstance(previous)));
}
previous = current;
}
previous = current;
}
}
}

View File

@ -557,37 +557,40 @@ public abstract class AbstractStringFieldDataTestCase extends AbstractFieldDataI
public void testTermsEnum() throws Exception {
fillExtendedMvSet();
LeafReaderContext atomicReaderContext = refreshReader();
writer.forceMerge(1);
List<LeafReaderContext> atomicReaderContexts = refreshReader();
IndexOrdinalsFieldData ifd = getForField("value");
AtomicOrdinalsFieldData afd = ifd.load(atomicReaderContext);
for (LeafReaderContext atomicReaderContext : atomicReaderContexts) {
AtomicOrdinalsFieldData afd = ifd.load(atomicReaderContext);
TermsEnum termsEnum = afd.getOrdinalsValues().termsEnum();
int size = 0;
while (termsEnum.next() != null) {
size++;
TermsEnum termsEnum = afd.getOrdinalsValues().termsEnum();
int size = 0;
while (termsEnum.next() != null) {
size++;
}
assertThat(size, equalTo(12));
assertThat(termsEnum.seekExact(new BytesRef("10")), is(true));
assertThat(termsEnum.term().utf8ToString(), equalTo("10"));
assertThat(termsEnum.next(), nullValue());
assertThat(termsEnum.seekExact(new BytesRef("08")), is(true));
assertThat(termsEnum.term().utf8ToString(), equalTo("08"));
size = 0;
while (termsEnum.next() != null) {
size++;
}
assertThat(size, equalTo(2));
termsEnum.seekExact(8);
assertThat(termsEnum.term().utf8ToString(), equalTo("07"));
size = 0;
while (termsEnum.next() != null) {
size++;
}
assertThat(size, equalTo(3));
}
assertThat(size, equalTo(12));
assertThat(termsEnum.seekExact(new BytesRef("10")), is(true));
assertThat(termsEnum.term().utf8ToString(), equalTo("10"));
assertThat(termsEnum.next(), nullValue());
assertThat(termsEnum.seekExact(new BytesRef("08")), is(true));
assertThat(termsEnum.term().utf8ToString(), equalTo("08"));
size = 0;
while (termsEnum.next() != null) {
size++;
}
assertThat(size, equalTo(2));
termsEnum.seekExact(8);
assertThat(termsEnum.term().utf8ToString(), equalTo("07"));
size = 0;
while (termsEnum.next() != null) {
size++;
}
assertThat(size, equalTo(3));
}
public void testGlobalOrdinalsGetRemovedOnceIndexReaderCloses() throws Exception {

View File

@ -30,6 +30,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.ParsedDocument;
import java.util.List;
import static org.hamcrest.Matchers.equalTo;
public class BinaryDVFieldDataTests extends AbstractFieldDataTestCase {
@ -75,30 +76,32 @@ public class BinaryDVFieldDataTests extends AbstractFieldDataTestCase {
d = mapper.parse("test", "test", "4", doc.bytes());
writer.addDocument(d.rootDoc());
LeafReaderContext reader = refreshReader();
List<LeafReaderContext> readers = refreshReader();
IndexFieldData<?> indexFieldData = getForField("field");
AtomicFieldData fieldData = indexFieldData.load(reader);
for (LeafReaderContext reader : readers) {
AtomicFieldData fieldData = indexFieldData.load(reader);
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
CollectionUtils.sortAndDedup(bytesList1);
bytesValues.setDocument(0);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(bytesList1.get(0))));
assertThat(bytesValues.valueAt(1), equalTo(new BytesRef(bytesList1.get(1))));
CollectionUtils.sortAndDedup(bytesList1);
bytesValues.setDocument(0);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(bytesList1.get(0))));
assertThat(bytesValues.valueAt(1), equalTo(new BytesRef(bytesList1.get(1))));
bytesValues.setDocument(1);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(bytes1)));
bytesValues.setDocument(1);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(bytes1)));
bytesValues.setDocument(2);
assertThat(bytesValues.count(), equalTo(0));
bytesValues.setDocument(2);
assertThat(bytesValues.count(), equalTo(0));
CollectionUtils.sortAndDedup(bytesList2);
bytesValues.setDocument(3);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(bytesList2.get(0))));
assertThat(bytesValues.valueAt(1), equalTo(new BytesRef(bytesList2.get(1))));
CollectionUtils.sortAndDedup(bytesList2);
bytesValues.setDocument(3);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0), equalTo(new BytesRef(bytesList2.get(0))));
assertThat(bytesValues.valueAt(1), equalTo(new BytesRef(bytesList2.get(1))));
}
}
private byte[] randomBytes() {

View File

@ -28,6 +28,7 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
import java.util.List;
import java.util.Random;
import static org.hamcrest.Matchers.equalTo;
@ -58,7 +59,7 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
writer.addDocument(d);
}
writer.forceMerge(1, true);
LeafReaderContext context = refreshReader();
List<LeafReaderContext> contexts = refreshReader();
final BuilderContext builderCtx = new BuilderContext(indexService.getIndexSettings().getSettings(), new ContentPath(1));
{
@ -68,11 +69,13 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
.fielddataFrequencyFilter(0, random.nextBoolean() ? 100 : 0.5d, 0)
.build(builderCtx).fieldType();
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
for (LeafReaderContext context : contexts) {
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
}
}
{
ifdService.clear();
@ -81,10 +84,12 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
.fielddataFrequencyFilter(random.nextBoolean() ? 101 : 101d/200.0d, 201, 100)
.build(builderCtx).fieldType();
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(1L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("5"));
for (LeafReaderContext context : contexts) {
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(1L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("5"));
}
}
{
@ -94,11 +99,13 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
.fielddataFrequencyFilter(random.nextBoolean() ? 101 : 101d/200.0d, Integer.MAX_VALUE, 101)
.build(builderCtx).fieldType();
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
for (LeafReaderContext context : contexts) {
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
}
}
{
@ -108,11 +115,13 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
.fielddataFrequencyFilter(random.nextBoolean() ? 101 : 101d/200.0d, Integer.MAX_VALUE, 101)
.build(builderCtx).fieldType();
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
for (LeafReaderContext context : contexts) {
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
}
}
}

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import org.elasticsearch.index.fielddata.plain.AbstractAtomicGeoPointFieldData;
import java.util.List;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
/**
@ -153,55 +154,64 @@ public class GeoFieldDataTests extends AbstractGeoFieldDataTestCase {
public void testSingleValueAllSet() throws Exception {
fillSingleValueAllSet();
IndexFieldData indexFieldData = getForField("value");
LeafReaderContext readerContext = refreshReader();
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
MultiGeoPointValues fieldValues = ((AbstractAtomicGeoPointFieldData)fieldData).getGeoPointValues();
assertValues(fieldValues, 0);
assertValues(fieldValues, 1);
assertValues(fieldValues, 2);
MultiGeoPointValues fieldValues = ((AbstractAtomicGeoPointFieldData)fieldData).getGeoPointValues();
assertValues(fieldValues, 0);
assertValues(fieldValues, 1);
assertValues(fieldValues, 2);
}
}
@Override
public void testSingleValueWithMissing() throws Exception {
fillSingleValueWithMissing();
IndexFieldData indexFieldData = getForField("value");
LeafReaderContext readerContext = refreshReader();
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
MultiGeoPointValues fieldValues = ((AbstractAtomicGeoPointFieldData)fieldData).getGeoPointValues();
assertValues(fieldValues, 0);
assertMissing(fieldValues, 1);
assertValues(fieldValues, 2);
MultiGeoPointValues fieldValues = ((AbstractAtomicGeoPointFieldData)fieldData).getGeoPointValues();
assertValues(fieldValues, 0);
assertMissing(fieldValues, 1);
assertValues(fieldValues, 2);
}
}
@Override
public void testMultiValueAllSet() throws Exception {
fillMultiValueAllSet();
IndexFieldData indexFieldData = getForField("value");
LeafReaderContext readerContext = refreshReader();
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
MultiGeoPointValues fieldValues = ((AbstractAtomicGeoPointFieldData)fieldData).getGeoPointValues();
assertValues(fieldValues, 0);
assertValues(fieldValues, 1);
assertValues(fieldValues, 2);
MultiGeoPointValues fieldValues = ((AbstractAtomicGeoPointFieldData)fieldData).getGeoPointValues();
assertValues(fieldValues, 0);
assertValues(fieldValues, 1);
assertValues(fieldValues, 2);
}
}
@Override
public void testMultiValueWithMissing() throws Exception {
fillMultiValueWithMissing();
IndexFieldData indexFieldData = getForField("value");
AtomicFieldData fieldData = indexFieldData.load(refreshReader());
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
assertThat(fieldData.ramBytesUsed(), greaterThanOrEqualTo(minRamBytesUsed()));
MultiGeoPointValues fieldValues = ((AbstractAtomicGeoPointFieldData)fieldData).getGeoPointValues();
MultiGeoPointValues fieldValues = ((AbstractAtomicGeoPointFieldData)fieldData).getGeoPointValues();
assertValues(fieldValues, 0);
assertMissing(fieldValues, 1);
assertValues(fieldValues, 2);
assertValues(fieldValues, 0);
assertMissing(fieldValues, 1);
assertValues(fieldValues, 2);
}
}
}

View File

@ -47,6 +47,7 @@ import org.elasticsearch.search.MultiValueMode;
import org.junit.Before;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
@ -124,44 +125,48 @@ public class ParentChildFieldDataTests extends AbstractFieldDataTestCase {
}
public void testGetBytesValues() throws Exception {
writer.forceMerge(1); // force merge to 1 segment so we can iterate through documents
IndexFieldData indexFieldData = getForField(childType);
AtomicFieldData fieldData = indexFieldData.load(refreshReader());
List<LeafReaderContext> readerContexts = refreshReader();
for (LeafReaderContext readerContext : readerContexts) {
AtomicFieldData fieldData = indexFieldData.load(readerContext);
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
bytesValues.setDocument(0);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("1"));
SortedBinaryDocValues bytesValues = fieldData.getBytesValues();
bytesValues.setDocument(0);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("1"));
bytesValues.setDocument(1);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("1"));
assertThat(bytesValues.valueAt(1).utf8ToString(), equalTo("2"));
bytesValues.setDocument(1);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("1"));
assertThat(bytesValues.valueAt(1).utf8ToString(), equalTo("2"));
bytesValues.setDocument(2);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("1"));
assertThat(bytesValues.valueAt(1).utf8ToString(), equalTo("3"));
bytesValues.setDocument(2);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("1"));
assertThat(bytesValues.valueAt(1).utf8ToString(), equalTo("3"));
bytesValues.setDocument(3);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("2"));
bytesValues.setDocument(3);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("2"));
bytesValues.setDocument(4);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("2"));
assertThat(bytesValues.valueAt(1).utf8ToString(), equalTo("4"));
bytesValues.setDocument(4);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("2"));
assertThat(bytesValues.valueAt(1).utf8ToString(), equalTo("4"));
bytesValues.setDocument(5);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("1"));
assertThat(bytesValues.valueAt(1).utf8ToString(), equalTo("5"));
bytesValues.setDocument(5);
assertThat(bytesValues.count(), equalTo(2));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("1"));
assertThat(bytesValues.valueAt(1).utf8ToString(), equalTo("5"));
bytesValues.setDocument(6);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("2"));
bytesValues.setDocument(6);
assertThat(bytesValues.count(), equalTo(1));
assertThat(bytesValues.valueAt(0).utf8ToString(), equalTo("2"));
bytesValues.setDocument(7);
assertThat(bytesValues.count(), equalTo(0));
bytesValues.setDocument(7);
assertThat(bytesValues.count(), equalTo(0));
}
}
public void testSorting() throws Exception {

View File

@ -97,7 +97,7 @@ public class FieldNamesFieldMapperTests extends ESSingleNodeTestCase {
.endObject()
.bytes());
assertFieldNames(set("a", "a.keyword", "b", "b.c", "_uid", "_type", "_version", "_seq_no", "_source", "_all"), doc);
assertFieldNames(set("a", "a.keyword", "b", "b.c", "_uid", "_type", "_version", "_seq_no", "_primary_term", "_source", "_all"), doc);
}
public void testExplicitEnabled() throws Exception {
@ -114,7 +114,7 @@ public class FieldNamesFieldMapperTests extends ESSingleNodeTestCase {
.endObject()
.bytes());
assertFieldNames(set("field", "field.keyword", "_uid", "_type", "_version", "_seq_no", "_source", "_all"), doc);
assertFieldNames(set("field", "field.keyword", "_uid", "_type", "_version", "_seq_no", "_primary_term", "_source", "_all"), doc);
}
public void testDisabled() throws Exception {

View File

@ -55,6 +55,7 @@ import org.elasticsearch.index.flush.FlushStats;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.seqno.SequenceNumbersService;
import org.elasticsearch.index.translog.Translog;
@ -102,12 +103,15 @@ public class IndexShardIT extends ESSingleNodeTestCase {
private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, long seqNo,
ParseContext.Document document, BytesReference source, Mapping mappingUpdate) {
Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE);
Field seqNoField = new NumericDocValuesField("_seq_no", seqNo);
Field versionField = new NumericDocValuesField("_version", 0);
SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID();
document.add(uidField);
document.add(versionField);
return new ParsedDocument(versionField, seqNoField, id, type, routing, Collections.singletonList(document), source,
mappingUpdate);
document.add(seqID.seqNo);
document.add(seqID.seqNoDocValue);
document.add(seqID.primaryTerm);
return new ParsedDocument(versionField, seqID, id, type, routing,
Collections.singletonList(document), source, mappingUpdate);
}
public void testLockTryingToDelete() throws Exception {

View File

@ -70,6 +70,7 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus;
@ -542,10 +543,13 @@ public class IndexShardTests extends IndexShardTestCase {
ParseContext.Document document, BytesReference source, Mapping mappingUpdate) {
Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE);
Field versionField = new NumericDocValuesField("_version", 0);
Field seqNoField = new NumericDocValuesField("_seq_no", 0);
SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID();
document.add(uidField);
document.add(versionField);
return new ParsedDocument(versionField, seqNoField, id, type, routing, Arrays.asList(document), source, mappingUpdate);
document.add(seqID.seqNo);
document.add(seqID.seqNoDocValue);
document.add(seqID.primaryTerm);
return new ParsedDocument(versionField, seqID, id, type, routing, Arrays.asList(document), source, mappingUpdate);
}
public void testIndexingOperationsListeners() throws IOException {

View File

@ -47,6 +47,7 @@ import org.elasticsearch.index.engine.InternalEngineTests.TranslogHandler;
import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.Store;
@ -273,11 +274,14 @@ public class RefreshListenersTests extends ESTestCase {
document.add(new TextField("test", testFieldValue, Field.Store.YES));
Field uidField = new Field("_uid", type + ":" + id, UidFieldMapper.Defaults.FIELD_TYPE);
Field versionField = new NumericDocValuesField("_version", Versions.MATCH_ANY);
Field seqNoField = new NumericDocValuesField("_seq_no", 0);
SeqNoFieldMapper.SequenceID seqID = SeqNoFieldMapper.SequenceID.emptySeqID();
document.add(uidField);
document.add(versionField);
document.add(seqID.seqNo);
document.add(seqID.seqNoDocValue);
document.add(seqID.primaryTerm);
BytesReference source = new BytesArray(new byte[] { 1 });
ParsedDocument doc = new ParsedDocument(versionField, seqNoField, id, type, null, Arrays.asList(document), source, null);
ParsedDocument doc = new ParsedDocument(versionField, seqID, id, type, null, Arrays.asList(document), source, null);
Engine.Index index = new Engine.Index(new Term("_uid", uid), doc);
return engine.index(index);
}

View File

@ -301,7 +301,7 @@ public class TranslogTests extends ESTestCase {
assertThat(stats.estimatedNumberOfOperations(), equalTo(0L));
assertThat(stats.getTranslogSizeInBytes(), equalTo(firstOperationPosition));
assertEquals(6, total.estimatedNumberOfOperations());
assertEquals(413, total.getTranslogSizeInBytes());
assertEquals(419, total.getTranslogSizeInBytes());
BytesStreamOutput out = new BytesStreamOutput();
total.writeTo(out);
@ -309,13 +309,13 @@ public class TranslogTests extends ESTestCase {
copy.readFrom(out.bytes().streamInput());
assertEquals(6, copy.estimatedNumberOfOperations());
assertEquals(413, copy.getTranslogSizeInBytes());
assertEquals(419, copy.getTranslogSizeInBytes());
try (XContentBuilder builder = XContentFactory.jsonBuilder()) {
builder.startObject();
copy.toXContent(builder, ToXContent.EMPTY_PARAMS);
builder.endObject();
assertEquals("{\"translog\":{\"operations\":6,\"size_in_bytes\":413}}", builder.string());
assertEquals("{\"translog\":{\"operations\":6,\"size_in_bytes\":419}}", builder.string());
}
try {
@ -1137,7 +1137,7 @@ public class TranslogTests extends ESTestCase {
try (Translog ignored = new Translog(config, translogGeneration, () -> SequenceNumbersService.UNASSIGNED_SEQ_NO)) {
fail("corrupted");
} catch (IllegalStateException ex) {
assertEquals(ex.getMessage(), "Checkpoint file translog-2.ckp already exists but has corrupted content expected: Checkpoint{offset=2298, numOps=55, translogFileGeneration=2, globalCheckpoint=-2} but got: Checkpoint{offset=0, numOps=0, translogFileGeneration=0, globalCheckpoint=-2}");
assertEquals(ex.getMessage(), "Checkpoint file translog-2.ckp already exists but has corrupted content expected: Checkpoint{offset=2353, numOps=55, translogFileGeneration=2, globalCheckpoint=-2} but got: Checkpoint{offset=0, numOps=0, translogFileGeneration=0, globalCheckpoint=-2}");
}
Checkpoint.write(FileChannel::open, config.getTranslogPath().resolve(Translog.getCommitCheckpointFileName(read.generation)), read, StandardOpenOption.WRITE, StandardOpenOption.TRUNCATE_EXISTING);
try (Translog translog = new Translog(config, translogGeneration, () -> SequenceNumbersService.UNASSIGNED_SEQ_NO)) {
@ -1303,6 +1303,7 @@ public class TranslogTests extends ESTestCase {
op = new Translog.Delete(
new Term("_uid", threadId + "_" + opCount),
opCount,
0,
1 + randomInt(100000),
randomFrom(VersionType.values()));
break;

View File

@ -359,7 +359,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase {
.field("query_field2", queryBuilder)
.endObject().bytes()
);
assertThat(doc.rootDoc().getFields().size(), equalTo(12)); // also includes all other meta fields
assertThat(doc.rootDoc().getFields().size(), equalTo(14)); // also includes all other meta fields
BytesRef queryBuilderAsBytes = doc.rootDoc().getField("query_field1.query_builder_field").binaryValue();
assertQueryBuilder(queryBuilderAsBytes, queryBuilder);
@ -389,7 +389,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase {
.field("query_field", queryBuilder)
.endObject().endObject().bytes()
);
assertThat(doc.rootDoc().getFields().size(), equalTo(9)); // also includes all other meta fields
assertThat(doc.rootDoc().getFields().size(), equalTo(11)); // also includes all other meta fields
BytesRef queryBuilderAsBytes = doc.rootDoc().getField("object_field.query_field.query_builder_field").binaryValue();
assertQueryBuilder(queryBuilderAsBytes, queryBuilder);
@ -400,7 +400,7 @@ public class PercolatorFieldMapperTests extends ESSingleNodeTestCase {
.endArray()
.endObject().bytes()
);
assertThat(doc.rootDoc().getFields().size(), equalTo(9)); // also includes all other meta fields
assertThat(doc.rootDoc().getFields().size(), equalTo(11)); // also includes all other meta fields
queryBuilderAsBytes = doc.rootDoc().getField("object_field.query_field.query_builder_field").binaryValue();
assertQueryBuilder(queryBuilderAsBytes, queryBuilder);