From ad052eca9386c1c2fda90070d3337eb4b874256b Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 28 Nov 2017 11:52:15 -0500 Subject: [PATCH 01/29] Keep index commits up to the global checkpoint We need to keep index commits and translog operations up to the current global checkpoint to allow us to throw away unsafe operations and increase the operation-based recovery chance. This is achieved by a new index deletion policy. --- .../lucene/index/ESIndexDeletionPolicy.java | 48 +++++ .../index/engine/CombinedDeletionPolicy.java | 44 ++--- .../index/engine/InternalEngine.java | 58 +++--- ...epUntilGlobalCheckpointDeletionPolicy.java | 125 ++++++++++++ .../index/translog/Translog.java | 4 +- .../translog/TranslogDeletionPolicy.java | 24 +++ .../engine/CombinedDeletionPolicyTests.java | 96 ++++++++-- .../index/engine/InternalEngineTests.java | 112 ++++++++++- ...ilGlobalCheckpointDeletionPolicyTests.java | 180 ++++++++++++++++++ .../index/shard/IndexShardTests.java | 11 +- .../index/translog/TranslogTests.java | 64 +++++-- .../indices/recovery/RecoveryTests.java | 19 +- 12 files changed, 678 insertions(+), 107 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java create mode 100644 core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java create mode 100644 core/src/test/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicyTests.java diff --git a/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java b/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java new file mode 100644 index 0000000000000..573a446c0ace4 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java @@ -0,0 +1,48 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.lucene.index; + +import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexDeletionPolicy; + +import java.io.IOException; +import java.util.List; + +/** + * An {@link IndexDeletionPolicy} that deletes unneeded index commits, and returns index commits are not deleted by this policy. + */ +public interface ESIndexDeletionPolicy { + + /** + * Similar to {@link IndexDeletionPolicy#onInit(List)} but returns a list of kept index commits. + * + * @param commits A list of index commits sorted by age (the 0th one is the oldest commit). + * @return a list of index commits that are not deleted by this policy. + */ + List onInit(List commits) throws IOException; + + /** + * Similar to {@link IndexDeletionPolicy#onCommit(List)} but returns a list of kept index commits. + * + * @param commits A list of index commits sorted by age (the 0th one is the oldest commit). + * @return a list of index commits that are not deleted by this policy. + */ + List onCommit(List commits) throws IOException; +} diff --git a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index add4a44390373..b67250563920b 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -21,7 +21,7 @@ import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexDeletionPolicy; -import org.apache.lucene.index.SnapshotDeletionPolicy; +import org.elasticsearch.common.lucene.index.ESIndexDeletionPolicy; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogDeletionPolicy; @@ -33,13 +33,11 @@ * making sure that all translog files that are needed to recover from the Lucene commit are not deleted. */ class CombinedDeletionPolicy extends IndexDeletionPolicy { - private final TranslogDeletionPolicy translogDeletionPolicy; private final EngineConfig.OpenMode openMode; + private final ESIndexDeletionPolicy indexDeletionPolicy; - private final SnapshotDeletionPolicy indexDeletionPolicy; - - CombinedDeletionPolicy(SnapshotDeletionPolicy indexDeletionPolicy, TranslogDeletionPolicy translogDeletionPolicy, + CombinedDeletionPolicy(ESIndexDeletionPolicy indexDeletionPolicy, TranslogDeletionPolicy translogDeletionPolicy, EngineConfig.OpenMode openMode) { this.indexDeletionPolicy = indexDeletionPolicy; this.translogDeletionPolicy = translogDeletionPolicy; @@ -48,7 +46,7 @@ class CombinedDeletionPolicy extends IndexDeletionPolicy { @Override public void onInit(List commits) throws IOException { - indexDeletionPolicy.onInit(commits); + final List keptCommits = indexDeletionPolicy.onInit(commits); switch (openMode) { case CREATE_INDEX_AND_TRANSLOG: break; @@ -57,7 +55,7 @@ public void onInit(List commits) throws IOException { break; case OPEN_INDEX_AND_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; - setLastCommittedTranslogGeneration(commits); + setLastCommittedTranslogGeneration(keptCommits); break; default: throw new IllegalArgumentException("unknown openMode [" + openMode + "]"); @@ -66,24 +64,24 @@ public void onInit(List commits) throws IOException { @Override public void onCommit(List commits) throws IOException { - indexDeletionPolicy.onCommit(commits); - setLastCommittedTranslogGeneration(commits); - } - - private void setLastCommittedTranslogGeneration(List commits) throws IOException { - // when opening an existing lucene index, we currently always open the last commit. - // we therefore use the translog gen as the one that will be required for recovery - final IndexCommit indexCommit = commits.get(commits.size() - 1); - assert indexCommit.isDeleted() == false : "last commit is deleted"; - long minGen = Long.parseLong(indexCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); - translogDeletionPolicy.setMinTranslogGenerationForRecovery(minGen); + assert commits.isEmpty() == false; + final List keptCommits = indexDeletionPolicy.onCommit(commits); + setLastCommittedTranslogGeneration(keptCommits); } - public SnapshotDeletionPolicy getIndexDeletionPolicy() { - return indexDeletionPolicy; - } + private void setLastCommittedTranslogGeneration(List keptCommits) throws IOException { + assert keptCommits.isEmpty() == false : "All index commits were deleted"; + assert keptCommits.stream().allMatch(commit -> commit.isDeleted() == false) : "Kept commits must not be deleted"; - public TranslogDeletionPolicy getTranslogDeletionPolicy() { - return translogDeletionPolicy; + final IndexCommit lastCommit = keptCommits.get(keptCommits.size() - 1); + final long lastGen = Long.parseLong(lastCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); + long minRequiredGen = lastGen; + for (IndexCommit indexCommit : keptCommits) { + long translogGen = Long.parseLong(indexCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); + minRequiredGen = Math.min(minRequiredGen, translogGen); + } + assert minRequiredGen <= lastGen; + translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen); + translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 6449c979de40a..1e2a645b310b0 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -128,7 +128,7 @@ public class InternalEngine extends Engine { private final String uidField; - private final CombinedDeletionPolicy deletionPolicy; + private final SnapshotDeletionPolicy snapshotDeletionPolicy; // How many callers are currently requesting index throttling. Currently there are only two situations where we do this: when merges // are falling behind and when writing indexing buffer to disk is too slow. When this is 0, there is no throttling, else we throttling @@ -167,8 +167,6 @@ public InternalEngine(EngineConfig engineConfig) { engineConfig.getIndexSettings().getTranslogRetentionSize().getBytes(), engineConfig.getIndexSettings().getTranslogRetentionAge().getMillis() ); - this.deletionPolicy = new CombinedDeletionPolicy( - new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()), translogDeletionPolicy, openMode); store.incRef(); IndexWriter writer = null; Translog translog = null; @@ -182,29 +180,13 @@ public InternalEngine(EngineConfig engineConfig) { mergeScheduler = scheduler = new EngineMergeScheduler(engineConfig.getShardId(), engineConfig.getIndexSettings()); throttle = new IndexThrottle(); try { - final SeqNoStats seqNoStats; - switch (openMode) { - case OPEN_INDEX_AND_TRANSLOG: - writer = createWriter(false); - final long globalCheckpoint = Translog.readGlobalCheckpoint(engineConfig.getTranslogConfig().getTranslogPath()); - seqNoStats = store.loadSeqNoStats(globalCheckpoint); - break; - case OPEN_INDEX_CREATE_TRANSLOG: - writer = createWriter(false); - seqNoStats = store.loadSeqNoStats(SequenceNumbers.UNASSIGNED_SEQ_NO); - break; - case CREATE_INDEX_AND_TRANSLOG: - writer = createWriter(true); - seqNoStats = new SeqNoStats( - SequenceNumbers.NO_OPS_PERFORMED, - SequenceNumbers.NO_OPS_PERFORMED, - SequenceNumbers.UNASSIGNED_SEQ_NO); - break; - default: - throw new IllegalArgumentException(openMode.toString()); - } + final SeqNoStats seqNoStats = loadSeqNoStats(openMode); logger.trace("recovered [{}]", seqNoStats); - seqNoService = seqNoServiceSupplier.apply(engineConfig, seqNoStats); + this.seqNoService = seqNoServiceSupplier.apply(engineConfig, seqNoStats); + this.snapshotDeletionPolicy = new SnapshotDeletionPolicy(new CombinedDeletionPolicy( + new KeepUntilGlobalCheckpointDeletionPolicy(seqNoService::getGlobalCheckpoint), translogDeletionPolicy, openMode) + ); + writer = createWriter(openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG); updateMaxUnsafeAutoIdTimestampFromWriter(writer); historyUUID = loadOrGenerateHistoryUUID(writer, engineConfig.getForceNewHistoryUUID()); Objects.requireNonNull(historyUUID, "history uuid should not be null"); @@ -377,6 +359,28 @@ static SequenceNumbersService sequenceNumberService( seqNoStats.getGlobalCheckpoint()); } + private SeqNoStats loadSeqNoStats(EngineConfig.OpenMode openMode) throws IOException { + final SeqNoStats seqNoStats; + switch (openMode) { + case OPEN_INDEX_AND_TRANSLOG: + final long globalCheckpoint = Translog.readGlobalCheckpoint(engineConfig.getTranslogConfig().getTranslogPath()); + seqNoStats = store.loadSeqNoStats(globalCheckpoint); + break; + case OPEN_INDEX_CREATE_TRANSLOG: + seqNoStats = store.loadSeqNoStats(SequenceNumbers.UNASSIGNED_SEQ_NO); + break; + case CREATE_INDEX_AND_TRANSLOG: + seqNoStats = new SeqNoStats( + SequenceNumbers.NO_OPS_PERFORMED, + SequenceNumbers.NO_OPS_PERFORMED, + SequenceNumbers.UNASSIGNED_SEQ_NO); + break; + default: + throw new IllegalArgumentException(openMode.toString()); + } + return seqNoStats; + } + @Override public InternalEngine recoverFromTranslog() throws IOException { flushLock.lock(); @@ -1642,7 +1646,7 @@ public IndexCommitRef acquireIndexCommit(final boolean flushFirst) throws Engine } try (ReleasableLock lock = readLock.acquire()) { logger.trace("pulling snapshot"); - return new IndexCommitRef(deletionPolicy.getIndexDeletionPolicy()); + return new IndexCommitRef(snapshotDeletionPolicy); } catch (IOException e) { throw new SnapshotFailedEngineException(shardId, e); } @@ -1823,7 +1827,7 @@ private IndexWriterConfig getIndexWriterConfig(boolean create) { final IndexWriterConfig iwc = new IndexWriterConfig(engineConfig.getAnalyzer()); iwc.setCommitOnClose(false); // we by default don't commit on close iwc.setOpenMode(create ? IndexWriterConfig.OpenMode.CREATE : IndexWriterConfig.OpenMode.APPEND); - iwc.setIndexDeletionPolicy(deletionPolicy); + iwc.setIndexDeletionPolicy(snapshotDeletionPolicy); // with tests.verbose, lucene sets this up: plumb to align with filesystem stream boolean verbose = false; try { diff --git a/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java new file mode 100644 index 0000000000000..4f43c8a434f5e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java @@ -0,0 +1,125 @@ +/* + * 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.index.IndexCommit; +import org.elasticsearch.common.lucene.index.ESIndexDeletionPolicy; +import org.elasticsearch.index.seqno.SequenceNumbers; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.LongSupplier; + +/** + * An {@link ESIndexDeletionPolicy} that deletes index commits that are not required for recovery. + * In particular, this policy will delete index commits whose max sequence number is smaller (or equal) than + * the current global checkpoint except the index commit which has the highest max sequence number among those. + */ +final class KeepUntilGlobalCheckpointDeletionPolicy implements ESIndexDeletionPolicy { + private final LongSupplier globalCheckpointSupplier; + + KeepUntilGlobalCheckpointDeletionPolicy(LongSupplier globalCheckpointSupplier) { + this.globalCheckpointSupplier = globalCheckpointSupplier; + } + + @Override + public List onInit(List commits) throws IOException { + if (commits.isEmpty()) { + return Collections.emptyList(); + } + return onCommit(commits); + } + + @Override + public List onCommit(List commits) throws IOException { + assert commits.isEmpty() == false : "onCommit() must be called with a non-empty list of commits"; + + final List keptCommits = new ArrayList<>(); + final int keptPosition = indexOfKeptCommits(commits); + final List duplicateIndexes = indexesOfDuplicateCommits(commits); + + for (int i = 0; i < commits.size() - 1; i++) { + final IndexCommit commit = commits.get(i); + if (i < keptPosition || duplicateIndexes.contains(i)) { + commit.delete(); + } else { + keptCommits.add(commit); + } + } + keptCommits.add(commits.get(commits.size() - 1)); // Always keep the last commit. + + assert keptCommits.stream().allMatch(c -> c.isDeleted() == false) : "All kept commits must not be deleted"; + return keptCommits; + } + + /** + * Find the index position of a safe index commit whose max sequence number is not greater than the global checkpoint. + */ + private int indexOfKeptCommits(List commits) throws IOException { + final long currentGlobalCheckpoint = globalCheckpointSupplier.getAsLong(); + + // Commits are sorted by age (the 0th one is the oldest commit). + for (int i = commits.size() - 1; i >= 0; i--) { + final Map commitUserData = commits.get(i).getUserData(); + // 5.x commits do not contain MAX_SEQ_NO. + if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO) == false) { + return i; + } + final long maxSeqNoFromCommit = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)); + if (maxSeqNoFromCommit <= currentGlobalCheckpoint) { + return i; + } + } + /* + * We may reach to this point in these cases: + * 1. In the previous 6.x, we keep only the last commit - which is likely not a safe commit if writes are in progress. + * Thus, after upgrading, we may not find a safe commit until we can reserve one. + * 2. In peer-recovery, if the file-based happens, a replica will be received the latest commit from a primary. + * However, that commit may not be a safe commit if writes are in progress in the primary. + */ + return -1; + } + + /** + * In some cases, we may have more than one index commits with the same max sequence number. + * We better scan and delete these duplicate index commits as soon as possible. + * + * @return index positions of duplicate commits. + */ + private List indexesOfDuplicateCommits(List commits) throws IOException { + final List duplicateEntries = new ArrayList<>(); + long lastMaxSeqNo = Long.MIN_VALUE; + for (int i = commits.size() - 1; i >= 0; i--) { + final Map commitUserData = commits.get(i).getUserData(); + // 5.x commits do not contain MAX_SEQ_NO. + if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO)) { + final long maxSeqNoFromCommit = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)); + if (lastMaxSeqNo == maxSeqNoFromCommit) { + duplicateEntries.add(i); + } + lastMaxSeqNo = maxSeqNoFromCommit; + } + } + return duplicateEntries; + } +} diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index 80033833899ec..9e2e7ddbd0653 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -372,14 +372,14 @@ long getMinFileGeneration() { * Returns the number of operations in the translog files that aren't committed to lucene. */ public int uncommittedOperations() { - return totalOperations(deletionPolicy.getMinTranslogGenerationForRecovery()); + return totalOperations(deletionPolicy.getTranslogGenerationOfLastCommit()); } /** * Returns the size in bytes of the translog files that aren't committed to lucene. */ public long uncommittedSizeInBytes() { - return sizeInBytesByMinGen(deletionPolicy.getMinTranslogGenerationForRecovery()); + return sizeInBytesByMinGen(deletionPolicy.getTranslogGenerationOfLastCommit()); } /** diff --git a/core/src/main/java/org/elasticsearch/index/translog/TranslogDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/translog/TranslogDeletionPolicy.java index adee4bd9fa9a0..8a2c7bd154f86 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/TranslogDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/translog/TranslogDeletionPolicy.java @@ -54,6 +54,11 @@ public void assertNoOpenTranslogRefs() { */ private long minTranslogGenerationForRecovery = 1; + /** + * This translog generation is used to calculate the number of uncommitted operations since the last index commit. + */ + private long translogGenerationOfLastCommit = 1; + private long retentionSizeInBytes; private long retentionAgeInMillis; @@ -76,6 +81,17 @@ public synchronized void setMinTranslogGenerationForRecovery(long newGen) { minTranslogGenerationForRecovery = newGen; } + /** + * Sets the translog generation of the last index commit. + */ + public synchronized void setTranslogGenerationOfLastCommit(long lastGen) { + if (lastGen < translogGenerationOfLastCommit) { + throw new IllegalArgumentException("translogGenerationOfLastCommit can't go backwards. new [" + lastGen + "] current [" + + translogGenerationOfLastCommit + "]"); + } + translogGenerationOfLastCommit = lastGen; + } + public synchronized void setRetentionSizeInBytes(long bytes) { retentionSizeInBytes = bytes; } @@ -193,6 +209,14 @@ public synchronized long getMinTranslogGenerationForRecovery() { return minTranslogGenerationForRecovery; } + /** + * Returns a translog generation that will be used to calculate the number of uncommitted operations since the last index commit. + * See {@link Translog#uncommittedOperations()} and {@link Translog#uncommittedSizeInBytes()} + */ + public synchronized long getTranslogGenerationOfLastCommit() { + return translogGenerationOfLastCommit; + } + synchronized long getTranslogRefCount(long gen) { final Counter counter = translogRefCounts.get(gen); return counter == null ? 0 : counter.get(); diff --git a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index 5d4385cbd384b..1637a4545fdb3 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -19,18 +19,24 @@ package org.elasticsearch.index.engine; +import com.carrotsearch.hppc.LongArrayList; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.SnapshotDeletionPolicy; +import org.elasticsearch.common.lucene.index.ESIndexDeletionPolicy; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogDeletionPolicy; import org.elasticsearch.test.ESTestCase; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Matchers.anyListOf; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -39,7 +45,7 @@ public class CombinedDeletionPolicyTests extends ESTestCase { public void testPassThrough() throws IOException { - SnapshotDeletionPolicy indexDeletionPolicy = mock(SnapshotDeletionPolicy.class); + ESIndexDeletionPolicy indexDeletionPolicy = mock(ESIndexDeletionPolicy.class); CombinedDeletionPolicy combinedDeletionPolicy = new CombinedDeletionPolicy(indexDeletionPolicy, createTranslogDeletionPolicy(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); List commitList = new ArrayList<>(); @@ -47,33 +53,99 @@ public void testPassThrough() throws IOException { for (int i = 0; i < count; i++) { commitList.add(mockIndexCommitWithTranslogGen(randomNonNegativeLong())); } + when(indexDeletionPolicy.onInit(anyListOf(IndexCommit.class))).thenReturn(commitList); + when(indexDeletionPolicy.onCommit(anyListOf(IndexCommit.class))).thenReturn(commitList); combinedDeletionPolicy.onInit(commitList); verify(indexDeletionPolicy, times(1)).onInit(commitList); combinedDeletionPolicy.onCommit(commitList); verify(indexDeletionPolicy, times(1)).onCommit(commitList); } - public void testSettingMinTranslogGen() throws IOException { - SnapshotDeletionPolicy indexDeletionPolicy = mock(SnapshotDeletionPolicy.class); - final TranslogDeletionPolicy translogDeletionPolicy = mock(TranslogDeletionPolicy.class); - CombinedDeletionPolicy combinedDeletionPolicy = new CombinedDeletionPolicy(indexDeletionPolicy, translogDeletionPolicy, - EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); - List commitList = new ArrayList<>(); - long count = randomIntBetween(10, 20); + public void testSettingTranslogGenerations() throws IOException { + final int count = randomIntBetween(10, 20); + final List commitList = new ArrayList<>(); + final LongArrayList translogGens = new LongArrayList(); long lastGen = 0; for (int i = 0; i < count; i++) { lastGen += randomIntBetween(10, 20000); commitList.add(mockIndexCommitWithTranslogGen(lastGen)); + translogGens.add(lastGen); } + + final AtomicInteger keepN = new AtomicInteger(between(1, commitList.size())); + final ESIndexDeletionPolicy keepLastNPolicy = new ESIndexDeletionPolicy() { + @Override + public List onInit(List commits) throws IOException { + return onCommit(commits); + } + + @Override + public List onCommit(List commits) throws IOException { + return new ArrayList<>(commits.subList(commits.size() - keepN.get(), commits.size())); + } + }; + final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(randomLong(), randomLong()); + CombinedDeletionPolicy combinedDeletionPolicy = new CombinedDeletionPolicy(keepLastNPolicy, translogDeletionPolicy, + EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); + combinedDeletionPolicy.onInit(commitList); - verify(translogDeletionPolicy, times(1)).setMinTranslogGenerationForRecovery(lastGen); - commitList.clear(); - for (int i = 0; i < count; i++) { + assertThat(translogDeletionPolicy.getTranslogGenerationOfLastCommit(), + equalTo(lastGen)); + assertThat(translogDeletionPolicy.getMinTranslogGenerationForRecovery(), + equalTo(translogGens.get(translogGens.size() - keepN.get()))); + + int moreCommits = randomIntBetween(0, 10); + for (int i = 0; i < moreCommits; i++) { lastGen += randomIntBetween(10, 20000); commitList.add(mockIndexCommitWithTranslogGen(lastGen)); + translogGens.add(lastGen); } + keepN.addAndGet(randomInt(moreCommits)); + combinedDeletionPolicy.onCommit(commitList); - verify(translogDeletionPolicy, times(1)).setMinTranslogGenerationForRecovery(lastGen); + assertThat(translogDeletionPolicy.getTranslogGenerationOfLastCommit(), + equalTo(lastGen)); + assertThat(translogDeletionPolicy.getMinTranslogGenerationForRecovery(), + equalTo(translogGens.get(translogGens.size() - keepN.get()))); + } + + public void testIgnoreSnapshottingCommits() throws Exception { + final ESIndexDeletionPolicy keepOnlyLastPolicy = new ESIndexDeletionPolicy() { + @Override + public List onInit(List commits) throws IOException { + return onCommit(commits); + } + @Override + public List onCommit(List commits) throws IOException { + for (int i = 0; i < commits.size() - 1; i++) { + commits.get(i).delete(); + } + return Arrays.asList(commits.get(commits.size() - 1)); + } + }; + final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(randomLong(), randomLong()); + CombinedDeletionPolicy combinedDeletionPolicy = new CombinedDeletionPolicy(keepOnlyLastPolicy, translogDeletionPolicy, + EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); + SnapshotDeletionPolicy snapshotDeletionPolicy = new SnapshotDeletionPolicy(combinedDeletionPolicy); + + final int count = randomIntBetween(10, 20); + final List commitList = new ArrayList<>(); + long lastGen = 0; + for (int i = 0; i < count; i++) { + lastGen += randomIntBetween(10, 20000); + commitList.add(mockIndexCommitWithTranslogGen(lastGen)); + } + snapshotDeletionPolicy.onInit(commitList); + assertThat(translogDeletionPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastGen)); + assertThat(translogDeletionPolicy.getMinTranslogGenerationForRecovery(), equalTo(lastGen)); + + snapshotDeletionPolicy.snapshot(); + long newGen = lastGen + randomIntBetween(10, 20000); + commitList.add(mockIndexCommitWithTranslogGen(newGen)); + snapshotDeletionPolicy.onCommit(commitList); + assertThat(translogDeletionPolicy.getTranslogGenerationOfLastCommit(), equalTo(newGen)); + // The previous commit is being snapshotted but ignored by the CombinedPolicy. + assertThat(translogDeletionPolicy.getMinTranslogGenerationForRecovery(), equalTo(newGen)); } IndexCommit mockIndexCommitWithTranslogGen(long gen) throws IOException { diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index c40923e3c7c92..b7eb4cf063ce6 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -105,7 +105,6 @@ import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; -import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.seqno.SequenceNumbersService; @@ -165,6 +164,7 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -945,17 +945,41 @@ public void testSearchResultRelease() throws Exception { } public void testCommitAdvancesMinTranslogForRecovery() throws IOException { + IOUtils.close(engine, store); + final Path translogPath = createTempDir(); + store = createStore(); + final AtomicBoolean inSync = new AtomicBoolean(randomBoolean()); + final BiFunction seqNoServiceSupplier = (config, seqNoStats) -> + new SequenceNumbersService( + config.getShardId(), + config.getAllocationId(), + config.getIndexSettings(), + seqNoStats.getMaxSeqNo(), + seqNoStats.getLocalCheckpoint(), + seqNoStats.getGlobalCheckpoint()) { + @Override + public long getGlobalCheckpoint() { + return inSync.get() ? getLocalCheckpoint() : SequenceNumbers.UNASSIGNED_SEQ_NO; + } + }; + engine = new InternalEngine(config(defaultSettings, store, translogPath, newMergePolicy(), null, null), seqNoServiceSupplier); ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); + engine.flush(); assertThat(engine.getTranslog().currentFileGeneration(), equalTo(2L)); - assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(2L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync.get() ? 2L : 1L)); + assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(2L)); + engine.flush(); assertThat(engine.getTranslog().currentFileGeneration(), equalTo(2L)); - assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(2L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync.get() ? 2L : 1L)); + assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(2L)); + engine.flush(true, true); assertThat(engine.getTranslog().currentFileGeneration(), equalTo(3L)); - assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(3L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync.get() ? 3L : 1L)); + assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(3L)); } public void testSyncedFlush() throws IOException { @@ -2377,10 +2401,26 @@ public void testTranslogCleanUpPostCommitCrash() throws Exception { ); indexSettings.updateIndexMetaData(builder.build()); + final BiFunction seqNoServiceSupplier = (config, seqNoStats) -> + new SequenceNumbersService( + config.getShardId(), + config.getAllocationId(), + config.getIndexSettings(), + seqNoStats.getMaxSeqNo(), + seqNoStats.getLocalCheckpoint(), + seqNoStats.getGlobalCheckpoint()) { + @Override + public long getGlobalCheckpoint() { + return getLocalCheckpoint(); + } + }; + try (Store store = createStore()) { AtomicBoolean throwErrorOnCommit = new AtomicBoolean(); final Path translogPath = createTempDir(); - try (InternalEngine engine = new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null)) { + try (InternalEngine engine = + new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null), seqNoServiceSupplier) { + @Override protected void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException { super.commitIndexWriter(writer, translog, syncId); @@ -2395,7 +2435,8 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s FlushFailedEngineException e = expectThrows(FlushFailedEngineException.class, engine::flush); assertThat(e.getCause().getMessage(), equalTo("power's out")); } - try (InternalEngine engine = new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null))) { + try (InternalEngine engine = + new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null), seqNoServiceSupplier)) { engine.recoverFromTranslog(); assertVisibleCount(engine, 1); final long committedGen = Long.valueOf( @@ -4088,4 +4129,63 @@ public void testSeqNoGenerator() throws IOException { } } + public void testKeepTranslogUpToGlobalCheckpoint() throws Exception { + IOUtils.close(engine, store); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final BiFunction seqNoServiceSupplier = (config, seqNoStats) -> + new SequenceNumbersService( + config.getShardId(), + config.getAllocationId(), + config.getIndexSettings(), + seqNoStats.getMaxSeqNo(), + seqNoStats.getLocalCheckpoint(), + seqNoStats.getGlobalCheckpoint()) { + @Override + public long getGlobalCheckpoint() { + return globalCheckpoint.get(); + } + }; + + final Path translogPath = createTempDir(); + store = createStore(); + try (InternalEngine engine + = new InternalEngine(config(defaultSettings, store, translogPath, NoMergePolicy.INSTANCE, null), seqNoServiceSupplier)) { + int numDocs = scaledRandomIntBetween(10, 100); + int uncommittedOps = 0; + for (int i = 0; i < numDocs; i++) { + ParseContext.Document document = testDocumentWithTextField(); + document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); + engine.index(indexForDoc(testParsedDocument(Integer.toString(i), null, document, B_1, null))); + uncommittedOps++; + if (frequently()) { + globalCheckpoint.set(randomIntBetween( + Math.toIntExact(engine.seqNoService().getGlobalCheckpoint()), + Math.toIntExact(engine.seqNoService().getLocalCheckpoint()))); + } + if (frequently()) { + engine.flush(randomBoolean(), true); + uncommittedOps = 0; + } + if (rarely()) { + engine.rollTranslogGeneration(); + } + try (Translog.Snapshot snapshot = engine.getTranslog().newSnapshot()) { + long requiredOps = engine.seqNoService().getMaxSeqNo() - Math.max(0, engine.seqNoService().getGlobalCheckpoint()); + assertThat("Should keep translog operations up to the global checkpoint", + (long) snapshot.totalOperations(), greaterThanOrEqualTo(requiredOps)); + } + assertThat(engine.getTranslog().uncommittedOperations(), equalTo(uncommittedOps)); + } + engine.flush(randomBoolean(), true); + } + // Reopen engine to test onInit with existing index commits. + try (InternalEngine engine + = new InternalEngine(config(defaultSettings, store, translogPath, NoMergePolicy.INSTANCE, null), seqNoServiceSupplier)) { + try (Translog.Snapshot snapshot = engine.getTranslog().newSnapshot()) { + long requiredOps = engine.seqNoService().getLocalCheckpoint() - Math.max(0, globalCheckpoint.get()); + assertThat("Should keep translog operations up to the global checkpoint", + (long) snapshot.totalOperations(), greaterThanOrEqualTo(requiredOps)); + } + } + } } diff --git a/core/src/test/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicyTests.java new file mode 100644 index 0000000000000..f6f23815a9b90 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicyTests.java @@ -0,0 +1,180 @@ +/* + * 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.index.IndexCommit; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import static com.carrotsearch.randomizedtesting.RandomizedTest.randomLongBetween; +import static java.util.Collections.singletonList; +import static org.hamcrest.Matchers.contains; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class KeepUntilGlobalCheckpointDeletionPolicyTests extends ESTestCase { + + public void testUnassignedGlobalCheckpointKeepAllCommits() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); + List commitList = new ArrayList<>(); + int totalCommits = between(1, 20); + for (int i = 0; i < totalCommits; i++) { + commitList.add(mockIndexCommitWithMaxSeqNo(randomNonNegativeLong())); + } + final IndexCommit[] allCommits = commitList.toArray(new IndexCommit[commitList.size()]); + assertThat("Unassigned global checkpoint keeps all commits", deletionPolicy.onInit(commitList), contains(allCommits)); + assertThat("Unassigned global checkpoint keeps all commits", deletionPolicy.onCommit(commitList), contains(allCommits)); + for (IndexCommit indexCommit : commitList) { + verify(indexCommit, never()).delete(); + } + } + + public void testKeepUpGlobalCheckpointKeepLastCommitOnly() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(); + final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); + List commitList = new ArrayList<>(); + int totalCommits = between(1, 20); + long lastMaxSeqNo = 0; + for (int i = 0; i < totalCommits; i++) { + lastMaxSeqNo += between(1, 1000); + commitList.add(mockIndexCommitWithMaxSeqNo(lastMaxSeqNo)); + } + final IndexCommit lastCommit = commitList.get(commitList.size() - 1); + globalCheckpoint.set(lastMaxSeqNo); + assertThat("Keep up global checkpoint keeps only the last commit", deletionPolicy.onInit(commitList), contains(lastCommit)); + for (int i = 0; i < commitList.size() - 1; i++) { + verify(commitList.get(i), times(1)).delete(); + } + assertThat("Keep up global checkpoint keeps only the last commit", deletionPolicy.onCommit(commitList), contains(lastCommit)); + for (int i = 0; i < commitList.size() - 1; i++) { + verify(commitList.get(i), times(2)).delete(); + } + verify(lastCommit, never()).delete(); + } + + public void testLaggingGlobalCheckpoint() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(); + final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); + List commitList = new ArrayList<>(); + List maxSeqNoList = new ArrayList<>(); + int totalCommits = between(2, 20); + long lastMaxSeqNo = 0; + for (int i = 0; i < totalCommits; i++) { + lastMaxSeqNo += between(1, 1000); + commitList.add(mockIndexCommitWithMaxSeqNo(lastMaxSeqNo)); + maxSeqNoList.add(lastMaxSeqNo); + } + + int pivot = randomInt(maxSeqNoList.size() - 2); + long currentGCP = randomLongBetween(maxSeqNoList.get(pivot), Math.max(maxSeqNoList.get(pivot), maxSeqNoList.get(pivot + 1) - 1)); + globalCheckpoint.set(currentGCP); + + final IndexCommit[] keptCommits = commitList.subList(pivot, commitList.size()).toArray(new IndexCommit[0]); + assertThat(deletionPolicy.onInit(commitList), contains(keptCommits)); + assertThat(deletionPolicy.onCommit(commitList), contains(keptCommits)); + } + + public void testLegacyIndex() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(randomInt(1000)); + final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); + + // Keep a single legacy commit + { + IndexCommit legacyCommit = mockLegacyIndexCommit(); + final List keptCommits = deletionPolicy.onInit(singletonList(legacyCommit)); + verify(legacyCommit, times(0)).delete(); + assertThat(keptCommits, contains(legacyCommit)); + } + + // Keep a safe commit, and delete a legacy commit. + { + IndexCommit legacyCommit = mockLegacyIndexCommit(); + IndexCommit safeCommit = mockIndexCommitWithMaxSeqNo(randomLongBetween(0, globalCheckpoint.get())); + + final List keptCommits = deletionPolicy.onCommit(Arrays.asList(legacyCommit, safeCommit)); + verify(legacyCommit, times(1)).delete(); + verify(safeCommit, times(0)).delete(); + assertThat(keptCommits, contains(safeCommit)); + } + + // Keep until the safe commit, and delete legacy commits + { + IndexCommit legacyCommit = mockLegacyIndexCommit(); + IndexCommit oldCommit = mockIndexCommitWithMaxSeqNo(randomLongBetween(0, globalCheckpoint.get())); + IndexCommit safeCommit = mockIndexCommitWithMaxSeqNo(randomLongBetween(0, globalCheckpoint.get())); + IndexCommit unsafeCommit = mockIndexCommitWithMaxSeqNo(globalCheckpoint.get() + between(1, 1000)); + + List keptCommits = deletionPolicy.onCommit(Arrays.asList(legacyCommit, oldCommit, safeCommit, unsafeCommit)); + verify(legacyCommit, times(1)).delete(); + verify(oldCommit, times(1)).delete(); + verify(safeCommit, times(0)).delete(); + verify(unsafeCommit, times(0)).delete(); + assertThat(keptCommits, contains(safeCommit, unsafeCommit)); + } + } + + public void testCleanupDuplicateCommits() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(); + final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); + + final long maxSeqNo = randomNonNegativeLong(); + IndexCommit commit1 = mockIndexCommitWithMaxSeqNo(maxSeqNo); + IndexCommit commit2 = mockIndexCommitWithMaxSeqNo(maxSeqNo); + + assertThat(deletionPolicy.onInit(Arrays.asList(commit1, commit2)), contains(commit2)); + verify(commit2, never()).delete(); + verify(commit1, times(1)).delete(); + + assertThat(deletionPolicy.onCommit(Arrays.asList(commit1, commit2)), contains(commit2)); + verify(commit2, never()).delete(); + verify(commit1, times(2)).delete(); + } + + IndexCommit mockIndexCommitWithMaxSeqNo(long maxSeqNo) throws IOException { + final Map userData = new HashMap<>(); + userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); + userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(randomNonNegativeLong())); + final IndexCommit commit = mock(IndexCommit.class); + when(commit.getUserData()).thenReturn(userData); + return commit; + } + + IndexCommit mockLegacyIndexCommit() throws IOException { + final Map userData = new HashMap<>(); + userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(randomNonNegativeLong())); + final IndexCommit commit = mock(IndexCommit.class); + when(commit.getUserData()).thenReturn(userData); + return commit; + } + +} diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 00f29dc252dc8..b29687186875c 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1045,8 +1045,9 @@ public void onFailure(Exception e) { closeShards(indexShard); } - public void testAcquireIndexCommit() throws IOException { - final IndexShard shard = newStartedShard(); + public void testAcquireIndexCommit() throws Exception { + boolean isPrimary = randomBoolean(); + final IndexShard shard = newStartedShard(isPrimary); int numDocs = randomInt(20); for (int i = 0; i < numDocs; i++) { indexDoc(shard, "type", "id_" + i); @@ -1063,6 +1064,12 @@ public void testAcquireIndexCommit() throws IOException { assertThat(reader.numDocs(), equalTo(flushFirst ? numDocs : 0)); } commit.close(); + // Make the global checkpoint in sync with the local checkpoint. + if (isPrimary) { + shard.getEngine().seqNoService().markAllocationIdAsInSync(shard.shardRouting.allocationId().getId(), numDocs + moreDocs - 1); + } else { + shard.updateGlobalCheckpointOnReplica(shard.getLocalCheckpoint(), "test"); + } flushShard(shard, true); // check it's clean up diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index 593e105921527..31319420ae051 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -153,17 +153,20 @@ protected Translog createTranslog(TranslogConfig config, String translogUUID) th } private void markCurrentGenAsCommitted(Translog translog) throws IOException { - commit(translog, translog.currentFileGeneration()); + long genToCommit = translog.currentFileGeneration(); + long genToRetain = randomLongBetween(translog.getDeletionPolicy().getMinTranslogGenerationForRecovery(), genToCommit); + commit(translog, genToRetain, genToCommit); } private void rollAndCommit(Translog translog) throws IOException { translog.rollGeneration(); - commit(translog, translog.currentFileGeneration()); + markCurrentGenAsCommitted(translog); } - private void commit(Translog translog, long genToCommit) throws IOException { + private void commit(Translog translog, long genToRetain, long genToCommit) throws IOException { final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); - deletionPolicy.setMinTranslogGenerationForRecovery(genToCommit); + deletionPolicy.setTranslogGenerationOfLastCommit(genToCommit); + deletionPolicy.setMinTranslogGenerationForRecovery(genToRetain); long minGenRequired = deletionPolicy.minTranslogGenRequired(translog.getReaders(), translog.getCurrent()); translog.trimUnreferencedReaders(); assertThat(minGenRequired, equalTo(translog.getMinFileGeneration())); @@ -440,6 +443,31 @@ public void testStats() throws IOException { } } + public void testUncommittedOperations() throws Exception { + final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); + deletionPolicy.setRetentionAgeInMillis(randomLong()); + deletionPolicy.setRetentionSizeInBytes(randomLong()); + + final int operations = scaledRandomIntBetween(10, 100); + int uncommittedOps = 0; + int operationsInLastGen = 0; + for (int i = 0; i < operations; i++) { + translog.add(new Translog.Index("test", Integer.toString(i), i, new byte[]{1})); + uncommittedOps++; + operationsInLastGen++; + if (rarely()) { + translog.rollGeneration(); + operationsInLastGen = 0; + } + assertThat(translog.uncommittedOperations(), equalTo(uncommittedOps)); + if (frequently()) { + markCurrentGenAsCommitted(translog); + assertThat(translog.uncommittedOperations(), equalTo(operationsInLastGen)); + uncommittedOps = operationsInLastGen; + } + } + } + public void testTotalTests() { final TranslogStats total = new TranslogStats(); final int n = randomIntBetween(0, 16); @@ -2413,8 +2441,9 @@ public void testRollGeneration() throws Exception { for (int i = 0; i <= rolls; i++) { assertFileIsPresent(translog, generation + i); } - commit(translog, generation + rolls); - assertThat(translog.currentFileGeneration(), equalTo(generation + rolls )); + long minGenForRecovery = randomLongBetween(generation, generation + rolls); + commit(translog, minGenForRecovery, generation + rolls); + assertThat(translog.currentFileGeneration(), equalTo(generation + rolls)); assertThat(translog.uncommittedOperations(), equalTo(0)); if (longRetention) { for (int i = 0; i <= rolls; i++) { @@ -2423,17 +2452,19 @@ public void testRollGeneration() throws Exception { deletionPolicy.setRetentionAgeInMillis(randomBoolean() ? 100 : -1); assertBusy(() -> { translog.trimUnreferencedReaders(); - for (int i = 0; i < rolls; i++) { - assertFileDeleted(translog, generation + i); + for (long i = 0; i < minGenForRecovery; i++) { + assertFileDeleted(translog, i); } }); } else { // immediate cleanup - for (int i = 0; i < rolls; i++) { - assertFileDeleted(translog, generation + i); + for (long i = 0; i < minGenForRecovery; i++) { + assertFileDeleted(translog, i); } } - assertFileIsPresent(translog, generation + rolls); + for (long i = minGenForRecovery; i < generation + rolls; i++) { + assertFileIsPresent(translog, i); + } } public void testMinSeqNoBasedAPI() throws IOException { @@ -2516,11 +2547,8 @@ public void testSimpleCommit() throws IOException { translog.rollGeneration(); } } - - final long generation = - randomIntBetween(1, Math.toIntExact(translog.currentFileGeneration())); - commit(translog, generation); - } + long lastGen = randomLongBetween(1, translog.currentFileGeneration()); + commit(translog, randomLongBetween(1, lastGen), lastGen); } public void testAcquiredLockIsPassedToDeletionPolicy() throws IOException { final int operations = randomIntBetween(1, 4096); @@ -2531,7 +2559,9 @@ public void testAcquiredLockIsPassedToDeletionPolicy() throws IOException { translog.rollGeneration(); } if (rarely()) { - commit(translog, randomLongBetween(deletionPolicy.getMinTranslogGenerationForRecovery(), translog.currentFileGeneration())); + long lastGen = randomLongBetween(deletionPolicy.getTranslogGenerationOfLastCommit(), translog.currentFileGeneration()); + long minGen = randomLongBetween(deletionPolicy.getMinTranslogGenerationForRecovery(), lastGen); + commit(translog, minGen, lastGen); } if (frequently()) { long minGen; diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index 55b7e22eb8a38..e1e2e37157f0f 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -113,27 +113,10 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { orgReplica.applyIndexOperationOnReplica(3, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, SourceToParse.source(orgReplica.shardId().getIndexName(), "type", "id2", new BytesArray("{}"), XContentType.JSON), u -> {}); - final int translogOps; + final int translogOps = 4; // 3 ops + seqno gaps if (randomBoolean()) { - if (randomBoolean()) { - logger.info("--> flushing shard (translog will be trimmed)"); - IndexMetaData.Builder builder = IndexMetaData.builder(orgReplica.indexSettings().getIndexMetaData()); - builder.settings(Settings.builder().put(orgReplica.indexSettings().getSettings()) - .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") - .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") - ); - orgReplica.indexSettings().updateIndexMetaData(builder.build()); - orgReplica.onSettingsChanged(); - translogOps = 3; // 2 ops + seqno gaps - } else { - logger.info("--> flushing shard (translog will be retained)"); - translogOps = 4; // 3 ops + seqno gaps - } flushShard(orgReplica); - } else { - translogOps = 4; // 3 ops + seqno gaps } - final IndexShard orgPrimary = shards.getPrimary(); shards.promoteReplicaToPrimary(orgReplica).get(); // wait for primary/replica sync to make sure seq# gap is closed. From 97e6cffd5a2e47b6c1b01f67cb9d0b08c4c41308 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 1 Dec 2017 13:58:44 -0500 Subject: [PATCH 02/29] add doc for policy --- .../common/lucene/index/ESIndexDeletionPolicy.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java b/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java index 573a446c0ace4..cb94b1ed120f5 100644 --- a/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java @@ -21,17 +21,22 @@ import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexDeletionPolicy; +import org.apache.lucene.index.SnapshotDeletionPolicy; import java.io.IOException; import java.util.List; /** - * An {@link IndexDeletionPolicy} that deletes unneeded index commits, and returns index commits are not deleted by this policy. + * A variant of {@link IndexDeletionPolicy} that deletes unneeded index commits, but also returns index commits are not deleted + * by this policy. Returning a list of kept index commits provides a reliable way to check if an index commit is retained or not + * as other deletion policy may suppress the delete request. For example, {@link SnapshotDeletionPolicy} suppresses delete requests + * for index commits that are being snapshotted. */ public interface ESIndexDeletionPolicy { /** * Similar to {@link IndexDeletionPolicy#onInit(List)} but returns a list of kept index commits. + * This is called once when a writer is first instantiated to give the policy a chance to remove old commit points. * * @param commits A list of index commits sorted by age (the 0th one is the oldest commit). * @return a list of index commits that are not deleted by this policy. @@ -40,6 +45,7 @@ public interface ESIndexDeletionPolicy { /** * Similar to {@link IndexDeletionPolicy#onCommit(List)} but returns a list of kept index commits. + * This is called each time the writer completed a commit. This gives the policy a chance to remove old commit points with each commit. * * @param commits A list of index commits sorted by age (the 0th one is the oldest commit). * @return a list of index commits that are not deleted by this policy. From 48381a271e1bf4db47ab2ba09c2d006b8c4e0d33 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 1 Dec 2017 13:59:19 -0500 Subject: [PATCH 03/29] use set --- ...eepUntilGlobalCheckpointDeletionPolicy.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java index 4f43c8a434f5e..25c5b8c2599cd 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java @@ -19,6 +19,10 @@ package org.elasticsearch.index.engine; +import com.carrotsearch.hppc.IntHashSet; +import com.carrotsearch.hppc.IntSet; +import com.carrotsearch.hppc.LongHashSet; +import com.carrotsearch.hppc.LongSet; import org.apache.lucene.index.IndexCommit; import org.elasticsearch.common.lucene.index.ESIndexDeletionPolicy; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -32,7 +36,7 @@ /** * An {@link ESIndexDeletionPolicy} that deletes index commits that are not required for recovery. - * In particular, this policy will delete index commits whose max sequence number is smaller (or equal) than + * In particular, this policy will delete index commits whose max sequence number is at most * the current global checkpoint except the index commit which has the highest max sequence number among those. */ final class KeepUntilGlobalCheckpointDeletionPolicy implements ESIndexDeletionPolicy { @@ -56,7 +60,7 @@ public List onCommit(List commits) throws IO final List keptCommits = new ArrayList<>(); final int keptPosition = indexOfKeptCommits(commits); - final List duplicateIndexes = indexesOfDuplicateCommits(commits); + final IntSet duplicateIndexes = indexesOfDuplicateCommits(commits); for (int i = 0; i < commits.size() - 1; i++) { final IndexCommit commit = commits.get(i); @@ -106,18 +110,18 @@ private int indexOfKeptCommits(List commits) throws IOExc * * @return index positions of duplicate commits. */ - private List indexesOfDuplicateCommits(List commits) throws IOException { - final List duplicateEntries = new ArrayList<>(); - long lastMaxSeqNo = Long.MIN_VALUE; + private IntSet indexesOfDuplicateCommits(List commits) throws IOException { + final LongSet seenMaxSeqNo = new LongHashSet(); + final IntSet duplicateEntries = new IntHashSet(); + for (int i = commits.size() - 1; i >= 0; i--) { final Map commitUserData = commits.get(i).getUserData(); // 5.x commits do not contain MAX_SEQ_NO. if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO)) { final long maxSeqNoFromCommit = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)); - if (lastMaxSeqNo == maxSeqNoFromCommit) { + if (seenMaxSeqNo.add(maxSeqNoFromCommit) == false) { duplicateEntries.add(i); } - lastMaxSeqNo = maxSeqNoFromCommit; } } return duplicateEntries; From 4b9ebb6354676a6f34b588dd2b9d0e49dfc9592f Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 1 Dec 2017 14:01:07 -0500 Subject: [PATCH 04/29] compact switch --- .../elasticsearch/index/engine/InternalEngine.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 1e2a645b310b0..69dadd1dbc801 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -25,7 +25,6 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LiveIndexWriterConfig; import org.apache.lucene.index.MergePolicy; @@ -360,25 +359,22 @@ static SequenceNumbersService sequenceNumberService( } private SeqNoStats loadSeqNoStats(EngineConfig.OpenMode openMode) throws IOException { - final SeqNoStats seqNoStats; switch (openMode) { case OPEN_INDEX_AND_TRANSLOG: final long globalCheckpoint = Translog.readGlobalCheckpoint(engineConfig.getTranslogConfig().getTranslogPath()); - seqNoStats = store.loadSeqNoStats(globalCheckpoint); - break; + return store.loadSeqNoStats(globalCheckpoint); + case OPEN_INDEX_CREATE_TRANSLOG: - seqNoStats = store.loadSeqNoStats(SequenceNumbers.UNASSIGNED_SEQ_NO); - break; + return store.loadSeqNoStats(SequenceNumbers.UNASSIGNED_SEQ_NO); + case CREATE_INDEX_AND_TRANSLOG: - seqNoStats = new SeqNoStats( + return new SeqNoStats( SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.UNASSIGNED_SEQ_NO); - break; default: throw new IllegalArgumentException(openMode.toString()); } - return seqNoStats; } @Override From 7f32e22da6dbc8208e4a6f11a4ea5ea28aaad49f Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 1 Dec 2017 14:01:45 -0500 Subject: [PATCH 05/29] format --- .../java/org/elasticsearch/index/translog/TranslogTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index 31319420ae051..57b3eb8e61e59 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -2548,7 +2548,8 @@ public void testSimpleCommit() throws IOException { } } long lastGen = randomLongBetween(1, translog.currentFileGeneration()); - commit(translog, randomLongBetween(1, lastGen), lastGen); } + commit(translog, randomLongBetween(1, lastGen), lastGen); + } public void testAcquiredLockIsPassedToDeletionPolicy() throws IOException { final int operations = randomIntBetween(1, 4096); From 94b735f426d5ccb120c99af275e92c652f22bd4e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 1 Dec 2017 22:32:51 -0500 Subject: [PATCH 06/29] add randomLong to ESTestCase --- .../src/main/java/org/elasticsearch/test/ESTestCase.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index e10411e5a435e..9cb0ce35c7591 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -459,6 +459,13 @@ public static int randomIntBetween(int min, int max) { return RandomNumbers.randomIntBetween(random(), min, max); } + /** + * A random long number between min (inclusive) and max (inclusive). + */ + public static long randomLongBetween(long min, long max) { + return RandomNumbers.randomLongBetween(random(), min, max); + } + /** * Returns a "scaled" number of iterations for loops which can have a variable * iteration count. This method is effectively From ab262b1d381cf25b7375fab63735d5e105cdf3f7 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 1 Dec 2017 23:26:55 -0500 Subject: [PATCH 07/29] move gcp policy to the combined policy --- .../lucene/index/ESIndexDeletionPolicy.java | 54 ----- .../index/engine/CombinedDeletionPolicy.java | 108 ++++++++- .../index/engine/InternalEngine.java | 4 +- ...epUntilGlobalCheckpointDeletionPolicy.java | 129 ----------- .../engine/CombinedDeletionPolicyTests.java | 210 ++++++++++-------- ...ilGlobalCheckpointDeletionPolicyTests.java | 180 --------------- 6 files changed, 212 insertions(+), 473 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java delete mode 100644 core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java delete mode 100644 core/src/test/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicyTests.java diff --git a/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java b/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java deleted file mode 100644 index cb94b1ed120f5..0000000000000 --- a/core/src/main/java/org/elasticsearch/common/lucene/index/ESIndexDeletionPolicy.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.common.lucene.index; - -import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexDeletionPolicy; -import org.apache.lucene.index.SnapshotDeletionPolicy; - -import java.io.IOException; -import java.util.List; - -/** - * A variant of {@link IndexDeletionPolicy} that deletes unneeded index commits, but also returns index commits are not deleted - * by this policy. Returning a list of kept index commits provides a reliable way to check if an index commit is retained or not - * as other deletion policy may suppress the delete request. For example, {@link SnapshotDeletionPolicy} suppresses delete requests - * for index commits that are being snapshotted. - */ -public interface ESIndexDeletionPolicy { - - /** - * Similar to {@link IndexDeletionPolicy#onInit(List)} but returns a list of kept index commits. - * This is called once when a writer is first instantiated to give the policy a chance to remove old commit points. - * - * @param commits A list of index commits sorted by age (the 0th one is the oldest commit). - * @return a list of index commits that are not deleted by this policy. - */ - List onInit(List commits) throws IOException; - - /** - * Similar to {@link IndexDeletionPolicy#onCommit(List)} but returns a list of kept index commits. - * This is called each time the writer completed a commit. This gives the policy a chance to remove old commit points with each commit. - * - * @param commits A list of index commits sorted by age (the 0th one is the oldest commit). - * @return a list of index commits that are not deleted by this policy. - */ - List onCommit(List commits) throws IOException; -} diff --git a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index b67250563920b..f4bbed51fea97 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -19,34 +19,45 @@ package org.elasticsearch.index.engine; +import com.carrotsearch.hppc.IntHashSet; +import com.carrotsearch.hppc.IntSet; +import com.carrotsearch.hppc.LongHashSet; +import com.carrotsearch.hppc.LongSet; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexDeletionPolicy; -import org.elasticsearch.common.lucene.index.ESIndexDeletionPolicy; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogDeletionPolicy; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.function.LongSupplier; /** * An {@link IndexDeletionPolicy} that coordinates between Lucene's commits and the retention of translog generation files, * making sure that all translog files that are needed to recover from the Lucene commit are not deleted. + *

+ * In particular, this policy will delete index commits whose max sequence number is at most + * the current global checkpoint except the index commit which has the highest max sequence number among those. */ -class CombinedDeletionPolicy extends IndexDeletionPolicy { +final class CombinedDeletionPolicy extends IndexDeletionPolicy { private final TranslogDeletionPolicy translogDeletionPolicy; private final EngineConfig.OpenMode openMode; - private final ESIndexDeletionPolicy indexDeletionPolicy; + private final LongSupplier globalCheckpointSupplier; - CombinedDeletionPolicy(ESIndexDeletionPolicy indexDeletionPolicy, TranslogDeletionPolicy translogDeletionPolicy, - EngineConfig.OpenMode openMode) { - this.indexDeletionPolicy = indexDeletionPolicy; - this.translogDeletionPolicy = translogDeletionPolicy; + CombinedDeletionPolicy(EngineConfig.OpenMode openMode, TranslogDeletionPolicy translogDeletionPolicy, + LongSupplier globalCheckpointSupplier) { this.openMode = openMode; + this.translogDeletionPolicy = translogDeletionPolicy; + this.globalCheckpointSupplier = globalCheckpointSupplier; } @Override public void onInit(List commits) throws IOException { - final List keptCommits = indexDeletionPolicy.onInit(commits); + final List keptCommits = deleteOldIndexCommits(commits); switch (openMode) { case CREATE_INDEX_AND_TRANSLOG: break; @@ -64,14 +75,13 @@ public void onInit(List commits) throws IOException { @Override public void onCommit(List commits) throws IOException { - assert commits.isEmpty() == false; - final List keptCommits = indexDeletionPolicy.onCommit(commits); + final List keptCommits = deleteOldIndexCommits(commits); setLastCommittedTranslogGeneration(keptCommits); } private void setLastCommittedTranslogGeneration(List keptCommits) throws IOException { assert keptCommits.isEmpty() == false : "All index commits were deleted"; - assert keptCommits.stream().allMatch(commit -> commit.isDeleted() == false) : "Kept commits must not be deleted"; + assert keptCommits.stream().allMatch(c -> c.isDeleted() == false) : "All kept commits must not be deleted"; final IndexCommit lastCommit = keptCommits.get(keptCommits.size() - 1); final long lastGen = Long.parseLong(lastCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); @@ -84,4 +94,80 @@ private void setLastCommittedTranslogGeneration(List keptCommits) t translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen); translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); } + + /** + * Deletes old index commits which are not required for operation based recovery. + */ + private List deleteOldIndexCommits(List commits) throws IOException { + if (commits.isEmpty()) { + return Collections.emptyList(); + } + + final List keptCommits = new ArrayList<>(); + final int keptPosition = indexOfKeptCommits(commits); + final IntSet duplicateIndexes = indexesOfDuplicateCommits(commits); + + for (int i = 0; i < commits.size() - 1; i++) { + final IndexCommit commit = commits.get(i); + if (i < keptPosition || duplicateIndexes.contains(i)) { + commit.delete(); + } else { + keptCommits.add(commit); + } + } + keptCommits.add(commits.get(commits.size() - 1)); // Always keep the last commit. + + return keptCommits; + } + + /** + * Find the index position of a safe index commit whose max sequence number is not greater than the global checkpoint. + */ + private int indexOfKeptCommits(List commits) throws IOException { + final long currentGlobalCheckpoint = globalCheckpointSupplier.getAsLong(); + + // Commits are sorted by age (the 0th one is the oldest commit). + for (int i = commits.size() - 1; i >= 0; i--) { + final Map commitUserData = commits.get(i).getUserData(); + // 5.x commits do not contain MAX_SEQ_NO. + if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO) == false) { + return i; + } + final long maxSeqNoFromCommit = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)); + if (maxSeqNoFromCommit <= currentGlobalCheckpoint) { + return i; + } + } + /* + * We may reach to this point in these cases: + * 1. In the previous 6.x, we keep only the last commit - which is likely not a safe commit if writes are in progress. + * Thus, after upgrading, we may not find a safe commit until we can reserve one. + * 2. In peer-recovery, if the file-based happens, a replica will be received the latest commit from a primary. + * However, that commit may not be a safe commit if writes are in progress in the primary. + */ + return -1; + } + + /** + * In some cases, we may have more than one index commits with the same max sequence number. + * We better scan and delete these duplicate index commits as soon as possible. + * + * @return index positions of duplicate commits. + */ + private IntSet indexesOfDuplicateCommits(List commits) throws IOException { + final LongSet seenMaxSeqNo = new LongHashSet(); + final IntSet duplicateIndexes = new IntHashSet(); + + for (int i = commits.size() - 1; i >= 0; i--) { + final Map commitUserData = commits.get(i).getUserData(); + // 5.x commits do not contain MAX_SEQ_NO. + if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO)) { + final long maxSeqNoFromCommit = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)); + if (seenMaxSeqNo.add(maxSeqNoFromCommit) == false) { + duplicateIndexes.add(i); + } + } + } + return duplicateIndexes; + } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 69dadd1dbc801..5c16a8b1731b2 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -182,8 +182,8 @@ public InternalEngine(EngineConfig engineConfig) { final SeqNoStats seqNoStats = loadSeqNoStats(openMode); logger.trace("recovered [{}]", seqNoStats); this.seqNoService = seqNoServiceSupplier.apply(engineConfig, seqNoStats); - this.snapshotDeletionPolicy = new SnapshotDeletionPolicy(new CombinedDeletionPolicy( - new KeepUntilGlobalCheckpointDeletionPolicy(seqNoService::getGlobalCheckpoint), translogDeletionPolicy, openMode) + this.snapshotDeletionPolicy = new SnapshotDeletionPolicy( + new CombinedDeletionPolicy(openMode, translogDeletionPolicy, seqNoService::getGlobalCheckpoint) ); writer = createWriter(openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG); updateMaxUnsafeAutoIdTimestampFromWriter(writer); diff --git a/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java deleted file mode 100644 index 25c5b8c2599cd..0000000000000 --- a/core/src/main/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicy.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -import com.carrotsearch.hppc.IntHashSet; -import com.carrotsearch.hppc.IntSet; -import com.carrotsearch.hppc.LongHashSet; -import com.carrotsearch.hppc.LongSet; -import org.apache.lucene.index.IndexCommit; -import org.elasticsearch.common.lucene.index.ESIndexDeletionPolicy; -import org.elasticsearch.index.seqno.SequenceNumbers; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.function.LongSupplier; - -/** - * An {@link ESIndexDeletionPolicy} that deletes index commits that are not required for recovery. - * In particular, this policy will delete index commits whose max sequence number is at most - * the current global checkpoint except the index commit which has the highest max sequence number among those. - */ -final class KeepUntilGlobalCheckpointDeletionPolicy implements ESIndexDeletionPolicy { - private final LongSupplier globalCheckpointSupplier; - - KeepUntilGlobalCheckpointDeletionPolicy(LongSupplier globalCheckpointSupplier) { - this.globalCheckpointSupplier = globalCheckpointSupplier; - } - - @Override - public List onInit(List commits) throws IOException { - if (commits.isEmpty()) { - return Collections.emptyList(); - } - return onCommit(commits); - } - - @Override - public List onCommit(List commits) throws IOException { - assert commits.isEmpty() == false : "onCommit() must be called with a non-empty list of commits"; - - final List keptCommits = new ArrayList<>(); - final int keptPosition = indexOfKeptCommits(commits); - final IntSet duplicateIndexes = indexesOfDuplicateCommits(commits); - - for (int i = 0; i < commits.size() - 1; i++) { - final IndexCommit commit = commits.get(i); - if (i < keptPosition || duplicateIndexes.contains(i)) { - commit.delete(); - } else { - keptCommits.add(commit); - } - } - keptCommits.add(commits.get(commits.size() - 1)); // Always keep the last commit. - - assert keptCommits.stream().allMatch(c -> c.isDeleted() == false) : "All kept commits must not be deleted"; - return keptCommits; - } - - /** - * Find the index position of a safe index commit whose max sequence number is not greater than the global checkpoint. - */ - private int indexOfKeptCommits(List commits) throws IOException { - final long currentGlobalCheckpoint = globalCheckpointSupplier.getAsLong(); - - // Commits are sorted by age (the 0th one is the oldest commit). - for (int i = commits.size() - 1; i >= 0; i--) { - final Map commitUserData = commits.get(i).getUserData(); - // 5.x commits do not contain MAX_SEQ_NO. - if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO) == false) { - return i; - } - final long maxSeqNoFromCommit = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)); - if (maxSeqNoFromCommit <= currentGlobalCheckpoint) { - return i; - } - } - /* - * We may reach to this point in these cases: - * 1. In the previous 6.x, we keep only the last commit - which is likely not a safe commit if writes are in progress. - * Thus, after upgrading, we may not find a safe commit until we can reserve one. - * 2. In peer-recovery, if the file-based happens, a replica will be received the latest commit from a primary. - * However, that commit may not be a safe commit if writes are in progress in the primary. - */ - return -1; - } - - /** - * In some cases, we may have more than one index commits with the same max sequence number. - * We better scan and delete these duplicate index commits as soon as possible. - * - * @return index positions of duplicate commits. - */ - private IntSet indexesOfDuplicateCommits(List commits) throws IOException { - final LongSet seenMaxSeqNo = new LongHashSet(); - final IntSet duplicateEntries = new IntHashSet(); - - for (int i = commits.size() - 1; i >= 0; i--) { - final Map commitUserData = commits.get(i).getUserData(); - // 5.x commits do not contain MAX_SEQ_NO. - if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO)) { - final long maxSeqNoFromCommit = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)); - if (seenMaxSeqNo.add(maxSeqNoFromCommit) == false) { - duplicateEntries.add(i); - } - } - } - return duplicateEntries; - } -} diff --git a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index 1637a4545fdb3..dfcf0c2d3e666 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -22,7 +22,7 @@ import com.carrotsearch.hppc.LongArrayList; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.SnapshotDeletionPolicy; -import org.elasticsearch.common.lucene.index.ESIndexDeletionPolicy; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogDeletionPolicy; import org.elasticsearch.test.ESTestCase; @@ -30,127 +30,143 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; +import java.util.HashMap; import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import static java.util.Collections.singletonList; +import static org.elasticsearch.index.engine.EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG; +import static org.elasticsearch.index.engine.EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; -import static org.mockito.Matchers.anyListOf; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class CombinedDeletionPolicyTests extends ESTestCase { - public void testPassThrough() throws IOException { - ESIndexDeletionPolicy indexDeletionPolicy = mock(ESIndexDeletionPolicy.class); - CombinedDeletionPolicy combinedDeletionPolicy = new CombinedDeletionPolicy(indexDeletionPolicy, createTranslogDeletionPolicy(), - EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); - List commitList = new ArrayList<>(); - long count = randomIntBetween(1, 3); - for (int i = 0; i < count; i++) { - commitList.add(mockIndexCommitWithTranslogGen(randomNonNegativeLong())); - } - when(indexDeletionPolicy.onInit(anyListOf(IndexCommit.class))).thenReturn(commitList); - when(indexDeletionPolicy.onCommit(anyListOf(IndexCommit.class))).thenReturn(commitList); - combinedDeletionPolicy.onInit(commitList); - verify(indexDeletionPolicy, times(1)).onInit(commitList); - combinedDeletionPolicy.onCommit(commitList); - verify(indexDeletionPolicy, times(1)).onCommit(commitList); - } + public void testCommitsKeepUpToGlobalCheckpoint() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(); + TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); - public void testSettingTranslogGenerations() throws IOException { - final int count = randomIntBetween(10, 20); + final LongArrayList maxSeqNoList = new LongArrayList(); + final LongArrayList translogGenList = new LongArrayList(); final List commitList = new ArrayList<>(); - final LongArrayList translogGens = new LongArrayList(); - long lastGen = 0; - for (int i = 0; i < count; i++) { - lastGen += randomIntBetween(10, 20000); - commitList.add(mockIndexCommitWithTranslogGen(lastGen)); - translogGens.add(lastGen); + int totalCommits = between(2, 20); + long lastMaxSeqNo = 0; + long lastTranslogGen = 0; + + for (int i = 0; i < totalCommits; i++) { + lastMaxSeqNo += between(1, 10000); + lastTranslogGen += between(1, 100); + commitList.add(mockIndexCommit(lastMaxSeqNo, lastTranslogGen)); + maxSeqNoList.add(lastMaxSeqNo); + translogGenList.add(lastTranslogGen); } - final AtomicInteger keepN = new AtomicInteger(between(1, commitList.size())); - final ESIndexDeletionPolicy keepLastNPolicy = new ESIndexDeletionPolicy() { - @Override - public List onInit(List commits) throws IOException { - return onCommit(commits); + int keptIndex = randomInt(commitList.size() - 2); + globalCheckpoint.set(randomLongBetween( + maxSeqNoList.get(keptIndex), Math.max(maxSeqNoList.get(keptIndex), maxSeqNoList.get(keptIndex + 1) - 1)) + ); + indexPolicy.onCommit(commitList); + + for (int i = 0; i < commitList.size(); i++) { + if (i < keptIndex) { + verify(commitList.get(i), times(1)).delete(); + } else { + verify(commitList.get(i), never()).delete(); } - - @Override - public List onCommit(List commits) throws IOException { - return new ArrayList<>(commits.subList(commits.size() - keepN.get(), commits.size())); - } - }; - final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(randomLong(), randomLong()); - CombinedDeletionPolicy combinedDeletionPolicy = new CombinedDeletionPolicy(keepLastNPolicy, translogDeletionPolicy, - EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); - - combinedDeletionPolicy.onInit(commitList); - assertThat(translogDeletionPolicy.getTranslogGenerationOfLastCommit(), - equalTo(lastGen)); - assertThat(translogDeletionPolicy.getMinTranslogGenerationForRecovery(), - equalTo(translogGens.get(translogGens.size() - keepN.get()))); - - int moreCommits = randomIntBetween(0, 10); - for (int i = 0; i < moreCommits; i++) { - lastGen += randomIntBetween(10, 20000); - commitList.add(mockIndexCommitWithTranslogGen(lastGen)); - translogGens.add(lastGen); } - keepN.addAndGet(randomInt(moreCommits)); - - combinedDeletionPolicy.onCommit(commitList); - assertThat(translogDeletionPolicy.getTranslogGenerationOfLastCommit(), - equalTo(lastGen)); - assertThat(translogDeletionPolicy.getMinTranslogGenerationForRecovery(), - equalTo(translogGens.get(translogGens.size() - keepN.get()))); + assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(translogGenList.get(keptIndex))); + assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); } public void testIgnoreSnapshottingCommits() throws Exception { - final ESIndexDeletionPolicy keepOnlyLastPolicy = new ESIndexDeletionPolicy() { - @Override - public List onInit(List commits) throws IOException { - return onCommit(commits); - } - @Override - public List onCommit(List commits) throws IOException { - for (int i = 0; i < commits.size() - 1; i++) { - commits.get(i).delete(); - } - return Arrays.asList(commits.get(commits.size() - 1)); - } - }; - final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(randomLong(), randomLong()); - CombinedDeletionPolicy combinedDeletionPolicy = new CombinedDeletionPolicy(keepOnlyLastPolicy, translogDeletionPolicy, - EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); - SnapshotDeletionPolicy snapshotDeletionPolicy = new SnapshotDeletionPolicy(combinedDeletionPolicy); + final AtomicLong globalCheckpoint = new AtomicLong(); + TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_CREATE_TRANSLOG, translogPolicy, globalCheckpoint::get); - final int count = randomIntBetween(10, 20); - final List commitList = new ArrayList<>(); - long lastGen = 0; - for (int i = 0; i < count; i++) { - lastGen += randomIntBetween(10, 20000); - commitList.add(mockIndexCommitWithTranslogGen(lastGen)); - } - snapshotDeletionPolicy.onInit(commitList); - assertThat(translogDeletionPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastGen)); - assertThat(translogDeletionPolicy.getMinTranslogGenerationForRecovery(), equalTo(lastGen)); + long firstMaxSeqNo = randomLongBetween(0, Long.MAX_VALUE - 1); + long secondMaxSeqNo = randomLongBetween(firstMaxSeqNo + 1, Long.MAX_VALUE); + + long lastTranslogGen = randomNonNegativeLong(); + final IndexCommit firstCommit = mockIndexCommit(firstMaxSeqNo, randomNonNegativeLong()); + final IndexCommit secondCommit = mockIndexCommit(secondMaxSeqNo, lastTranslogGen); + SnapshotDeletionPolicy snapshotDeletionPolicy = new SnapshotDeletionPolicy(indexPolicy); + snapshotDeletionPolicy.onInit(Arrays.asList(firstCommit)); snapshotDeletionPolicy.snapshot(); - long newGen = lastGen + randomIntBetween(10, 20000); - commitList.add(mockIndexCommitWithTranslogGen(newGen)); - snapshotDeletionPolicy.onCommit(commitList); - assertThat(translogDeletionPolicy.getTranslogGenerationOfLastCommit(), equalTo(newGen)); - // The previous commit is being snapshotted but ignored by the CombinedPolicy. - assertThat(translogDeletionPolicy.getMinTranslogGenerationForRecovery(), equalTo(newGen)); + assertThat(snapshotDeletionPolicy.getSnapshots(), contains(firstCommit)); + + // SnapshotPolicy prevents the first commit from deleting, but CombinedPolicy does not retain its translog. + globalCheckpoint.set(randomLongBetween(secondMaxSeqNo, Long.MAX_VALUE)); + snapshotDeletionPolicy.onCommit(Arrays.asList(firstCommit, secondCommit)); + verify(firstCommit, never()).delete(); + verify(secondCommit, never()).delete(); + assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(lastTranslogGen)); + assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); + } + + public void testLegacyIndex() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(); + TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); + + long legacyTranslogGen = randomNonNegativeLong(); + IndexCommit legacyCommit = mockLegacyIndexCommit(legacyTranslogGen); + indexPolicy.onInit(singletonList(legacyCommit)); + verify(legacyCommit, never()).delete(); + assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(legacyTranslogGen)); + assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(legacyTranslogGen)); + + globalCheckpoint.set(randomNonNegativeLong()); + long safeTranslogGen = randomLongBetween(legacyTranslogGen, Long.MAX_VALUE); + IndexCommit safeCommit = mockIndexCommit(randomLongBetween(0, globalCheckpoint.get()), safeTranslogGen); + + indexPolicy.onCommit(Arrays.asList(legacyCommit, safeCommit)); + verify(legacyCommit, times(1)).delete(); + verify(safeCommit, times(0)).delete(); + assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(safeTranslogGen)); + assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(safeTranslogGen)); + } + + public void testCleanupDuplicateCommits() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(); + TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); + + final long maxSeqNo = randomNonNegativeLong(); + final long lastTranslogGen = randomNonNegativeLong(); + IndexCommit commit1 = mockIndexCommit(maxSeqNo, randomNonNegativeLong()); + IndexCommit commit2 = mockIndexCommit(maxSeqNo, lastTranslogGen); + + indexPolicy.onInit(Arrays.asList(commit1, commit2)); + verify(commit2, never()).delete(); + verify(commit1, times(1)).delete(); + + assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); + assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(lastTranslogGen)); + } + + IndexCommit mockIndexCommit(long maxSeqNo, long translogGen) throws IOException { + final Map userData = new HashMap<>(); + userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); + userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGen)); + final IndexCommit commit = mock(IndexCommit.class); + when(commit.getUserData()).thenReturn(userData); + return commit; } - IndexCommit mockIndexCommitWithTranslogGen(long gen) throws IOException { - IndexCommit commit = mock(IndexCommit.class); - when(commit.getUserData()).thenReturn(Collections.singletonMap(Translog.TRANSLOG_GENERATION_KEY, Long.toString(gen))); + IndexCommit mockLegacyIndexCommit(long translogGen) throws IOException { + final Map userData = new HashMap<>(); + userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGen)); + final IndexCommit commit = mock(IndexCommit.class); + when(commit.getUserData()).thenReturn(userData); return commit; } } diff --git a/core/src/test/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicyTests.java deleted file mode 100644 index f6f23815a9b90..0000000000000 --- a/core/src/test/java/org/elasticsearch/index/engine/KeepUntilGlobalCheckpointDeletionPolicyTests.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -import org.apache.lucene.index.IndexCommit; -import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.test.ESTestCase; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; - -import static com.carrotsearch.randomizedtesting.RandomizedTest.randomLongBetween; -import static java.util.Collections.singletonList; -import static org.hamcrest.Matchers.contains; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class KeepUntilGlobalCheckpointDeletionPolicyTests extends ESTestCase { - - public void testUnassignedGlobalCheckpointKeepAllCommits() throws Exception { - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); - final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); - List commitList = new ArrayList<>(); - int totalCommits = between(1, 20); - for (int i = 0; i < totalCommits; i++) { - commitList.add(mockIndexCommitWithMaxSeqNo(randomNonNegativeLong())); - } - final IndexCommit[] allCommits = commitList.toArray(new IndexCommit[commitList.size()]); - assertThat("Unassigned global checkpoint keeps all commits", deletionPolicy.onInit(commitList), contains(allCommits)); - assertThat("Unassigned global checkpoint keeps all commits", deletionPolicy.onCommit(commitList), contains(allCommits)); - for (IndexCommit indexCommit : commitList) { - verify(indexCommit, never()).delete(); - } - } - - public void testKeepUpGlobalCheckpointKeepLastCommitOnly() throws Exception { - final AtomicLong globalCheckpoint = new AtomicLong(); - final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); - List commitList = new ArrayList<>(); - int totalCommits = between(1, 20); - long lastMaxSeqNo = 0; - for (int i = 0; i < totalCommits; i++) { - lastMaxSeqNo += between(1, 1000); - commitList.add(mockIndexCommitWithMaxSeqNo(lastMaxSeqNo)); - } - final IndexCommit lastCommit = commitList.get(commitList.size() - 1); - globalCheckpoint.set(lastMaxSeqNo); - assertThat("Keep up global checkpoint keeps only the last commit", deletionPolicy.onInit(commitList), contains(lastCommit)); - for (int i = 0; i < commitList.size() - 1; i++) { - verify(commitList.get(i), times(1)).delete(); - } - assertThat("Keep up global checkpoint keeps only the last commit", deletionPolicy.onCommit(commitList), contains(lastCommit)); - for (int i = 0; i < commitList.size() - 1; i++) { - verify(commitList.get(i), times(2)).delete(); - } - verify(lastCommit, never()).delete(); - } - - public void testLaggingGlobalCheckpoint() throws Exception { - final AtomicLong globalCheckpoint = new AtomicLong(); - final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); - List commitList = new ArrayList<>(); - List maxSeqNoList = new ArrayList<>(); - int totalCommits = between(2, 20); - long lastMaxSeqNo = 0; - for (int i = 0; i < totalCommits; i++) { - lastMaxSeqNo += between(1, 1000); - commitList.add(mockIndexCommitWithMaxSeqNo(lastMaxSeqNo)); - maxSeqNoList.add(lastMaxSeqNo); - } - - int pivot = randomInt(maxSeqNoList.size() - 2); - long currentGCP = randomLongBetween(maxSeqNoList.get(pivot), Math.max(maxSeqNoList.get(pivot), maxSeqNoList.get(pivot + 1) - 1)); - globalCheckpoint.set(currentGCP); - - final IndexCommit[] keptCommits = commitList.subList(pivot, commitList.size()).toArray(new IndexCommit[0]); - assertThat(deletionPolicy.onInit(commitList), contains(keptCommits)); - assertThat(deletionPolicy.onCommit(commitList), contains(keptCommits)); - } - - public void testLegacyIndex() throws Exception { - final AtomicLong globalCheckpoint = new AtomicLong(randomInt(1000)); - final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); - - // Keep a single legacy commit - { - IndexCommit legacyCommit = mockLegacyIndexCommit(); - final List keptCommits = deletionPolicy.onInit(singletonList(legacyCommit)); - verify(legacyCommit, times(0)).delete(); - assertThat(keptCommits, contains(legacyCommit)); - } - - // Keep a safe commit, and delete a legacy commit. - { - IndexCommit legacyCommit = mockLegacyIndexCommit(); - IndexCommit safeCommit = mockIndexCommitWithMaxSeqNo(randomLongBetween(0, globalCheckpoint.get())); - - final List keptCommits = deletionPolicy.onCommit(Arrays.asList(legacyCommit, safeCommit)); - verify(legacyCommit, times(1)).delete(); - verify(safeCommit, times(0)).delete(); - assertThat(keptCommits, contains(safeCommit)); - } - - // Keep until the safe commit, and delete legacy commits - { - IndexCommit legacyCommit = mockLegacyIndexCommit(); - IndexCommit oldCommit = mockIndexCommitWithMaxSeqNo(randomLongBetween(0, globalCheckpoint.get())); - IndexCommit safeCommit = mockIndexCommitWithMaxSeqNo(randomLongBetween(0, globalCheckpoint.get())); - IndexCommit unsafeCommit = mockIndexCommitWithMaxSeqNo(globalCheckpoint.get() + between(1, 1000)); - - List keptCommits = deletionPolicy.onCommit(Arrays.asList(legacyCommit, oldCommit, safeCommit, unsafeCommit)); - verify(legacyCommit, times(1)).delete(); - verify(oldCommit, times(1)).delete(); - verify(safeCommit, times(0)).delete(); - verify(unsafeCommit, times(0)).delete(); - assertThat(keptCommits, contains(safeCommit, unsafeCommit)); - } - } - - public void testCleanupDuplicateCommits() throws Exception { - final AtomicLong globalCheckpoint = new AtomicLong(); - final KeepUntilGlobalCheckpointDeletionPolicy deletionPolicy = new KeepUntilGlobalCheckpointDeletionPolicy(globalCheckpoint::get); - - final long maxSeqNo = randomNonNegativeLong(); - IndexCommit commit1 = mockIndexCommitWithMaxSeqNo(maxSeqNo); - IndexCommit commit2 = mockIndexCommitWithMaxSeqNo(maxSeqNo); - - assertThat(deletionPolicy.onInit(Arrays.asList(commit1, commit2)), contains(commit2)); - verify(commit2, never()).delete(); - verify(commit1, times(1)).delete(); - - assertThat(deletionPolicy.onCommit(Arrays.asList(commit1, commit2)), contains(commit2)); - verify(commit2, never()).delete(); - verify(commit1, times(2)).delete(); - } - - IndexCommit mockIndexCommitWithMaxSeqNo(long maxSeqNo) throws IOException { - final Map userData = new HashMap<>(); - userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); - userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(randomNonNegativeLong())); - final IndexCommit commit = mock(IndexCommit.class); - when(commit.getUserData()).thenReturn(userData); - return commit; - } - - IndexCommit mockLegacyIndexCommit() throws IOException { - final Map userData = new HashMap<>(); - userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(randomNonNegativeLong())); - final IndexCommit commit = mock(IndexCommit.class); - when(commit.getUserData()).thenReturn(userData); - return commit; - } - -} From 72a4190da4bacc4d9e851303547e0cb9929efedc Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 4 Dec 2017 14:11:53 -0500 Subject: [PATCH 08/29] Use the kept position --- .../index/engine/CombinedDeletionPolicy.java | 81 +++++-------------- .../engine/CombinedDeletionPolicyTests.java | 18 ----- 2 files changed, 18 insertions(+), 81 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index f4bbed51fea97..d1ac6db4e18bc 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -19,10 +19,6 @@ package org.elasticsearch.index.engine; -import com.carrotsearch.hppc.IntHashSet; -import com.carrotsearch.hppc.IntSet; -import com.carrotsearch.hppc.LongHashSet; -import com.carrotsearch.hppc.LongSet; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexDeletionPolicy; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -30,8 +26,6 @@ import org.elasticsearch.index.translog.TranslogDeletionPolicy; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.function.LongSupplier; @@ -57,7 +51,7 @@ final class CombinedDeletionPolicy extends IndexDeletionPolicy { @Override public void onInit(List commits) throws IOException { - final List keptCommits = deleteOldIndexCommits(commits); + final int keptPosition = deleteOldIndexCommits(commits); switch (openMode) { case CREATE_INDEX_AND_TRANSLOG: break; @@ -66,7 +60,7 @@ public void onInit(List commits) throws IOException { break; case OPEN_INDEX_AND_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; - setLastCommittedTranslogGeneration(keptCommits); + setLastCommittedTranslogGeneration(commits, keptPosition); break; default: throw new IllegalArgumentException("unknown openMode [" + openMode + "]"); @@ -75,22 +69,20 @@ public void onInit(List commits) throws IOException { @Override public void onCommit(List commits) throws IOException { - final List keptCommits = deleteOldIndexCommits(commits); - setLastCommittedTranslogGeneration(keptCommits); + final int keptKeptPosition = deleteOldIndexCommits(commits); + setLastCommittedTranslogGeneration(commits, keptKeptPosition); } - private void setLastCommittedTranslogGeneration(List keptCommits) throws IOException { - assert keptCommits.isEmpty() == false : "All index commits were deleted"; - assert keptCommits.stream().allMatch(c -> c.isDeleted() == false) : "All kept commits must not be deleted"; - - final IndexCommit lastCommit = keptCommits.get(keptCommits.size() - 1); + private void setLastCommittedTranslogGeneration(List commits, final int keptPosition) throws IOException { + final IndexCommit lastCommit = commits.get(commits.size() - 1); + assert lastCommit.isDeleted() == false : "The last commit should not be deleted"; final long lastGen = Long.parseLong(lastCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); - long minRequiredGen = lastGen; - for (IndexCommit indexCommit : keptCommits) { - long translogGen = Long.parseLong(indexCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); - minRequiredGen = Math.min(minRequiredGen, translogGen); - } - assert minRequiredGen <= lastGen; + + final IndexCommit keptCommit = commits.get(keptPosition); + assert keptCommit.isDeleted() == false : "The kept commit should not be deleted"; + final long minRequiredGen = Long.parseLong(keptCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); + + assert minRequiredGen <= lastGen : "MinRequiredGen must not be greater than LastGen"; translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen); translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); } @@ -98,26 +90,12 @@ private void setLastCommittedTranslogGeneration(List keptCommits) t /** * Deletes old index commits which are not required for operation based recovery. */ - private List deleteOldIndexCommits(List commits) throws IOException { - if (commits.isEmpty()) { - return Collections.emptyList(); - } - - final List keptCommits = new ArrayList<>(); + private int deleteOldIndexCommits(List commits) throws IOException { final int keptPosition = indexOfKeptCommits(commits); - final IntSet duplicateIndexes = indexesOfDuplicateCommits(commits); - - for (int i = 0; i < commits.size() - 1; i++) { - final IndexCommit commit = commits.get(i); - if (i < keptPosition || duplicateIndexes.contains(i)) { - commit.delete(); - } else { - keptCommits.add(commit); - } + for (int i = 0; i < keptPosition; i++) { + commits.get(i).delete(); } - keptCommits.add(commits.get(commits.size() - 1)); // Always keep the last commit. - - return keptCommits; + return keptPosition; } /** @@ -145,29 +123,6 @@ private int indexOfKeptCommits(List commits) throws IOExc * 2. In peer-recovery, if the file-based happens, a replica will be received the latest commit from a primary. * However, that commit may not be a safe commit if writes are in progress in the primary. */ - return -1; - } - - /** - * In some cases, we may have more than one index commits with the same max sequence number. - * We better scan and delete these duplicate index commits as soon as possible. - * - * @return index positions of duplicate commits. - */ - private IntSet indexesOfDuplicateCommits(List commits) throws IOException { - final LongSet seenMaxSeqNo = new LongHashSet(); - final IntSet duplicateIndexes = new IntHashSet(); - - for (int i = commits.size() - 1; i >= 0; i--) { - final Map commitUserData = commits.get(i).getUserData(); - // 5.x commits do not contain MAX_SEQ_NO. - if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO)) { - final long maxSeqNoFromCommit = Long.parseLong(commitUserData.get(SequenceNumbers.MAX_SEQ_NO)); - if (seenMaxSeqNo.add(maxSeqNoFromCommit) == false) { - duplicateIndexes.add(i); - } - } - } - return duplicateIndexes; + return 0; } } diff --git a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index dfcf0c2d3e666..2f1622a1c271d 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -135,24 +135,6 @@ public void testLegacyIndex() throws Exception { assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(safeTranslogGen)); } - public void testCleanupDuplicateCommits() throws Exception { - final AtomicLong globalCheckpoint = new AtomicLong(); - TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); - - final long maxSeqNo = randomNonNegativeLong(); - final long lastTranslogGen = randomNonNegativeLong(); - IndexCommit commit1 = mockIndexCommit(maxSeqNo, randomNonNegativeLong()); - IndexCommit commit2 = mockIndexCommit(maxSeqNo, lastTranslogGen); - - indexPolicy.onInit(Arrays.asList(commit1, commit2)); - verify(commit2, never()).delete(); - verify(commit1, times(1)).delete(); - - assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(lastTranslogGen)); - assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(lastTranslogGen)); - } - IndexCommit mockIndexCommit(long maxSeqNo, long translogGen) throws IOException { final Map userData = new HashMap<>(); userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); From 6b2646bb5d313c5cde59159163ab35e8c56c2a9d Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 6 Dec 2017 15:29:40 -0500 Subject: [PATCH 09/29] Delete all commits in OPEN_INDEX_CREATE_TRANSLOG --- .../index/engine/CombinedDeletionPolicy.java | 30 +++++++++++-------- .../engine/CombinedDeletionPolicyTests.java | 2 +- 2 files changed, 19 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index d1ac6db4e18bc..cf7c00b5e42d5 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -51,16 +51,21 @@ final class CombinedDeletionPolicy extends IndexDeletionPolicy { @Override public void onInit(List commits) throws IOException { - final int keptPosition = deleteOldIndexCommits(commits); switch (openMode) { case CREATE_INDEX_AND_TRANSLOG: + assert commits.isEmpty() : "index is created, but we have commits"; break; case OPEN_INDEX_CREATE_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; + // When an engine starts with OPEN_INDEX_CREATE_TRANSLOG, it will create a fresh commit immediately. + // We can safely delete all existing commits here. + for (IndexCommit commit : commits) { + commit.delete(); + } break; case OPEN_INDEX_AND_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; - setLastCommittedTranslogGeneration(commits, keptPosition); + onCommit(commits); break; default: throw new IllegalArgumentException("unknown openMode [" + openMode + "]"); @@ -69,19 +74,18 @@ public void onInit(List commits) throws IOException { @Override public void onCommit(List commits) throws IOException { - final int keptKeptPosition = deleteOldIndexCommits(commits); - setLastCommittedTranslogGeneration(commits, keptKeptPosition); - } - - private void setLastCommittedTranslogGeneration(List commits, final int keptPosition) throws IOException { + final IndexCommit keptCommit = deleteOldIndexCommits(commits); final IndexCommit lastCommit = commits.get(commits.size() - 1); - assert lastCommit.isDeleted() == false : "The last commit should not be deleted"; - final long lastGen = Long.parseLong(lastCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); + setLastCommittedTranslogGeneration(keptCommit, lastCommit); + } - final IndexCommit keptCommit = commits.get(keptPosition); + private void setLastCommittedTranslogGeneration(final IndexCommit keptCommit, final IndexCommit lastCommit) throws IOException { assert keptCommit.isDeleted() == false : "The kept commit should not be deleted"; final long minRequiredGen = Long.parseLong(keptCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); + assert lastCommit.isDeleted() == false : "The last commit should not be deleted"; + final long lastGen = Long.parseLong(lastCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); + assert minRequiredGen <= lastGen : "MinRequiredGen must not be greater than LastGen"; translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen); translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); @@ -89,13 +93,15 @@ private void setLastCommittedTranslogGeneration(List comm /** * Deletes old index commits which are not required for operation based recovery. + * + * @return returns the min required index commit for recovery. */ - private int deleteOldIndexCommits(List commits) throws IOException { + private IndexCommit deleteOldIndexCommits(List commits) throws IOException { final int keptPosition = indexOfKeptCommits(commits); for (int i = 0; i < keptPosition; i++) { commits.get(i).delete(); } - return keptPosition; + return commits.get(keptPosition); } /** diff --git a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index 2f1622a1c271d..b9006fdf19c96 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -89,7 +89,7 @@ public void testCommitsKeepUpToGlobalCheckpoint() throws Exception { public void testIgnoreSnapshottingCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_CREATE_TRANSLOG, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); long firstMaxSeqNo = randomLongBetween(0, Long.MAX_VALUE - 1); long secondMaxSeqNo = randomLongBetween(firstMaxSeqNo + 1, Long.MAX_VALUE); From b7177cddb22daf63a4436a927649c5cdab64a15e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 6 Dec 2017 15:58:05 -0500 Subject: [PATCH 10/29] more assert --- .../index/engine/CombinedDeletionPolicy.java | 14 +++++++------- .../index/translog/TranslogDeletionPolicy.java | 12 ++++++------ 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index cf7c00b5e42d5..9e2e9959c5276 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -76,17 +76,17 @@ public void onInit(List commits) throws IOException { public void onCommit(List commits) throws IOException { final IndexCommit keptCommit = deleteOldIndexCommits(commits); final IndexCommit lastCommit = commits.get(commits.size() - 1); - setLastCommittedTranslogGeneration(keptCommit, lastCommit); + updateTranslogDeletionPolicy(keptCommit, lastCommit); } - private void setLastCommittedTranslogGeneration(final IndexCommit keptCommit, final IndexCommit lastCommit) throws IOException { - assert keptCommit.isDeleted() == false : "The kept commit should not be deleted"; - final long minRequiredGen = Long.parseLong(keptCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); + private void updateTranslogDeletionPolicy(final IndexCommit minRequiredCommit, final IndexCommit lastCommit) throws IOException { + assert minRequiredCommit.isDeleted() == false : "The minimum required commit must not be deleted"; + final long minRequiredGen = Long.parseLong(minRequiredCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); - assert lastCommit.isDeleted() == false : "The last commit should not be deleted"; + assert lastCommit.isDeleted() == false : "The last commit must not be deleted"; final long lastGen = Long.parseLong(lastCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)); - assert minRequiredGen <= lastGen : "MinRequiredGen must not be greater than LastGen"; + assert minRequiredGen <= lastGen : "minRequiredGen must not be greater than lastGen"; translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen); translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); } @@ -105,7 +105,7 @@ private IndexCommit deleteOldIndexCommits(List commits) t } /** - * Find the index position of a safe index commit whose max sequence number is not greater than the global checkpoint. + * Find the highest index position of a safe index commit whose max sequence number is not greater than the global checkpoint. */ private int indexOfKeptCommits(List commits) throws IOException { final long currentGlobalCheckpoint = globalCheckpointSupplier.getAsLong(); diff --git a/core/src/main/java/org/elasticsearch/index/translog/TranslogDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/translog/TranslogDeletionPolicy.java index 8a2c7bd154f86..5eba198378a1d 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/TranslogDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/translog/TranslogDeletionPolicy.java @@ -74,9 +74,9 @@ public TranslogDeletionPolicy(long retentionSizeInBytes, long retentionAgeInMill } public synchronized void setMinTranslogGenerationForRecovery(long newGen) { - if (newGen < minTranslogGenerationForRecovery) { - throw new IllegalArgumentException("minTranslogGenerationForRecovery can't go backwards. new [" + newGen + "] current [" + - minTranslogGenerationForRecovery + "]"); + if (newGen < minTranslogGenerationForRecovery || newGen > translogGenerationOfLastCommit) { + throw new IllegalArgumentException("Invalid minTranslogGenerationForRecovery can't go backwards; new [" + newGen + "]," + + "current [" + minTranslogGenerationForRecovery + "], lastGen [" + translogGenerationOfLastCommit + "]"); } minTranslogGenerationForRecovery = newGen; } @@ -85,9 +85,9 @@ public synchronized void setMinTranslogGenerationForRecovery(long newGen) { * Sets the translog generation of the last index commit. */ public synchronized void setTranslogGenerationOfLastCommit(long lastGen) { - if (lastGen < translogGenerationOfLastCommit) { - throw new IllegalArgumentException("translogGenerationOfLastCommit can't go backwards. new [" + lastGen + "] current [" + - translogGenerationOfLastCommit + "]"); + if (lastGen < translogGenerationOfLastCommit || lastGen < minTranslogGenerationForRecovery) { + throw new IllegalArgumentException("Invalid translogGenerationOfLastCommit; new [" + lastGen + "]," + + "current [" + translogGenerationOfLastCommit + "], minRequiredGen [" + minTranslogGenerationForRecovery + "]"); } translogGenerationOfLastCommit = lastGen; } From 12cae9226ee7c585ed61994a2d3eee3903295435 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 5 Dec 2017 09:51:20 -0500 Subject: [PATCH 11/29] call lastGen() before minGen() --- .../index/engine/CombinedDeletionPolicyTests.java | 2 +- .../index/translog/TranslogDeletionPolicyTests.java | 3 +++ .../java/org/elasticsearch/index/translog/TranslogTests.java | 4 ++++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index b9006fdf19c96..bcfef9cf1a90e 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -95,7 +95,7 @@ public void testIgnoreSnapshottingCommits() throws Exception { long secondMaxSeqNo = randomLongBetween(firstMaxSeqNo + 1, Long.MAX_VALUE); long lastTranslogGen = randomNonNegativeLong(); - final IndexCommit firstCommit = mockIndexCommit(firstMaxSeqNo, randomNonNegativeLong()); + final IndexCommit firstCommit = mockIndexCommit(firstMaxSeqNo, randomLongBetween(0, lastTranslogGen)); final IndexCommit secondCommit = mockIndexCommit(secondMaxSeqNo, lastTranslogGen); SnapshotDeletionPolicy snapshotDeletionPolicy = new SnapshotDeletionPolicy(indexPolicy); diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogDeletionPolicyTests.java index f62d292730e43..39fc182623f95 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogDeletionPolicyTests.java @@ -53,6 +53,7 @@ public void testNoRetention() throws IOException { assertMinGenRequired(deletionPolicy, readersAndWriter, 1L); final int committedReader = randomIntBetween(0, allGens.size() - 1); final long committedGen = allGens.get(committedReader).generation; + deletionPolicy.setTranslogGenerationOfLastCommit(randomLongBetween(committedGen, Long.MAX_VALUE)); deletionPolicy.setMinTranslogGenerationForRecovery(committedGen); assertMinGenRequired(deletionPolicy, readersAndWriter, committedGen); } finally { @@ -109,6 +110,7 @@ public void testRetentionHierarchy() throws IOException { allGens.add(readersAndWriter.v2()); try { TranslogDeletionPolicy deletionPolicy = new MockDeletionPolicy(now, Long.MAX_VALUE, Long.MAX_VALUE); + deletionPolicy.setTranslogGenerationOfLastCommit(Long.MAX_VALUE); deletionPolicy.setMinTranslogGenerationForRecovery(Long.MAX_VALUE); int selectedReader = randomIntBetween(0, allGens.size() - 1); final long selectedGenerationByAge = allGens.get(selectedReader).generation; @@ -122,6 +124,7 @@ public void testRetentionHierarchy() throws IOException { // make a new policy as committed gen can't go backwards (for now) deletionPolicy = new MockDeletionPolicy(now, size, maxAge); long committedGen = randomFrom(allGens).generation; + deletionPolicy.setTranslogGenerationOfLastCommit(randomLongBetween(committedGen, Long.MAX_VALUE)); deletionPolicy.setMinTranslogGenerationForRecovery(committedGen); assertMinGenRequired(deletionPolicy, readersAndWriter, Math.min(committedGen, Math.max(selectedGenerationByAge, selectedGenerationBySize))); diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index 57b3eb8e61e59..dc050949fe3cc 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -852,6 +852,7 @@ public void doRun() throws BrokenBarrierException, InterruptedException, IOExcep translog.rollGeneration(); // expose the new checkpoint (simulating a commit), before we trim the translog lastCommittedLocalCheckpoint.set(localCheckpoint); + deletionPolicy.setTranslogGenerationOfLastCommit(translog.currentFileGeneration()); deletionPolicy.setMinTranslogGenerationForRecovery( translog.getMinGenerationForSeqNo(localCheckpoint + 1).translogFileGeneration); translog.trimUnreferencedReaders(); @@ -1850,6 +1851,7 @@ public void testRecoveryFromAFutureGenerationCleansUp() throws IOException { translog.close(); TranslogConfig config = translog.getConfig(); final TranslogDeletionPolicy deletionPolicy = new TranslogDeletionPolicy(-1, -1); + deletionPolicy.setTranslogGenerationOfLastCommit(randomLongBetween(comittedGeneration, Long.MAX_VALUE)); deletionPolicy.setMinTranslogGenerationForRecovery(comittedGeneration); translog = new Translog(config, translog.getTranslogUUID(), deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO); assertThat(translog.getMinFileGeneration(), equalTo(1L)); @@ -1895,6 +1897,7 @@ public void testRecoveryFromFailureOnTrimming() throws IOException { translog.rollGeneration(); } } + deletionPolicy.setTranslogGenerationOfLastCommit(randomLongBetween(comittedGeneration, translog.currentFileGeneration())); deletionPolicy.setMinTranslogGenerationForRecovery(comittedGeneration); fail.failRandomly(); try { @@ -1904,6 +1907,7 @@ public void testRecoveryFromFailureOnTrimming() throws IOException { } } final TranslogDeletionPolicy deletionPolicy = new TranslogDeletionPolicy(-1, -1); + deletionPolicy.setTranslogGenerationOfLastCommit(randomLongBetween(comittedGeneration, Long.MAX_VALUE)); deletionPolicy.setMinTranslogGenerationForRecovery(comittedGeneration); try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { // we don't know when things broke exactly From dbdacc450ae0524f6d00fe1d4b9c9d4adb1e0d26 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 6 Dec 2017 16:38:31 -0500 Subject: [PATCH 12/29] improve tests --- .../index/engine/InternalEngineTests.java | 28 ++++++++++++++----- .../index/shard/IndexShardTests.java | 2 +- 2 files changed, 22 insertions(+), 8 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index ef62c10e6b6ec..3dfd50a59eeeb 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -164,8 +164,8 @@ import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasKey; -import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -4132,7 +4132,7 @@ public void testSeqNoGenerator() throws IOException { } } - public void testKeepTranslogUpToGlobalCheckpoint() throws Exception { + public void testKeepTranslogAfterGlobalCheckpoint() throws Exception { IOUtils.close(engine, store); final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); final BiFunction seqNoServiceSupplier = (config, seqNoStats) -> @@ -4149,10 +4149,19 @@ public long getGlobalCheckpoint() { } }; + final IndexSettings indexSettings = new IndexSettings(defaultSettings.getIndexMetaData(), defaultSettings.getNodeSettings(), + defaultSettings.getScopedSettings()); + IndexMetaData.Builder builder = IndexMetaData.builder(indexSettings.getIndexMetaData()); + builder.settings(Settings.builder().put(indexSettings.getSettings()) + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), randomFrom("-1", "10nanos", "500ms", "7s", "2m", "60m")) + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), randomFrom("-1", "5b", "2kb", "1mb", "50gb")) + ); + indexSettings.updateIndexMetaData(builder.build()); + final Path translogPath = createTempDir(); store = createStore(); try (InternalEngine engine - = new InternalEngine(config(defaultSettings, store, translogPath, NoMergePolicy.INSTANCE, null), seqNoServiceSupplier)) { + = new InternalEngine(config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null), seqNoServiceSupplier)) { int numDocs = scaledRandomIntBetween(10, 100); int uncommittedOps = 0; for (int i = 0; i < numDocs; i++) { @@ -4183,11 +4192,16 @@ public long getGlobalCheckpoint() { } // Reopen engine to test onInit with existing index commits. try (InternalEngine engine - = new InternalEngine(config(defaultSettings, store, translogPath, NoMergePolicy.INSTANCE, null), seqNoServiceSupplier)) { + = new InternalEngine(config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null), seqNoServiceSupplier)) { + final Set seqNoList = new HashSet<>(); try (Translog.Snapshot snapshot = engine.getTranslog().newSnapshot()) { - long requiredOps = engine.seqNoService().getLocalCheckpoint() - Math.max(0, globalCheckpoint.get()); - assertThat("Should keep translog operations up to the global checkpoint", - (long) snapshot.totalOperations(), greaterThanOrEqualTo(requiredOps)); + Translog.Operation op; + while ((op = snapshot.next()) != null) { + seqNoList.add(op.seqNo()); + } + } + for (long i = Math.max(0, globalCheckpoint.get() + 1); i <= engine.seqNoService().getLocalCheckpoint(); i++) { + assertThat("Translog should keep op with seqno [" + i + "]", seqNoList, hasItem(i)); } } } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 5c9934cd399df..5a6f1aa43d285 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1072,7 +1072,7 @@ public void testAcquireIndexCommit() throws Exception { commit.close(); // Make the global checkpoint in sync with the local checkpoint. if (isPrimary) { - shard.getEngine().seqNoService().markAllocationIdAsInSync(shard.shardRouting.allocationId().getId(), numDocs + moreDocs - 1); + shard.updateGlobalCheckpointForShard(shard.shardRouting.allocationId().getId(), shard.getLocalCheckpoint()); } else { shard.updateGlobalCheckpointOnReplica(shard.getLocalCheckpoint(), "test"); } From d07af94439cc2b2829b6286d3d95d846a0b1c69e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 6 Dec 2017 17:03:15 -0500 Subject: [PATCH 13/29] tighten onInit() --- .../engine/CombinedDeletionPolicyTests.java | 47 +++++++++++++++---- 1 file changed, 37 insertions(+), 10 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index bcfef9cf1a90e..90ec0d4f01c7d 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -45,11 +45,12 @@ import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; public class CombinedDeletionPolicyTests extends ESTestCase { - public void testCommitsKeepUpToGlobalCheckpoint() throws Exception { + public void testKeepCommitsAfterGlobalCheckpoint() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); @@ -69,10 +70,11 @@ public void testCommitsKeepUpToGlobalCheckpoint() throws Exception { translogGenList.add(lastTranslogGen); } - int keptIndex = randomInt(commitList.size() - 2); - globalCheckpoint.set(randomLongBetween( - maxSeqNoList.get(keptIndex), Math.max(maxSeqNoList.get(keptIndex), maxSeqNoList.get(keptIndex + 1) - 1)) - ); + int keptIndex = randomInt(commitList.size() - 1); + final long lower = maxSeqNoList.get(keptIndex); + final long upper = keptIndex == commitList.size() - 1 ? + Long.MAX_VALUE : Math.max(maxSeqNoList.get(keptIndex), maxSeqNoList.get(keptIndex + 1) - 1); + globalCheckpoint.set(randomLongBetween(lower, upper)); indexPolicy.onCommit(commitList); for (int i = 0; i < commitList.size(); i++) { @@ -124,15 +126,40 @@ public void testLegacyIndex() throws Exception { assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(legacyTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(legacyTranslogGen)); - globalCheckpoint.set(randomNonNegativeLong()); long safeTranslogGen = randomLongBetween(legacyTranslogGen, Long.MAX_VALUE); - IndexCommit safeCommit = mockIndexCommit(randomLongBetween(0, globalCheckpoint.get()), safeTranslogGen); + long maxSeqNo = randomLongBetween(1, Long.MAX_VALUE); + final IndexCommit freshCommit = mockIndexCommit(maxSeqNo, safeTranslogGen); - indexPolicy.onCommit(Arrays.asList(legacyCommit, safeCommit)); - verify(legacyCommit, times(1)).delete(); - verify(safeCommit, times(0)).delete(); + globalCheckpoint.set(randomLongBetween(0, maxSeqNo - 1)); + indexPolicy.onCommit(Arrays.asList(legacyCommit, freshCommit)); + verify(legacyCommit, times(0)).delete(); + verify(freshCommit, times(0)).delete(); + assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(legacyTranslogGen)); assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(safeTranslogGen)); + + // Make the fresh commit safe. + globalCheckpoint.set(randomLongBetween(maxSeqNo, Long.MAX_VALUE)); + indexPolicy.onCommit(Arrays.asList(legacyCommit, freshCommit)); + verify(legacyCommit, times(1)).delete(); + verify(freshCommit, times(0)).delete(); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(safeTranslogGen)); + assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(safeTranslogGen)); + } + + public void testDeleteAllCommitWhenTruncateTranslog() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + TranslogDeletionPolicy translogPolicy = mock(TranslogDeletionPolicy.class); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_CREATE_TRANSLOG, translogPolicy, globalCheckpoint::get); + final int numCommits = between(1, 10); + final List commitList = new ArrayList<>(numCommits); + for (int i = 0; i < numCommits; i++) { + commitList.add(mockIndexCommit(randomNonNegativeLong(), randomNonNegativeLong())); + } + indexPolicy.onInit(commitList); + verifyZeroInteractions(translogPolicy); + for (IndexCommit commit : commitList) { + verify(commit, times(1)).delete(); + } } IndexCommit mockIndexCommit(long maxSeqNo, long translogGen) throws IOException { From 639cac618f0c67a5cec0ecef89b67dbfe794031d Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 6 Dec 2017 17:19:32 -0500 Subject: [PATCH 14/29] test advance minGen --- .../org/elasticsearch/index/engine/InternalEngineTests.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 3dfd50a59eeeb..6b1cc1888fc34 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -981,6 +981,12 @@ public long getGlobalCheckpoint() { assertThat(engine.getTranslog().currentFileGeneration(), equalTo(3L)); assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync.get() ? 3L : 1L)); assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(3L)); + + inSync.set(true); + engine.flush(true, true); + assertThat(engine.getTranslog().currentFileGeneration(), equalTo(4L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(4L)); + assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(4L)); } public void testSyncedFlush() throws IOException { From d1465839b2828bab7c7ed05ae8f6749dfc591809 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 6 Dec 2017 18:11:09 -0500 Subject: [PATCH 15/29] use matcher --- .../index/engine/InternalEngineTests.java | 19 +++---- .../index/translog/SnapshotMatchers.java | 53 +++++++++++++++++++ 2 files changed, 59 insertions(+), 13 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 6b1cc1888fc34..471059f1a2767 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -158,13 +158,13 @@ import static org.elasticsearch.index.engine.Engine.Operation.Origin.PEER_RECOVERY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; +import static org.elasticsearch.index.translog.SnapshotMatchers.containsSeqNoRange; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; @@ -4187,27 +4187,20 @@ public long getGlobalCheckpoint() { if (rarely()) { engine.rollTranslogGeneration(); } + assertThat(engine.getTranslog().uncommittedOperations(), equalTo(uncommittedOps)); try (Translog.Snapshot snapshot = engine.getTranslog().newSnapshot()) { - long requiredOps = engine.seqNoService().getMaxSeqNo() - Math.max(0, engine.seqNoService().getGlobalCheckpoint()); - assertThat("Should keep translog operations up to the global checkpoint", - (long) snapshot.totalOperations(), greaterThanOrEqualTo(requiredOps)); + assertThat(snapshot, + containsSeqNoRange(Math.max(0, globalCheckpoint.get() + 1), engine.seqNoService().getLocalCheckpoint())); } - assertThat(engine.getTranslog().uncommittedOperations(), equalTo(uncommittedOps)); } engine.flush(randomBoolean(), true); } // Reopen engine to test onInit with existing index commits. try (InternalEngine engine = new InternalEngine(config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null), seqNoServiceSupplier)) { - final Set seqNoList = new HashSet<>(); try (Translog.Snapshot snapshot = engine.getTranslog().newSnapshot()) { - Translog.Operation op; - while ((op = snapshot.next()) != null) { - seqNoList.add(op.seqNo()); - } - } - for (long i = Math.max(0, globalCheckpoint.get() + 1); i <= engine.seqNoService().getLocalCheckpoint(); i++) { - assertThat("Translog should keep op with seqno [" + i + "]", seqNoList, hasItem(i)); + assertThat(snapshot, + containsSeqNoRange(Math.max(0, globalCheckpoint.get() + 1), engine.seqNoService().getLocalCheckpoint())); } } } diff --git a/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java b/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java index 4ca6057bd6bc9..daa08dc234b3a 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java +++ b/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.translog; +import com.carrotsearch.hppc.LongHashSet; +import com.carrotsearch.hppc.LongSet; import org.elasticsearch.ElasticsearchException; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -61,6 +63,16 @@ public static Matcher containsOperationsInAnyOrder(Collection return new ContainingInAnyOrderMatcher(expectedOperations); } + /** + * Consumes a snapshot and makes sure its operations have sequence numbers in the given range (both inclusive). + * + * @param minSeqNo inclusive + * @param maxSeqNo inclusive + */ + public static Matcher containsSeqNoRange(long minSeqNo, long maxSeqNo) { + return new ContainingSeqNoRangeMatcher(minSeqNo, maxSeqNo); + } + public static class SizeMatcher extends TypeSafeMatcher { private final int size; @@ -190,4 +202,45 @@ public void describeTo(Description description) { .appendText(" in any order."); } } + + static class ContainingSeqNoRangeMatcher extends TypeSafeMatcher { + private final long minSeqNo; + private final long maxSeqNo; + private final List notFoundSeqNo = new ArrayList<>(); + + ContainingSeqNoRangeMatcher(long minSeqNo, long maxSeqNo) { + this.minSeqNo = minSeqNo; + this.maxSeqNo = maxSeqNo; + } + + @Override + protected boolean matchesSafely(Translog.Snapshot snapshot) { + try { + final LongSet seqNoList = new LongHashSet(); + Translog.Operation op; + while ((op = snapshot.next()) != null) { + seqNoList.add(op.seqNo()); + } + for (long i = minSeqNo; i <= maxSeqNo; i++) { + if (seqNoList.contains(i) == false) { + notFoundSeqNo.add(i); + } + } + return notFoundSeqNo.isEmpty(); + } catch (IOException ex) { + throw new ElasticsearchException("failed to read snapshot content", ex); + } + } + + @Override + protected void describeMismatchSafely(Translog.Snapshot snapshot, Description mismatchDescription) { + mismatchDescription + .appendText("not found seqno").appendValueList("[", ", ", "]", notFoundSeqNo); + } + + @Override + public void describeTo(Description description) { + description.appendText("snapshot contains all seqno from [" + minSeqNo + " to " + maxSeqNo + "]"); + } + } } From 2a6198e87f2e0f649c355b36829b8b82c40639f2 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 6 Dec 2017 19:40:22 -0500 Subject: [PATCH 16/29] inline small functions --- .../index/engine/CombinedDeletionPolicy.java | 26 ++++++------------- .../index/translog/SnapshotMatchers.java | 7 ++--- 2 files changed, 10 insertions(+), 23 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index 9e2e9959c5276..8238b7a9fac75 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -57,8 +57,9 @@ public void onInit(List commits) throws IOException { break; case OPEN_INDEX_CREATE_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; - // When an engine starts with OPEN_INDEX_CREATE_TRANSLOG, it will create a fresh commit immediately. - // We can safely delete all existing commits here. + // When an engine starts with OPEN_INDEX_CREATE_TRANSLOG, we can safely delete all existing index commits because: + // 1. The engine will create a fresh index commit with the same actual data immediately. + // 2. We should never refer translog of these existing commits as they belong to another engine. for (IndexCommit commit : commits) { commit.delete(); } @@ -74,9 +75,11 @@ public void onInit(List commits) throws IOException { @Override public void onCommit(List commits) throws IOException { - final IndexCommit keptCommit = deleteOldIndexCommits(commits); - final IndexCommit lastCommit = commits.get(commits.size() - 1); - updateTranslogDeletionPolicy(keptCommit, lastCommit); + final int keptPosition = indexOfKeptCommits(commits); + for (int i = 0; i < keptPosition; i++) { + commits.get(i).delete(); + } + updateTranslogDeletionPolicy(commits.get(keptPosition), commits.get(commits.size() - 1)); } private void updateTranslogDeletionPolicy(final IndexCommit minRequiredCommit, final IndexCommit lastCommit) throws IOException { @@ -91,19 +94,6 @@ private void updateTranslogDeletionPolicy(final IndexCommit minRequiredCommit, f translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen); } - /** - * Deletes old index commits which are not required for operation based recovery. - * - * @return returns the min required index commit for recovery. - */ - private IndexCommit deleteOldIndexCommits(List commits) throws IOException { - final int keptPosition = indexOfKeptCommits(commits); - for (int i = 0; i < keptPosition; i++) { - commits.get(i).delete(); - } - return commits.get(keptPosition); - } - /** * Find the highest index position of a safe index commit whose max sequence number is not greater than the global checkpoint. */ diff --git a/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java b/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java index daa08dc234b3a..21f7dd9481c5e 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java +++ b/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java @@ -64,10 +64,7 @@ public static Matcher containsOperationsInAnyOrder(Collection } /** - * Consumes a snapshot and makes sure its operations have sequence numbers in the given range (both inclusive). - * - * @param minSeqNo inclusive - * @param maxSeqNo inclusive + * Consumes a snapshot and makes sure that its operations have all seqno between minSeqNo(inclusive) and maxSeqNo(inclusive). */ public static Matcher containsSeqNoRange(long minSeqNo, long maxSeqNo) { return new ContainingSeqNoRangeMatcher(minSeqNo, maxSeqNo); @@ -235,7 +232,7 @@ protected boolean matchesSafely(Translog.Snapshot snapshot) { @Override protected void describeMismatchSafely(Translog.Snapshot snapshot, Description mismatchDescription) { mismatchDescription - .appendText("not found seqno").appendValueList("[", ", ", "]", notFoundSeqNo); + .appendText("not found seqno ").appendValueList("[", ", ", "]", notFoundSeqNo); } @Override From b819a8be6d6caac110ec8d29a2421f9673336382 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 6 Dec 2017 21:08:53 -0500 Subject: [PATCH 17/29] test: update local checkpoint --- .../java/org/elasticsearch/index/shard/IndexShardTests.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 5a6f1aa43d285..682ef5b89e207 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1072,9 +1072,11 @@ public void testAcquireIndexCommit() throws Exception { commit.close(); // Make the global checkpoint in sync with the local checkpoint. if (isPrimary) { - shard.updateGlobalCheckpointForShard(shard.shardRouting.allocationId().getId(), shard.getLocalCheckpoint()); + final String allocationId = shard.shardRouting.allocationId().getId(); + shard.updateLocalCheckpointForShard(allocationId, numDocs + moreDocs - 1); + shard.updateGlobalCheckpointForShard(allocationId, shard.getLocalCheckpoint()); } else { - shard.updateGlobalCheckpointOnReplica(shard.getLocalCheckpoint(), "test"); + shard.updateGlobalCheckpointOnReplica(numDocs + moreDocs - 1, "test"); } flushShard(shard, true); From b29623d8cfc2f79f7d512c7d3cb3de340a46c022 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Wed, 6 Dec 2017 22:29:30 -0500 Subject: [PATCH 18/29] add comment onInit() --- .../elasticsearch/index/engine/CombinedDeletionPolicy.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index 8238b7a9fac75..1e3d784f58b83 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -59,10 +59,8 @@ public void onInit(List commits) throws IOException { assert commits.isEmpty() == false : "index is opened, but we have no commits"; // When an engine starts with OPEN_INDEX_CREATE_TRANSLOG, we can safely delete all existing index commits because: // 1. The engine will create a fresh index commit with the same actual data immediately. - // 2. We should never refer translog of these existing commits as they belong to another engine. - for (IndexCommit commit : commits) { - commit.delete(); - } + // 2. We should never refer these existing commits again as they belong to another engine. + commits.forEach(IndexCommit::delete); break; case OPEN_INDEX_AND_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; From 60a204fe543574a7a14ba630dbc1d9162675967c Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Thu, 7 Dec 2017 11:42:01 -0500 Subject: [PATCH 19/29] resume out of order test --- .../indices/recovery/RecoveryTests.java | 46 +++++++++++++++++-- 1 file changed, 43 insertions(+), 3 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index e1e2e37157f0f..79f575106d338 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -39,12 +39,16 @@ import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; +import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; @@ -100,6 +104,8 @@ public void testRetentionPolicyChangeDuringRecovery() throws Exception { public void testRecoveryWithOutOfOrderDelete() throws Exception { try (ReplicationGroup shards = createGroup(1)) { shards.startAll(); + final boolean isolateFirstBatch = randomBoolean(); + final boolean duplicateStaleIndexOp = randomBoolean(); // create out of order delete and index op on replica final IndexShard orgReplica = shards.getReplicas().get(0); orgReplica.applyDeleteOperationOnReplica(1, 2, "type", "id", VersionType.EXTERNAL, u -> {}); @@ -108,14 +114,41 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { SourceToParse.source(orgReplica.shardId().getIndexName(), "type", "id", new BytesArray("{}"), XContentType.JSON), u -> {}); + if (isolateFirstBatch) { + flushShard(orgReplica, true); + if (duplicateStaleIndexOp) { + orgReplica.applyIndexOperationOnReplica(0, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(orgReplica.shardId().getIndexName(), "type", "id", new BytesArray("{}"), XContentType.JSON), + u -> { + }); + } + } // index a second item into the second generation, skipping seq# 2. Local checkpoint is now 1, which will make this generation // stick around orgReplica.applyIndexOperationOnReplica(3, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, SourceToParse.source(orgReplica.shardId().getIndexName(), "type", "id2", new BytesArray("{}"), XContentType.JSON), u -> {}); - final int translogOps = 4; // 3 ops + seqno gaps + final List expectedTranslogOps = new ArrayList<>(Arrays.asList(1L, 0L, 3L, 2L)); // 2L is NoOp. if (randomBoolean()) { - flushShard(orgReplica); + if (randomBoolean()) { + logger.info("--> flushing shard (translog will be trimmed)"); + IndexMetaData.Builder builder = IndexMetaData.builder(orgReplica.indexSettings().getIndexMetaData()); + builder.settings(Settings.builder().put(orgReplica.indexSettings().getSettings()) + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") + ); + orgReplica.indexSettings().updateIndexMetaData(builder.build()); + orgReplica.onSettingsChanged(); + orgReplica.updateGlobalCheckpointOnReplica(1L, "test"); + if (isolateFirstBatch) { + // An operation with seqno=1L won't be retained in the translog. + expectedTranslogOps.remove(1L); + if (duplicateStaleIndexOp == false) { + expectedTranslogOps.remove(0L); + } + } + } + flushShard(orgReplica, true); } final IndexShard orgPrimary = shards.getPrimary(); shards.promoteReplicaToPrimary(orgReplica).get(); // wait for primary/replica sync to make sure seq# gap is closed. @@ -124,7 +157,14 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { shards.recoverReplica(newReplica); shards.assertAllEqual(1); - assertThat(newReplica.getTranslog().totalOperations(), equalTo(translogOps)); + try (Translog.Snapshot snapshot = newReplica.getTranslog().newSnapshot()) { + final List actualOps = new ArrayList<>(); + Translog.Operation op; + while ((op = snapshot.next()) != null) { + actualOps.add(op.seqNo()); + } + assertThat(actualOps, containsInAnyOrder(expectedTranslogOps.toArray(new Long[expectedTranslogOps.size()]))); + } } } From 6b74ab14f0bd070d7247c3ff97d7ba9c3141f1d0 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 8 Dec 2017 21:49:53 -0500 Subject: [PATCH 20/29] use TRANSLOG_UUID_KEY to valid commit --- .../index/engine/CombinedDeletionPolicy.java | 14 +++-- .../engine/CombinedDeletionPolicyTests.java | 57 +++++++++++++++---- 2 files changed, 55 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index 1e3d784f58b83..ad3614eae37e9 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -57,10 +57,11 @@ public void onInit(List commits) throws IOException { break; case OPEN_INDEX_CREATE_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; - // When an engine starts with OPEN_INDEX_CREATE_TRANSLOG, we can safely delete all existing index commits because: - // 1. The engine will create a fresh index commit with the same actual data immediately. - // 2. We should never refer these existing commits again as they belong to another engine. - commits.forEach(IndexCommit::delete); + // When an engine starts with OPEN_INDEX_CREATE_TRANSLOG, we can delete all commits except the last commit, + // without updating the translog policy because these commits do not belong this engine. + for (int i = 0; i < commits.size() - 1; i++) { + commits.get(i).delete(); + } break; case OPEN_INDEX_AND_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; @@ -97,10 +98,15 @@ private void updateTranslogDeletionPolicy(final IndexCommit minRequiredCommit, f */ private int indexOfKeptCommits(List commits) throws IOException { final long currentGlobalCheckpoint = globalCheckpointSupplier.getAsLong(); + final String expectedTranslogUUID = commits.get(commits.size() - 1).getUserData().get(Translog.TRANSLOG_UUID_KEY); // Commits are sorted by age (the 0th one is the oldest commit). for (int i = commits.size() - 1; i >= 0; i--) { final Map commitUserData = commits.get(i).getUserData(); + // Index commit with a different TRANSLOG_UUID_KEY don't belong to this engine, skip it. + if (expectedTranslogUUID.equals(commitUserData.get(Translog.TRANSLOG_UUID_KEY)) == false) { + return i + 1; + } // 5.x commits do not contain MAX_SEQ_NO. if (commitUserData.containsKey(SequenceNumbers.MAX_SEQ_NO) == false) { return i; diff --git a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index 90ec0d4f01c7d..009898ea27c63 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -33,6 +33,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; import static java.util.Collections.singletonList; @@ -61,11 +62,11 @@ public void testKeepCommitsAfterGlobalCheckpoint() throws Exception { int totalCommits = between(2, 20); long lastMaxSeqNo = 0; long lastTranslogGen = 0; - + final UUID translogUUID = UUID.randomUUID(); for (int i = 0; i < totalCommits; i++) { lastMaxSeqNo += between(1, 10000); lastTranslogGen += between(1, 100); - commitList.add(mockIndexCommit(lastMaxSeqNo, lastTranslogGen)); + commitList.add(mockIndexCommit(lastMaxSeqNo, translogUUID, lastTranslogGen)); maxSeqNoList.add(lastMaxSeqNo); translogGenList.add(lastTranslogGen); } @@ -90,6 +91,7 @@ public void testKeepCommitsAfterGlobalCheckpoint() throws Exception { public void testIgnoreSnapshottingCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); + final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); @@ -97,8 +99,8 @@ public void testIgnoreSnapshottingCommits() throws Exception { long secondMaxSeqNo = randomLongBetween(firstMaxSeqNo + 1, Long.MAX_VALUE); long lastTranslogGen = randomNonNegativeLong(); - final IndexCommit firstCommit = mockIndexCommit(firstMaxSeqNo, randomLongBetween(0, lastTranslogGen)); - final IndexCommit secondCommit = mockIndexCommit(secondMaxSeqNo, lastTranslogGen); + final IndexCommit firstCommit = mockIndexCommit(firstMaxSeqNo, translogUUID, randomLongBetween(0, lastTranslogGen)); + final IndexCommit secondCommit = mockIndexCommit(secondMaxSeqNo, translogUUID, lastTranslogGen); SnapshotDeletionPolicy snapshotDeletionPolicy = new SnapshotDeletionPolicy(indexPolicy); snapshotDeletionPolicy.onInit(Arrays.asList(firstCommit)); @@ -116,11 +118,13 @@ public void testIgnoreSnapshottingCommits() throws Exception { public void testLegacyIndex() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); + final UUID translogUUID = UUID.randomUUID(); + TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); long legacyTranslogGen = randomNonNegativeLong(); - IndexCommit legacyCommit = mockLegacyIndexCommit(legacyTranslogGen); + IndexCommit legacyCommit = mockLegacyIndexCommit(translogUUID, legacyTranslogGen); indexPolicy.onInit(singletonList(legacyCommit)); verify(legacyCommit, never()).delete(); assertThat(translogPolicy.getMinTranslogGenerationForRecovery(), equalTo(legacyTranslogGen)); @@ -128,7 +132,7 @@ public void testLegacyIndex() throws Exception { long safeTranslogGen = randomLongBetween(legacyTranslogGen, Long.MAX_VALUE); long maxSeqNo = randomLongBetween(1, Long.MAX_VALUE); - final IndexCommit freshCommit = mockIndexCommit(maxSeqNo, safeTranslogGen); + final IndexCommit freshCommit = mockIndexCommit(maxSeqNo, translogUUID, safeTranslogGen); globalCheckpoint.set(randomLongBetween(0, maxSeqNo - 1)); indexPolicy.onCommit(Arrays.asList(legacyCommit, freshCommit)); @@ -146,33 +150,62 @@ public void testLegacyIndex() throws Exception { assertThat(translogPolicy.getTranslogGenerationOfLastCommit(), equalTo(safeTranslogGen)); } - public void testDeleteAllCommitWhenTruncateTranslog() throws Exception { + public void testDeleteInvalidCommits() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(randomNonNegativeLong()); + TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_CREATE_TRANSLOG, translogPolicy, globalCheckpoint::get); + + final int invalidCommits = between(1, 10); + final List commitList = new ArrayList<>(); + for (int i = 0; i < invalidCommits; i++) { + commitList.add(mockIndexCommit(randomNonNegativeLong(), UUID.randomUUID(), randomNonNegativeLong())); + } + + final UUID expectedTranslogUUID = UUID.randomUUID(); + long lastTranslogGen = 0; + final int validCommits = between(1, 10); + for (int i = 0; i < validCommits; i++) { + lastTranslogGen += between(1, 1000); + commitList.add(mockIndexCommit(randomNonNegativeLong(), expectedTranslogUUID, lastTranslogGen)); + } + + // We should never keep invalid commits regardless of the value of the global checkpoint. + indexPolicy.onCommit(commitList); + for (int i = 0; i < invalidCommits - 1; i++) { + verify(commitList.get(i), times(1)).delete(); + } + } + + public void testDeleteAllCommitExceptLastWhenTruncateTranslog() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); TranslogDeletionPolicy translogPolicy = mock(TranslogDeletionPolicy.class); CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_CREATE_TRANSLOG, translogPolicy, globalCheckpoint::get); final int numCommits = between(1, 10); final List commitList = new ArrayList<>(numCommits); for (int i = 0; i < numCommits; i++) { - commitList.add(mockIndexCommit(randomNonNegativeLong(), randomNonNegativeLong())); + commitList.add(mockIndexCommit(randomNonNegativeLong(), UUID.randomUUID(), randomNonNegativeLong())); } indexPolicy.onInit(commitList); verifyZeroInteractions(translogPolicy); - for (IndexCommit commit : commitList) { - verify(commit, times(1)).delete(); + for (int i = 0; i < numCommits - 1; i++) { + verify(commitList.get(i), times(1)).delete(); } + verifyZeroInteractions(commitList.get(numCommits - 1)); } - IndexCommit mockIndexCommit(long maxSeqNo, long translogGen) throws IOException { + IndexCommit mockIndexCommit(long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); + userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGen)); final IndexCommit commit = mock(IndexCommit.class); when(commit.getUserData()).thenReturn(userData); return commit; } - IndexCommit mockLegacyIndexCommit(long translogGen) throws IOException { + IndexCommit mockLegacyIndexCommit(UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); + userData.put(Translog.TRANSLOG_UUID_KEY, translogUUID.toString()); userData.put(Translog.TRANSLOG_GENERATION_KEY, Long.toString(translogGen)); final IndexCommit commit = mock(IndexCommit.class); when(commit.getUserData()).thenReturn(userData); From d8705b4c3a3e349484d6a0611900368e11a81e8e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 8 Dec 2017 22:14:00 -0500 Subject: [PATCH 21/29] minor feedbacks --- .../org/elasticsearch/index/engine/InternalEngine.java | 2 -- .../elasticsearch/index/engine/InternalEngineTests.java | 7 +++++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 0c085fd4cda01..6d4a3d14da3ef 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -366,10 +366,8 @@ private SeqNoStats loadSeqNoStats(EngineConfig.OpenMode openMode) throws IOExcep case OPEN_INDEX_AND_TRANSLOG: final long globalCheckpoint = Translog.readGlobalCheckpoint(engineConfig.getTranslogConfig().getTranslogPath()); return store.loadSeqNoStats(globalCheckpoint); - case OPEN_INDEX_CREATE_TRANSLOG: return store.loadSeqNoStats(SequenceNumbers.UNASSIGNED_SEQ_NO); - case CREATE_INDEX_AND_TRANSLOG: return new SeqNoStats( SequenceNumbers.NO_OPS_PERFORMED, diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 471059f1a2767..4e7284f239625 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -84,6 +84,7 @@ import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -4159,8 +4160,10 @@ public long getGlobalCheckpoint() { defaultSettings.getScopedSettings()); IndexMetaData.Builder builder = IndexMetaData.builder(indexSettings.getIndexMetaData()); builder.settings(Settings.builder().put(indexSettings.getSettings()) - .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), randomFrom("-1", "10nanos", "500ms", "7s", "2m", "60m")) - .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), randomFrom("-1", "5b", "2kb", "1mb", "50gb")) + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), + randomBoolean() ? "-1" : TimeValue.timeValueMillis(randomNonNegativeLong()).getStringRep()) + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), + randomBoolean() ? "-1" : new ByteSizeValue(randomNonNegativeLong()).toString()) ); indexSettings.updateIndexMetaData(builder.build()); From 7ec417cec3bc0c8bb64ac603dd84b50e1c0d42ff Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 9 Dec 2017 12:01:29 -0500 Subject: [PATCH 22/29] use size retention translog --- .../indices/recovery/RecoveryTests.java | 59 +++++++------------ 1 file changed, 21 insertions(+), 38 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index 79f575106d338..15bff3223aa8d 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -22,11 +22,13 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.NoMergePolicy; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; @@ -39,16 +41,12 @@ import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; -import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; -import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; @@ -104,52 +102,44 @@ public void testRetentionPolicyChangeDuringRecovery() throws Exception { public void testRecoveryWithOutOfOrderDelete() throws Exception { try (ReplicationGroup shards = createGroup(1)) { shards.startAll(); - final boolean isolateFirstBatch = randomBoolean(); - final boolean duplicateStaleIndexOp = randomBoolean(); // create out of order delete and index op on replica final IndexShard orgReplica = shards.getReplicas().get(0); orgReplica.applyDeleteOperationOnReplica(1, 2, "type", "id", VersionType.EXTERNAL, u -> {}); orgReplica.getTranslog().rollGeneration(); // isolate the delete in it's own generation + orgReplica.applyIndexOperationOnReplica(0, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, SourceToParse.source(orgReplica.shardId().getIndexName(), "type", "id", new BytesArray("{}"), XContentType.JSON), u -> {}); - - if (isolateFirstBatch) { - flushShard(orgReplica, true); - if (duplicateStaleIndexOp) { - orgReplica.applyIndexOperationOnReplica(0, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(orgReplica.shardId().getIndexName(), "type", "id", new BytesArray("{}"), XContentType.JSON), - u -> { - }); - } - } - // index a second item into the second generation, skipping seq# 2. Local checkpoint is now 1, which will make this generation - // stick around + long uncommittedBytes = orgReplica.translogStats().getUncommittedSizeInBytes(); + // Flushing a new commit with local checkpoint=1 allows to delete the 1st translog file. + orgReplica.flush(new FlushRequest().force(true).waitIfOngoing(true)); + // index a second item into the second generation, skipping seq# 2. orgReplica.applyIndexOperationOnReplica(3, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, SourceToParse.source(orgReplica.shardId().getIndexName(), "type", "id2", new BytesArray("{}"), XContentType.JSON), u -> {}); - - final List expectedTranslogOps = new ArrayList<>(Arrays.asList(1L, 0L, 3L, 2L)); // 2L is NoOp. + final int translogOps; if (randomBoolean()) { if (randomBoolean()) { logger.info("--> flushing shard (translog will be trimmed)"); IndexMetaData.Builder builder = IndexMetaData.builder(orgReplica.indexSettings().getIndexMetaData()); + // Adjusts the translog size retention policy so that it will retain only the 2nd but not 1st translog gen. builder.settings(Settings.builder().put(orgReplica.indexSettings().getSettings()) .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") - .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1") + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), + new ByteSizeValue(uncommittedBytes + 1).toString()) ); orgReplica.indexSettings().updateIndexMetaData(builder.build()); orgReplica.onSettingsChanged(); - orgReplica.updateGlobalCheckpointOnReplica(1L, "test"); - if (isolateFirstBatch) { - // An operation with seqno=1L won't be retained in the translog. - expectedTranslogOps.remove(1L); - if (duplicateStaleIndexOp == false) { - expectedTranslogOps.remove(0L); - } - } + translogOps = 3; // 2 ops + seqno gaps + } else { + logger.info("--> flushing shard (translog will be retained)"); + translogOps = 4; // 3 ops + seqno gaps } - flushShard(orgReplica, true); + orgReplica.updateGlobalCheckpointOnReplica(1L, "test"); + flushShard(orgReplica); + } else { + translogOps = 4; // 3 ops + seqno gaps } + final IndexShard orgPrimary = shards.getPrimary(); shards.promoteReplicaToPrimary(orgReplica).get(); // wait for primary/replica sync to make sure seq# gap is closed. @@ -157,14 +147,7 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { shards.recoverReplica(newReplica); shards.assertAllEqual(1); - try (Translog.Snapshot snapshot = newReplica.getTranslog().newSnapshot()) { - final List actualOps = new ArrayList<>(); - Translog.Operation op; - while ((op = snapshot.next()) != null) { - actualOps.add(op.seqNo()); - } - assertThat(actualOps, containsInAnyOrder(expectedTranslogOps.toArray(new Long[expectedTranslogOps.size()]))); - } + assertThat(newReplica.getTranslog().totalOperations(), equalTo(translogOps)); } } From 6733b750d714955d2715eb1d67f8e31305e48b9e Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 12 Dec 2017 10:21:19 -0500 Subject: [PATCH 23/29] do not use size-retention translog --- .../indices/recovery/RecoveryTests.java | 49 +++++++++++++------ 1 file changed, 33 insertions(+), 16 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index 15bff3223aa8d..f8f8470d78019 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; @@ -100,44 +99,62 @@ public void testRetentionPolicyChangeDuringRecovery() throws Exception { } public void testRecoveryWithOutOfOrderDelete() throws Exception { + /* + * Flow of this test: + * - delete #1 + * - roll generation (to create gen 2) + * - index #0 + * - index #3 + * - flush (commit point has max_seqno 3, and local checkpoint 1 -> points at gen 2, previous commit point is maintained) + * - index #2 + * - index #5 + * - update the global checkpoint to #3 + * - If flush and the translog retention disabled, the delete #1 will be removed while index #0 is still retained. + */ try (ReplicationGroup shards = createGroup(1)) { shards.startAll(); // create out of order delete and index op on replica final IndexShard orgReplica = shards.getReplicas().get(0); + final String indexName = orgReplica.shardId().getIndexName(); + + // delete #1 orgReplica.applyDeleteOperationOnReplica(1, 2, "type", "id", VersionType.EXTERNAL, u -> {}); orgReplica.getTranslog().rollGeneration(); // isolate the delete in it's own generation - + // index #0 orgReplica.applyIndexOperationOnReplica(0, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(orgReplica.shardId().getIndexName(), "type", "id", new BytesArray("{}"), XContentType.JSON), - u -> {}); - long uncommittedBytes = orgReplica.translogStats().getUncommittedSizeInBytes(); + SourceToParse.source(indexName, "type", "id", new BytesArray("{}"), XContentType.JSON), u -> {}); + // index #3 + orgReplica.applyIndexOperationOnReplica(3, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id-3", new BytesArray("{}"), XContentType.JSON), u -> {}); // Flushing a new commit with local checkpoint=1 allows to delete the 1st translog file. orgReplica.flush(new FlushRequest().force(true).waitIfOngoing(true)); - // index a second item into the second generation, skipping seq# 2. - orgReplica.applyIndexOperationOnReplica(3, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, - SourceToParse.source(orgReplica.shardId().getIndexName(), "type", "id2", new BytesArray("{}"), XContentType.JSON), u -> {}); + // index #2 + orgReplica.applyIndexOperationOnReplica(2, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id-2", new BytesArray("{}"), XContentType.JSON), u -> {}); + orgReplica.updateGlobalCheckpointOnReplica(3L, "test"); + // index #5 -> force NoOp #4. + orgReplica.applyIndexOperationOnReplica(5, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, + SourceToParse.source(indexName, "type", "id-5", new BytesArray("{}"), XContentType.JSON), u -> {}); + final int translogOps; if (randomBoolean()) { if (randomBoolean()) { logger.info("--> flushing shard (translog will be trimmed)"); IndexMetaData.Builder builder = IndexMetaData.builder(orgReplica.indexSettings().getIndexMetaData()); - // Adjusts the translog size retention policy so that it will retain only the 2nd but not 1st translog gen. builder.settings(Settings.builder().put(orgReplica.indexSettings().getSettings()) .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1") - .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), - new ByteSizeValue(uncommittedBytes + 1).toString()) - ); + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1")); orgReplica.indexSettings().updateIndexMetaData(builder.build()); orgReplica.onSettingsChanged(); - translogOps = 3; // 2 ops + seqno gaps + translogOps = 5; // 4 ops + seqno gaps (delete #1 is removed) } else { logger.info("--> flushing shard (translog will be retained)"); - translogOps = 4; // 3 ops + seqno gaps + translogOps = 6; // 5 ops + seqno gaps } orgReplica.updateGlobalCheckpointOnReplica(1L, "test"); flushShard(orgReplica); } else { - translogOps = 4; // 3 ops + seqno gaps + translogOps = 6; // 5 ops + seqno gaps } final IndexShard orgPrimary = shards.getPrimary(); @@ -145,7 +162,7 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { IndexShard newReplica = shards.addReplicaWithExistingPath(orgPrimary.shardPath(), orgPrimary.routingEntry().currentNodeId()); shards.recoverReplica(newReplica); - shards.assertAllEqual(1); + shards.assertAllEqual(3); assertThat(newReplica.getTranslog().totalOperations(), equalTo(translogOps)); } From 3a67d4d1e2bef96ccaeddaf8fe5b2889e923263d Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 12 Dec 2017 11:59:03 -0500 Subject: [PATCH 24/29] do nothing with OPEN_INDEX_CREATE_TRANSLOG --- .../index/engine/CombinedDeletionPolicy.java | 10 ++++------ .../engine/CombinedDeletionPolicyTests.java | 18 ------------------ 2 files changed, 4 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index ad3614eae37e9..2d46ae1d8a863 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/core/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -57,11 +57,8 @@ public void onInit(List commits) throws IOException { break; case OPEN_INDEX_CREATE_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; - // When an engine starts with OPEN_INDEX_CREATE_TRANSLOG, we can delete all commits except the last commit, - // without updating the translog policy because these commits do not belong this engine. - for (int i = 0; i < commits.size() - 1; i++) { - commits.get(i).delete(); - } + // When an engine starts with OPEN_INDEX_CREATE_TRANSLOG, a new fresh index commit will be created immediately. + // We therefore can simply skip processing here as `onCommit` will be called right after with a new commit. break; case OPEN_INDEX_AND_TRANSLOG: assert commits.isEmpty() == false : "index is opened, but we have no commits"; @@ -95,6 +92,7 @@ private void updateTranslogDeletionPolicy(final IndexCommit minRequiredCommit, f /** * Find the highest index position of a safe index commit whose max sequence number is not greater than the global checkpoint. + * Index commits with different translog UUID will be filtered out as they don't belong to this engine. */ private int indexOfKeptCommits(List commits) throws IOException { final long currentGlobalCheckpoint = globalCheckpointSupplier.getAsLong(); @@ -103,7 +101,7 @@ private int indexOfKeptCommits(List commits) throws IOExc // Commits are sorted by age (the 0th one is the oldest commit). for (int i = commits.size() - 1; i >= 0; i--) { final Map commitUserData = commits.get(i).getUserData(); - // Index commit with a different TRANSLOG_UUID_KEY don't belong to this engine, skip it. + // Ignore index commits with different translog uuid. if (expectedTranslogUUID.equals(commitUserData.get(Translog.TRANSLOG_UUID_KEY)) == false) { return i + 1; } diff --git a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index 009898ea27c63..0fc6195161a0a 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -46,7 +46,6 @@ import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; public class CombinedDeletionPolicyTests extends ESTestCase { @@ -176,23 +175,6 @@ public void testDeleteInvalidCommits() throws Exception { } } - public void testDeleteAllCommitExceptLastWhenTruncateTranslog() throws Exception { - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); - TranslogDeletionPolicy translogPolicy = mock(TranslogDeletionPolicy.class); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_CREATE_TRANSLOG, translogPolicy, globalCheckpoint::get); - final int numCommits = between(1, 10); - final List commitList = new ArrayList<>(numCommits); - for (int i = 0; i < numCommits; i++) { - commitList.add(mockIndexCommit(randomNonNegativeLong(), UUID.randomUUID(), randomNonNegativeLong())); - } - indexPolicy.onInit(commitList); - verifyZeroInteractions(translogPolicy); - for (int i = 0; i < numCommits - 1; i++) { - verify(commitList.get(i), times(1)).delete(); - } - verifyZeroInteractions(commitList.get(numCommits - 1)); - } - IndexCommit mockIndexCommit(long maxSeqNo, UUID translogUUID, long translogGen) throws IOException { final Map userData = new HashMap<>(); userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo)); From 53702f9c78b5df08bddf7be52099c00fab639eee Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 12 Dec 2017 12:07:06 -0500 Subject: [PATCH 25/29] simplify the recovery tests --- .../elasticsearch/indices/recovery/RecoveryTests.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index f8f8470d78019..689627d664246 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -100,7 +100,7 @@ public void testRetentionPolicyChangeDuringRecovery() throws Exception { public void testRecoveryWithOutOfOrderDelete() throws Exception { /* - * Flow of this test: + * The flow of this test: * - delete #1 * - roll generation (to create gen 2) * - index #0 @@ -108,8 +108,7 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { * - flush (commit point has max_seqno 3, and local checkpoint 1 -> points at gen 2, previous commit point is maintained) * - index #2 * - index #5 - * - update the global checkpoint to #3 - * - If flush and the translog retention disabled, the delete #1 will be removed while index #0 is still retained. + * - If flush and the translog retention disabled, delete #1 will be removed while index #0 is still retained and replayed. */ try (ReplicationGroup shards = createGroup(1)) { shards.startAll(); @@ -126,7 +125,7 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { // index #3 orgReplica.applyIndexOperationOnReplica(3, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, SourceToParse.source(indexName, "type", "id-3", new BytesArray("{}"), XContentType.JSON), u -> {}); - // Flushing a new commit with local checkpoint=1 allows to delete the 1st translog file. + // Flushing a new commit with local checkpoint=1 allows to delete the translog gen #1. orgReplica.flush(new FlushRequest().force(true).waitIfOngoing(true)); // index #2 orgReplica.applyIndexOperationOnReplica(2, 1, VersionType.EXTERNAL, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, @@ -146,12 +145,11 @@ public void testRecoveryWithOutOfOrderDelete() throws Exception { .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1")); orgReplica.indexSettings().updateIndexMetaData(builder.build()); orgReplica.onSettingsChanged(); - translogOps = 5; // 4 ops + seqno gaps (delete #1 is removed) + translogOps = 5; // 4 ops + seqno gaps (delete #1 is removed but index #0 will be replayed). } else { logger.info("--> flushing shard (translog will be retained)"); translogOps = 6; // 5 ops + seqno gaps } - orgReplica.updateGlobalCheckpointOnReplica(1L, "test"); flushShard(orgReplica); } else { translogOps = 6; // 5 ops + seqno gaps From 5644456f7e80147ee05e7c37e63b59d37ce3c9e5 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 12 Dec 2017 12:52:46 -0500 Subject: [PATCH 26/29] just test keep only one safe commit --- .../index/engine/InternalEngineTests.java | 49 +++++-------------- .../index/translog/SnapshotMatchers.java | 48 ------------------ 2 files changed, 12 insertions(+), 85 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 4e7284f239625..5a98a2becc3ac 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -84,7 +84,6 @@ import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -159,7 +158,6 @@ import static org.elasticsearch.index.engine.Engine.Operation.Origin.PEER_RECOVERY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; -import static org.elasticsearch.index.translog.SnapshotMatchers.containsSeqNoRange; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.Matchers.equalTo; @@ -167,6 +165,7 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -4139,9 +4138,9 @@ public void testSeqNoGenerator() throws IOException { } } - public void testKeepTranslogAfterGlobalCheckpoint() throws Exception { + public void testKeepOnlyOneSafeCommit() throws Exception { IOUtils.close(engine, store); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final AtomicLong globalCheckpoint = new AtomicLong(0); final BiFunction seqNoServiceSupplier = (config, seqNoStats) -> new SequenceNumbersService( config.getShardId(), @@ -4156,28 +4155,14 @@ public long getGlobalCheckpoint() { } }; - final IndexSettings indexSettings = new IndexSettings(defaultSettings.getIndexMetaData(), defaultSettings.getNodeSettings(), - defaultSettings.getScopedSettings()); - IndexMetaData.Builder builder = IndexMetaData.builder(indexSettings.getIndexMetaData()); - builder.settings(Settings.builder().put(indexSettings.getSettings()) - .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), - randomBoolean() ? "-1" : TimeValue.timeValueMillis(randomNonNegativeLong()).getStringRep()) - .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), - randomBoolean() ? "-1" : new ByteSizeValue(randomNonNegativeLong()).toString()) - ); - indexSettings.updateIndexMetaData(builder.build()); - - final Path translogPath = createTempDir(); store = createStore(); try (InternalEngine engine - = new InternalEngine(config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null), seqNoServiceSupplier)) { + = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null, seqNoServiceSupplier)) { int numDocs = scaledRandomIntBetween(10, 100); - int uncommittedOps = 0; for (int i = 0; i < numDocs; i++) { ParseContext.Document document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); engine.index(indexForDoc(testParsedDocument(Integer.toString(i), null, document, B_1, null))); - uncommittedOps++; if (frequently()) { globalCheckpoint.set(randomIntBetween( Math.toIntExact(engine.seqNoService().getGlobalCheckpoint()), @@ -4185,26 +4170,16 @@ public long getGlobalCheckpoint() { } if (frequently()) { engine.flush(randomBoolean(), true); - uncommittedOps = 0; - } - if (rarely()) { - engine.rollTranslogGeneration(); - } - assertThat(engine.getTranslog().uncommittedOperations(), equalTo(uncommittedOps)); - try (Translog.Snapshot snapshot = engine.getTranslog().newSnapshot()) { - assertThat(snapshot, - containsSeqNoRange(Math.max(0, globalCheckpoint.get() + 1), engine.seqNoService().getLocalCheckpoint())); + List safeCommits = new ArrayList<>(); + final List existingCommits = DirectoryReader.listCommits(store.directory()); + for (IndexCommit commit : existingCommits) { + if (Long.parseLong(commit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)) <= globalCheckpoint.get()) { + safeCommits.add(commit); + } + } + assertThat("Should keep only one safe commit", safeCommits, hasSize(1)); } } - engine.flush(randomBoolean(), true); - } - // Reopen engine to test onInit with existing index commits. - try (InternalEngine engine - = new InternalEngine(config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null), seqNoServiceSupplier)) { - try (Translog.Snapshot snapshot = engine.getTranslog().newSnapshot()) { - assertThat(snapshot, - containsSeqNoRange(Math.max(0, globalCheckpoint.get() + 1), engine.seqNoService().getLocalCheckpoint())); - } } } } diff --git a/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java b/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java index 21f7dd9481c5e..c2f92b1f0870a 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java +++ b/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java @@ -63,13 +63,6 @@ public static Matcher containsOperationsInAnyOrder(Collection return new ContainingInAnyOrderMatcher(expectedOperations); } - /** - * Consumes a snapshot and makes sure that its operations have all seqno between minSeqNo(inclusive) and maxSeqNo(inclusive). - */ - public static Matcher containsSeqNoRange(long minSeqNo, long maxSeqNo) { - return new ContainingSeqNoRangeMatcher(minSeqNo, maxSeqNo); - } - public static class SizeMatcher extends TypeSafeMatcher { private final int size; @@ -199,45 +192,4 @@ public void describeTo(Description description) { .appendText(" in any order."); } } - - static class ContainingSeqNoRangeMatcher extends TypeSafeMatcher { - private final long minSeqNo; - private final long maxSeqNo; - private final List notFoundSeqNo = new ArrayList<>(); - - ContainingSeqNoRangeMatcher(long minSeqNo, long maxSeqNo) { - this.minSeqNo = minSeqNo; - this.maxSeqNo = maxSeqNo; - } - - @Override - protected boolean matchesSafely(Translog.Snapshot snapshot) { - try { - final LongSet seqNoList = new LongHashSet(); - Translog.Operation op; - while ((op = snapshot.next()) != null) { - seqNoList.add(op.seqNo()); - } - for (long i = minSeqNo; i <= maxSeqNo; i++) { - if (seqNoList.contains(i) == false) { - notFoundSeqNo.add(i); - } - } - return notFoundSeqNo.isEmpty(); - } catch (IOException ex) { - throw new ElasticsearchException("failed to read snapshot content", ex); - } - } - - @Override - protected void describeMismatchSafely(Translog.Snapshot snapshot, Description mismatchDescription) { - mismatchDescription - .appendText("not found seqno ").appendValueList("[", ", ", "]", notFoundSeqNo); - } - - @Override - public void describeTo(Description description) { - description.appendText("snapshot contains all seqno from [" + minSeqNo + " to " + maxSeqNo + "]"); - } - } } From dad3a1bcf581f0a81de6c5d9e586485b6d4730c6 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 12 Dec 2017 15:22:08 -0500 Subject: [PATCH 27/29] assert #forceNewHistoryUUID --- .../java/org/elasticsearch/index/engine/InternalEngine.java | 4 ++++ .../org/elasticsearch/index/engine/InternalEngineTests.java | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 6d4a3d14da3ef..ea91125c496a1 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -187,6 +187,10 @@ public InternalEngine(EngineConfig engineConfig) { ); writer = createWriter(openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG); updateMaxUnsafeAutoIdTimestampFromWriter(writer); + assert engineConfig.getForceNewHistoryUUID() == false + || openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG + || openMode == EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG + : "OpenMode must be either CREATE_INDEX_AND_TRANSLOG or OPEN_INDEX_CREATE_TRANSLOG if forceNewHistoryUUID is true"; historyUUID = loadOrGenerateHistoryUUID(writer, engineConfig.getForceNewHistoryUUID()); Objects.requireNonNull(historyUUID, "history uuid should not be null"); indexWriter = writer; diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 5a98a2becc3ac..438a4e5e9b5f0 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -79,6 +79,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; @@ -2674,13 +2675,16 @@ public void testHistoryUUIDCanBeForced() throws IOException { EngineConfig config = engine.config(); EngineConfig newConfig = new EngineConfig( - randomBoolean() ? EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG : EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG, + randomBoolean() ? EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG : EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG, shardId, allocationId.getId(), threadPool, config.getIndexSettings(), null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), true, config.getTranslogConfig(), TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), new NoneCircuitBreakerService()); + if (newConfig.getOpenMode() == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG) { + Lucene.cleanLuceneIndex(store.directory()); + } engine = new InternalEngine(newConfig); if (newConfig.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { engine.recoverFromTranslog(); From b18725cac4751ae05a2ccfc900990d55c4e31ee5 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 12 Dec 2017 16:16:06 -0500 Subject: [PATCH 28/29] keeps ops after local checkpoint of safe commit --- .../index/engine/InternalEngineTests.java | 37 +++++++++----- .../index/translog/SnapshotMatchers.java | 48 +++++++++++++++++++ 2 files changed, 74 insertions(+), 11 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 438a4e5e9b5f0..64146f2805a2b 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -114,6 +114,7 @@ import org.elasticsearch.index.shard.ShardUtils; import org.elasticsearch.index.store.DirectoryUtils; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.SnapshotMatchers; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; @@ -4142,7 +4143,7 @@ public void testSeqNoGenerator() throws IOException { } } - public void testKeepOnlyOneSafeCommit() throws Exception { + public void testKeepTranslogAfterGlobalCheckpoint() throws Exception { IOUtils.close(engine, store); final AtomicLong globalCheckpoint = new AtomicLong(0); final BiFunction seqNoServiceSupplier = (config, seqNoStats) -> @@ -4159,14 +4160,22 @@ public long getGlobalCheckpoint() { } }; + final IndexSettings indexSettings = new IndexSettings(defaultSettings.getIndexMetaData(), defaultSettings.getNodeSettings(), + defaultSettings.getScopedSettings()); + IndexMetaData.Builder builder = IndexMetaData.builder(indexSettings.getIndexMetaData()) + .settings(Settings.builder().put(indexSettings.getSettings()) + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), randomFrom("-1", "100micros", "30m")) + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), randomFrom("-1", "512b", "1gb"))); + indexSettings.updateIndexMetaData(builder.build()); + store = createStore(); try (InternalEngine engine - = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null, seqNoServiceSupplier)) { + = createEngine(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null, seqNoServiceSupplier)) { int numDocs = scaledRandomIntBetween(10, 100); - for (int i = 0; i < numDocs; i++) { + for (int docId = 0; docId < numDocs; docId++) { ParseContext.Document document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); - engine.index(indexForDoc(testParsedDocument(Integer.toString(i), null, document, B_1, null))); + engine.index(indexForDoc(testParsedDocument(Integer.toString(docId), null, document, B_1, null))); if (frequently()) { globalCheckpoint.set(randomIntBetween( Math.toIntExact(engine.seqNoService().getGlobalCheckpoint()), @@ -4174,14 +4183,20 @@ public long getGlobalCheckpoint() { } if (frequently()) { engine.flush(randomBoolean(), true); - List safeCommits = new ArrayList<>(); - final List existingCommits = DirectoryReader.listCommits(store.directory()); - for (IndexCommit commit : existingCommits) { - if (Long.parseLong(commit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)) <= globalCheckpoint.get()) { - safeCommits.add(commit); - } + final List commits = DirectoryReader.listCommits(store.directory()); + // Keep only one safe commit as the oldest commit. + final IndexCommit safeCommit = commits.get(0); + assertThat(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)), + lessThanOrEqualTo(globalCheckpoint.get())); + for (int i = 1; i < commits.size(); i++) { + assertThat(Long.parseLong(commits.get(i).getUserData().get(SequenceNumbers.MAX_SEQ_NO)), + greaterThan(globalCheckpoint.get())); + } + // Make sure we keep all translog operations after the local checkpoint of the safe commit. + long localCheckpointFromSafeCommit = Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + try (Translog.Snapshot snapshot = engine.getTranslog().newSnapshot()) { + assertThat(snapshot, SnapshotMatchers.containsSeqNoRange(localCheckpointFromSafeCommit + 1, docId)); } - assertThat("Should keep only one safe commit", safeCommits, hasSize(1)); } } } diff --git a/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java b/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java index c2f92b1f0870a..21f7dd9481c5e 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java +++ b/core/src/test/java/org/elasticsearch/index/translog/SnapshotMatchers.java @@ -63,6 +63,13 @@ public static Matcher containsOperationsInAnyOrder(Collection return new ContainingInAnyOrderMatcher(expectedOperations); } + /** + * Consumes a snapshot and makes sure that its operations have all seqno between minSeqNo(inclusive) and maxSeqNo(inclusive). + */ + public static Matcher containsSeqNoRange(long minSeqNo, long maxSeqNo) { + return new ContainingSeqNoRangeMatcher(minSeqNo, maxSeqNo); + } + public static class SizeMatcher extends TypeSafeMatcher { private final int size; @@ -192,4 +199,45 @@ public void describeTo(Description description) { .appendText(" in any order."); } } + + static class ContainingSeqNoRangeMatcher extends TypeSafeMatcher { + private final long minSeqNo; + private final long maxSeqNo; + private final List notFoundSeqNo = new ArrayList<>(); + + ContainingSeqNoRangeMatcher(long minSeqNo, long maxSeqNo) { + this.minSeqNo = minSeqNo; + this.maxSeqNo = maxSeqNo; + } + + @Override + protected boolean matchesSafely(Translog.Snapshot snapshot) { + try { + final LongSet seqNoList = new LongHashSet(); + Translog.Operation op; + while ((op = snapshot.next()) != null) { + seqNoList.add(op.seqNo()); + } + for (long i = minSeqNo; i <= maxSeqNo; i++) { + if (seqNoList.contains(i) == false) { + notFoundSeqNo.add(i); + } + } + return notFoundSeqNo.isEmpty(); + } catch (IOException ex) { + throw new ElasticsearchException("failed to read snapshot content", ex); + } + } + + @Override + protected void describeMismatchSafely(Translog.Snapshot snapshot, Description mismatchDescription) { + mismatchDescription + .appendText("not found seqno ").appendValueList("[", ", ", "]", notFoundSeqNo); + } + + @Override + public void describeTo(Description description) { + description.appendText("snapshot contains all seqno from [" + minSeqNo + " to " + maxSeqNo + "]"); + } + } } From 09b0f91bb8ca54e9a2935237897d6f84c3e8df3f Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 12 Dec 2017 16:36:14 -0500 Subject: [PATCH 29/29] explain the assertion --- .../java/org/elasticsearch/index/engine/InternalEngine.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index ea91125c496a1..a2897100e4c3f 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -190,7 +190,8 @@ public InternalEngine(EngineConfig engineConfig) { assert engineConfig.getForceNewHistoryUUID() == false || openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG || openMode == EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG - : "OpenMode must be either CREATE_INDEX_AND_TRANSLOG or OPEN_INDEX_CREATE_TRANSLOG if forceNewHistoryUUID is true"; + : "OpenMode must be either CREATE_INDEX_AND_TRANSLOG or OPEN_INDEX_CREATE_TRANSLOG if forceNewHistoryUUID; " + + "openMode [" + openMode + "], forceNewHistoryUUID [" + engineConfig.getForceNewHistoryUUID() + "]"; historyUUID = loadOrGenerateHistoryUUID(writer, engineConfig.getForceNewHistoryUUID()); Objects.requireNonNull(historyUUID, "history uuid should not be null"); indexWriter = writer;