From f3a8f954c8b329a15287b739b105378127f6ec5d Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 25 Apr 2018 11:17:17 +0200 Subject: [PATCH 01/18] [CCR] use IndexSearcher to read operations from Lucene index instead of using the translog --- .../xpack/ccr/FollowIndexIT.java | 4 + .../ccr/action/FollowExistingIndexAction.java | 122 ++++++----- .../xpack/ccr/action/ShardChangesAction.java | 202 +++++++++++++++--- .../xpack/ccr/ShardChangesIT.java | 128 ++++++++++- .../FollowExistingIndexActionTests.java | 66 ++++++ .../ccr/action/ShardChangesActionTests.java | 109 +++++++++- .../ccr/action/ShardChangesRequestTests.java | 1 - 7 files changed, 537 insertions(+), 95 deletions(-) create mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexActionTests.java diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index 09556bf748726..7ef0d5703cb21 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -44,6 +44,10 @@ public void testFollowIndex() throws Exception { final String leaderIndexName = "test_index1"; if (runningAgainstLeaderCluster) { logger.info("Running against leader cluster"); + Settings indexSettings = Settings.builder() + .put("index.soft_deletes", true) + .build(); + createIndex(leaderIndexName, indexSettings); for (int i = 0; i < numDocs; i++) { logger.info("Indexing doc [{}]", i); index(client(), leaderIndexName, Integer.toString(i), "field", i); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexAction.java index 44628f1da5b3c..f2a63908b965f 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexAction.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.persistent.PersistentTasksService; @@ -205,73 +206,78 @@ protected void doExecute(Request request, ActionListener listener) { */ void start(Request request, String clusterNameAlias, IndexMetaData leaderIndexMetadata, IndexMetaData followIndexMetadata, ActionListener handler) { - if (leaderIndexMetadata == null) { - handler.onFailure(new IllegalArgumentException("leader index [" + request.leaderIndex + "] does not exist")); - return; - } - - if (followIndexMetadata == null) { - handler.onFailure(new IllegalArgumentException("follow index [" + request.followIndex + "] does not exist")); - return; - } - - if (leaderIndexMetadata.getNumberOfShards() != followIndexMetadata.getNumberOfShards()) { - handler.onFailure(new IllegalArgumentException("leader index primary shards [" + - leaderIndexMetadata.getNumberOfShards() + "] does not match with the number of " + - "shards of the follow index [" + followIndexMetadata.getNumberOfShards() + "]")); - // TODO: other validation checks - } else { - final int numShards = followIndexMetadata.getNumberOfShards(); - final AtomicInteger counter = new AtomicInteger(numShards); - final AtomicReferenceArray responses = new AtomicReferenceArray<>(followIndexMetadata.getNumberOfShards()); - for (int i = 0; i < numShards; i++) { - final int shardId = i; - String taskId = followIndexMetadata.getIndexUUID() + "-" + shardId; - ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, - new ShardId(followIndexMetadata.getIndex(), shardId), - new ShardId(leaderIndexMetadata.getIndex(), shardId), - request.batchSize, request.concurrentProcessors, request.processorMaxTranslogBytes); - persistentTasksService.startPersistentTask(taskId, ShardFollowTask.NAME, shardFollowTask, - new ActionListener>() { - @Override - public void onResponse(PersistentTasksCustomMetaData.PersistentTask task) { - responses.set(shardId, task); - finalizeResponse(); - } + validate(leaderIndexMetadata, followIndexMetadata, request); + + final int numShards = followIndexMetadata.getNumberOfShards(); + final AtomicInteger counter = new AtomicInteger(numShards); + final AtomicReferenceArray responses = new AtomicReferenceArray<>(followIndexMetadata.getNumberOfShards()); + for (int i = 0; i < numShards; i++) { + final int shardId = i; + String taskId = followIndexMetadata.getIndexUUID() + "-" + shardId; + ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, + new ShardId(followIndexMetadata.getIndex(), shardId), + new ShardId(leaderIndexMetadata.getIndex(), shardId), + request.batchSize, request.concurrentProcessors, request.processorMaxTranslogBytes); + persistentTasksService.startPersistentTask(taskId, ShardFollowTask.NAME, shardFollowTask, + new ActionListener>() { + @Override + public void onResponse(PersistentTasksCustomMetaData.PersistentTask task) { + responses.set(shardId, task); + finalizeResponse(); + } - @Override - public void onFailure(Exception e) { - responses.set(shardId, e); - finalizeResponse(); - } + @Override + public void onFailure(Exception e) { + responses.set(shardId, e); + finalizeResponse(); + } - void finalizeResponse() { - Exception error = null; - if (counter.decrementAndGet() == 0) { - for (int j = 0; j < responses.length(); j++) { - Object response = responses.get(j); - if (response instanceof Exception) { - if (error == null) { - error = (Exception) response; - } else { - error.addSuppressed((Throwable) response); - } + void finalizeResponse() { + Exception error = null; + if (counter.decrementAndGet() == 0) { + for (int j = 0; j < responses.length(); j++) { + Object response = responses.get(j); + if (response instanceof Exception) { + if (error == null) { + error = (Exception) response; + } else { + error.addSuppressed((Throwable) response); } } + } - if (error == null) { - // include task ids? - handler.onResponse(new Response()); - } else { - // TODO: cancel all started tasks - handler.onFailure(error); - } + if (error == null) { + // include task ids? + handler.onResponse(new Response()); + } else { + // TODO: cancel all started tasks + handler.onFailure(error); } } } - ); - } + }); } } } + + static void validate(IndexMetaData leaderIndex, IndexMetaData followIndex, Request request) { + if (leaderIndex == null) { + throw new IllegalArgumentException("leader index [" + request.leaderIndex + "] does not exist"); + } + + if (followIndex == null) { + throw new IllegalArgumentException("follow index [" + request.followIndex + "] does not exist"); + } + + if (leaderIndex.getSettings().getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false) == false) { + throw new IllegalArgumentException("leader index [" + request.leaderIndex + "] does not have soft deletes enabled"); + } + + if (leaderIndex.getNumberOfShards() != followIndex.getNumberOfShards()) { + throw new IllegalArgumentException("leader index primary shards [" + leaderIndex.getNumberOfShards() + + "] does not match with the number of shards of the follow index [" + followIndex.getNumberOfShards() + "]"); + } + // TODO: other validation checks + } + } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java index d5c774f53a0ed..7521a22133691 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java @@ -5,6 +5,25 @@ */ package org.elasticsearch.xpack.ccr.action; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FilterDirectoryReader; +import org.apache.lucene.index.FilterLeafReader; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.ReaderUtil; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.SortedNumericSortField; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; @@ -20,8 +39,19 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.fieldvisitor.FieldsVisitor; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardNotStartedException; import org.elasticsearch.index.shard.IndexShardState; @@ -34,11 +64,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; import java.util.List; import java.util.Objects; -import java.util.PriorityQueue; -import java.util.Queue; import static org.elasticsearch.action.ValidateActions.addValidationError; @@ -227,6 +254,7 @@ protected Response shardOperation(Request request, ShardId shardId) throws IOExc IndexService indexService = indicesService.indexServiceSafe(request.getShard().getIndex()); IndexShard indexShard = indexService.getShard(request.getShard().id()); + request.maxSeqNo = Math.min(request.maxSeqNo, indexShard.getGlobalCheckpoint()); return getOperationsBetween(indexShard, request.minSeqNo, request.maxSeqNo, request.maxTranslogsBytes); } @@ -257,40 +285,152 @@ static Response getOperationsBetween(IndexShard indexShard, long minSeqNo, long throw new IndexShardNotStartedException(indexShard.shardId(), indexShard.state()); } - long seenBytes = 0; - long nextExpectedSeqNo = minSeqNo; - final Queue orderedOps = new PriorityQueue<>(Comparator.comparingLong(Translog.Operation::seqNo)); + // TODO: Somehow this needs to be an internal refresh (SearcherScope.INTERNAL) + indexShard.refresh("shard_changes_api"); + // TODO: Somehow this needs to acquire an internal searcher (SearcherScope.INTERNAL) + try (Engine.Searcher searcher = indexShard.acquireSearcher("shard_changes_api")) { + List operations = getOperationsBetween(minSeqNo, maxSeqNo, byteLimit, + searcher.getDirectoryReader(), indexShard.mapperService()); + return new Response(operations.toArray(EMPTY_OPERATIONS_ARRAY)); + } + } + + static List getOperationsBetween(long minSeqNo, long maxSeqNo, long byteLimit, + DirectoryReader indexReader, MapperService mapperService) throws IOException { + IndexSearcher searcher = new IndexSearcher(new CCRIndexReader(indexReader)); + searcher.setQueryCache(null); + + MappedFieldType seqNoFieldType = mapperService.fullName(SeqNoFieldMapper.NAME); + assert mapperService.types().size() == 1; + String type = mapperService.types().iterator().next(); + + int size = ((int) (maxSeqNo - minSeqNo)) + 1; + Sort sort = new Sort(new SortedNumericSortField(seqNoFieldType.name(), SortField.Type.LONG)); + Query query; + if (mapperService.hasNested()) { + BooleanQuery.Builder builder = new BooleanQuery.Builder(); + builder.add(Queries.newNonNestedFilter(mapperService.getIndexSettings().getIndexVersionCreated()), Occur.FILTER); + builder.add(seqNoFieldType.rangeQuery(minSeqNo, maxSeqNo, true, true, null, null, null, null), Occur.FILTER); + query = builder.build(); + } else { + query = seqNoFieldType.rangeQuery(minSeqNo, maxSeqNo, true, true, null, null, null, null); + } + TopDocs topDocs = searcher.search(query, size, sort); + if (topDocs.scoreDocs.length != size) { + String message = "not all operations between min_seq_no [" + minSeqNo + "] and max_seq_no [" + maxSeqNo + "] found"; + throw new IllegalStateException(message); + } + long seenBytes = 0; final List operations = new ArrayList<>(); - try (Translog.Snapshot snapshot = indexShard.newTranslogSnapshotBetween(minSeqNo, maxSeqNo)) { - for (Translog.Operation unorderedOp = snapshot.next(); unorderedOp != null; unorderedOp = snapshot.next()) { - if (unorderedOp.seqNo() < minSeqNo || unorderedOp.seqNo() > maxSeqNo) { - continue; - } - - orderedOps.add(unorderedOp); - while (orderedOps.peek() != null && orderedOps.peek().seqNo() == nextExpectedSeqNo) { - Translog.Operation orderedOp = orderedOps.poll(); - if (seenBytes < byteLimit) { - nextExpectedSeqNo++; - seenBytes += orderedOp.estimateSize(); - operations.add(orderedOp); - if (nextExpectedSeqNo > maxSeqNo) { - return new Response(operations.toArray(EMPTY_OPERATIONS_ARRAY)); - } - } else { - return new Response(operations.toArray(EMPTY_OPERATIONS_ARRAY)); + for (ScoreDoc scoreDoc : topDocs.scoreDocs) { + FieldsVisitor fieldsVisitor = new FieldsVisitor(true); + searcher.doc(scoreDoc.doc, fieldsVisitor); + fieldsVisitor.postProcess(mapperService); + + String id = fieldsVisitor.uid().id(); + String routing = fieldsVisitor.routing(); + if (fieldsVisitor.source() == null) { + throw new IllegalArgumentException("no source found for document with id [" + id + "]"); + } + byte[] source = fieldsVisitor.source().toBytesRef().bytes; + + // TODO: optimize this so that we fetch doc values data in segment and doc id order (see: DocValueFieldsFetchSubPhase): + int leafReaderIndex = ReaderUtil.subIndex(scoreDoc.doc, searcher.getIndexReader().leaves()); + LeafReaderContext leafReaderContext = searcher.getIndexReader().leaves().get(leafReaderIndex); + int segmentDocId = scoreDoc.doc - leafReaderContext.docBase; + long version = readNumberDvValue(leafReaderContext, VersionFieldMapper.NAME, segmentDocId); + long seqNo = readNumberDvValue(leafReaderContext, SeqNoFieldMapper.NAME, segmentDocId); + long primaryTerm = readNumberDvValue(leafReaderContext, SeqNoFieldMapper.PRIMARY_TERM_NAME, segmentDocId); + + // TODO: handle NOOPs + // NOTE: versionType can always be INTERNAL, + // because version logic has already been taken care of when indexing into leader shard. + final VersionType versionType = VersionType.INTERNAL; + final Translog.Operation op; + if (isDeleteOperation(leafReaderContext, segmentDocId)) { + BytesRef idBytes = Uid.encodeId(id); + Term uidForDelete = new Term(IdFieldMapper.NAME, idBytes); + op = new Translog.Delete(type, id, uidForDelete, seqNo, primaryTerm, version, versionType); + } else { + // NOTE: autoGeneratedIdTimestamp can always be -1, + // because auto id generation has already been performed when inxdexing into leader shard. + final int autoGeneratedId = -1; + op = new Translog.Index(type, id, seqNo, primaryTerm, version, versionType, source, routing, autoGeneratedId); + } + seenBytes += op.estimateSize(); + operations.add(op); + if (seenBytes > byteLimit) { + return operations; + } + } + return operations; + } + + private static long readNumberDvValue(LeafReaderContext leafReaderContext, String fieldName, int segmentDocId) throws IOException { + NumericDocValues versionDvField = leafReaderContext.reader().getNumericDocValues(fieldName); + assert versionDvField != null : fieldName + " field is missing"; + boolean advanced = versionDvField.advanceExact(segmentDocId); + assert advanced; + return versionDvField.longValue(); + } + + private static boolean isDeleteOperation(LeafReaderContext leafReaderContext, int segmentDocId) throws IOException { + NumericDocValues softDeleteField = leafReaderContext.reader().getNumericDocValues(Lucene.SOFT_DELETE_FIELD); + if (softDeleteField == null) { + return false; + } + + boolean advanced = softDeleteField.advanceExact(segmentDocId); + if (advanced == false) { + return false; + } + + long value = softDeleteField.longValue(); + return value == 1L; + } + + static final class CCRIndexReader extends FilterDirectoryReader { + + static final class CCRSubReaderWrapper extends SubReaderWrapper { + @Override + public LeafReader wrap(LeafReader in) { + return new FilterLeafReader(in) { + @Override + public CacheHelper getCoreCacheHelper() { + return null; + } + + @Override + public CacheHelper getReaderCacheHelper() { + return null; + } + + @Override + public int numDocs() { + return maxDoc(); + } + + @Override + public Bits getLiveDocs() { + return null; } - } + }; } } - if (nextExpectedSeqNo >= maxSeqNo) { - return new Response(operations.toArray(EMPTY_OPERATIONS_ARRAY)); - } else { - String message = "Not all operations between min_seq_no [" + minSeqNo + "] and max_seq_no [" + maxSeqNo + - "] found, tracker checkpoint [" + nextExpectedSeqNo + "]"; - throw new IllegalStateException(message); + CCRIndexReader(DirectoryReader in) throws IOException { + super(in, new CCRSubReaderWrapper()); + } + + @Override + protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { + return new CCRIndexReader(in); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return in.getReaderCacheHelper(); } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java index 3e7811873014b..4855ca58eb008 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java @@ -34,6 +34,8 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; @@ -157,7 +159,8 @@ public void testGetOperationsBasedOnGlobalSequenceId() throws Exception { public void testFollowIndex() throws Exception { final int numberOfPrimaryShards = randomIntBetween(1, 3); - final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, Collections.emptyMap()); + final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, + Collections.singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); final String followerIndexSettings = @@ -213,6 +216,70 @@ public void testFollowIndex() throws Exception { assertBusy(assertExpectedDocumentRunnable(i)); } + final int numDocsToDelete = randomIntBetween(2, 64); + for (int i = 0; i < numDocsToDelete; i++) { + client().prepareDelete("index1", "doc", Integer.toString(i)).get(); + } + + for (int i = 0; i < numDocsToDelete; i++) { + assertDocumentDeletedRunnable(i); + } + + final UnfollowIndexAction.Request unfollowRequest = new UnfollowIndexAction.Request(); + unfollowRequest.setFollowIndex("index2"); + client().execute(UnfollowIndexAction.INSTANCE, unfollowRequest).get(); + + assertBusy(() -> { + final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + final PersistentTasksCustomMetaData tasks = clusterState.getMetaData().custom(PersistentTasksCustomMetaData.TYPE); + assertThat(tasks.tasks().size(), equalTo(0)); + }); + } + + public void testFollowIndexWithNestedField() throws Exception { + final String leaderIndexSettings = + getIndexSettingsWithNestedMapping(1, Collections.singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); + + final String followerIndexSettings = + getIndexSettingsWithNestedMapping(1, Collections.singletonMap(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), "true")); + assertAcked(client().admin().indices().prepareCreate("index2").setSource(followerIndexSettings, XContentType.JSON)); + + ensureGreen("index1", "index2"); + + final FollowExistingIndexAction.Request followRequest = new FollowExistingIndexAction.Request(); + followRequest.setLeaderIndex("index1"); + followRequest.setFollowIndex("index2"); + client().execute(FollowExistingIndexAction.INSTANCE, followRequest).get(); + + final int numDocs = randomIntBetween(2, 64); + for (int i = 0; i < numDocs; i++) { + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + builder.field("field", "value"); + builder.startArray("objects"); + { + builder.startObject(); + builder.field("field", i); + builder.endObject(); + } + builder.endArray(); + builder.endObject(); + client().prepareIndex("index1", "doc", Integer.toString(i)).setSource(builder).get(); + } + } + + for (int i = 0; i < numDocs; i++) { + int value = i; + assertBusy(() -> { + final GetResponse getResponse = client().prepareGet("index2", "doc", Integer.toString(value)).get(); + assertTrue(getResponse.isExists()); + assertTrue((getResponse.getSource().containsKey("field"))); + assertThat(XContentMapValues.extractValue("objects.field", getResponse.getSource()), + equalTo(Collections.singletonList(value))); + }); + } + final UnfollowIndexAction.Request unfollowRequest = new UnfollowIndexAction.Request(); unfollowRequest.setFollowIndex("index2"); client().execute(UnfollowIndexAction.INSTANCE, unfollowRequest).get(); @@ -287,6 +354,13 @@ private CheckedRunnable assertExpectedDocumentRunnable(final int valu }; } + private CheckedRunnable assertDocumentDeletedRunnable(final int value) { + return () -> { + final GetResponse getResponse = client().prepareGet("index2", "doc", Integer.toString(value)).get(); + assertFalse(getResponse.isExists()); + }; + } + private String getIndexSettings(final int numberOfPrimaryShards, final Map additionalIndexSettings) throws IOException { final String settings; try (XContentBuilder builder = jsonBuilder()) { @@ -324,4 +398,56 @@ private String getIndexSettings(final int numberOfPrimaryShards, final Map additionalIndexSettings) throws IOException { + final String settings; + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + { + builder.startObject("settings"); + { + builder.field("index.number_of_shards", numberOfPrimaryShards); + for (final Map.Entry additionalSetting : additionalIndexSettings.entrySet()) { + builder.field(additionalSetting.getKey(), additionalSetting.getValue()); + } + } + builder.endObject(); + builder.startObject("mappings"); + { + builder.startObject("doc"); + { + builder.startObject("properties"); + { + builder.startObject("objects"); + { + builder.field("type", "nested"); + builder.startObject("properties"); + { + builder.startObject("field"); + { + builder.field("type", "long"); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + builder.startObject("field"); + { + builder.field("type", "keyword"); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + settings = BytesReference.bytes(builder).utf8ToString(); + } + return settings; + } + } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexActionTests.java new file mode 100644 index 0000000000000..4c2185d31799c --- /dev/null +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexActionTests.java @@ -0,0 +1,66 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ccr.action; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.equalTo; + +public class FollowExistingIndexActionTests extends ESTestCase { + + public void testValidation() { + FollowExistingIndexAction.Request request = new FollowExistingIndexAction.Request(); + request.setLeaderIndex("index1"); + request.setFollowIndex("index2"); + + { + Exception e = expectThrows(IllegalArgumentException.class, () -> FollowExistingIndexAction.validate(null, null, request)); + assertThat(e.getMessage(), equalTo("leader index [index1] does not exist")); + } + { + IndexMetaData leaderIMD = createIMD("index1", 5); + Exception e = expectThrows(IllegalArgumentException.class, () -> FollowExistingIndexAction.validate(leaderIMD, null, request)); + assertThat(e.getMessage(), equalTo("follow index [index2] does not exist")); + } + { + IndexMetaData leaderIMD = createIMD("index1", 5); + IndexMetaData followIMD = createIMD("index2", 5); + Exception e = expectThrows(IllegalArgumentException.class, + () -> FollowExistingIndexAction.validate(leaderIMD, followIMD, request)); + assertThat(e.getMessage(), equalTo("leader index [index1] does not have soft deletes enabled")); + } + { + IndexMetaData leaderIMD = createIMD("index1", 5, IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true"); + IndexMetaData followIMD = createIMD("index2", 4); + Exception e = expectThrows(IllegalArgumentException.class, + () -> FollowExistingIndexAction.validate(leaderIMD, followIMD, request)); + assertThat(e.getMessage(), + equalTo("leader index primary shards [5] does not match with the number of shards of the follow index [4]")); + } + { + IndexMetaData leaderIMD = createIMD("index1", 5, IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true"); + IndexMetaData followIMD = createIMD("index2", 5); + FollowExistingIndexAction.validate(leaderIMD, followIMD, request); + } + } + + private static IndexMetaData createIMD(String index, int numShards, String... settings) { + assert settings.length % 2 == 0; + Settings.Builder settingsBuilder = settings(Version.CURRENT); + for (int i = 0; i < settings.length; i += 2) { + settingsBuilder.put(settings[i], settings[i + 1]); + } + return IndexMetaData.builder(index).settings(settingsBuilder) + .numberOfShards(numShards) + .numberOfReplicas(0) + .setRoutingNumShards(numShards).build(); + } + +} diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index b91c43d74e757..d7d68e0a2dc0d 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -5,6 +5,20 @@ */ package org.elasticsearch.xpack.ccr.action; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.index.Term; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; @@ -13,6 +27,12 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardNotStartedException; import org.elasticsearch.index.translog.Translog; @@ -20,6 +40,7 @@ import org.mockito.Mockito; import java.util.Arrays; +import java.util.List; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.LongStream; @@ -62,14 +83,14 @@ public void testGetOperationsBetween() throws Exception { // get operations for a range no operations exists: Exception e = expectThrows(IllegalStateException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites, numWrites + 1, Long.MAX_VALUE)); - assertThat(e.getMessage(), containsString("Not all operations between min_seq_no [" + numWrites + "] and max_seq_no [" + - (numWrites + 1) +"] found, tracker checkpoint [")); + assertThat(e.getMessage(), equalTo("not all operations between min_seq_no [" + numWrites + "] and max_seq_no [" + + (numWrites + 1) +"] found")); // get operations for a range some operations do not exist: e = expectThrows(IllegalStateException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites - 10, numWrites + 10, Long.MAX_VALUE)); - assertThat(e.getMessage(), containsString("Not all operations between min_seq_no [" + (numWrites - 10) + "] and max_seq_no [" + - (numWrites + 10) +"] found, tracker checkpoint [")); + assertThat(e.getMessage(), equalTo("not all operations between min_seq_no [" + (numWrites - 10) + "] and max_seq_no [" + + (numWrites + 10) +"] found")); } public void testGetOperationsBetweenWhenShardNotStarted() throws Exception { @@ -109,4 +130,84 @@ public void testGetOperationsBetweenExceedByteLimit() throws Exception { assertThat(r.getOperations()[11].seqNo(), equalTo(11L)); } + public void testReadOpsFromLuceneIndex() throws Exception { + // MapperService is needed by FieldsVisitor.postProcess(...) + final Settings settings = Settings.builder() + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", 0) + .build(); + final MapperService mapperService = createIndex("index", settings, "_doc").mapperService(); + + try (Directory directory = newDirectory()) { + IndexWriterConfig indexWriterConfig = newIndexWriterConfig(); + indexWriterConfig.setMergePolicy(NoMergePolicy.INSTANCE); + try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory, indexWriterConfig)) { + int numDocs = randomIntBetween(10, 100); + for (int i = 0; i < numDocs; i++) { + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); + seqID.seqNo.setLongValue(i); + seqID.seqNoDocValue.setLongValue(i); + seqID.primaryTerm.setLongValue(1); + + Document document = new Document(); + document.add(seqID.seqNo); + document.add(seqID.seqNoDocValue); + document.add(seqID.primaryTerm); + document.add(new NumericDocValuesField(VersionFieldMapper.NAME, 1L)); + document.add(new StoredField(SourceFieldMapper.NAME, new BytesRef("{}"))); + document.add(new StringField(IdFieldMapper.NAME, Uid.encodeId(Integer.toString(i)), Field.Store.YES)); + + iw.addDocument(document); + if (randomBoolean()) { + iw.deleteDocuments(new Term(IdFieldMapper.NAME, Integer.toString(i))); + } + } + try(DirectoryReader ir = DirectoryReader.open(iw.w)) { + ShardChangesAction.CCRIndexReader ccrIndexReader = new ShardChangesAction.CCRIndexReader(ir); + assertThat(ccrIndexReader.numDocs(), equalTo(ccrIndexReader.maxDoc())); + assertThat(ccrIndexReader.numDeletedDocs(), equalTo(0)); + + List ops = + ShardChangesAction.getOperationsBetween(0, numDocs - 1, Long.MAX_VALUE, ir, mapperService); + assertThat(ops.size(), equalTo(numDocs)); + } + } + + } + } + + public void testReadOpsFromLuceneIndex_missingFields() throws Exception { + // MapperService is needed by FieldsVisitor.postProcess(...) + final Settings settings = Settings.builder() + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", 0) + .build(); + final MapperService mapperService = createIndex("index", settings, "_doc").mapperService(); + + try (Directory directory = newDirectory()) { + IndexWriterConfig indexWriterConfig = newIndexWriterConfig(); + indexWriterConfig.setMergePolicy(NoMergePolicy.INSTANCE); + try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory, indexWriterConfig)) { + SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); + seqID.seqNo.setLongValue(0); + seqID.seqNoDocValue.setLongValue(0); + seqID.primaryTerm.setLongValue(1); + + Document document = new Document(); + document.add(seqID.seqNo); + document.add(seqID.seqNoDocValue); + document.add(seqID.primaryTerm); + document.add(new NumericDocValuesField(VersionFieldMapper.NAME, 1L)); + document.add(new StoredField(IdFieldMapper.NAME, Uid.encodeId("0"))); + iw.addDocument(document); + + try(DirectoryReader ir = DirectoryReader.open(iw.w)) { + Exception e = expectThrows(IllegalArgumentException.class, + () -> ShardChangesAction.getOperationsBetween(0, 0, Long.MAX_VALUE, ir, mapperService)); + assertThat(e.getMessage(), equalTo("no source found for document with id [0]")); + } + } + } + } + } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java index a0b68c3699bf6..3f30545576046 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.ccr.action; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.AbstractStreamableTestCase; From 341eb39461ac1413af9849695923bc49bcfcd87f Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 25 Apr 2018 11:35:42 +0200 Subject: [PATCH 02/18] moved CCRIndexReader to Lucene.java and added a simple test --- .../elasticsearch/common/lucene/Lucene.java | 54 +++++++++++++++++++ .../common/lucene/LuceneTests.java | 32 +++++++++++ .../xpack/ccr/action/ShardChangesAction.java | 50 +---------------- .../ccr/action/ShardChangesActionTests.java | 7 --- 4 files changed, 87 insertions(+), 56 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index f707ecc1fe65c..52a024601d628 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -30,6 +30,7 @@ import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.FilterLeafReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFileNames; @@ -839,4 +840,57 @@ public int length() { public static NumericDocValuesField newSoftDeleteField() { return new NumericDocValuesField(SOFT_DELETE_FIELD, 1); } + + /** + * Wraps the provided {@link DirectoryReader} and return a new {@link DirectoryReader} instance that ignores + * deleted documents. + */ + public static DirectoryReader ignoreDeletes(DirectoryReader in) throws IOException { + return new NoDeletesIndexReader(in); + } + + private static final class NoDeletesIndexReader extends FilterDirectoryReader { + + private static final class NoDeletesIndexReaderWrapper extends SubReaderWrapper { + @Override + public LeafReader wrap(LeafReader in) { + return new FilterLeafReader(in) { + @Override + public CacheHelper getCoreCacheHelper() { + return null; + } + + @Override + public CacheHelper getReaderCacheHelper() { + return null; + } + + @Override + public int numDocs() { + return maxDoc(); + } + + @Override + public Bits getLiveDocs() { + return null; + } + }; + } + } + + private NoDeletesIndexReader(DirectoryReader in) throws IOException { + super(in, new NoDeletesIndexReaderWrapper()); + } + + @Override + protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { + return new NoDeletesIndexReader(in); + } + + @Override + public CacheHelper getReaderCacheHelper() { + return in.getReaderCacheHelper(); + } + } + } diff --git a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java index 753aedea01e02..bf275ef8c9fb7 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java @@ -23,6 +23,8 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.Field.Store; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.StoredField; import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.DirectoryReader; @@ -42,6 +44,9 @@ import org.apache.lucene.store.MMapDirectory; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -53,6 +58,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import static org.hamcrest.Matchers.equalTo; + public class LuceneTests extends ESTestCase { public void testWaitForIndex() throws Exception { final MockDirectoryWrapper dir = newMockDirectory(); @@ -406,4 +413,29 @@ public void testMMapHackSupported() throws Exception { // add assume's here if needed for certain platforms, but we should know if it does not work. assertTrue("MMapDirectory does not support unmapping: " + MMapDirectory.UNMAP_NOT_SUPPORTED_REASON, MMapDirectory.UNMAP_SUPPORTED); } + + public void testIgnoreDeletes() throws IOException { + try (Directory directory = newDirectory()) { + IndexWriterConfig indexWriterConfig = newIndexWriterConfig(); + indexWriterConfig.setMergePolicy(NoMergePolicy.INSTANCE); + try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory, indexWriterConfig)) { + int numDocs = randomIntBetween(0, 32); + for (int i = 0; i < numDocs; i++) { + Document document = new Document(); + document.add(new StringField("field", Integer.toString(i), Store.NO)); + + iw.addDocument(document); + if (randomBoolean()) { + iw.deleteDocuments(new Term("field", Integer.toString(i))); + } + } + try(DirectoryReader ir = DirectoryReader.open(iw.w)) { + DirectoryReader noDeletesReader = Lucene.ignoreDeletes(ir); + assertThat(noDeletesReader.numDocs(), equalTo(noDeletesReader.maxDoc())); + assertThat(noDeletesReader.numDeletedDocs(), equalTo(0)); + } + } + + } + } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java index 7521a22133691..1cbe7b0db423a 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java @@ -6,9 +6,6 @@ package org.elasticsearch.xpack.ccr.action; import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.FilterDirectoryReader; -import org.apache.lucene.index.FilterLeafReader; -import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.ReaderUtil; @@ -22,7 +19,6 @@ import org.apache.lucene.search.SortField; import org.apache.lucene.search.SortedNumericSortField; import org.apache.lucene.search.TopDocs; -import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; @@ -297,7 +293,7 @@ static Response getOperationsBetween(IndexShard indexShard, long minSeqNo, long static List getOperationsBetween(long minSeqNo, long maxSeqNo, long byteLimit, DirectoryReader indexReader, MapperService mapperService) throws IOException { - IndexSearcher searcher = new IndexSearcher(new CCRIndexReader(indexReader)); + IndexSearcher searcher = new IndexSearcher(Lucene.ignoreDeletes(indexReader)); searcher.setQueryCache(null); MappedFieldType seqNoFieldType = mapperService.fullName(SeqNoFieldMapper.NAME); @@ -390,48 +386,4 @@ private static boolean isDeleteOperation(LeafReaderContext leafReaderContext, in return value == 1L; } - static final class CCRIndexReader extends FilterDirectoryReader { - - static final class CCRSubReaderWrapper extends SubReaderWrapper { - @Override - public LeafReader wrap(LeafReader in) { - return new FilterLeafReader(in) { - @Override - public CacheHelper getCoreCacheHelper() { - return null; - } - - @Override - public CacheHelper getReaderCacheHelper() { - return null; - } - - @Override - public int numDocs() { - return maxDoc(); - } - - @Override - public Bits getLiveDocs() { - return null; - } - }; - } - } - - CCRIndexReader(DirectoryReader in) throws IOException { - super(in, new CCRSubReaderWrapper()); - } - - @Override - protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { - return new CCRIndexReader(in); - } - - @Override - public CacheHelper getReaderCacheHelper() { - return in.getReaderCacheHelper(); - } - } - } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index d7d68e0a2dc0d..0d9f0992b4119 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -11,8 +11,6 @@ import org.apache.lucene.document.StoredField; import org.apache.lucene.document.StringField; import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.RandomIndexWriter; @@ -45,7 +43,6 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; public class ShardChangesActionTests extends ESSingleNodeTestCase { @@ -163,10 +160,6 @@ public void testReadOpsFromLuceneIndex() throws Exception { } } try(DirectoryReader ir = DirectoryReader.open(iw.w)) { - ShardChangesAction.CCRIndexReader ccrIndexReader = new ShardChangesAction.CCRIndexReader(ir); - assertThat(ccrIndexReader.numDocs(), equalTo(ccrIndexReader.maxDoc())); - assertThat(ccrIndexReader.numDeletedDocs(), equalTo(0)); - List ops = ShardChangesAction.getOperationsBetween(0, numDocs - 1, Long.MAX_VALUE, ir, mapperService); assertThat(ops.size(), equalTo(numDocs)); From 59b69e330ef105f3c6010ffdba6d95ec8dd4bf7e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 4 May 2018 13:59:45 -0400 Subject: [PATCH 03/18] use existing Lucene --- .../elasticsearch/common/lucene/Lucene.java | 53 ------------------- .../common/lucene/LuceneTests.java | 32 ----------- 2 files changed, 85 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index 1a623c948fbd2..0fbb0c0e2b23f 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -892,57 +892,4 @@ public CacheHelper getReaderCacheHelper() { public static NumericDocValuesField newSoftDeleteField() { return new NumericDocValuesField(SOFT_DELETE_FIELD, 1); } - - /** - * Wraps the provided {@link DirectoryReader} and return a new {@link DirectoryReader} instance that ignores - * deleted documents. - */ - public static DirectoryReader ignoreDeletes(DirectoryReader in) throws IOException { - return new NoDeletesIndexReader(in); - } - - private static final class NoDeletesIndexReader extends FilterDirectoryReader { - - private static final class NoDeletesIndexReaderWrapper extends SubReaderWrapper { - @Override - public LeafReader wrap(LeafReader in) { - return new FilterLeafReader(in) { - @Override - public CacheHelper getCoreCacheHelper() { - return null; - } - - @Override - public CacheHelper getReaderCacheHelper() { - return null; - } - - @Override - public int numDocs() { - return maxDoc(); - } - - @Override - public Bits getLiveDocs() { - return null; - } - }; - } - } - - private NoDeletesIndexReader(DirectoryReader in) throws IOException { - super(in, new NoDeletesIndexReaderWrapper()); - } - - @Override - protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { - return new NoDeletesIndexReader(in); - } - - @Override - public CacheHelper getReaderCacheHelper() { - return in.getReaderCacheHelper(); - } - } - } diff --git a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java index bf275ef8c9fb7..753aedea01e02 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/LuceneTests.java @@ -23,8 +23,6 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.Field.Store; -import org.apache.lucene.document.NumericDocValuesField; -import org.apache.lucene.document.StoredField; import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.DirectoryReader; @@ -44,9 +42,6 @@ import org.apache.lucene.store.MMapDirectory; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.Bits; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -58,8 +53,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; -import static org.hamcrest.Matchers.equalTo; - public class LuceneTests extends ESTestCase { public void testWaitForIndex() throws Exception { final MockDirectoryWrapper dir = newMockDirectory(); @@ -413,29 +406,4 @@ public void testMMapHackSupported() throws Exception { // add assume's here if needed for certain platforms, but we should know if it does not work. assertTrue("MMapDirectory does not support unmapping: " + MMapDirectory.UNMAP_NOT_SUPPORTED_REASON, MMapDirectory.UNMAP_SUPPORTED); } - - public void testIgnoreDeletes() throws IOException { - try (Directory directory = newDirectory()) { - IndexWriterConfig indexWriterConfig = newIndexWriterConfig(); - indexWriterConfig.setMergePolicy(NoMergePolicy.INSTANCE); - try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory, indexWriterConfig)) { - int numDocs = randomIntBetween(0, 32); - for (int i = 0; i < numDocs; i++) { - Document document = new Document(); - document.add(new StringField("field", Integer.toString(i), Store.NO)); - - iw.addDocument(document); - if (randomBoolean()) { - iw.deleteDocuments(new Term("field", Integer.toString(i))); - } - } - try(DirectoryReader ir = DirectoryReader.open(iw.w)) { - DirectoryReader noDeletesReader = Lucene.ignoreDeletes(ir); - assertThat(noDeletesReader.numDocs(), equalTo(noDeletesReader.maxDoc())); - assertThat(noDeletesReader.numDeletedDocs(), equalTo(0)); - } - } - - } - } } From 1b690935f181a1ce31fabf4e78071f186ca68d72 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 4 May 2018 13:59:51 -0400 Subject: [PATCH 04/18] Move to lucene snapshot --- .../elasticsearch/index/engine/Engine.java | 7 + .../index/engine/InternalEngine.java | 38 +++ .../index/engine/LuceneChangesSnapshot.java | 216 ++++++++++++++++++ .../elasticsearch/index/shard/IndexShard.java | 14 +- .../engine/LuceneChangesSnapshotTests.java | 130 +++++++++++ .../index/engine/EngineTestCase.java | 2 +- .../xpack/ccr/action/ShardChangesAction.java | 138 +---------- .../ccr/action/ShardChangesActionTests.java | 112 +-------- 8 files changed, 416 insertions(+), 241 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index 179134699925c..d0ea8fb2ab2a9 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -58,6 +58,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; @@ -609,6 +610,12 @@ public Translog.Location getTranslogLastWriteLocation() { return getTranslog().getLastWriteLocation(); } + /** + * Creates a new "translog" snapshot containing changes in the requesting range from the Lucene index + */ + public abstract Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService mapperService, + long minSeqNo, long maxSeqNo, boolean requiredFullRange) throws IOException; + protected final void ensureOpen(Exception suppressed) { if (isClosed.get()) { AlreadyClosedException ace = new AlreadyClosedException(shardId + " engine is closed", failedEngine.get()); diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 8f0260f7c4788..92ea64e1c080a 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -65,6 +65,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; @@ -145,6 +146,7 @@ public class InternalEngine extends Engine { private final CounterMetric numDocUpdates = new CounterMetric(); private final NumericDocValuesField softDeleteField = Lucene.newSoftDeleteField(); private final boolean softDeleteEnabled; + private final LastRefreshedCheckpointListener lastRefreshedCheckpointListener; /** * How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh. IndexingMemoryController polls this @@ -221,6 +223,8 @@ public InternalEngine(EngineConfig engineConfig) { for (ReferenceManager.RefreshListener listener: engineConfig.getInternalRefreshListener()) { this.internalSearcherManager.addListener(listener); } + this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getCheckpoint()); + this.internalSearcherManager.addListener(lastRefreshedCheckpointListener); success = true; } finally { if (success == false) { @@ -2325,6 +2329,22 @@ long getNumDocUpdates() { return numDocUpdates.count(); } + /** + * Creates a new "translog" snapshot containing changes between minSeqNo and maxSeqNo + * from the Lucene index. + */ + public Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService mapperService, + long minSeqNo, long maxSeqNo) throws IOException { + // TODO: Should we defer the refresh until we really need it? + ensureOpen(); + if (lastRefreshedCheckpointListener.refreshedLocalCheckpoint.get() < maxSeqNo) { + refresh(source, SearcherScope.INTERNAL); + } + return new LuceneChangesSnapshot(() -> acquireSearcher(source, SearcherScope.INTERNAL), + mapperService, minSeqNo, maxSeqNo, true, () -> { + }); + } + @Override public boolean isRecovering() { return pendingTranslogRecovery.get(); @@ -2371,4 +2391,22 @@ public long softUpdateDocuments(Term term, Iterable Math.max(prev, pendingCheckpoint)); + } + } + } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java new file mode 100644 index 0000000000000..b5488b2a7cf28 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -0,0 +1,216 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.ReaderUtil; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.SortedNumericSortField; +import org.apache.lucene.search.TopDocs; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.fieldvisitor.FieldsVisitor; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.VersionFieldMapper; +import org.elasticsearch.index.translog.Translog; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.function.Supplier; + +/** + * A {@link Translog.Snapshot} from changes in a Lucene index + */ +final class LuceneChangesSnapshot implements Translog.Snapshot { + private final long fromSeqNo, toSeqNo; + private long lastSeenSeqNo; + private int skippedOperations; + private final boolean requiredFullRange; + + private final IndexSearcher searcher; + private final MapperService mapperService; + private int docIndex; + private final TopDocs topDocs; + + private final Closeable onClose; + + /** + * Creates a new "translog" snapshot from Lucene including all operations whose seq# between the specified range. + * + * @param searcherFactory the engine searcher factory (prefer the internal searcher) + * @param mapperService the mapper service which will be mainly used to resolve the document's type and uid + * @param fromSeqNo the min requesting seq# - inclusive + * @param toSeqNo the maximum requesting seq# - inclusive + * @param requiredFullRange if true, the snapshot must contains all seq# between fromSeqNo and toSeqNo + * @param onClose a callback to be called when this snapshot is closed + */ + LuceneChangesSnapshot(Supplier searcherFactory, MapperService mapperService, + long fromSeqNo, long toSeqNo, boolean requiredFullRange, Closeable onClose) throws IOException { + if (fromSeqNo < 0 || toSeqNo < 0 || fromSeqNo > toSeqNo) { + throw new IllegalArgumentException("Invalid range; from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "]"); + } + this.mapperService = mapperService; + this.fromSeqNo = fromSeqNo; + this.toSeqNo = toSeqNo; + this.lastSeenSeqNo = fromSeqNo - 1; + this.requiredFullRange = requiredFullRange; + boolean success = false; + final Engine.Searcher engineSearcher = searcherFactory.get(); + try { + this.searcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); + this.topDocs = searchOperations(searcher); + success = true; + this.onClose = () -> IOUtils.close(onClose, engineSearcher); + } finally { + if (success == false) { + IOUtils.close(engineSearcher); + } + } + } + + @Override + public void close() throws IOException { + onClose.close(); + } + + @Override + public int totalOperations() { + return Math.toIntExact(topDocs.totalHits); + } + + @Override + public int overriddenOperations() { + return skippedOperations; + } + + @Override + public Translog.Operation next() throws IOException { + final Translog.Operation op = nextOp(); + if (requiredFullRange == false || lastSeenSeqNo == toSeqNo) { + if (op != null){ + lastSeenSeqNo = op.seqNo(); + } + return op; + } + final long expectedSeqNo = lastSeenSeqNo + 1; + if (op == null || op.seqNo() != expectedSeqNo) { + throw new IllegalStateException("not all operations between min_seqno [" + fromSeqNo + "] " + + "and max_seqno [" + toSeqNo + "] found; expected seqno [" + expectedSeqNo + "]; found [" + op + "]"); + } + lastSeenSeqNo = op.seqNo(); + return op; + } + + private Translog.Operation nextOp() throws IOException { + final ScoreDoc[] scoreDocs = topDocs.scoreDocs; + for (; docIndex < scoreDocs.length; docIndex++) { + if (scoreDocs[docIndex].doc == DocIdSetIterator.NO_MORE_DOCS) { + return null; + } + final Translog.Operation op = readDocAsOperation(scoreDocs[docIndex].doc); + if (op != null) { + return op; + } + } + return null; + } + + private TopDocs searchOperations(IndexSearcher searcher) throws IOException { + final Query rangeQuery = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, fromSeqNo, toSeqNo); + final Sort sortedBySeqNoThenByTerm = new Sort( + new SortedNumericSortField(SeqNoFieldMapper.NAME, SortField.Type.LONG), + new SortedNumericSortField(SeqNoFieldMapper.PRIMARY_TERM_NAME, SortField.Type.LONG, true) + ); + return searcher.search(rangeQuery, Integer.MAX_VALUE, sortedBySeqNoThenByTerm); + } + + private Translog.Operation readDocAsOperation(int docID) throws IOException { + final List leaves = searcher.getIndexReader().leaves(); + final LeafReaderContext leaf = leaves.get(ReaderUtil.subIndex(docID, leaves)); + final int segmentDocID = docID - leaf.docBase; + final long seqNo = readNumericDV(leaf, SeqNoFieldMapper.NAME, segmentDocID); + + // This operation has seen and will be skipped anyway - do not visit other fields. + if (seqNo == lastSeenSeqNo) { + skippedOperations++; + return null; + } + + final long primaryTerm = readNumericDV(leaf, SeqNoFieldMapper.PRIMARY_TERM_NAME, segmentDocID); + final FieldsVisitor fields = new FieldsVisitor(true); + searcher.doc(docID, fields); + fields.postProcess(mapperService); + + final Translog.Operation op; + final boolean isTombstone = isTombstoneOperation(leaf, segmentDocID); + if (isTombstone && fields.uid() == null) { + op = new Translog.NoOp(seqNo, primaryTerm, ""); // TODO: store reason in ignored fields? + assert readNumericDV(leaf, Lucene.SOFT_DELETE_FIELD, segmentDocID) == 1 + : "Noop operation but soft_deletes field is not set [" + op + "]"; + } else { + final String id = fields.uid().id(); + final String type = fields.uid().type(); + final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); + final long version = readNumericDV(leaf, VersionFieldMapper.NAME, segmentDocID); + if (isTombstone) { + op = new Translog.Delete(type, id, uid, seqNo, primaryTerm, version, VersionType.INTERNAL); + assert readNumericDV(leaf, Lucene.SOFT_DELETE_FIELD, segmentDocID) == 1 + : "Delete operation but soft_deletes field is not set [" + op + "]"; + } else { + final BytesReference source = fields.source(); + // TODO: pass the latest timestamp from engine. + final long autoGeneratedIdTimestamp = -1; + op = new Translog.Index(type, id, seqNo, primaryTerm, version, VersionType.INTERNAL, + source.toBytesRef().bytes, fields.routing(), autoGeneratedIdTimestamp); + } + } + return op; + } + + private boolean isTombstoneOperation(LeafReaderContext leaf, int segmentDocID) throws IOException { + final NumericDocValues tombstoneDV = leaf.reader().getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); + if (tombstoneDV != null && tombstoneDV.advanceExact(segmentDocID)) { + return tombstoneDV.longValue() == 1; + } + return false; + } + + private long readNumericDV(LeafReaderContext leaf, String field, int segmentDocID) throws IOException { + final NumericDocValues dv = leaf.reader().getNumericDocValues(field); + if (dv == null || dv.advanceExact(segmentDocID) == false) { + throw new IllegalStateException("DocValues for field [" + field + "] is not found"); + } + return dv.longValue(); + } +} diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 6f0e1fc7ce54d..06b2fe7aa6101 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1595,11 +1595,7 @@ public Closeable acquireTranslogRetentionLock() { * The caller has to close the returned snapshot after finishing the reading. */ public Translog.Snapshot newTranslogSnapshotFromMinSeqNo(long minSeqNo) throws IOException { - return newTranslogSnapshotBetween(minSeqNo, Long.MAX_VALUE); - } - - public Translog.Snapshot newTranslogSnapshotBetween(long minSeqNo, long maxSeqNo) throws IOException { - return getEngine().newTranslogSnapshotBetween(minSeqNo, maxSeqNo); + return getEngine().newTranslogSnapshotBetween(minSeqNo, Long.MAX_VALUE); } /** @@ -1609,6 +1605,14 @@ public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { return getEngine().estimateTranslogOperationsFromMinSeq(minSeqNo); } + /** + * Creates a new "translog" snapshot for reading translog operations between the requesting range. + * The caller has to close the returned snapshot after finishing the reading. + */ + public Translog.Snapshot newLuceneChangesSnapshot(String source, long minSeqNo, long maxSeqNo, boolean requiredFullRange) throws IOException { + return getEngine().newLuceneChangesSnapshot(source, mapperService, minSeqNo, maxSeqNo, requiredFullRange); + } + public List segments(boolean verbose) { return getEngine().segments(verbose); } diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java new file mode 100644 index 0000000000000..560981187afc9 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -0,0 +1,130 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.translog.Translog; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; + +public class LuceneChangesSnapshotTests extends EngineTestCase { + + public void testEmptyEngine() throws Exception { + MapperService mapper = createMapperService("test"); + long fromSeqNo = randomNonNegativeLong(); + long toSeqNo = randomLongBetween(fromSeqNo, Long.MAX_VALUE); + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, true)) { + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat(error.getMessage(), + containsString("not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); + } + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, false)) { + assertThat(drainAll(snapshot), empty()); + } + } + + public void testRequiredFullRange() throws Exception { + MapperService mapper = createMapperService("test"); + int numOps = between(0, 100); + for (int i = 0; i < numOps; i++) { + String id = Integer.toString(randomIntBetween(i, i + 5)); + ParsedDocument doc = createParsedDoc(id, null); + if (randomBoolean()) { + engine.index(indexForDoc(doc)); + } else { + engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); + } + if (rarely()) { + engine.flush(); + } + } + int iters = between(1, 10); + for (int i = 0; i < iters; i++) { + + } + } + + public void testDedupByPrimaryTerm() throws Exception { + MapperService mapper = createMapperService("test"); + Map latestOperations = new HashMap<>(); + int numOps = scaledRandomIntBetween(100, 2000); + List seqNos = LongStream.range(0, numOps).boxed().collect(Collectors.toList()); + Randomness.shuffle(seqNos); + for (int i = 0; i < numOps; i++) { + if (randomBoolean()) { + primaryTerm.set(randomLongBetween(primaryTerm.get(), Long.MAX_VALUE)); + engine.rollTranslogGeneration(); + } + if (randomBoolean()) { + primaryTerm.set(randomLongBetween(1, primaryTerm.get())); + } + String id = Integer.toString(randomIntBetween(i, i + 5)); + ParsedDocument doc = createParsedDoc(id, null); + final long seqNo = seqNos.remove(0); + if (randomBoolean()) { + engine.index(replicaIndexForDoc(doc, randomNonNegativeLong(), seqNo, false)); + } else { + engine.delete(replicaDeleteForDoc(doc.id(), randomNonNegativeLong(), seqNo, threadPool.relativeTimeInMillis())); + } + latestOperations.put(seqNo, primaryTerm.get()); + if (rarely()) { + engine.flush(); + } + } + final boolean requiredFullRange = randomBoolean(); + long fromSeqNo = randomLongBetween(0, numOps); + long toSeqNo = randomLongBetween(fromSeqNo, requiredFullRange ? numOps : numOps * 2); + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, requiredFullRange)) { + List ops = drainAll(snapshot); + for (Translog.Operation op : ops) { + assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); + } + } + } + + public void testOutOfOrderOperations() throws Exception { + + } + + public void testUpdateAndReplicateOperationsConcurrently() throws Exception { + // + } + + List drainAll(Translog.Snapshot snapshot) throws IOException { + List operations = new ArrayList<>(); + Translog.Operation op; + while ((op = snapshot.next()) != null) { + operations.add(op); + } + return operations; + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 76f5f7e51fcd0..1dfe8ad0b1cec 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -561,7 +561,7 @@ protected Engine.Index replicaIndexForDoc(ParsedDocument doc, long version, long } protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, long startTime) { - return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, VersionType.EXTERNAL, + return new Engine.Delete("test", id, newUid(id), seqNo, primaryTerm.get(), version, VersionType.EXTERNAL, Engine.Operation.Origin.REPLICA, startTime); } protected static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java index 1cbe7b0db423a..0e61521f95f21 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardChangesAction.java @@ -5,21 +5,6 @@ */ package org.elasticsearch.xpack.ccr.action; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.ReaderUtil; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.BooleanClause.Occur; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; -import org.apache.lucene.search.SortedNumericSortField; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; @@ -35,19 +20,8 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.fieldvisitor.FieldsVisitor; -import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardNotStartedException; import org.elasticsearch.index.shard.IndexShardState; @@ -280,110 +254,18 @@ static Response getOperationsBetween(IndexShard indexShard, long minSeqNo, long if (indexShard.state() != IndexShardState.STARTED) { throw new IndexShardNotStartedException(indexShard.shardId(), indexShard.state()); } - - // TODO: Somehow this needs to be an internal refresh (SearcherScope.INTERNAL) - indexShard.refresh("shard_changes_api"); - // TODO: Somehow this needs to acquire an internal searcher (SearcherScope.INTERNAL) - try (Engine.Searcher searcher = indexShard.acquireSearcher("shard_changes_api")) { - List operations = getOperationsBetween(minSeqNo, maxSeqNo, byteLimit, - searcher.getDirectoryReader(), indexShard.mapperService()); - return new Response(operations.toArray(EMPTY_OPERATIONS_ARRAY)); - } - } - - static List getOperationsBetween(long minSeqNo, long maxSeqNo, long byteLimit, - DirectoryReader indexReader, MapperService mapperService) throws IOException { - IndexSearcher searcher = new IndexSearcher(Lucene.ignoreDeletes(indexReader)); - searcher.setQueryCache(null); - - MappedFieldType seqNoFieldType = mapperService.fullName(SeqNoFieldMapper.NAME); - assert mapperService.types().size() == 1; - String type = mapperService.types().iterator().next(); - - int size = ((int) (maxSeqNo - minSeqNo)) + 1; - Sort sort = new Sort(new SortedNumericSortField(seqNoFieldType.name(), SortField.Type.LONG)); - Query query; - if (mapperService.hasNested()) { - BooleanQuery.Builder builder = new BooleanQuery.Builder(); - builder.add(Queries.newNonNestedFilter(mapperService.getIndexSettings().getIndexVersionCreated()), Occur.FILTER); - builder.add(seqNoFieldType.rangeQuery(minSeqNo, maxSeqNo, true, true, null, null, null, null), Occur.FILTER); - query = builder.build(); - } else { - query = seqNoFieldType.rangeQuery(minSeqNo, maxSeqNo, true, true, null, null, null, null); - } - TopDocs topDocs = searcher.search(query, size, sort); - if (topDocs.scoreDocs.length != size) { - String message = "not all operations between min_seq_no [" + minSeqNo + "] and max_seq_no [" + maxSeqNo + "] found"; - throw new IllegalStateException(message); - } - - long seenBytes = 0; + int seenBytes = 0; final List operations = new ArrayList<>(); - for (ScoreDoc scoreDoc : topDocs.scoreDocs) { - FieldsVisitor fieldsVisitor = new FieldsVisitor(true); - searcher.doc(scoreDoc.doc, fieldsVisitor); - fieldsVisitor.postProcess(mapperService); - - String id = fieldsVisitor.uid().id(); - String routing = fieldsVisitor.routing(); - if (fieldsVisitor.source() == null) { - throw new IllegalArgumentException("no source found for document with id [" + id + "]"); - } - byte[] source = fieldsVisitor.source().toBytesRef().bytes; - - // TODO: optimize this so that we fetch doc values data in segment and doc id order (see: DocValueFieldsFetchSubPhase): - int leafReaderIndex = ReaderUtil.subIndex(scoreDoc.doc, searcher.getIndexReader().leaves()); - LeafReaderContext leafReaderContext = searcher.getIndexReader().leaves().get(leafReaderIndex); - int segmentDocId = scoreDoc.doc - leafReaderContext.docBase; - long version = readNumberDvValue(leafReaderContext, VersionFieldMapper.NAME, segmentDocId); - long seqNo = readNumberDvValue(leafReaderContext, SeqNoFieldMapper.NAME, segmentDocId); - long primaryTerm = readNumberDvValue(leafReaderContext, SeqNoFieldMapper.PRIMARY_TERM_NAME, segmentDocId); - - // TODO: handle NOOPs - // NOTE: versionType can always be INTERNAL, - // because version logic has already been taken care of when indexing into leader shard. - final VersionType versionType = VersionType.INTERNAL; - final Translog.Operation op; - if (isDeleteOperation(leafReaderContext, segmentDocId)) { - BytesRef idBytes = Uid.encodeId(id); - Term uidForDelete = new Term(IdFieldMapper.NAME, idBytes); - op = new Translog.Delete(type, id, uidForDelete, seqNo, primaryTerm, version, versionType); - } else { - // NOTE: autoGeneratedIdTimestamp can always be -1, - // because auto id generation has already been performed when inxdexing into leader shard. - final int autoGeneratedId = -1; - op = new Translog.Index(type, id, seqNo, primaryTerm, version, versionType, source, routing, autoGeneratedId); - } - seenBytes += op.estimateSize(); - operations.add(op); - if (seenBytes > byteLimit) { - return operations; + try (Translog.Snapshot snapshot = indexShard.newLuceneChangesSnapshot("ccr", minSeqNo, maxSeqNo, true)) { + Translog.Operation op; + while ((op = snapshot.next()) != null) { + operations.add(op); + seenBytes += op.estimateSize(); + if (seenBytes > byteLimit) { + break; + } } } - return operations; + return new Response(operations.toArray(EMPTY_OPERATIONS_ARRAY)); } - - private static long readNumberDvValue(LeafReaderContext leafReaderContext, String fieldName, int segmentDocId) throws IOException { - NumericDocValues versionDvField = leafReaderContext.reader().getNumericDocValues(fieldName); - assert versionDvField != null : fieldName + " field is missing"; - boolean advanced = versionDvField.advanceExact(segmentDocId); - assert advanced; - return versionDvField.longValue(); - } - - private static boolean isDeleteOperation(LeafReaderContext leafReaderContext, int segmentDocId) throws IOException { - NumericDocValues softDeleteField = leafReaderContext.reader().getNumericDocValues(Lucene.SOFT_DELETE_FIELD); - if (softDeleteField == null) { - return false; - } - - boolean advanced = softDeleteField.advanceExact(segmentDocId); - if (advanced == false) { - return false; - } - - long value = softDeleteField.longValue(); - return value == 1L; - } - } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index 0d9f0992b4119..92f1d09309264 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -5,32 +5,12 @@ */ package org.elasticsearch.xpack.ccr.action; -import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; -import org.apache.lucene.document.NumericDocValuesField; -import org.apache.lucene.document.StoredField; -import org.apache.lucene.document.StringField; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.NoMergePolicy; -import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.index.Term; -import org.apache.lucene.store.Directory; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; -import org.elasticsearch.index.mapper.SourceFieldMapper; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.mapper.VersionFieldMapper; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardNotStartedException; import org.elasticsearch.index.translog.Translog; @@ -39,10 +19,10 @@ import java.util.Arrays; import java.util.List; -import java.util.Set; import java.util.stream.Collectors; import java.util.stream.LongStream; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; public class ShardChangesActionTests extends ESSingleNodeTestCase { @@ -51,7 +31,6 @@ public void testGetOperationsBetween() throws Exception { final Settings settings = Settings.builder() .put("index.number_of_shards", 1) .put("index.number_of_replicas", 0) - .put("index.translog.generation_threshold_size", new ByteSizeValue(randomIntBetween(8, 64), ByteSizeUnit.KB)) .build(); final IndexService indexService = createIndex("index", settings); @@ -66,27 +45,22 @@ public void testGetOperationsBetween() throws Exception { for (int iter = 0; iter < iters; iter++) { int min = randomIntBetween(0, numWrites - 1); int max = randomIntBetween(min, numWrites - 1); - final ShardChangesAction.Response r = ShardChangesAction.getOperationsBetween(indexShard, min, max, Long.MAX_VALUE); - /* - * We are not guaranteed that operations are returned to us in order they are in the translog (if our read crosses multiple - * generations) so the best we can assert is that we see the expected operations. - */ - final Set seenSeqNos = Arrays.stream(r.getOperations()).map(Translog.Operation::seqNo).collect(Collectors.toSet()); - final Set expectedSeqNos = LongStream.range(min, max + 1).boxed().collect(Collectors.toSet()); + final List seenSeqNos = Arrays.stream(r.getOperations()).map(Translog.Operation::seqNo).collect(Collectors.toList()); + final List expectedSeqNos = LongStream.range(min, max + 1).boxed().collect(Collectors.toList()); assertThat(seenSeqNos, equalTo(expectedSeqNos)); } // get operations for a range no operations exists: Exception e = expectThrows(IllegalStateException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites, numWrites + 1, Long.MAX_VALUE)); - assertThat(e.getMessage(), equalTo("not all operations between min_seq_no [" + numWrites + "] and max_seq_no [" + + assertThat(e.getMessage(), containsString("not all operations between min_seqno [" + numWrites + "] and max_seqno [" + (numWrites + 1) +"] found")); // get operations for a range some operations do not exist: e = expectThrows(IllegalStateException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites - 10, numWrites + 10, Long.MAX_VALUE)); - assertThat(e.getMessage(), equalTo("not all operations between min_seq_no [" + (numWrites - 10) + "] and max_seq_no [" + + assertThat(e.getMessage(), containsString("not all operations between min_seqno [" + (numWrites - 10) + "] and max_seqno [" + (numWrites + 10) +"] found")); } @@ -127,80 +101,4 @@ public void testGetOperationsBetweenExceedByteLimit() throws Exception { assertThat(r.getOperations()[11].seqNo(), equalTo(11L)); } - public void testReadOpsFromLuceneIndex() throws Exception { - // MapperService is needed by FieldsVisitor.postProcess(...) - final Settings settings = Settings.builder() - .put("index.number_of_shards", 1) - .put("index.number_of_replicas", 0) - .build(); - final MapperService mapperService = createIndex("index", settings, "_doc").mapperService(); - - try (Directory directory = newDirectory()) { - IndexWriterConfig indexWriterConfig = newIndexWriterConfig(); - indexWriterConfig.setMergePolicy(NoMergePolicy.INSTANCE); - try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory, indexWriterConfig)) { - int numDocs = randomIntBetween(10, 100); - for (int i = 0; i < numDocs; i++) { - SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); - seqID.seqNo.setLongValue(i); - seqID.seqNoDocValue.setLongValue(i); - seqID.primaryTerm.setLongValue(1); - - Document document = new Document(); - document.add(seqID.seqNo); - document.add(seqID.seqNoDocValue); - document.add(seqID.primaryTerm); - document.add(new NumericDocValuesField(VersionFieldMapper.NAME, 1L)); - document.add(new StoredField(SourceFieldMapper.NAME, new BytesRef("{}"))); - document.add(new StringField(IdFieldMapper.NAME, Uid.encodeId(Integer.toString(i)), Field.Store.YES)); - - iw.addDocument(document); - if (randomBoolean()) { - iw.deleteDocuments(new Term(IdFieldMapper.NAME, Integer.toString(i))); - } - } - try(DirectoryReader ir = DirectoryReader.open(iw.w)) { - List ops = - ShardChangesAction.getOperationsBetween(0, numDocs - 1, Long.MAX_VALUE, ir, mapperService); - assertThat(ops.size(), equalTo(numDocs)); - } - } - - } - } - - public void testReadOpsFromLuceneIndex_missingFields() throws Exception { - // MapperService is needed by FieldsVisitor.postProcess(...) - final Settings settings = Settings.builder() - .put("index.number_of_shards", 1) - .put("index.number_of_replicas", 0) - .build(); - final MapperService mapperService = createIndex("index", settings, "_doc").mapperService(); - - try (Directory directory = newDirectory()) { - IndexWriterConfig indexWriterConfig = newIndexWriterConfig(); - indexWriterConfig.setMergePolicy(NoMergePolicy.INSTANCE); - try (RandomIndexWriter iw = new RandomIndexWriter(random(), directory, indexWriterConfig)) { - SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); - seqID.seqNo.setLongValue(0); - seqID.seqNoDocValue.setLongValue(0); - seqID.primaryTerm.setLongValue(1); - - Document document = new Document(); - document.add(seqID.seqNo); - document.add(seqID.seqNoDocValue); - document.add(seqID.primaryTerm); - document.add(new NumericDocValuesField(VersionFieldMapper.NAME, 1L)); - document.add(new StoredField(IdFieldMapper.NAME, Uid.encodeId("0"))); - iw.addDocument(document); - - try(DirectoryReader ir = DirectoryReader.open(iw.w)) { - Exception e = expectThrows(IllegalArgumentException.class, - () -> ShardChangesAction.getOperationsBetween(0, 0, Long.MAX_VALUE, ir, mapperService)); - assertThat(e.getMessage(), equalTo("no source found for document with id [0]")); - } - } - } - } - } From f86dc1deb7fd8cbda089a871827ce48c9aabeba1 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 4 May 2018 23:31:18 -0400 Subject: [PATCH 05/18] Use the changes snapshot --- .../index/engine/InternalEngine.java | 11 +-- .../index/engine/EngineTestCase.java | 78 ++----------------- 2 files changed, 8 insertions(+), 81 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 07a084e8c33a9..b1d74a3d58fde 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -2346,20 +2346,15 @@ long getNumDocUpdates() { return numDocUpdates.count(); } - /** - * Creates a new "translog" snapshot containing changes between minSeqNo and maxSeqNo - * from the Lucene index. - */ public Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService mapperService, - long minSeqNo, long maxSeqNo) throws IOException { + long minSeqNo, long maxSeqNo, boolean requiredFullRange) throws IOException { // TODO: Should we defer the refresh until we really need it? ensureOpen(); if (lastRefreshedCheckpointListener.refreshedLocalCheckpoint.get() < maxSeqNo) { refresh(source, SearcherScope.INTERNAL); } - return new LuceneChangesSnapshot(() -> acquireSearcher(source, SearcherScope.INTERNAL), - mapperService, minSeqNo, maxSeqNo, true, () -> { - }); + return new LuceneChangesSnapshot(() -> acquireSearcher(source, SearcherScope.INTERNAL), mapperService, + minSeqNo, maxSeqNo, requiredFullRange, () -> { }); } @Override diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 9025ff89ca5aa..7b8730a862e89 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -28,21 +28,14 @@ import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.ReaderUtil; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.ReferenceManager; -import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; -import org.apache.lucene.search.SortedNumericSortField; import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; @@ -67,7 +60,6 @@ import org.elasticsearch.index.MapperTestUtils; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.codec.CodecService; -import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; @@ -116,7 +108,6 @@ import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.notNullValue; public abstract class EngineTestCase extends ESTestCase { @@ -707,76 +698,17 @@ public static void assertOpsOnReplica( * The returned operations are sorted and de-duplicated, thus each sequence number will be have at most one operation. */ public static List readAllOperationsInLucene(Engine engine, MapperService mapper) throws IOException { - engine.refresh("test"); final List operations = new ArrayList<>(); - try (Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL)) { - final IndexSearcher indexSearcher = new IndexSearcher(Lucene.wrapAllDocsLive(searcher.getDirectoryReader())); - final Sort sortedBySeqNoThenByTerm = new Sort( - new SortedNumericSortField(SeqNoFieldMapper.NAME, SortField.Type.LONG), - new SortedNumericSortField(SeqNoFieldMapper.PRIMARY_TERM_NAME, SortField.Type.LONG, true) - ); - final TopDocs allDocs = indexSearcher.search(new MatchAllDocsQuery(), Integer.MAX_VALUE, sortedBySeqNoThenByTerm); - long lastSeenSeqNo = SequenceNumbers.NO_OPS_PERFORMED; - for (ScoreDoc scoreDoc : allDocs.scoreDocs) { - final Translog.Operation op = readOperationInLucene(indexSearcher, mapper, scoreDoc.doc); - if (op.seqNo() != lastSeenSeqNo) { - operations.add(op); - lastSeenSeqNo = op.seqNo(); - } + long maxSeqNo = Math.max(0, engine.getLocalCheckpointTracker().getMaxSeqNo()); + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, 0, maxSeqNo, false)) { + Translog.Operation op; + while ((op = snapshot.next()) != null){ + operations.add(op); } } return operations; } - private static Translog.Operation readOperationInLucene(IndexSearcher searcher, MapperService mapper, int docID) throws IOException { - final List leaves = searcher.getIndexReader().leaves(); - final int leafIndex = ReaderUtil.subIndex(docID, leaves); - final int segmentDocID = docID - leaves.get(leafIndex).docBase; - final long seqNo = readNumericDV(leaves.get(leafIndex), SeqNoFieldMapper.NAME, segmentDocID); - final long primaryTerm = readNumericDV(leaves.get(leafIndex), SeqNoFieldMapper.PRIMARY_TERM_NAME, segmentDocID); - final FieldsVisitor fields = new FieldsVisitor(true); - searcher.doc(docID, fields); - fields.postProcess(mapper); - final Translog.Operation op; - final boolean isTombstone = isTombstoneOperation(leaves.get(leafIndex), segmentDocID); - if (isTombstone && fields.uid() == null) { - op = new Translog.NoOp(seqNo, primaryTerm, ""); - assert readNumericDV(leaves.get(leafIndex), Lucene.SOFT_DELETE_FIELD, segmentDocID) == 1 - : "Noop operation but soft_deletes field is not set"; - } else { - final String id = fields.uid().id(); - final String type = fields.uid().type(); - final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); - final long version = readNumericDV(leaves.get(leafIndex), VersionFieldMapper.NAME, segmentDocID); - if (isTombstone) { - op = new Translog.Delete(type, id, uid, seqNo, primaryTerm, version, VersionType.INTERNAL); - assert readNumericDV(leaves.get(leafIndex), Lucene.SOFT_DELETE_FIELD, segmentDocID) == 1 - : "Delete operation but soft_deletes field is not set"; - } else { - final BytesReference source = fields.source(); - op = new Translog.Index(type, id, seqNo, primaryTerm, version, VersionType.INTERNAL, source.toBytesRef().bytes, - fields.routing(), -1); - } - } - return op; - } - - private static boolean isTombstoneOperation(LeafReaderContext leaf, int segmentDocID) throws IOException { - final NumericDocValues tombstoneDV = leaf.reader().getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); - if (tombstoneDV != null && tombstoneDV.advanceExact(segmentDocID)) { - return tombstoneDV.longValue() == 1; - } - return false; - } - - private static long readNumericDV(LeafReaderContext leaf, String field, int segmentDocID) throws IOException { - final NumericDocValues dv = leaf.reader().getNumericDocValues(field); - if (dv == null || dv.advanceExact(segmentDocID) == false) { - throw new IllegalStateException("DocValues for field [" + field + "] is not found"); - } - return dv.longValue(); - } - /** * Asserts the provided engine has a consistent document history between translog and Lucene index. */ From 1fe57c0b8468434746843885ab803994b7fbf5e7 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 5 May 2018 00:39:53 -0400 Subject: [PATCH 06/18] More test --- .../index/engine/InternalEngine.java | 10 +- .../index/engine/InternalEngineTests.java | 46 --------- .../engine/LuceneChangesSnapshotTests.java | 97 +++++++++++++++++-- 3 files changed, 99 insertions(+), 54 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index b1d74a3d58fde..3c5b5eafbeb74 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -2350,7 +2350,7 @@ public Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService m long minSeqNo, long maxSeqNo, boolean requiredFullRange) throws IOException { // TODO: Should we defer the refresh until we really need it? ensureOpen(); - if (lastRefreshedCheckpointListener.refreshedLocalCheckpoint.get() < maxSeqNo) { + if (lastRefreshedCheckpoint() < maxSeqNo) { refresh(source, SearcherScope.INTERNAL); } return new LuceneChangesSnapshot(() -> acquireSearcher(source, SearcherScope.INTERNAL), mapperService, @@ -2404,6 +2404,12 @@ public long softUpdateDocuments(Term term, Iterable operations = generateSingleDocHistory(false, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "1"); - assertOperationHistoryInLucene(operations); - } - - public void testLuceneHistoryOnReplica() throws Exception { - final List operations = generateSingleDocHistory(true, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); - Randomness.shuffle(operations); - assertOperationHistoryInLucene(operations); - } - - private void assertOperationHistoryInLucene(List operations) throws IOException { - final MergePolicy keepSoftDeleteDocsMP = new SoftDeletesRetentionMergePolicy( - Lucene.SOFT_DELETE_FIELD, () -> new MatchAllDocsQuery(), engine.config().getMergePolicy()); - Set expectedSeqNos = new HashSet<>(); - try (Store store = createStore(); - Engine engine = createEngine(config(defaultSettings, store, createTempDir(), keepSoftDeleteDocsMP, null))) { - for (Engine.Operation op : operations) { - if (op instanceof Engine.Index) { - Engine.IndexResult indexResult = engine.index((Engine.Index) op); - assertThat(indexResult.getFailure(), nullValue()); - expectedSeqNos.add(indexResult.getSeqNo()); - } else { - Engine.DeleteResult deleteResult = engine.delete((Engine.Delete) op); - assertThat(deleteResult.getFailure(), nullValue()); - expectedSeqNos.add(deleteResult.getSeqNo()); - } - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(); - } - if (rarely()) { - engine.forceMerge(true); - } - } - MapperService mapperService = createMapperService("test"); - List actualOps = readAllOperationsInLucene(engine, mapperService); - assertThat(actualOps.stream().map(o -> o.seqNo()).collect(Collectors.toList()), containsInAnyOrder(expectedSeqNos.toArray())); - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); - } - } - private static void trimUnsafeCommits(EngineConfig config) throws IOException { final Store store = config.getStore(); final TranslogConfig translogConfig = config.getTranslogConfig(); diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index 560981187afc9..63313aa0d22fd 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -19,22 +19,36 @@ package org.elasticsearch.index.engine; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.test.IndexSettingsModule; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.LongStream; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.nullValue; public class LuceneChangesSnapshotTests extends EngineTestCase { @@ -69,14 +83,25 @@ public void testRequiredFullRange() throws Exception { } int iters = between(1, 10); for (int i = 0; i < iters; i++) { - + int fromSeqNo = between(0, numOps * 2); + int toSeqNo = between(Math.max(numOps + 1, fromSeqNo), numOps * 10); + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, true)) { + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat(error.getMessage(), + containsString("not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); + } + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, false)) { + List ops = drainAll(snapshot); + int readOps = Math.min(toSeqNo, numOps) - Math.min(fromSeqNo, numOps); + assertThat(ops, hasSize(readOps)); + } } } public void testDedupByPrimaryTerm() throws Exception { MapperService mapper = createMapperService("test"); Map latestOperations = new HashMap<>(); - int numOps = scaledRandomIntBetween(100, 2000); + int numOps = scaledRandomIntBetween(10, 2000); List seqNos = LongStream.range(0, numOps).boxed().collect(Collectors.toList()); Randomness.shuffle(seqNos); for (int i = 0; i < numOps; i++) { @@ -99,6 +124,9 @@ public void testDedupByPrimaryTerm() throws Exception { if (rarely()) { engine.flush(); } + if (rarely()){ + engine.forceMerge(randomBoolean(), between(1, 2), false, false, false); + } } final boolean requiredFullRange = randomBoolean(); long fromSeqNo = randomLongBetween(0, numOps); @@ -111,19 +139,76 @@ public void testDedupByPrimaryTerm() throws Exception { } } - public void testOutOfOrderOperations() throws Exception { + public void testHistoryOnPrimary() throws Exception { + final List operations = generateSingleDocHistory(false, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), primaryTerm.get(), 10, 1000, "1"); + assertOperationHistoryInLucene(operations, true); + } + public void testHistoryOnReplica() throws Exception { + final List operations = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 1000, "2"); + Randomness.shuffle(operations); + assertOperationHistoryInLucene(operations, false); } - public void testUpdateAndReplicateOperationsConcurrently() throws Exception { - // + private void assertOperationHistoryInLucene(List operations, boolean requiredFullRange) throws IOException { + Set expectedSeqNos = new HashSet<>(); + Settings settings = Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), operations.size()) + .build(); + final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + try (Store store = createStore(); + InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get))) { + for (Engine.Operation op : operations) { + if (op instanceof Engine.Index) { + Engine.IndexResult indexResult = engine.index((Engine.Index) op); + assertThat(indexResult.getFailure(), nullValue()); + expectedSeqNos.add(indexResult.getSeqNo()); + } else { + Engine.DeleteResult deleteResult = engine.delete((Engine.Delete) op); + assertThat(deleteResult.getFailure(), nullValue()); + expectedSeqNos.add(deleteResult.getSeqNo()); + } + if (randomBoolean()) { + globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); + } + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + if (rarely()) { + engine.forceMerge(true); + } + } + long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); + long toSeqNo = requiredFullRange ? maxSeqNo : randomLongBetween(maxSeqNo, Long.MAX_VALUE); + MapperService mapperService = createMapperService("test"); + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, 0, toSeqNo, requiredFullRange)) { + List actualOps = drainAll(snapshot); + List sortedSeqNo = new ArrayList<>(expectedSeqNos); + Collections.sort(sortedSeqNo); + assertThat(actualOps.stream().map(o -> o.seqNo()).collect(Collectors.toList()), + containsInAnyOrder(sortedSeqNo.toArray())); + } + + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + } } List drainAll(Translog.Snapshot snapshot) throws IOException { List operations = new ArrayList<>(); Translog.Operation op; while ((op = snapshot.next()) != null) { - operations.add(op); + final Translog.Operation newOp = op; + assert operations.stream().allMatch(o -> o.seqNo() < newOp.seqNo()) : "Operations [" + operations + "], op [" + op + "]"; + operations.add(newOp); } return operations; } From ce6d8da32afb14b4bf0e90f981802e172f9f3610 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 5 May 2018 11:00:48 -0400 Subject: [PATCH 07/18] backout mapping changes --- .../ccr/action/FollowExistingIndexAction.java | 122 ++++++++--------- .../xpack/ccr/ShardChangesIT.java | 128 +----------------- .../FollowExistingIndexActionTests.java | 66 --------- 3 files changed, 59 insertions(+), 257 deletions(-) delete mode 100644 x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexActionTests.java diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexAction.java index f2a63908b965f..44628f1da5b3c 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexAction.java @@ -24,7 +24,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.persistent.PersistentTasksService; @@ -206,78 +205,73 @@ protected void doExecute(Request request, ActionListener listener) { */ void start(Request request, String clusterNameAlias, IndexMetaData leaderIndexMetadata, IndexMetaData followIndexMetadata, ActionListener handler) { - validate(leaderIndexMetadata, followIndexMetadata, request); - - final int numShards = followIndexMetadata.getNumberOfShards(); - final AtomicInteger counter = new AtomicInteger(numShards); - final AtomicReferenceArray responses = new AtomicReferenceArray<>(followIndexMetadata.getNumberOfShards()); - for (int i = 0; i < numShards; i++) { - final int shardId = i; - String taskId = followIndexMetadata.getIndexUUID() + "-" + shardId; - ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, - new ShardId(followIndexMetadata.getIndex(), shardId), - new ShardId(leaderIndexMetadata.getIndex(), shardId), - request.batchSize, request.concurrentProcessors, request.processorMaxTranslogBytes); - persistentTasksService.startPersistentTask(taskId, ShardFollowTask.NAME, shardFollowTask, - new ActionListener>() { - @Override - public void onResponse(PersistentTasksCustomMetaData.PersistentTask task) { - responses.set(shardId, task); - finalizeResponse(); - } + if (leaderIndexMetadata == null) { + handler.onFailure(new IllegalArgumentException("leader index [" + request.leaderIndex + "] does not exist")); + return; + } - @Override - public void onFailure(Exception e) { - responses.set(shardId, e); - finalizeResponse(); - } + if (followIndexMetadata == null) { + handler.onFailure(new IllegalArgumentException("follow index [" + request.followIndex + "] does not exist")); + return; + } - void finalizeResponse() { - Exception error = null; - if (counter.decrementAndGet() == 0) { - for (int j = 0; j < responses.length(); j++) { - Object response = responses.get(j); - if (response instanceof Exception) { - if (error == null) { - error = (Exception) response; - } else { - error.addSuppressed((Throwable) response); + if (leaderIndexMetadata.getNumberOfShards() != followIndexMetadata.getNumberOfShards()) { + handler.onFailure(new IllegalArgumentException("leader index primary shards [" + + leaderIndexMetadata.getNumberOfShards() + "] does not match with the number of " + + "shards of the follow index [" + followIndexMetadata.getNumberOfShards() + "]")); + // TODO: other validation checks + } else { + final int numShards = followIndexMetadata.getNumberOfShards(); + final AtomicInteger counter = new AtomicInteger(numShards); + final AtomicReferenceArray responses = new AtomicReferenceArray<>(followIndexMetadata.getNumberOfShards()); + for (int i = 0; i < numShards; i++) { + final int shardId = i; + String taskId = followIndexMetadata.getIndexUUID() + "-" + shardId; + ShardFollowTask shardFollowTask = new ShardFollowTask(clusterNameAlias, + new ShardId(followIndexMetadata.getIndex(), shardId), + new ShardId(leaderIndexMetadata.getIndex(), shardId), + request.batchSize, request.concurrentProcessors, request.processorMaxTranslogBytes); + persistentTasksService.startPersistentTask(taskId, ShardFollowTask.NAME, shardFollowTask, + new ActionListener>() { + @Override + public void onResponse(PersistentTasksCustomMetaData.PersistentTask task) { + responses.set(shardId, task); + finalizeResponse(); + } + + @Override + public void onFailure(Exception e) { + responses.set(shardId, e); + finalizeResponse(); + } + + void finalizeResponse() { + Exception error = null; + if (counter.decrementAndGet() == 0) { + for (int j = 0; j < responses.length(); j++) { + Object response = responses.get(j); + if (response instanceof Exception) { + if (error == null) { + error = (Exception) response; + } else { + error.addSuppressed((Throwable) response); + } } } - } - if (error == null) { - // include task ids? - handler.onResponse(new Response()); - } else { - // TODO: cancel all started tasks - handler.onFailure(error); + if (error == null) { + // include task ids? + handler.onResponse(new Response()); + } else { + // TODO: cancel all started tasks + handler.onFailure(error); + } } } } - }); + ); + } } } } - - static void validate(IndexMetaData leaderIndex, IndexMetaData followIndex, Request request) { - if (leaderIndex == null) { - throw new IllegalArgumentException("leader index [" + request.leaderIndex + "] does not exist"); - } - - if (followIndex == null) { - throw new IllegalArgumentException("follow index [" + request.followIndex + "] does not exist"); - } - - if (leaderIndex.getSettings().getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false) == false) { - throw new IllegalArgumentException("leader index [" + request.leaderIndex + "] does not have soft deletes enabled"); - } - - if (leaderIndex.getNumberOfShards() != followIndex.getNumberOfShards()) { - throw new IllegalArgumentException("leader index primary shards [" + leaderIndex.getNumberOfShards() + - "] does not match with the number of shards of the follow index [" + followIndex.getNumberOfShards() + "]"); - } - // TODO: other validation checks - } - } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java index 259818912e1f6..998ea1d4d9c28 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java @@ -16,8 +16,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.common.xcontent.support.XContentMapValues; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; @@ -142,8 +140,7 @@ public void testGetOperationsBasedOnGlobalSequenceId() throws Exception { public void testFollowIndex() throws Exception { final int numberOfPrimaryShards = randomIntBetween(1, 3); - final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, - Collections.singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, Collections.emptyMap()); assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); final String followerIndexSettings = @@ -199,70 +196,6 @@ public void testFollowIndex() throws Exception { assertBusy(assertExpectedDocumentRunnable(i)); } - final int numDocsToDelete = randomIntBetween(2, 64); - for (int i = 0; i < numDocsToDelete; i++) { - client().prepareDelete("index1", "doc", Integer.toString(i)).get(); - } - - for (int i = 0; i < numDocsToDelete; i++) { - assertDocumentDeletedRunnable(i); - } - - final UnfollowIndexAction.Request unfollowRequest = new UnfollowIndexAction.Request(); - unfollowRequest.setFollowIndex("index2"); - client().execute(UnfollowIndexAction.INSTANCE, unfollowRequest).get(); - - assertBusy(() -> { - final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); - final PersistentTasksCustomMetaData tasks = clusterState.getMetaData().custom(PersistentTasksCustomMetaData.TYPE); - assertThat(tasks.tasks().size(), equalTo(0)); - }); - } - - public void testFollowIndexWithNestedField() throws Exception { - final String leaderIndexSettings = - getIndexSettingsWithNestedMapping(1, Collections.singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); - assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); - - final String followerIndexSettings = - getIndexSettingsWithNestedMapping(1, Collections.singletonMap(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), "true")); - assertAcked(client().admin().indices().prepareCreate("index2").setSource(followerIndexSettings, XContentType.JSON)); - - ensureGreen("index1", "index2"); - - final FollowExistingIndexAction.Request followRequest = new FollowExistingIndexAction.Request(); - followRequest.setLeaderIndex("index1"); - followRequest.setFollowIndex("index2"); - client().execute(FollowExistingIndexAction.INSTANCE, followRequest).get(); - - final int numDocs = randomIntBetween(2, 64); - for (int i = 0; i < numDocs; i++) { - try (XContentBuilder builder = jsonBuilder()) { - builder.startObject(); - builder.field("field", "value"); - builder.startArray("objects"); - { - builder.startObject(); - builder.field("field", i); - builder.endObject(); - } - builder.endArray(); - builder.endObject(); - client().prepareIndex("index1", "doc", Integer.toString(i)).setSource(builder).get(); - } - } - - for (int i = 0; i < numDocs; i++) { - int value = i; - assertBusy(() -> { - final GetResponse getResponse = client().prepareGet("index2", "doc", Integer.toString(value)).get(); - assertTrue(getResponse.isExists()); - assertTrue((getResponse.getSource().containsKey("field"))); - assertThat(XContentMapValues.extractValue("objects.field", getResponse.getSource()), - equalTo(Collections.singletonList(value))); - }); - } - final UnfollowIndexAction.Request unfollowRequest = new UnfollowIndexAction.Request(); unfollowRequest.setFollowIndex("index2"); client().execute(UnfollowIndexAction.INSTANCE, unfollowRequest).get(); @@ -337,13 +270,6 @@ private CheckedRunnable assertExpectedDocumentRunnable(final int valu }; } - private CheckedRunnable assertDocumentDeletedRunnable(final int value) { - return () -> { - final GetResponse getResponse = client().prepareGet("index2", "doc", Integer.toString(value)).get(); - assertFalse(getResponse.isExists()); - }; - } - private String getIndexSettings(final int numberOfPrimaryShards, final Map additionalIndexSettings) throws IOException { final String settings; try (XContentBuilder builder = jsonBuilder()) { @@ -381,56 +307,4 @@ private String getIndexSettings(final int numberOfPrimaryShards, final Map additionalIndexSettings) throws IOException { - final String settings; - try (XContentBuilder builder = jsonBuilder()) { - builder.startObject(); - { - builder.startObject("settings"); - { - builder.field("index.number_of_shards", numberOfPrimaryShards); - for (final Map.Entry additionalSetting : additionalIndexSettings.entrySet()) { - builder.field(additionalSetting.getKey(), additionalSetting.getValue()); - } - } - builder.endObject(); - builder.startObject("mappings"); - { - builder.startObject("doc"); - { - builder.startObject("properties"); - { - builder.startObject("objects"); - { - builder.field("type", "nested"); - builder.startObject("properties"); - { - builder.startObject("field"); - { - builder.field("type", "long"); - } - builder.endObject(); - } - builder.endObject(); - } - builder.endObject(); - builder.startObject("field"); - { - builder.field("type", "keyword"); - } - builder.endObject(); - } - builder.endObject(); - } - builder.endObject(); - } - builder.endObject(); - } - builder.endObject(); - settings = BytesReference.bytes(builder).utf8ToString(); - } - return settings; - } - } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexActionTests.java deleted file mode 100644 index 4c2185d31799c..0000000000000 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/FollowExistingIndexActionTests.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ccr.action; - -import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.test.ESTestCase; - -import static org.hamcrest.Matchers.equalTo; - -public class FollowExistingIndexActionTests extends ESTestCase { - - public void testValidation() { - FollowExistingIndexAction.Request request = new FollowExistingIndexAction.Request(); - request.setLeaderIndex("index1"); - request.setFollowIndex("index2"); - - { - Exception e = expectThrows(IllegalArgumentException.class, () -> FollowExistingIndexAction.validate(null, null, request)); - assertThat(e.getMessage(), equalTo("leader index [index1] does not exist")); - } - { - IndexMetaData leaderIMD = createIMD("index1", 5); - Exception e = expectThrows(IllegalArgumentException.class, () -> FollowExistingIndexAction.validate(leaderIMD, null, request)); - assertThat(e.getMessage(), equalTo("follow index [index2] does not exist")); - } - { - IndexMetaData leaderIMD = createIMD("index1", 5); - IndexMetaData followIMD = createIMD("index2", 5); - Exception e = expectThrows(IllegalArgumentException.class, - () -> FollowExistingIndexAction.validate(leaderIMD, followIMD, request)); - assertThat(e.getMessage(), equalTo("leader index [index1] does not have soft deletes enabled")); - } - { - IndexMetaData leaderIMD = createIMD("index1", 5, IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true"); - IndexMetaData followIMD = createIMD("index2", 4); - Exception e = expectThrows(IllegalArgumentException.class, - () -> FollowExistingIndexAction.validate(leaderIMD, followIMD, request)); - assertThat(e.getMessage(), - equalTo("leader index primary shards [5] does not match with the number of shards of the follow index [4]")); - } - { - IndexMetaData leaderIMD = createIMD("index1", 5, IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true"); - IndexMetaData followIMD = createIMD("index2", 5); - FollowExistingIndexAction.validate(leaderIMD, followIMD, request); - } - } - - private static IndexMetaData createIMD(String index, int numShards, String... settings) { - assert settings.length % 2 == 0; - Settings.Builder settingsBuilder = settings(Version.CURRENT); - for (int i = 0; i < settings.length; i += 2) { - settingsBuilder.put(settings[i], settings[i + 1]); - } - return IndexMetaData.builder(index).settings(settingsBuilder) - .numberOfShards(numShards) - .numberOfReplicas(0) - .setRoutingNumShards(numShards).build(); - } - -} From 974c44caafba5ded0b2d03f8ee5f2878a5b2f222 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 5 May 2018 11:10:12 -0400 Subject: [PATCH 08/18] harden tests --- .../index/engine/InternalEngine.java | 11 +- .../index/engine/LuceneChangesSnapshot.java | 26 +- .../elasticsearch/index/shard/IndexShard.java | 14 +- .../index/engine/InternalEngineTests.java | 82 ++--- .../engine/LuceneChangesSnapshotTests.java | 285 ++++++++++-------- .../index/engine/EngineTestCase.java | 77 ++++- .../index/shard/IndexShardTestCase.java | 17 +- .../ccr/action/ShardChangesActionTests.java | 4 +- .../ccr/action/ShardChangesRequestTests.java | 1 + 9 files changed, 317 insertions(+), 200 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 3c5b5eafbeb74..3c8d7d6c48c26 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -2353,6 +2353,7 @@ public Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService m if (lastRefreshedCheckpoint() < maxSeqNo) { refresh(source, SearcherScope.INTERNAL); } + refresh(source, SearcherScope.INTERNAL); return new LuceneChangesSnapshot(() -> acquireSearcher(source, SearcherScope.INTERNAL), mapperService, minSeqNo, maxSeqNo, requiredFullRange, () -> { }); } @@ -2408,13 +2409,13 @@ public long softUpdateDocuments(Term term, Iterable Math.max(prev, pendingCheckpoint)); + refreshedCheckpoint.getAndUpdate(prev -> Math.max(prev, pendingCheckpoint)); } } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index b5488b2a7cf28..1687063a6806a 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -66,13 +66,13 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { private final Closeable onClose; /** - * Creates a new "translog" snapshot from Lucene including all operations whose seq# between the specified range. + * Creates a new "translog" snapshot from Lucene for reading operations whose seq# in the specified range. * * @param searcherFactory the engine searcher factory (prefer the internal searcher) * @param mapperService the mapper service which will be mainly used to resolve the document's type and uid * @param fromSeqNo the min requesting seq# - inclusive * @param toSeqNo the maximum requesting seq# - inclusive - * @param requiredFullRange if true, the snapshot must contains all seq# between fromSeqNo and toSeqNo + * @param requiredFullRange if true, the snapshot will strictly check for the existence of operations between fromSeqNo and toSeqNo * @param onClose a callback to be called when this snapshot is closed */ LuceneChangesSnapshot(Supplier searcherFactory, MapperService mapperService, @@ -89,6 +89,7 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { final Engine.Searcher engineSearcher = searcherFactory.get(); try { this.searcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); + this.searcher.setQueryCache(null); this.topDocs = searchOperations(searcher); success = true; this.onClose = () -> IOUtils.close(onClose, engineSearcher); @@ -117,18 +118,16 @@ public int overriddenOperations() { @Override public Translog.Operation next() throws IOException { final Translog.Operation op = nextOp(); - if (requiredFullRange == false || lastSeenSeqNo == toSeqNo) { - if (op != null){ - lastSeenSeqNo = op.seqNo(); + if (requiredFullRange && lastSeenSeqNo < toSeqNo) { + final long expectedSeqNo = lastSeenSeqNo + 1; + if (op == null || op.seqNo() != expectedSeqNo) { + throw new IllegalStateException("Not all operations between min_seqno [" + fromSeqNo + "] " + + "and max_seqno [" + toSeqNo + "] found; expected seqno [" + expectedSeqNo + "]; found [" + op + "]"); } - return op; } - final long expectedSeqNo = lastSeenSeqNo + 1; - if (op == null || op.seqNo() != expectedSeqNo) { - throw new IllegalStateException("not all operations between min_seqno [" + fromSeqNo + "] " + - "and max_seqno [" + toSeqNo + "] found; expected seqno [" + expectedSeqNo + "]; found [" + op + "]"); + if (op != null) { + lastSeenSeqNo = op.seqNo(); } - lastSeenSeqNo = op.seqNo(); return op; } @@ -138,7 +137,7 @@ private Translog.Operation nextOp() throws IOException { if (scoreDocs[docIndex].doc == DocIdSetIterator.NO_MORE_DOCS) { return null; } - final Translog.Operation op = readDocAsOperation(scoreDocs[docIndex].doc); + final Translog.Operation op = readDocAsOp(scoreDocs[docIndex].doc); if (op != null) { return op; } @@ -155,12 +154,11 @@ private TopDocs searchOperations(IndexSearcher searcher) throws IOException { return searcher.search(rangeQuery, Integer.MAX_VALUE, sortedBySeqNoThenByTerm); } - private Translog.Operation readDocAsOperation(int docID) throws IOException { + private Translog.Operation readDocAsOp(int docID) throws IOException { final List leaves = searcher.getIndexReader().leaves(); final LeafReaderContext leaf = leaves.get(ReaderUtil.subIndex(docID, leaves)); final int segmentDocID = docID - leaf.docBase; final long seqNo = readNumericDV(leaf, SeqNoFieldMapper.NAME, segmentDocID); - // This operation has seen and will be skipped anyway - do not visit other fields. if (seqNo == lastSeenSeqNo) { skippedOperations++; diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 06b2fe7aa6101..1b6cabb948de0 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1606,10 +1606,18 @@ public int estimateTranslogOperationsFromMinSeq(long minSeqNo) { } /** - * Creates a new "translog" snapshot for reading translog operations between the requesting range. + * Creates a new "translog" snapshot from Lucene for reading operations whose seqno is between minSeqNo and maxSeqNo. * The caller has to close the returned snapshot after finishing the reading. - */ - public Translog.Snapshot newLuceneChangesSnapshot(String source, long minSeqNo, long maxSeqNo, boolean requiredFullRange) throws IOException { + * + * @param source the source of the request + * @param minSeqNo the min_seqno to read - inclusive + * @param maxSeqNo the max_seqno to read - inclusive + * @param requiredFullRange if true then {@link Translog.Snapshot#next()} will throw {@link IllegalStateException} + * if any operation between minSeqNo and maxSeqNo is missing. This parameter should be only + * enabled when the requesting range is below the global checkpoint. + */ + public Translog.Snapshot newLuceneChangesSnapshot(String source, long minSeqNo, long maxSeqNo, + boolean requiredFullRange) throws IOException { return getEngine().newLuceneChangesSnapshot(source, mapperService, minSeqNo, maxSeqNo, requiredFullRange); } diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index c171942ff7d15..c5ec026837345 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1545,42 +1545,6 @@ public void testConcurrentOutOfOrderDocsOnReplica() throws IOException, Interrup assertVisibleCount(engine, totalExpectedOps); } - private void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { - Thread[] thread = new Thread[randomIntBetween(3, 5)]; - CountDownLatch startGun = new CountDownLatch(thread.length); - AtomicInteger offset = new AtomicInteger(-1); - for (int i = 0; i < thread.length; i++) { - thread[i] = new Thread(() -> { - startGun.countDown(); - try { - startGun.await(); - } catch (InterruptedException e) { - throw new AssertionError(e); - } - int docOffset; - while ((docOffset = offset.incrementAndGet()) < ops.size()) { - try { - final Engine.Operation op = ops.get(docOffset); - if (op instanceof Engine.Index) { - engine.index((Engine.Index) op); - } else { - engine.delete((Engine.Delete) op); - } - if ((docOffset + 1) % 4 == 0) { - engine.refresh("test"); - } - } catch (IOException e) { - throw new AssertionError(e); - } - } - }); - thread[i].start(); - } - for (int i = 0; i < thread.length; i++) { - thread[i].join(); - } - } - public void testInternalVersioningOnPrimary() throws IOException { final List ops = generateSingleDocHistory(false, VersionType.INTERNAL, 2, 2, 20, "1"); assertOpsOnPrimary(ops, Versions.NOT_FOUND, true, engine); @@ -4704,6 +4668,52 @@ public void testTrimUnsafeCommits() throws Exception { } } + public void testLuceneHistoryOnPrimary() throws Exception { + final List operations = generateSingleDocHistory(false, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "1"); + assertOperationHistoryInLucene(operations); + } + + public void testLuceneHistoryOnReplica() throws Exception { + final List operations = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); + Randomness.shuffle(operations); + assertOperationHistoryInLucene(operations); + } + + private void assertOperationHistoryInLucene(List operations) throws IOException { + final MergePolicy keepSoftDeleteDocsMP = new SoftDeletesRetentionMergePolicy( + Lucene.SOFT_DELETE_FIELD, () -> new MatchAllDocsQuery(), engine.config().getMergePolicy()); + Set expectedSeqNos = new HashSet<>(); + try (Store store = createStore(); + Engine engine = createEngine(config(defaultSettings, store, createTempDir(), keepSoftDeleteDocsMP, null))) { + for (Engine.Operation op : operations) { + if (op instanceof Engine.Index) { + Engine.IndexResult indexResult = engine.index((Engine.Index) op); + assertThat(indexResult.getFailure(), nullValue()); + expectedSeqNos.add(indexResult.getSeqNo()); + } else { + Engine.DeleteResult deleteResult = engine.delete((Engine.Delete) op); + assertThat(deleteResult.getFailure(), nullValue()); + expectedSeqNos.add(deleteResult.getSeqNo()); + } + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + if (rarely()) { + engine.forceMerge(true); + } + } + MapperService mapperService = createMapperService("test"); + List actualOps = readAllOperationsInLucene(engine, mapperService); + assertThat(actualOps.stream().map(o -> o.seqNo()).collect(Collectors.toList()), containsInAnyOrder(expectedSeqNos.toArray())); + assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + } + } + private static void trimUnsafeCommits(EngineConfig config) throws IOException { final Store store = config.getStore(); final TranslogConfig translogConfig = config.getTranslogConfig(); diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index 63313aa0d22fd..1813816cc1cf8 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -19,56 +19,50 @@ package org.elasticsearch.index.engine; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.SnapshotMatchers; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.test.IndexSettingsModule; +import org.junit.Before; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; +import java.util.Arrays; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import java.util.stream.LongStream; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.nullValue; public class LuceneChangesSnapshotTests extends EngineTestCase { + private MapperService mapperService; - public void testEmptyEngine() throws Exception { - MapperService mapper = createMapperService("test"); + @Before + public void createMapper() throws Exception { + mapperService = createMapperService("test"); + } + + public void testBasics() throws Exception { long fromSeqNo = randomNonNegativeLong(); long toSeqNo = randomLongBetween(fromSeqNo, Long.MAX_VALUE); - try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, true)) { + // Empty engine + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, true)) { IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); assertThat(error.getMessage(), - containsString("not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); + containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); } - try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, false)) { - assertThat(drainAll(snapshot), empty()); + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, false)) { + assertThat(snapshot, SnapshotMatchers.size(0)); } - } - - public void testRequiredFullRange() throws Exception { - MapperService mapper = createMapperService("test"); - int numOps = between(0, 100); + int numOps = between(1, 100); + int refreshedSeqNo = -1; for (int i = 0; i < numOps; i++) { String id = Integer.toString(randomIntBetween(i, i + 5)); ParsedDocument doc = createParsedDoc(id, null); @@ -78,135 +72,182 @@ public void testRequiredFullRange() throws Exception { engine.delete(new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get())); } if (rarely()) { - engine.flush(); + if (randomBoolean()) { + engine.flush(); + } else { + engine.refresh("test"); + } + refreshedSeqNo = i; } } - int iters = between(1, 10); - for (int i = 0; i < iters; i++) { - int fromSeqNo = between(0, numOps * 2); - int toSeqNo = between(Math.max(numOps + 1, fromSeqNo), numOps * 10); - try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, true)) { + if (refreshedSeqNo == -1) { + fromSeqNo = between(0, numOps); + toSeqNo = randomLongBetween(fromSeqNo, numOps * 2); + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false, ()->{})) { + assertThat(snapshot, SnapshotMatchers.size(0)); + } + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true, ()->{})) { IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); assertThat(error.getMessage(), - containsString("not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); + containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); } - try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, false)) { - List ops = drainAll(snapshot); - int readOps = Math.min(toSeqNo, numOps) - Math.min(fromSeqNo, numOps); - assertThat(ops, hasSize(readOps)); + }else { + fromSeqNo = randomLongBetween(0, refreshedSeqNo); + toSeqNo = randomLongBetween(refreshedSeqNo + 1, numOps * 2); + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false, ()->{})) { + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); } + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true, ()->{})) { + IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); + assertThat(error.getMessage(), + containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); + } + toSeqNo = randomLongBetween(fromSeqNo, refreshedSeqNo); + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true, ()->{})) { + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); + } + } + // Get snapshot via engine will auto refresh + fromSeqNo = randomLongBetween(0, numOps - 1); + toSeqNo = randomLongBetween(fromSeqNo, numOps - 1); + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, randomBoolean())) { + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); } } public void testDedupByPrimaryTerm() throws Exception { - MapperService mapper = createMapperService("test"); Map latestOperations = new HashMap<>(); - int numOps = scaledRandomIntBetween(10, 2000); - List seqNos = LongStream.range(0, numOps).boxed().collect(Collectors.toList()); - Randomness.shuffle(seqNos); - for (int i = 0; i < numOps; i++) { - if (randomBoolean()) { - primaryTerm.set(randomLongBetween(primaryTerm.get(), Long.MAX_VALUE)); + List terms = Arrays.asList(between(1, 1000), between(1000, 2000)); + try (Store store = createStore(); + InternalEngine engine = createEngine(store, createTempDir())) { + for (long term : terms) { + final List ops = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE), term, 2, 20, "1"); + primaryTerm.set(Math.max(primaryTerm.get(), term)); engine.rollTranslogGeneration(); + for (Engine.Operation op : ops) { + // Only ops after local checkpoint get into the engine + if (engine.getLocalCheckpointTracker().getCheckpoint() < op.seqNo()) { + latestOperations.put(op.seqNo(), op.primaryTerm()); + } + if (op instanceof Engine.Index) { + engine.index((Engine.Index) op); + } else if (op instanceof Engine.Delete) { + engine.delete((Engine.Delete) op); + } + if (rarely()) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + } } - if (randomBoolean()) { - primaryTerm.set(randomLongBetween(1, primaryTerm.get())); + long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, 0, maxSeqNo, false)) { + Translog.Operation op; + while ((op = snapshot.next()) != null) { + assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); + } } - String id = Integer.toString(randomIntBetween(i, i + 5)); - ParsedDocument doc = createParsedDoc(id, null); - final long seqNo = seqNos.remove(0); - if (randomBoolean()) { - engine.index(replicaIndexForDoc(doc, randomNonNegativeLong(), seqNo, false)); + } + } + + public void testUpdateAndReadChangesConcurrently() throws Exception { + Follower[] followers = new Follower[between(1, 3)]; + CountDownLatch readyLatch = new CountDownLatch(followers.length + 1); + AtomicBoolean isDone = new AtomicBoolean(); + for (int i = 0; i < followers.length; i++) { + followers[i] = new Follower(engine, isDone, readyLatch); + followers[i].start(); + } + boolean onPrimary = randomBoolean(); + List operations = new ArrayList<>(); + int numOps = scaledRandomIntBetween(1, 1000); + for (int i = 0; i < numOps; i++) { + String id = Integer.toString(randomIntBetween(1, 10)); + ParsedDocument doc = createParsedDoc(id, randomAlphaOfLengthBetween(1, 5)); + final Engine.Operation op; + if (onPrimary) { + if (randomBoolean()) { + op = new Engine.Index(newUid(doc), primaryTerm.get(), doc); + } else { + op = new Engine.Delete(doc.type(), doc.id(), newUid(doc.id()), primaryTerm.get()); + } } else { - engine.delete(replicaDeleteForDoc(doc.id(), randomNonNegativeLong(), seqNo, threadPool.relativeTimeInMillis())); - } - latestOperations.put(seqNo, primaryTerm.get()); - if (rarely()) { - engine.flush(); - } - if (rarely()){ - engine.forceMerge(randomBoolean(), between(1, 2), false, false, false); + if (randomBoolean()) { + op = replicaIndexForDoc(doc, randomNonNegativeLong(), i, randomBoolean()); + } else { + op = replicaDeleteForDoc(doc.id(), randomNonNegativeLong(), i, randomNonNegativeLong()); + } } + operations.add(op); } - final boolean requiredFullRange = randomBoolean(); - long fromSeqNo = randomLongBetween(0, numOps); - long toSeqNo = randomLongBetween(fromSeqNo, requiredFullRange ? numOps : numOps * 2); - try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapper, fromSeqNo, toSeqNo, requiredFullRange)) { - List ops = drainAll(snapshot); - for (Translog.Operation op : ops) { - assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); - } + readyLatch.countDown(); + concurrentlyApplyOps(operations, engine); + assertThat(engine.getLocalCheckpointTracker().getCheckpoint(), equalTo(operations.size() - 1L)); + isDone.set(true); + for (Follower follower : followers) { + follower.join(); } } - public void testHistoryOnPrimary() throws Exception { - final List operations = generateSingleDocHistory(false, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), primaryTerm.get(), 10, 1000, "1"); - assertOperationHistoryInLucene(operations, true); - } + class Follower extends Thread { + private final Engine leader; + private final TranslogHandler translogHandler; + private final AtomicBoolean isDone; + private final CountDownLatch readLatch; - public void testHistoryOnReplica() throws Exception { - final List operations = generateSingleDocHistory(true, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 1000, "2"); - Randomness.shuffle(operations); - assertOperationHistoryInLucene(operations, false); - } + Follower(Engine leader, AtomicBoolean isDone, CountDownLatch readLatch) { + this.leader = leader; + this.isDone = isDone; + this.readLatch = readLatch; + this.translogHandler = new TranslogHandler(xContentRegistry(), IndexSettingsModule.newIndexSettings(shardId.getIndexName(), + engine.engineConfig.getIndexSettings().getSettings())); + } - private void assertOperationHistoryInLucene(List operations, boolean requiredFullRange) throws IOException { - Set expectedSeqNos = new HashSet<>(); - Settings settings = Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), operations.size()) - .build(); - final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - try (Store store = createStore(); - InternalEngine engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get))) { - for (Engine.Operation op : operations) { - if (op instanceof Engine.Index) { - Engine.IndexResult indexResult = engine.index((Engine.Index) op); - assertThat(indexResult.getFailure(), nullValue()); - expectedSeqNos.add(indexResult.getSeqNo()); - } else { - Engine.DeleteResult deleteResult = engine.delete((Engine.Delete) op); - assertThat(deleteResult.getFailure(), nullValue()); - expectedSeqNos.add(deleteResult.getSeqNo()); - } - if (randomBoolean()) { - globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); - } - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(); - } - if (rarely()) { - engine.forceMerge(true); + void pullOperations(Engine follower) throws IOException { + long leaderCheckpoint = leader.getLocalCheckpointTracker().getCheckpoint(); + long followerCheckpoint = follower.getLocalCheckpointTracker().getCheckpoint(); + if (followerCheckpoint < leaderCheckpoint) { + long fromSeqNo = followerCheckpoint + 1; + long batchSize = randomLongBetween(0, 100); + long toSeqNo = Math.min(fromSeqNo + batchSize, leaderCheckpoint); + try (Translog.Snapshot snapshot = leader.newLuceneChangesSnapshot("test", mapperService, fromSeqNo, toSeqNo, true)) { + translogHandler.run(follower, snapshot); } } - long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); - long toSeqNo = requiredFullRange ? maxSeqNo : randomLongBetween(maxSeqNo, Long.MAX_VALUE); - MapperService mapperService = createMapperService("test"); - try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, 0, toSeqNo, requiredFullRange)) { - List actualOps = drainAll(snapshot); - List sortedSeqNo = new ArrayList<>(expectedSeqNos); - Collections.sort(sortedSeqNo); - assertThat(actualOps.stream().map(o -> o.seqNo()).collect(Collectors.toList()), - containsInAnyOrder(sortedSeqNo.toArray())); - } + } - assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService); + @Override + public void run() { + try (Store store = createStore(); + InternalEngine follower = createEngine(store, createTempDir())) { + readLatch.countDown(); + readLatch.await(); + while (isDone.get() == false || + follower.getLocalCheckpointTracker().getCheckpoint() < leader.getLocalCheckpointTracker().getCheckpoint()) { + pullOperations(follower); + } + assertConsistentHistoryBetweenTranslogAndLuceneIndex(follower, mapperService); + assertThat(getDocIds(follower, true), equalTo(getDocIds(leader, true))); + } catch (Exception ex) { + throw new AssertionError(ex); + } } } - List drainAll(Translog.Snapshot snapshot) throws IOException { + private List drainAll(Translog.Snapshot snapshot) throws IOException { List operations = new ArrayList<>(); Translog.Operation op; while ((op = snapshot.next()) != null) { final Translog.Operation newOp = op; + logger.error("Reading [{}]", op); assert operations.stream().allMatch(o -> o.seqNo() < newOp.seqNo()) : "Operations [" + operations + "], op [" + op + "]"; operations.add(newOp); } diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 7b8730a862e89..ff1e1675c4d86 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -22,12 +22,15 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StoredField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.Term; @@ -38,6 +41,7 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; @@ -93,8 +97,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; import java.util.function.Function; @@ -601,7 +609,7 @@ public static List generateSingleDocHistory(boolean forReplica default: throw new UnsupportedOperationException("unknown version type: " + versionType); } - if (randomBoolean()) { + if (true || randomBoolean()) { op = new Engine.Index(id, testParsedDocument(docId, null, testDocumentWithTextField(valuePrefix + i), B_1, null), forReplica && i >= startWithSeqNo ? i * 2 : SequenceNumbers.UNASSIGNED_SEQ_NO, forReplica && i >= startWithSeqNo && incrementTermWhenIntroducingSeqNo ? primaryTerm + 1 : primaryTerm, @@ -693,6 +701,71 @@ public static void assertOpsOnReplica( } } + protected void concurrentlyApplyOps(List ops, InternalEngine engine) throws InterruptedException { + Thread[] thread = new Thread[randomIntBetween(3, 5)]; + CountDownLatch startGun = new CountDownLatch(thread.length); + AtomicInteger offset = new AtomicInteger(-1); + for (int i = 0; i < thread.length; i++) { + thread[i] = new Thread(() -> { + startGun.countDown(); + try { + startGun.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + int docOffset; + while ((docOffset = offset.incrementAndGet()) < ops.size()) { + try { + final Engine.Operation op = ops.get(docOffset); + if (op instanceof Engine.Index) { + engine.index((Engine.Index) op); + } else if (op instanceof Engine.Delete){ + engine.delete((Engine.Delete) op); + } else { + engine.noOp((Engine.NoOp) op); + } + if ((docOffset + 1) % 4 == 0) { + engine.refresh("test"); + } + if (rarely()) { + engine.flush(); + } + } catch (IOException e) { + throw new AssertionError(e); + } + } + }); + thread[i].start(); + } + for (int i = 0; i < thread.length; i++) { + thread[i].join(); + } + } + + /** + * Gets all docId from the given engine. + */ + public static Set getDocIds(Engine engine, boolean refresh) throws IOException { + if (refresh) { + engine.refresh("test_get_doc_ids"); + } + try (Engine.Searcher searcher = engine.acquireSearcher("test_get_doc_ids")) { + Set ids = new HashSet<>(); + for (LeafReaderContext leafContext : searcher.reader().leaves()) { + LeafReader reader = leafContext.reader(); + Bits liveDocs = reader.getLiveDocs(); + for (int i = 0; i < reader.maxDoc(); i++) { + if (liveDocs == null || liveDocs.get(i)) { + Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); + BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); + ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); + } + } + } + return ids; + } + } + /** * Reads all engine operations that have been processed by the engine from Lucene index. * The returned operations are sorted and de-duplicated, thus each sequence number will be have at most one operation. @@ -739,7 +812,7 @@ public static void assertConsistentHistoryBetweenTranslogAndLuceneIndex(Engine e } } assertThat(luceneOp, notNullValue()); - assertThat(luceneOp.primaryTerm(), equalTo(translogOp.primaryTerm())); + assertThat(luceneOp.toString(), luceneOp.primaryTerm(), equalTo(translogOp.primaryTerm())); assertThat(luceneOp.opType(), equalTo(translogOp.opType())); if (luceneOp.opType() == Translog.Operation.Type.INDEX) { assertThat(luceneOp.getSource().source, equalTo(translogOp.getSource().source)); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index e23f5531cc6c8..79fd216c0d9b8 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -518,22 +518,7 @@ private Store.MetadataSnapshot getMetadataSnapshotOrEmpty(IndexShard replica) th } protected Set getShardDocUIDs(final IndexShard shard) throws IOException { - shard.refresh("get_uids"); - try (Engine.Searcher searcher = shard.acquireSearcher("test")) { - Set ids = new HashSet<>(); - for (LeafReaderContext leafContext : searcher.reader().leaves()) { - LeafReader reader = leafContext.reader(); - Bits liveDocs = reader.getLiveDocs(); - for (int i = 0; i < reader.maxDoc(); i++) { - if (liveDocs == null || liveDocs.get(i)) { - Document uuid = reader.document(i, Collections.singleton(IdFieldMapper.NAME)); - BytesRef binaryID = uuid.getBinaryValue(IdFieldMapper.NAME); - ids.add(Uid.decodeId(Arrays.copyOfRange(binaryID.bytes, binaryID.offset, binaryID.offset + binaryID.length))); - } - } - } - return ids; - } + return EngineTestCase.getDocIds(shard.getEngine(), true); } protected void assertDocCount(IndexShard shard, int docDount) throws IOException { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java index 92f1d09309264..27081f3a1e60c 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesActionTests.java @@ -54,13 +54,13 @@ public void testGetOperationsBetween() throws Exception { // get operations for a range no operations exists: Exception e = expectThrows(IllegalStateException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites, numWrites + 1, Long.MAX_VALUE)); - assertThat(e.getMessage(), containsString("not all operations between min_seqno [" + numWrites + "] and max_seqno [" + + assertThat(e.getMessage(), containsString("Not all operations between min_seqno [" + numWrites + "] and max_seqno [" + (numWrites + 1) +"] found")); // get operations for a range some operations do not exist: e = expectThrows(IllegalStateException.class, () -> ShardChangesAction.getOperationsBetween(indexShard, numWrites - 10, numWrites + 10, Long.MAX_VALUE)); - assertThat(e.getMessage(), containsString("not all operations between min_seqno [" + (numWrites - 10) + "] and max_seqno [" + + assertThat(e.getMessage(), containsString("Not all operations between min_seqno [" + (numWrites - 10) + "] and max_seqno [" + (numWrites + 10) +"] found")); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java index 3f30545576046..a0b68c3699bf6 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesRequestTests.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.ccr.action; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.AbstractStreamableTestCase; From 23b8c51fe3dac36424f2838fa1492f5b827ddbb9 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sun, 6 May 2018 21:34:33 -0400 Subject: [PATCH 09/18] Simulate rollback in test --- .../engine/LuceneChangesSnapshotTests.java | 58 +++++++++---------- 1 file changed, 28 insertions(+), 30 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index 1813816cc1cf8..cd9e03d6ebac2 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -37,7 +37,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; -import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -123,37 +122,36 @@ public void testBasics() throws Exception { public void testDedupByPrimaryTerm() throws Exception { Map latestOperations = new HashMap<>(); List terms = Arrays.asList(between(1, 1000), between(1000, 2000)); - try (Store store = createStore(); - InternalEngine engine = createEngine(store, createTempDir())) { - for (long term : terms) { - final List ops = generateSingleDocHistory(true, - randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE), term, 2, 20, "1"); - primaryTerm.set(Math.max(primaryTerm.get(), term)); - engine.rollTranslogGeneration(); - for (Engine.Operation op : ops) { - // Only ops after local checkpoint get into the engine - if (engine.getLocalCheckpointTracker().getCheckpoint() < op.seqNo()) { - latestOperations.put(op.seqNo(), op.primaryTerm()); - } - if (op instanceof Engine.Index) { - engine.index((Engine.Index) op); - } else if (op instanceof Engine.Delete) { - engine.delete((Engine.Delete) op); - } - if (rarely()) { - engine.refresh("test"); - } - if (rarely()) { - engine.flush(); - } + for (long term : terms) { + final List ops = generateSingleDocHistory(true, + randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE), term, 2, 20, "1"); + primaryTerm.set(Math.max(primaryTerm.get(), term)); + engine.rollTranslogGeneration(); + for (Engine.Operation op : ops) { + // We need to simulate a rollback here as only ops after local checkpoint get into the engine + if (op.seqNo() <= engine.getLocalCheckpointTracker().getCheckpoint()) { + engine.getLocalCheckpointTracker().resetCheckpoint(randomLongBetween(-1, op.seqNo() - 1)); + engine.rollTranslogGeneration(); } - } - long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); - try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, 0, maxSeqNo, false)) { - Translog.Operation op; - while ((op = snapshot.next()) != null) { - assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); + if (op instanceof Engine.Index) { + engine.index((Engine.Index) op); + } else if (op instanceof Engine.Delete) { + engine.delete((Engine.Delete) op); + } + latestOperations.put(op.seqNo(), op.primaryTerm()); + if (rarely()) { + engine.refresh("test"); } + if (rarely()) { + engine.flush(); + } + } + } + long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); + try (Translog.Snapshot snapshot = engine.newLuceneChangesSnapshot("test", mapperService, 0, maxSeqNo, false)) { + Translog.Operation op; + while ((op = snapshot.next()) != null) { + assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); } } } From f2415e78066b8927affda31bd2d69ead4185bd18 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sun, 6 May 2018 22:54:28 -0400 Subject: [PATCH 10/18] Remove onClose callback --- .../java/org/elasticsearch/index/engine/Engine.java | 2 +- .../org/elasticsearch/index/engine/InternalEngine.java | 2 +- .../index/engine/LuceneChangesSnapshot.java | 5 ++--- .../index/engine/LuceneChangesSnapshotTests.java | 10 +++++----- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index d0ea8fb2ab2a9..f3a4c8b13bd3e 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -611,7 +611,7 @@ public Translog.Location getTranslogLastWriteLocation() { } /** - * Creates a new "translog" snapshot containing changes in the requesting range from the Lucene index + * Creates a new "translog" snapshot from Lucene for reading operations whose seqno in the requesting seqno range */ public abstract Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService mapperService, long minSeqNo, long maxSeqNo, boolean requiredFullRange) throws IOException; diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 3c8d7d6c48c26..3fca5b3bbc4f1 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -2355,7 +2355,7 @@ public Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService m } refresh(source, SearcherScope.INTERNAL); return new LuceneChangesSnapshot(() -> acquireSearcher(source, SearcherScope.INTERNAL), mapperService, - minSeqNo, maxSeqNo, requiredFullRange, () -> { }); + minSeqNo, maxSeqNo, requiredFullRange); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index 1687063a6806a..ad79ebbdb7c33 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -73,10 +73,9 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { * @param fromSeqNo the min requesting seq# - inclusive * @param toSeqNo the maximum requesting seq# - inclusive * @param requiredFullRange if true, the snapshot will strictly check for the existence of operations between fromSeqNo and toSeqNo - * @param onClose a callback to be called when this snapshot is closed */ LuceneChangesSnapshot(Supplier searcherFactory, MapperService mapperService, - long fromSeqNo, long toSeqNo, boolean requiredFullRange, Closeable onClose) throws IOException { + long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { if (fromSeqNo < 0 || toSeqNo < 0 || fromSeqNo > toSeqNo) { throw new IllegalArgumentException("Invalid range; from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "]"); } @@ -92,7 +91,7 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { this.searcher.setQueryCache(null); this.topDocs = searchOperations(searcher); success = true; - this.onClose = () -> IOUtils.close(onClose, engineSearcher); + this.onClose = engineSearcher; } finally { if (success == false) { IOUtils.close(engineSearcher); diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index cd9e03d6ebac2..9b872980e4931 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -83,11 +83,11 @@ public void testBasics() throws Exception { fromSeqNo = between(0, numOps); toSeqNo = randomLongBetween(fromSeqNo, numOps * 2); try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false, ()->{})) { + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false)) { assertThat(snapshot, SnapshotMatchers.size(0)); } try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true, ()->{})) { + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); assertThat(error.getMessage(), containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); @@ -96,18 +96,18 @@ public void testBasics() throws Exception { fromSeqNo = randomLongBetween(0, refreshedSeqNo); toSeqNo = randomLongBetween(refreshedSeqNo + 1, numOps * 2); try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false, ()->{})) { + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false)) { assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); } try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true, ()->{})) { + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); assertThat(error.getMessage(), containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); } toSeqNo = randomLongBetween(fromSeqNo, refreshedSeqNo); try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true, ()->{})) { + () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); } } From 29a145e62ecab87b47851e9b2b4470c5bd504adf Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 7 May 2018 17:49:18 -0400 Subject: [PATCH 11/18] =?UTF-8?q?Boaz=E2=80=99s=20feedbacks?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../index/engine/InternalEngine.java | 5 ++-- .../index/engine/LuceneChangesSnapshot.java | 24 +++++++++---------- .../engine/LuceneChangesSnapshotTests.java | 10 ++++---- 3 files changed, 19 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 3fca5b3bbc4f1..be92f2d377908 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -2353,8 +2353,7 @@ public Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService m if (lastRefreshedCheckpoint() < maxSeqNo) { refresh(source, SearcherScope.INTERNAL); } - refresh(source, SearcherScope.INTERNAL); - return new LuceneChangesSnapshot(() -> acquireSearcher(source, SearcherScope.INTERNAL), mapperService, + return new LuceneChangesSnapshot(acquireSearcher(source, SearcherScope.INTERNAL), mapperService, minSeqNo, maxSeqNo, requiredFullRange); } @@ -2406,7 +2405,7 @@ public long softUpdateDocuments(Term term, Iterable searcherFactory, MapperService mapperService, + LuceneChangesSnapshot(Engine.Searcher engineSearcher, MapperService mapperService, long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { if (fromSeqNo < 0 || toSeqNo < 0 || fromSeqNo > toSeqNo) { throw new IllegalArgumentException("Invalid range; from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "]"); @@ -85,13 +84,12 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { this.lastSeenSeqNo = fromSeqNo - 1; this.requiredFullRange = requiredFullRange; boolean success = false; - final Engine.Searcher engineSearcher = searcherFactory.get(); try { - this.searcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); - this.searcher.setQueryCache(null); - this.topDocs = searchOperations(searcher); - success = true; + this.indexSearcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); + this.indexSearcher.setQueryCache(null); + this.topDocs = searchOperations(indexSearcher); this.onClose = engineSearcher; + success = true; } finally { if (success == false) { IOUtils.close(engineSearcher); @@ -117,7 +115,7 @@ public int overriddenOperations() { @Override public Translog.Operation next() throws IOException { final Translog.Operation op = nextOp(); - if (requiredFullRange && lastSeenSeqNo < toSeqNo) { + if (requiredFullRange && lastSeenSeqNo != toSeqNo) { final long expectedSeqNo = lastSeenSeqNo + 1; if (op == null || op.seqNo() != expectedSeqNo) { throw new IllegalStateException("Not all operations between min_seqno [" + fromSeqNo + "] " + @@ -125,6 +123,8 @@ public Translog.Operation next() throws IOException { } } if (op != null) { + assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() : "Unexpected operation; " + + "last_seen_seqno [" + lastSeenSeqNo + "], from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "], op [" + op + "]"; lastSeenSeqNo = op.seqNo(); } return op; @@ -154,7 +154,7 @@ private TopDocs searchOperations(IndexSearcher searcher) throws IOException { } private Translog.Operation readDocAsOp(int docID) throws IOException { - final List leaves = searcher.getIndexReader().leaves(); + final List leaves = indexSearcher.getIndexReader().leaves(); final LeafReaderContext leaf = leaves.get(ReaderUtil.subIndex(docID, leaves)); final int segmentDocID = docID - leaf.docBase; final long seqNo = readNumericDV(leaf, SeqNoFieldMapper.NAME, segmentDocID); @@ -166,7 +166,7 @@ private Translog.Operation readDocAsOp(int docID) throws IOException { final long primaryTerm = readNumericDV(leaf, SeqNoFieldMapper.PRIMARY_TERM_NAME, segmentDocID); final FieldsVisitor fields = new FieldsVisitor(true); - searcher.doc(docID, fields); + indexSearcher.doc(docID, fields); fields.postProcess(mapperService); final Translog.Operation op; diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index 9b872980e4931..24a7a2157e737 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -83,11 +83,11 @@ public void testBasics() throws Exception { fromSeqNo = between(0, numOps); toSeqNo = randomLongBetween(fromSeqNo, numOps * 2); try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false)) { + engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false)) { assertThat(snapshot, SnapshotMatchers.size(0)); } try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { + engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); assertThat(error.getMessage(), containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); @@ -96,18 +96,18 @@ public void testBasics() throws Exception { fromSeqNo = randomLongBetween(0, refreshedSeqNo); toSeqNo = randomLongBetween(refreshedSeqNo + 1, numOps * 2); try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false)) { + engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false)) { assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); } try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { + engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); assertThat(error.getMessage(), containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); } toSeqNo = randomLongBetween(fromSeqNo, refreshedSeqNo); try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - () -> engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { + engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); } } From 2b559b5280cf51826bb1c2645dfb5f24e96d70b7 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 8 May 2018 09:06:50 -0400 Subject: [PATCH 12/18] Capture and set checkpoint --- .../java/org/elasticsearch/index/engine/InternalEngine.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index be92f2d377908..f44116ef7c8d6 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -2423,7 +2423,7 @@ public void beforeRefresh() { @Override public void afterRefresh(boolean didRefresh) { if (didRefresh) { - refreshedCheckpoint.getAndUpdate(prev -> Math.max(prev, pendingCheckpoint)); + refreshedCheckpoint.set(pendingCheckpoint); } } } From 3abe88e5b1d2db35de7053a4a12c4c1c18b2d9e5 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 8 May 2018 12:27:17 -0400 Subject: [PATCH 13/18] range check --- .../index/engine/LuceneChangesSnapshot.java | 57 +++++++++++-------- .../engine/LuceneChangesSnapshotTests.java | 3 + 2 files changed, 36 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index 95668cbe93cc2..07db2cb3b58a4 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -27,7 +27,6 @@ import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; import org.apache.lucene.search.SortedNumericSortField; @@ -59,7 +58,7 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { private final IndexSearcher indexSearcher; private final MapperService mapperService; - private int docIndex; + private int docIndex = 0; private final TopDocs topDocs; private final Closeable onClose; @@ -114,31 +113,47 @@ public int overriddenOperations() { @Override public Translog.Operation next() throws IOException { - final Translog.Operation op = nextOp(); - if (requiredFullRange && lastSeenSeqNo != toSeqNo) { - final long expectedSeqNo = lastSeenSeqNo + 1; - if (op == null || op.seqNo() != expectedSeqNo) { - throw new IllegalStateException("Not all operations between min_seqno [" + fromSeqNo + "] " + - "and max_seqno [" + toSeqNo + "] found; expected seqno [" + expectedSeqNo + "]; found [" + op + "]"); + Translog.Operation op; + while ((op = nextOp()) != null) { + // Only pick the first seen seq# + if (op.seqNo() == lastSeenSeqNo) { + skippedOperations++; + } else { + assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() : "Unexpected operation; " + + "last_seen_seqno [" + lastSeenSeqNo + "], from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "], op [" + op + "]"; + break; } } + if (requiredFullRange) { + rangeCheck(op); + } if (op != null) { - assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() : "Unexpected operation; " + - "last_seen_seqno [" + lastSeenSeqNo + "], from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "], op [" + op + "]"; lastSeenSeqNo = op.seqNo(); } return op; } - private Translog.Operation nextOp() throws IOException { - final ScoreDoc[] scoreDocs = topDocs.scoreDocs; - for (; docIndex < scoreDocs.length; docIndex++) { - if (scoreDocs[docIndex].doc == DocIdSetIterator.NO_MORE_DOCS) { - return null; + private void rangeCheck(Translog.Operation op) { + if (op == null) { + if (lastSeenSeqNo < toSeqNo) { + throw new IllegalStateException("Not all operations between min_seqno [" + fromSeqNo + "] " + + "and max_seqno [" + toSeqNo + "] found; prematurely terminated last_seen_seqno [" + lastSeenSeqNo + "]"); + } + } else { + final long expectedSeqNo = lastSeenSeqNo + 1; + if (op.seqNo() != expectedSeqNo) { + throw new IllegalStateException("Not all operations between min_seqno [" + fromSeqNo + "] " + + "and max_seqno [" + toSeqNo + "] found; expected seqno [" + expectedSeqNo + "]; found [" + op + "]"); } - final Translog.Operation op = readDocAsOp(scoreDocs[docIndex].doc); - if (op != null) { - return op; + } + } + + private Translog.Operation nextOp() throws IOException { + if (docIndex < topDocs.scoreDocs.length) { + final int docId = topDocs.scoreDocs[docIndex].doc; + docIndex++; + if (docId != DocIdSetIterator.NO_MORE_DOCS) { + return readDocAsOp(docId); } } return null; @@ -158,12 +173,6 @@ private Translog.Operation readDocAsOp(int docID) throws IOException { final LeafReaderContext leaf = leaves.get(ReaderUtil.subIndex(docID, leaves)); final int segmentDocID = docID - leaf.docBase; final long seqNo = readNumericDV(leaf, SeqNoFieldMapper.NAME, segmentDocID); - // This operation has seen and will be skipped anyway - do not visit other fields. - if (seqNo == lastSeenSeqNo) { - skippedOperations++; - return null; - } - final long primaryTerm = readNumericDV(leaf, SeqNoFieldMapper.PRIMARY_TERM_NAME, segmentDocID); final FieldsVisitor fields = new FieldsVisitor(true); indexSearcher.doc(docID, fields); diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index 24a7a2157e737..6a3659dbf1fbe 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -122,6 +122,7 @@ public void testBasics() throws Exception { public void testDedupByPrimaryTerm() throws Exception { Map latestOperations = new HashMap<>(); List terms = Arrays.asList(between(1, 1000), between(1000, 2000)); + int totalOps = 0; for (long term : terms) { final List ops = generateSingleDocHistory(true, randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL, VersionType.EXTERNAL_GTE), term, 2, 20, "1"); @@ -145,6 +146,7 @@ public void testDedupByPrimaryTerm() throws Exception { if (rarely()) { engine.flush(); } + totalOps++; } } long maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo(); @@ -153,6 +155,7 @@ public void testDedupByPrimaryTerm() throws Exception { while ((op = snapshot.next()) != null) { assertThat(op.toString(), op.primaryTerm(), equalTo(latestOperations.get(op.seqNo()))); } + assertThat(snapshot.overriddenOperations(), equalTo(totalOps - latestOperations.size())); } } From b357a5417876f01167ebf69a96189f99b5c46636 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 8 May 2018 14:23:54 -0400 Subject: [PATCH 14/18] nested docs --- .../index/engine/LuceneChangesSnapshot.java | 36 +++--- .../xpack/ccr/ShardChangesIT.java | 108 ++++++++++++++++++ 2 files changed, 130 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index 07db2cb3b58a4..e5d9f5ce60be1 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -113,14 +113,10 @@ public int overriddenOperations() { @Override public Translog.Operation next() throws IOException { - Translog.Operation op; - while ((op = nextOp()) != null) { - // Only pick the first seen seq# - if (op.seqNo() == lastSeenSeqNo) { - skippedOperations++; - } else { - assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() : "Unexpected operation; " + - "last_seen_seqno [" + lastSeenSeqNo + "], from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "], op [" + op + "]"; + Translog.Operation op = null; + for (int docId = nextDocId(); docId != DocIdSetIterator.NO_MORE_DOCS; docId = nextDocId()) { + op = readDocAsOp(docId); + if (op != null) { break; } } @@ -148,15 +144,14 @@ private void rangeCheck(Translog.Operation op) { } } - private Translog.Operation nextOp() throws IOException { + private int nextDocId() { if (docIndex < topDocs.scoreDocs.length) { final int docId = topDocs.scoreDocs[docIndex].doc; docIndex++; - if (docId != DocIdSetIterator.NO_MORE_DOCS) { - return readDocAsOp(docId); - } + return docId; + } else { + return DocIdSetIterator.NO_MORE_DOCS; } - return null; } private TopDocs searchOperations(IndexSearcher searcher) throws IOException { @@ -172,8 +167,19 @@ private Translog.Operation readDocAsOp(int docID) throws IOException { final List leaves = indexSearcher.getIndexReader().leaves(); final LeafReaderContext leaf = leaves.get(ReaderUtil.subIndex(docID, leaves)); final int segmentDocID = docID - leaf.docBase; + final NumericDocValues primaryTermDV = leaf.reader().getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME); + // We don't have to read the nested child documents - those docs don't have primary terms. + if (primaryTermDV == null || primaryTermDV.advanceExact(segmentDocID) == false) { + skippedOperations++; + return null; + } + final long primaryTerm = primaryTermDV.longValue(); final long seqNo = readNumericDV(leaf, SeqNoFieldMapper.NAME, segmentDocID); - final long primaryTerm = readNumericDV(leaf, SeqNoFieldMapper.PRIMARY_TERM_NAME, segmentDocID); + // Only pick the first seen seq# + if (seqNo == lastSeenSeqNo) { + skippedOperations++; + return null; + } final FieldsVisitor fields = new FieldsVisitor(true); indexSearcher.doc(docID, fields); fields.postProcess(mapperService); @@ -201,6 +207,8 @@ assert readNumericDV(leaf, Lucene.SOFT_DELETE_FIELD, segmentDocID) == 1 source.toBytesRef().bytes, fields.routing(), autoGeneratedIdTimestamp); } } + assert fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && lastSeenSeqNo < op.seqNo() : "Unexpected operation; " + + "last_seen_seqno [" + lastSeenSeqNo + "], from_seqno [" + fromSeqNo + "], to_seqno [" + toSeqNo + "], op [" + op + "]"; return op; } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java index 38a129b3a51ef..02ef001817eb5 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/ShardChangesIT.java @@ -17,6 +17,8 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; @@ -219,6 +221,61 @@ public void testFollowIndex() throws Exception { }); } + public void testFollowIndexWithNestedField() throws Exception { + final String leaderIndexSettings = + getIndexSettingsWithNestedMapping(1, Collections.singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + assertAcked(client().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); + + final String followerIndexSettings = + getIndexSettingsWithNestedMapping(1, Collections.singletonMap(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), "true")); + assertAcked(client().admin().indices().prepareCreate("index2").setSource(followerIndexSettings, XContentType.JSON)); + + ensureGreen("index1", "index2"); + + final FollowExistingIndexAction.Request followRequest = new FollowExistingIndexAction.Request(); + followRequest.setLeaderIndex("index1"); + followRequest.setFollowIndex("index2"); + client().execute(FollowExistingIndexAction.INSTANCE, followRequest).get(); + + final int numDocs = randomIntBetween(2, 64); + for (int i = 0; i < numDocs; i++) { + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + builder.field("field", "value"); + builder.startArray("objects"); + { + builder.startObject(); + builder.field("field", i); + builder.endObject(); + } + builder.endArray(); + builder.endObject(); + client().prepareIndex("index1", "doc", Integer.toString(i)).setSource(builder).get(); + } + } + + for (int i = 0; i < numDocs; i++) { + int value = i; + assertBusy(() -> { + final GetResponse getResponse = client().prepareGet("index2", "doc", Integer.toString(value)).get(); + assertTrue(getResponse.isExists()); + assertTrue((getResponse.getSource().containsKey("field"))); + assertThat(XContentMapValues.extractValue("objects.field", getResponse.getSource()), + equalTo(Collections.singletonList(value))); + }); + } + + final UnfollowIndexAction.Request unfollowRequest = new UnfollowIndexAction.Request(); + unfollowRequest.setFollowIndex("index2"); + client().execute(UnfollowIndexAction.INSTANCE, unfollowRequest).get(); + + assertBusy(() -> { + final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + final PersistentTasksCustomMetaData tasks = clusterState.getMetaData().custom(PersistentTasksCustomMetaData.TYPE); + assertThat(tasks.tasks().size(), equalTo(0)); + }); + } + public void testFollowNonExistentIndex() throws Exception { assertAcked(client().admin().indices().prepareCreate("test-leader").get()); assertAcked(client().admin().indices().prepareCreate("test-follower").get()); @@ -319,4 +376,55 @@ private String getIndexSettings(final int numberOfPrimaryShards, final Map additionalIndexSettings) throws IOException { + final String settings; + try (XContentBuilder builder = jsonBuilder()) { + builder.startObject(); + { + builder.startObject("settings"); + { + builder.field("index.number_of_shards", numberOfPrimaryShards); + for (final Map.Entry additionalSetting : additionalIndexSettings.entrySet()) { + builder.field(additionalSetting.getKey(), additionalSetting.getValue()); + } + } + builder.endObject(); + builder.startObject("mappings"); + { + builder.startObject("doc"); + { + builder.startObject("properties"); + { + builder.startObject("objects"); + { + builder.field("type", "nested"); + builder.startObject("properties"); + { + builder.startObject("field"); + { + builder.field("type", "long"); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + builder.startObject("field"); + { + builder.field("type", "keyword"); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + settings = BytesReference.bytes(builder).utf8ToString(); + } + return settings; + } } From 09c48ea33710125e861401d28d2a238250a4f13c Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 8 May 2018 17:31:24 -0400 Subject: [PATCH 15/18] index.soft_deletes -> index.soft_deletes.enabled --- .../test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index 7ef0d5703cb21..ed3413969b396 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -45,7 +45,7 @@ public void testFollowIndex() throws Exception { if (runningAgainstLeaderCluster) { logger.info("Running against leader cluster"); Settings indexSettings = Settings.builder() - .put("index.soft_deletes", true) + .put("index.soft_deletes.enabled", true) .build(); createIndex(leaderIndexName, indexSettings); for (int i = 0; i < numDocs; i++) { From aa1f1c054cc671e37b2154da2790bb98138051aa Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 9 May 2018 11:08:53 -0400 Subject: [PATCH 16/18] Cache DocValues --- .../index/engine/LuceneChangesSnapshot.java | 116 +++++++++++++++--- 1 file changed, 96 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index fb8eedd342791..bb662bc5f2c23 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.engine; import org.apache.lucene.document.LongPoint; +import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.ReaderUtil; @@ -62,6 +63,7 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { private final TopDocs topDocs; private final Closeable onClose; + private final CombinedDocValues[] docValues; // cache of docvalues /** * Creates a new "translog" snapshot from Lucene for reading operations whose seq# in the specified range. @@ -88,6 +90,11 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { this.indexSearcher.setQueryCache(null); this.topDocs = searchOperations(indexSearcher); this.onClose = engineSearcher; + final List leaves = indexSearcher.getIndexReader().leaves(); + this.docValues = new CombinedDocValues[leaves.size()]; + for (LeafReaderContext leaf : leaves) { + this.docValues[leaf.ord] = new CombinedDocValues(leaf.reader()); + } success = true; } finally { if (success == false) { @@ -167,39 +174,36 @@ private Translog.Operation readDocAsOp(int docID) throws IOException { final List leaves = indexSearcher.getIndexReader().leaves(); final LeafReaderContext leaf = leaves.get(ReaderUtil.subIndex(docID, leaves)); final int segmentDocID = docID - leaf.docBase; - final NumericDocValues primaryTermDV = leaf.reader().getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME); + final long primaryTerm = docValues[leaf.ord].docPrimaryTerm(segmentDocID); // We don't have to read the nested child documents - those docs don't have primary terms. - if (primaryTermDV == null || primaryTermDV.advanceExact(segmentDocID) == false) { + if (primaryTerm == -1) { skippedOperations++; return null; } - final long primaryTerm = primaryTermDV.longValue(); - final long seqNo = readNumericDV(leaf, SeqNoFieldMapper.NAME, segmentDocID); + final long seqNo = docValues[leaf.ord].docSeqNo(segmentDocID); // Only pick the first seen seq# if (seqNo == lastSeenSeqNo) { skippedOperations++; return null; } - final long version = readNumericDV(leaf, VersionFieldMapper.NAME, segmentDocID); + final long version = docValues[leaf.ord].docVersion(segmentDocID); final FieldsVisitor fields = new FieldsVisitor(true); indexSearcher.doc(docID, fields); fields.postProcess(mapperService); final Translog.Operation op; - final boolean isTombstone = isTombstoneOperation(leaf, segmentDocID); + final boolean isTombstone = docValues[leaf.ord].isTombstone(segmentDocID); if (isTombstone && fields.uid() == null) { op = new Translog.NoOp(seqNo, primaryTerm, ""); // TODO: store reason in ignored fields? - assert readNumericDV(leaf, Lucene.SOFT_DELETE_FIELD, segmentDocID) == 1 - : "Noop operation but soft_deletes field is not set [" + op + "]"; assert version == 1L : "Noop tombstone should have version 1L; actual version [" + version + "]"; + assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Noop but soft_deletes field is not set [" + op + "]"; } else { final String id = fields.uid().id(); final String type = fields.uid().type(); final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); if (isTombstone) { op = new Translog.Delete(type, id, uid, seqNo, primaryTerm, version, VersionType.INTERNAL); - assert readNumericDV(leaf, Lucene.SOFT_DELETE_FIELD, segmentDocID) == 1 - : "Delete operation but soft_deletes field is not set [" + op + "]"; + assert assertDocSoftDeleted(leaf.reader(), segmentDocID) : "Delete op but soft_deletes field is not set [" + op + "]"; } else { final BytesReference source = fields.source(); // TODO: pass the latest timestamp from engine. @@ -213,19 +217,91 @@ assert readNumericDV(leaf, Lucene.SOFT_DELETE_FIELD, segmentDocID) == 1 return op; } - private boolean isTombstoneOperation(LeafReaderContext leaf, int segmentDocID) throws IOException { - final NumericDocValues tombstoneDV = leaf.reader().getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); - if (tombstoneDV != null && tombstoneDV.advanceExact(segmentDocID)) { - return tombstoneDV.longValue() == 1; + private boolean assertDocSoftDeleted(LeafReader leafReader, int segmentDocId) throws IOException { + final NumericDocValues dv = leafReader.getNumericDocValues(Lucene.SOFT_DELETE_FIELD); + if (dv == null || dv.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + Lucene.SOFT_DELETE_FIELD + "] is not found"); } - return false; + return dv.longValue() == 1; } - private long readNumericDV(LeafReaderContext leaf, String field, int segmentDocID) throws IOException { - final NumericDocValues dv = leaf.reader().getNumericDocValues(field); - if (dv == null || dv.advanceExact(segmentDocID) == false) { - throw new IllegalStateException("DocValues for field [" + field + "] is not found"); + private static final class CombinedDocValues { + private final LeafReader leafReader; + private NumericDocValues versionDV; + private NumericDocValues seqNoDV; + private NumericDocValues primaryTermDV; + private NumericDocValues tombstoneDV; + + CombinedDocValues(LeafReader leafReader) { + this.leafReader = leafReader; + } + + NumericDocValues reloadIfNeed(NumericDocValues dv, String field, int segmentDocId) throws IOException { + if (dv == null || dv.docID() > segmentDocId) { + dv = leafReader.getNumericDocValues(field); + } + if (dv == null || dv.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + field + "] is not found"); + } + return dv; + } + + long docVersion(int segmentDocId) throws IOException { + versionDV = reloadIfNeed(versionDV, VersionFieldMapper.NAME, segmentDocId); + return versionDV.longValue(); + } + + long docSeqNo(int segmentDocID) throws IOException { + seqNoDV = reloadIfNeed(seqNoDV, SeqNoFieldMapper.NAME, segmentDocID); + return seqNoDV.longValue(); + } + + long docPrimaryTerm(int segmentDocId) throws IOException { + if (primaryTermDV == null || primaryTermDV.docID() > segmentDocId) { + primaryTermDV = leafReader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME); + } + // Use -1 for docs which don't have primary term. The caller considers those docs as nested docs. + if (primaryTermDV == null || primaryTermDV.advanceExact(segmentDocId) == false) { + return -1; + } + return primaryTermDV.longValue(); + } + + boolean isTombstone(int segmentDocId) throws IOException { + if (tombstoneDV == null || tombstoneDV.docID() > segmentDocId) { + tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); + if (tombstoneDV == null) { + tombstoneDV = EMPTY_DOC_VALUES; // tombstones are rare - use dummy so that we won't have to reload many times. + } + } + return tombstoneDV.advanceExact(segmentDocId) && tombstoneDV.longValue() > 0; } - return dv.longValue(); } + + private static final NumericDocValues EMPTY_DOC_VALUES = new NumericDocValues() { + @Override + public long longValue() { + return 0; + } + @Override + public boolean advanceExact(int target) { + return false; + } + @Override + public int docID() { + return 0; + } + @Override + public int nextDoc() { + throw new UnsupportedOperationException(); + } + @Override + public int advance(int target) { + throw new UnsupportedOperationException(); + } + @Override + public long cost() { + throw new UnsupportedOperationException(); + } + }; } From c3b0e7a5c063eebb771114fa4ad628dd54fd5f3f Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 9 May 2018 11:26:29 -0400 Subject: [PATCH 17/18] Let caller release searcher when failed to open snapshot --- .../index/engine/InternalEngine.java | 10 ++++- .../index/engine/LuceneChangesSnapshot.java | 27 +++++-------- .../engine/LuceneChangesSnapshotTests.java | 38 ++++++++++++++----- 3 files changed, 45 insertions(+), 30 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 940829e60aeb2..e14ecc47c877f 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -2356,8 +2356,14 @@ public Translog.Snapshot newLuceneChangesSnapshot(String source, MapperService m if (lastRefreshedCheckpoint() < maxSeqNo) { refresh(source, SearcherScope.INTERNAL); } - return new LuceneChangesSnapshot(acquireSearcher(source, SearcherScope.INTERNAL), mapperService, - minSeqNo, maxSeqNo, requiredFullRange); + Searcher searcher = acquireSearcher(source, SearcherScope.INTERNAL); + try { + LuceneChangesSnapshot snapshot = new LuceneChangesSnapshot(searcher, mapperService, minSeqNo, maxSeqNo, requiredFullRange); + searcher = null; + return snapshot; + } finally { + IOUtils.close(searcher); + } } @Override diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index bb662bc5f2c23..20cc98cf74e9f 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -34,7 +34,6 @@ import org.apache.lucene.search.TopDocs; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.fieldvisitor.FieldsVisitor; import org.elasticsearch.index.mapper.IdFieldMapper; @@ -68,7 +67,7 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { /** * Creates a new "translog" snapshot from Lucene for reading operations whose seq# in the specified range. * - * @param engineSearcher the internal engine searcher - this snapshot will take over the provided searcher + * @param engineSearcher the internal engine searcher which will be taken over if the snapshot is opened successfully * @param mapperService the mapper service which will be mainly used to resolve the document's type and uid * @param fromSeqNo the min requesting seq# - inclusive * @param toSeqNo the maximum requesting seq# - inclusive @@ -84,23 +83,15 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { this.toSeqNo = toSeqNo; this.lastSeenSeqNo = fromSeqNo - 1; this.requiredFullRange = requiredFullRange; - boolean success = false; - try { - this.indexSearcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); - this.indexSearcher.setQueryCache(null); - this.topDocs = searchOperations(indexSearcher); - this.onClose = engineSearcher; - final List leaves = indexSearcher.getIndexReader().leaves(); - this.docValues = new CombinedDocValues[leaves.size()]; - for (LeafReaderContext leaf : leaves) { - this.docValues[leaf.ord] = new CombinedDocValues(leaf.reader()); - } - success = true; - } finally { - if (success == false) { - IOUtils.close(engineSearcher); - } + this.indexSearcher = new IndexSearcher(Lucene.wrapAllDocsLive(engineSearcher.getDirectoryReader())); + this.indexSearcher.setQueryCache(null); + this.topDocs = searchOperations(indexSearcher); + final List leaves = indexSearcher.getIndexReader().leaves(); + this.docValues = new CombinedDocValues[leaves.size()]; + for (LeafReaderContext leaf : leaves) { + this.docValues[leaf.ord] = new CombinedDocValues(leaf.reader()); } + this.onClose = engineSearcher; } @Override diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index 6a3659dbf1fbe..17a9303678078 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.engine; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParsedDocument; @@ -82,33 +83,50 @@ public void testBasics() throws Exception { if (refreshedSeqNo == -1) { fromSeqNo = between(0, numOps); toSeqNo = randomLongBetween(fromSeqNo, numOps * 2); - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false)) { + + Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot(searcher, mapperService, fromSeqNo, toSeqNo, false)) { + searcher = null; assertThat(snapshot, SnapshotMatchers.size(0)); + } finally { + IOUtils.close(searcher); } - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { + + searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot(searcher, mapperService, fromSeqNo, toSeqNo, true)) { + searcher = null; IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); assertThat(error.getMessage(), containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); + }finally { + IOUtils.close(searcher); } }else { fromSeqNo = randomLongBetween(0, refreshedSeqNo); toSeqNo = randomLongBetween(refreshedSeqNo + 1, numOps * 2); - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, false)) { + Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot(searcher, mapperService, fromSeqNo, toSeqNo, false)) { + searcher = null; assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, refreshedSeqNo)); + }finally { + IOUtils.close(searcher); } - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { + searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot(searcher, mapperService, fromSeqNo, toSeqNo, true)) { + searcher = null; IllegalStateException error = expectThrows(IllegalStateException.class, () -> drainAll(snapshot)); assertThat(error.getMessage(), containsString("Not all operations between min_seqno [" + fromSeqNo + "] and max_seqno [" + toSeqNo + "] found")); + }finally { + IOUtils.close(searcher); } toSeqNo = randomLongBetween(fromSeqNo, refreshedSeqNo); - try (Translog.Snapshot snapshot = new LuceneChangesSnapshot( - engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL), mapperService, fromSeqNo, toSeqNo, true)) { + searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL); + try (Translog.Snapshot snapshot = new LuceneChangesSnapshot(searcher, mapperService, fromSeqNo, toSeqNo, true)) { + searcher = null; assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(fromSeqNo, toSeqNo)); + }finally { + IOUtils.close(searcher); } } // Get snapshot via engine will auto refresh From 3b8c63b655b1de6279d69ef967536b51622b742e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 9 May 2018 13:41:57 -0400 Subject: [PATCH 18/18] Load DocValues eagerly --- .../index/engine/LuceneChangesSnapshot.java | 82 ++++++++----------- 1 file changed, 32 insertions(+), 50 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java index 20cc98cf74e9f..127787c7224bd 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LuceneChangesSnapshot.java @@ -46,6 +46,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.List; +import java.util.Objects; /** * A {@link Translog.Snapshot} from changes in a Lucene index @@ -62,7 +63,7 @@ final class LuceneChangesSnapshot implements Translog.Snapshot { private final TopDocs topDocs; private final Closeable onClose; - private final CombinedDocValues[] docValues; // cache of docvalues + private final CombinedDocValues[] docValues; // Cache of DocValues /** * Creates a new "translog" snapshot from Lucene for reading operations whose seq# in the specified range. @@ -209,11 +210,11 @@ private Translog.Operation readDocAsOp(int docID) throws IOException { } private boolean assertDocSoftDeleted(LeafReader leafReader, int segmentDocId) throws IOException { - final NumericDocValues dv = leafReader.getNumericDocValues(Lucene.SOFT_DELETE_FIELD); - if (dv == null || dv.advanceExact(segmentDocId) == false) { + final NumericDocValues ndv = leafReader.getNumericDocValues(Lucene.SOFT_DELETE_FIELD); + if (ndv == null || ndv.advanceExact(segmentDocId) == false) { throw new IllegalStateException("DocValues for field [" + Lucene.SOFT_DELETE_FIELD + "] is not found"); } - return dv.longValue() == 1; + return ndv.longValue() == 1; } private static final class CombinedDocValues { @@ -223,76 +224,57 @@ private static final class CombinedDocValues { private NumericDocValues primaryTermDV; private NumericDocValues tombstoneDV; - CombinedDocValues(LeafReader leafReader) { + CombinedDocValues(LeafReader leafReader) throws IOException { this.leafReader = leafReader; + this.versionDV = Objects.requireNonNull(leafReader.getNumericDocValues(VersionFieldMapper.NAME), "VersionDV is missing"); + this.seqNoDV = Objects.requireNonNull(leafReader.getNumericDocValues(SeqNoFieldMapper.NAME), "SeqNoDV is missing"); + this.primaryTermDV = Objects.requireNonNull( + leafReader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME), "PrimaryTermDV is missing"); + this.tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); } - NumericDocValues reloadIfNeed(NumericDocValues dv, String field, int segmentDocId) throws IOException { - if (dv == null || dv.docID() > segmentDocId) { - dv = leafReader.getNumericDocValues(field); + long docVersion(int segmentDocId) throws IOException { + if (versionDV.docID() > segmentDocId) { + versionDV = Objects.requireNonNull(leafReader.getNumericDocValues(VersionFieldMapper.NAME), "VersionDV is missing"); } - if (dv == null || dv.advanceExact(segmentDocId) == false) { - throw new IllegalStateException("DocValues for field [" + field + "] is not found"); + if (versionDV.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + VersionFieldMapper.NAME + "] is not found"); } - return dv; - } - - long docVersion(int segmentDocId) throws IOException { - versionDV = reloadIfNeed(versionDV, VersionFieldMapper.NAME, segmentDocId); return versionDV.longValue(); } - long docSeqNo(int segmentDocID) throws IOException { - seqNoDV = reloadIfNeed(seqNoDV, SeqNoFieldMapper.NAME, segmentDocID); + long docSeqNo(int segmentDocId) throws IOException { + if (seqNoDV.docID() > segmentDocId) { + seqNoDV = Objects.requireNonNull(leafReader.getNumericDocValues(SeqNoFieldMapper.NAME), "SeqNoDV is missing"); + } + if (seqNoDV.advanceExact(segmentDocId) == false) { + throw new IllegalStateException("DocValues for field [" + SeqNoFieldMapper.NAME + "] is not found"); + } return seqNoDV.longValue(); } long docPrimaryTerm(int segmentDocId) throws IOException { - if (primaryTermDV == null || primaryTermDV.docID() > segmentDocId) { + if (primaryTermDV == null) { + return -1L; + } + if (primaryTermDV.docID() > segmentDocId) { primaryTermDV = leafReader.getNumericDocValues(SeqNoFieldMapper.PRIMARY_TERM_NAME); } // Use -1 for docs which don't have primary term. The caller considers those docs as nested docs. - if (primaryTermDV == null || primaryTermDV.advanceExact(segmentDocId) == false) { + if (primaryTermDV.advanceExact(segmentDocId) == false) { return -1; } return primaryTermDV.longValue(); } boolean isTombstone(int segmentDocId) throws IOException { - if (tombstoneDV == null || tombstoneDV.docID() > segmentDocId) { + if (tombstoneDV == null) { + return false; + } + if (tombstoneDV.docID() > segmentDocId) { tombstoneDV = leafReader.getNumericDocValues(SeqNoFieldMapper.TOMBSTONE_NAME); - if (tombstoneDV == null) { - tombstoneDV = EMPTY_DOC_VALUES; // tombstones are rare - use dummy so that we won't have to reload many times. - } } return tombstoneDV.advanceExact(segmentDocId) && tombstoneDV.longValue() > 0; } } - - private static final NumericDocValues EMPTY_DOC_VALUES = new NumericDocValues() { - @Override - public long longValue() { - return 0; - } - @Override - public boolean advanceExact(int target) { - return false; - } - @Override - public int docID() { - return 0; - } - @Override - public int nextDoc() { - throw new UnsupportedOperationException(); - } - @Override - public int advance(int target) { - throw new UnsupportedOperationException(); - } - @Override - public long cost() { - throw new UnsupportedOperationException(); - } - }; }