|
@@ -81,7 +81,6 @@ import org.elasticsearch.common.bytes.BytesArray;
|
|
|
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;
|
|
@@ -119,7 +118,6 @@ import org.elasticsearch.index.translog.SnapshotMatchers;
|
|
|
import org.elasticsearch.index.translog.Translog;
|
|
|
import org.elasticsearch.index.translog.TranslogConfig;
|
|
|
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
|
|
-import org.elasticsearch.test.IndexSettingsModule;
|
|
|
import org.hamcrest.MatcherAssert;
|
|
|
import org.hamcrest.Matchers;
|
|
|
|
|
@@ -133,7 +131,6 @@ import java.util.Arrays;
|
|
|
import java.util.Base64;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Comparator;
|
|
|
-import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.LinkedHashMap;
|
|
|
import java.util.List;
|
|
@@ -149,6 +146,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
import java.util.function.BiFunction;
|
|
|
+import java.util.function.Function;
|
|
|
import java.util.function.LongSupplier;
|
|
|
import java.util.function.Supplier;
|
|
|
import java.util.function.ToLongBiFunction;
|
|
@@ -641,7 +639,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
InternalEngine engine = createEngine(store, translog);
|
|
|
engine.close();
|
|
|
|
|
|
- engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG));
|
|
|
+ engine = new InternalEngine(engine.config());
|
|
|
assertTrue(engine.isRecovering());
|
|
|
engine.recoverFromTranslog();
|
|
|
Engine.Searcher searcher = wrapper.wrap(engine.acquireSearcher("test"));
|
|
@@ -656,7 +654,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
engine.index(indexForDoc(doc));
|
|
|
engine.close();
|
|
|
|
|
|
- engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG));
|
|
|
+ engine = new InternalEngine(engine.config());
|
|
|
expectThrows(IllegalStateException.class, () -> engine.flush(true, true));
|
|
|
assertTrue(engine.isRecovering());
|
|
|
engine.recoverFromTranslog();
|
|
@@ -690,7 +688,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
Engine recoveringEngine = null;
|
|
|
try {
|
|
|
- recoveringEngine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG));
|
|
|
+ recoveringEngine = new InternalEngine(engine.config());
|
|
|
recoveringEngine.recoverFromTranslog();
|
|
|
try (Engine.Searcher searcher = recoveringEngine.acquireSearcher("test")) {
|
|
|
final TotalHitCountCollector collector = new TotalHitCountCollector();
|
|
@@ -718,20 +716,19 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
Engine recoveringEngine = null;
|
|
|
try {
|
|
|
- final AtomicBoolean flushed = new AtomicBoolean();
|
|
|
- recoveringEngine = new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)) {
|
|
|
+ final AtomicBoolean committed = new AtomicBoolean();
|
|
|
+ recoveringEngine = new InternalEngine(initialEngine.config()) {
|
|
|
+
|
|
|
@Override
|
|
|
- public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException {
|
|
|
- assertThat(getTranslog().uncommittedOperations(), equalTo(docs));
|
|
|
- final CommitId commitId = super.flush(force, waitIfOngoing);
|
|
|
- flushed.set(true);
|
|
|
- return commitId;
|
|
|
+ protected void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException {
|
|
|
+ committed.set(true);
|
|
|
+ super.commitIndexWriter(writer, translog, syncId);
|
|
|
}
|
|
|
};
|
|
|
|
|
|
assertThat(recoveringEngine.getTranslog().uncommittedOperations(), equalTo(docs));
|
|
|
recoveringEngine.recoverFromTranslog();
|
|
|
- assertTrue(flushed.get());
|
|
|
+ assertTrue(committed.get());
|
|
|
} finally {
|
|
|
IOUtils.close(recoveringEngine);
|
|
|
}
|
|
@@ -762,7 +759,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
}
|
|
|
}
|
|
|
initialEngine.close();
|
|
|
- recoveringEngine = new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG));
|
|
|
+ recoveringEngine = new InternalEngine(initialEngine.config());
|
|
|
recoveringEngine.recoverFromTranslog();
|
|
|
try (Engine.Searcher searcher = recoveringEngine.acquireSearcher("test")) {
|
|
|
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), docs);
|
|
@@ -1009,9 +1006,9 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
IOUtils.close(engine, store);
|
|
|
final Path translogPath = createTempDir();
|
|
|
store = createStore();
|
|
|
- final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
final LongSupplier globalCheckpointSupplier = () -> globalCheckpoint.get();
|
|
|
- engine = new InternalEngine(config(defaultSettings, store, translogPath, newMergePolicy(), null, null, globalCheckpointSupplier));
|
|
|
+ engine = createEngine(config(defaultSettings, store, translogPath, newMergePolicy(), null, null, globalCheckpointSupplier));
|
|
|
ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null);
|
|
|
engine.index(indexForDoc(doc));
|
|
|
boolean inSync = randomBoolean();
|
|
@@ -1021,17 +1018,17 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
engine.flush();
|
|
|
assertThat(engine.getTranslog().currentFileGeneration(), equalTo(3L));
|
|
|
- assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 3L : 2L));
|
|
|
+ assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 3L : 1L));
|
|
|
assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(3L));
|
|
|
|
|
|
engine.flush();
|
|
|
assertThat(engine.getTranslog().currentFileGeneration(), equalTo(3L));
|
|
|
- assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 3L : 2L));
|
|
|
+ assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 3L : 1L));
|
|
|
assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(3L));
|
|
|
|
|
|
engine.flush(true, true);
|
|
|
assertThat(engine.getTranslog().currentFileGeneration(), equalTo(4L));
|
|
|
- assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 4L : 2L));
|
|
|
+ assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 4L : 1L));
|
|
|
assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(4L));
|
|
|
|
|
|
globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint());
|
|
@@ -1043,7 +1040,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
public void testSyncedFlush() throws IOException {
|
|
|
try (Store store = createStore();
|
|
|
- Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new LogByteSizeMergePolicy(), null))) {
|
|
|
+ Engine engine = createEngine(defaultSettings, store, createTempDir(), new LogByteSizeMergePolicy(), null)) {
|
|
|
final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20);
|
|
|
ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null);
|
|
|
engine.index(indexForDoc(doc));
|
|
@@ -1069,8 +1066,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
final int iters = randomIntBetween(2, 5); // run this a couple of times to get some coverage
|
|
|
for (int i = 0; i < iters; i++) {
|
|
|
try (Store store = createStore();
|
|
|
- InternalEngine engine = new InternalEngine(config(defaultSettings, store, createTempDir(),
|
|
|
- new LogDocMergePolicy(), null))) {
|
|
|
+ InternalEngine engine =
|
|
|
+ createEngine(config(defaultSettings, store, createTempDir(), new LogDocMergePolicy(), null))) {
|
|
|
final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20);
|
|
|
Engine.Index doc1 = indexForDoc(testParsedDocument("1", null, testDocumentWithTextField(), B_1, null));
|
|
|
engine.index(doc1);
|
|
@@ -1125,7 +1122,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
}
|
|
|
|
|
|
public void testSyncedFlushSurvivesEngineRestart() throws IOException {
|
|
|
- final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
IOUtils.close(store, engine);
|
|
|
store = createStore();
|
|
|
engine = createEngine(store, primaryTranslogDir, globalCheckpoint::get);
|
|
@@ -1144,12 +1141,13 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
} else {
|
|
|
engine.flushAndClose();
|
|
|
}
|
|
|
- engine = new InternalEngine(copy(config, randomFrom(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG)));
|
|
|
-
|
|
|
- if (engine.config().getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG && randomBoolean()) {
|
|
|
- engine.recoverFromTranslog();
|
|
|
+ if (randomBoolean()) {
|
|
|
+ EngineDiskUtils.createNewTranslog(store.directory(), config.getTranslogConfig().getTranslogPath(),
|
|
|
+ SequenceNumbers.UNASSIGNED_SEQ_NO, shardId);
|
|
|
}
|
|
|
- assertEquals(engine.config().getOpenMode().toString(), engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId);
|
|
|
+ engine = new InternalEngine(config);
|
|
|
+ engine.recoverFromTranslog();
|
|
|
+ assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId);
|
|
|
}
|
|
|
|
|
|
public void testSyncedFlushVanishesOnReplay() throws IOException {
|
|
@@ -1165,7 +1163,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
engine.index(indexForDoc(doc));
|
|
|
EngineConfig config = engine.config();
|
|
|
engine.close();
|
|
|
- engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG));
|
|
|
+ engine = new InternalEngine(config);
|
|
|
engine.recoverFromTranslog();
|
|
|
assertNull("Sync ID must be gone since we have a document to replay", engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID));
|
|
|
}
|
|
@@ -1270,7 +1268,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
public void testForceMerge() throws IOException {
|
|
|
try (Store store = createStore();
|
|
|
- Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(),
|
|
|
+ Engine engine = createEngine(config(defaultSettings, store, createTempDir(),
|
|
|
new LogByteSizeMergePolicy(), null))) { // use log MP here we test some behavior in ESMP
|
|
|
int numDocs = randomIntBetween(10, 100);
|
|
|
for (int i = 0; i < numDocs; i++) {
|
|
@@ -2051,7 +2049,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
InternalEngine recoveringEngine = null;
|
|
|
try {
|
|
|
- recoveringEngine = new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG));
|
|
|
+ recoveringEngine = new InternalEngine(initialEngine.config());
|
|
|
recoveringEngine.recoverFromTranslog();
|
|
|
|
|
|
assertEquals(primarySeqNo, recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo());
|
|
@@ -2078,10 +2076,9 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
// this test writes documents to the engine while concurrently flushing/commit
|
|
|
// and ensuring that the commit points contain the correct sequence number data
|
|
|
public void testConcurrentWritesAndCommits() throws Exception {
|
|
|
+ List<Engine.IndexCommitRef> commits = new ArrayList<>();
|
|
|
try (Store store = createStore();
|
|
|
- InternalEngine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), newMergePolicy(), null))) {
|
|
|
- final List<Engine.IndexCommitRef> commits = new ArrayList<>();
|
|
|
-
|
|
|
+ InternalEngine engine = createEngine(config(defaultSettings, store, createTempDir(), newMergePolicy(), null))) {
|
|
|
final int numIndexingThreads = scaledRandomIntBetween(2, 4);
|
|
|
final int numDocsPerThread = randomIntBetween(500, 1000);
|
|
|
final CyclicBarrier barrier = new CyclicBarrier(numIndexingThreads + 1);
|
|
@@ -2243,7 +2240,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
public void testEnableGcDeletes() throws Exception {
|
|
|
try (Store store = createStore();
|
|
|
- Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), newMergePolicy(), null))) {
|
|
|
+ Engine engine = createEngine(config(defaultSettings, store, createTempDir(), newMergePolicy(), null))) {
|
|
|
engine.config().setEnableGcDeletes(false);
|
|
|
|
|
|
final BiFunction<String, Engine.SearcherScope, Searcher> searcherFactory = engine::acquireSearcher;
|
|
@@ -2326,7 +2323,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
InternalEngine holder;
|
|
|
try {
|
|
|
holder = createEngine(store, translogPath);
|
|
|
- } catch (EngineCreationFailureException ex) {
|
|
|
+ } catch (EngineCreationFailureException | IOException ex) {
|
|
|
assertEquals(store.refCount(), refCount);
|
|
|
continue;
|
|
|
}
|
|
@@ -2372,9 +2369,9 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
} catch (EngineCreationFailureException ex) {
|
|
|
// expected
|
|
|
}
|
|
|
- // now it should be OK.
|
|
|
- EngineConfig config = copy(config(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null),
|
|
|
- EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG);
|
|
|
+ // when a new translog is created it should be ok
|
|
|
+ EngineDiskUtils.createNewTranslog(store.directory(), primaryTranslogDir, SequenceNumbers.UNASSIGNED_SEQ_NO, shardId);
|
|
|
+ EngineConfig config = config(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null);
|
|
|
engine = new InternalEngine(config);
|
|
|
}
|
|
|
|
|
@@ -2421,21 +2418,6 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException {
|
|
|
- assertVisibleCount(engine, numDocs, true);
|
|
|
- }
|
|
|
-
|
|
|
- private static void assertVisibleCount(InternalEngine engine, int numDocs, boolean refresh) throws IOException {
|
|
|
- if (refresh) {
|
|
|
- engine.refresh("test");
|
|
|
- }
|
|
|
- try (Searcher searcher = engine.acquireSearcher("test")) {
|
|
|
- final TotalHitCountCollector collector = new TotalHitCountCollector();
|
|
|
- searcher.searcher().search(new MatchAllDocsQuery(), collector);
|
|
|
- assertThat(collector.getTotalHits(), equalTo(numDocs));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
public void testTranslogCleanUpPostCommitCrash() throws Exception {
|
|
|
IndexSettings indexSettings = new IndexSettings(defaultSettings.getIndexMetaData(), defaultSettings.getNodeSettings(),
|
|
|
defaultSettings.getScopedSettings());
|
|
@@ -2449,8 +2431,9 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
try (Store store = createStore()) {
|
|
|
AtomicBoolean throwErrorOnCommit = new AtomicBoolean();
|
|
|
final Path translogPath = createTempDir();
|
|
|
- final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
final LongSupplier globalCheckpointSupplier = () -> globalCheckpoint.get();
|
|
|
+ EngineDiskUtils.createEmpty(store.directory(), translogPath, shardId);
|
|
|
try (InternalEngine engine =
|
|
|
new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null,
|
|
|
globalCheckpointSupplier)) {
|
|
@@ -2463,6 +2446,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
}
|
|
|
}
|
|
|
}) {
|
|
|
+ engine.recoverFromTranslog();
|
|
|
final ParsedDocument doc1 = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null);
|
|
|
engine.index(indexForDoc(doc1));
|
|
|
globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint());
|
|
@@ -2495,7 +2479,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
}
|
|
|
assertVisibleCount(engine, numDocs);
|
|
|
engine.close();
|
|
|
- engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG));
|
|
|
+ engine = new InternalEngine(engine.config());
|
|
|
+ engine.skipTranslogRecovery();
|
|
|
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
|
|
|
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10));
|
|
|
assertThat(topDocs.totalHits, equalTo(0L));
|
|
@@ -2535,7 +2520,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
parser.mappingUpdate = dynamicUpdate();
|
|
|
|
|
|
engine.close();
|
|
|
- engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, inSyncGlobalCheckpointSupplier)); // we need to reuse the engine config unless the parser.mappingModified won't work
|
|
|
+ engine = new InternalEngine(copy(engine.config(), inSyncGlobalCheckpointSupplier)); // we need to reuse the engine config unless the parser.mappingModified won't work
|
|
|
engine.recoverFromTranslog();
|
|
|
|
|
|
assertVisibleCount(engine, numDocs, false);
|
|
@@ -2622,10 +2607,10 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
TranslogConfig translogConfig = new TranslogConfig(shardId, translog.location(), config.getIndexSettings(),
|
|
|
BigArrays.NON_RECYCLING_INSTANCE);
|
|
|
|
|
|
- EngineConfig brokenConfig = new EngineConfig(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, shardId, allocationId.getId(),
|
|
|
+ EngineConfig brokenConfig = new EngineConfig(shardId, allocationId.getId(),
|
|
|
threadPool, config.getIndexSettings(), null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(),
|
|
|
new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(),
|
|
|
- IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, TimeValue.timeValueMinutes(5),
|
|
|
+ IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5),
|
|
|
config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(),
|
|
|
new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
try {
|
|
@@ -2638,94 +2623,6 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
assertVisibleCount(engine, numDocs, false);
|
|
|
}
|
|
|
|
|
|
- public void testHistoryUUIDIsSetIfMissing() throws IOException {
|
|
|
- final int numDocs = randomIntBetween(0, 3);
|
|
|
- for (int i = 0; i < numDocs; i++) {
|
|
|
- ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null);
|
|
|
- Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
|
|
|
- Engine.IndexResult index = engine.index(firstIndexRequest);
|
|
|
- assertThat(index.getVersion(), equalTo(1L));
|
|
|
- }
|
|
|
- assertVisibleCount(engine, numDocs);
|
|
|
- engine.close();
|
|
|
-
|
|
|
- IndexWriterConfig iwc = new IndexWriterConfig(null)
|
|
|
- .setCommitOnClose(false)
|
|
|
- // we don't want merges to happen here - we call maybe merge on the engine
|
|
|
- // later once we stared it up otherwise we would need to wait for it here
|
|
|
- // we also don't specify a codec here and merges should use the engines for this index
|
|
|
- .setMergePolicy(NoMergePolicy.INSTANCE)
|
|
|
- .setOpenMode(IndexWriterConfig.OpenMode.APPEND);
|
|
|
- try (IndexWriter writer = new IndexWriter(store.directory(), iwc)) {
|
|
|
- Map<String, String> newCommitData = new HashMap<>();
|
|
|
- for (Map.Entry<String, String> entry: writer.getLiveCommitData()) {
|
|
|
- if (entry.getKey().equals(Engine.HISTORY_UUID_KEY) == false) {
|
|
|
- newCommitData.put(entry.getKey(), entry.getValue());
|
|
|
- }
|
|
|
- }
|
|
|
- writer.setLiveCommitData(newCommitData.entrySet());
|
|
|
- writer.commit();
|
|
|
- }
|
|
|
-
|
|
|
- final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder()
|
|
|
- .put(defaultSettings.getSettings())
|
|
|
- .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_6_0_0_beta1)
|
|
|
- .build());
|
|
|
-
|
|
|
- EngineConfig config = engine.config();
|
|
|
-
|
|
|
- EngineConfig newConfig = new EngineConfig(
|
|
|
- randomBoolean() ? EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG : EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG,
|
|
|
- shardId, allocationId.getId(),
|
|
|
- threadPool, indexSettings, null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(),
|
|
|
- new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(),
|
|
|
- IndexSearcher.getDefaultQueryCachingPolicy(), false, config.getTranslogConfig(), TimeValue.timeValueMinutes(5),
|
|
|
- config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(),
|
|
|
- new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
- engine = new InternalEngine(newConfig);
|
|
|
- if (newConfig.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) {
|
|
|
- engine.recoverFromTranslog();
|
|
|
- assertVisibleCount(engine, numDocs, false);
|
|
|
- } else {
|
|
|
- assertVisibleCount(engine, 0, false);
|
|
|
- }
|
|
|
- assertThat(engine.getHistoryUUID(), notNullValue());
|
|
|
- }
|
|
|
-
|
|
|
- public void testHistoryUUIDCanBeForced() throws IOException {
|
|
|
- final int numDocs = randomIntBetween(0, 3);
|
|
|
- for (int i = 0; i < numDocs; i++) {
|
|
|
- ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null);
|
|
|
- Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
|
|
|
- Engine.IndexResult index = engine.index(firstIndexRequest);
|
|
|
- assertThat(index.getVersion(), equalTo(1L));
|
|
|
- }
|
|
|
- assertVisibleCount(engine, numDocs);
|
|
|
- final String oldHistoryUUID = engine.getHistoryUUID();
|
|
|
- engine.close();
|
|
|
- EngineConfig config = engine.config();
|
|
|
-
|
|
|
- EngineConfig newConfig = new EngineConfig(
|
|
|
- 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(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
- 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();
|
|
|
- assertVisibleCount(engine, numDocs, false);
|
|
|
- } else {
|
|
|
- assertVisibleCount(engine, 0, false);
|
|
|
- }
|
|
|
- assertThat(engine.getHistoryUUID(), not(equalTo(oldHistoryUUID)));
|
|
|
- }
|
|
|
-
|
|
|
public void testShardNotAvailableExceptionWhenEngineClosedConcurrently() throws IOException, InterruptedException {
|
|
|
AtomicReference<Exception> exception = new AtomicReference<>();
|
|
|
String operation = randomFrom("optimize", "refresh", "flush");
|
|
@@ -2818,74 +2715,6 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void testCurrentTranslogIDisCommitted() throws IOException {
|
|
|
- final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
- try (Store store = createStore()) {
|
|
|
- EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get);
|
|
|
-
|
|
|
- // create
|
|
|
- {
|
|
|
- ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null);
|
|
|
- Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false);
|
|
|
-
|
|
|
- try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG))){
|
|
|
- assertFalse(engine.isRecovering());
|
|
|
- engine.index(firstIndexRequest);
|
|
|
- globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint());
|
|
|
- expectThrows(IllegalStateException.class, () -> engine.recoverFromTranslog());
|
|
|
- Map<String, String> userData = engine.getLastCommittedSegmentInfos().getUserData();
|
|
|
- assertEquals("2", userData.get(Translog.TRANSLOG_GENERATION_KEY));
|
|
|
- assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY));
|
|
|
- }
|
|
|
- }
|
|
|
- // open and recover tlog
|
|
|
- {
|
|
|
- for (int i = 0; i < 2; i++) {
|
|
|
- try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) {
|
|
|
- assertTrue(engine.isRecovering());
|
|
|
- Map<String, String> userData = engine.getLastCommittedSegmentInfos().getUserData();
|
|
|
- if (i == 0) {
|
|
|
- assertEquals("2", userData.get(Translog.TRANSLOG_GENERATION_KEY));
|
|
|
- } else {
|
|
|
- assertEquals("4", userData.get(Translog.TRANSLOG_GENERATION_KEY));
|
|
|
- }
|
|
|
- assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY));
|
|
|
- engine.recoverFromTranslog();
|
|
|
- userData = engine.getLastCommittedSegmentInfos().getUserData();
|
|
|
- assertEquals("4", userData.get(Translog.TRANSLOG_GENERATION_KEY));
|
|
|
- assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY));
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- // open index with new tlog
|
|
|
- {
|
|
|
- try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG))) {
|
|
|
- Map<String, String> userData = engine.getLastCommittedSegmentInfos().getUserData();
|
|
|
- assertEquals("2", userData.get(Translog.TRANSLOG_GENERATION_KEY));
|
|
|
- assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY));
|
|
|
- expectThrows(IllegalStateException.class, () -> engine.recoverFromTranslog());
|
|
|
- assertEquals(2, engine.getTranslog().currentFileGeneration());
|
|
|
- assertEquals(0L, engine.getTranslog().uncommittedOperations());
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // open and recover tlog with empty tlog
|
|
|
- {
|
|
|
- for (int i = 0; i < 2; i++) {
|
|
|
- try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) {
|
|
|
- Map<String, String> userData = engine.getLastCommittedSegmentInfos().getUserData();
|
|
|
- assertEquals("2", userData.get(Translog.TRANSLOG_GENERATION_KEY));
|
|
|
- assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY));
|
|
|
- engine.recoverFromTranslog();
|
|
|
- userData = engine.getLastCommittedSegmentInfos().getUserData();
|
|
|
- assertEquals("no changes - nothing to commit", "2", userData.get(Translog.TRANSLOG_GENERATION_KEY));
|
|
|
- assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY));
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
private static class ThrowingIndexWriter extends IndexWriter {
|
|
|
private AtomicReference<Supplier<Exception>> failureToThrow = new AtomicReference<>();
|
|
|
|
|
@@ -3367,21 +3196,22 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
public void testEngineMaxTimestampIsInitialized() throws IOException {
|
|
|
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
final long timestamp1 = Math.abs(randomNonNegativeLong());
|
|
|
final Path storeDir = createTempDir();
|
|
|
final Path translogDir = createTempDir();
|
|
|
final long timestamp2 = randomNonNegativeLong();
|
|
|
final long maxTimestamp12 = Math.max(timestamp1, timestamp2);
|
|
|
- try (Store store = createStore(newFSDirectory(storeDir));
|
|
|
- Engine engine = new InternalEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) {
|
|
|
+ final Function<Store, EngineConfig> configSupplier =
|
|
|
+ store -> config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null, null, globalCheckpoint::get);
|
|
|
+ try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = createEngine(configSupplier.apply(store))) {
|
|
|
assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
|
|
|
final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(),
|
|
|
new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
|
|
|
engine.index(appendOnlyPrimary(doc, true, timestamp1));
|
|
|
assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
|
|
|
}
|
|
|
- try (Store store = createStore(newFSDirectory(storeDir));
|
|
|
- Engine engine = new InternalEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) {
|
|
|
+ try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = new InternalEngine(configSupplier.apply(store))) {
|
|
|
assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
|
|
|
engine.recoverFromTranslog();
|
|
|
assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
|
|
@@ -3389,13 +3219,16 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
|
|
|
engine.index(appendOnlyPrimary(doc, true, timestamp2));
|
|
|
assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
|
|
|
+ globalCheckpoint.set(1); // make sure flush cleans up commits for later.
|
|
|
engine.flush();
|
|
|
}
|
|
|
- try (Store store = createStore(newFSDirectory(storeDir));
|
|
|
- Engine engine = new InternalEngine(
|
|
|
- copy(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null),
|
|
|
- randomFrom(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG)))) {
|
|
|
- assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
|
|
|
+ try (Store store = createStore(newFSDirectory(storeDir))) {
|
|
|
+ if (randomBoolean() || true) {
|
|
|
+ EngineDiskUtils.createNewTranslog(store.directory(), translogDir, SequenceNumbers.NO_OPS_PERFORMED, shardId);
|
|
|
+ }
|
|
|
+ try (Engine engine = new InternalEngine(configSupplier.apply(store))) {
|
|
|
+ assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp());
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -3491,7 +3324,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
}
|
|
|
}
|
|
|
});
|
|
|
- InternalEngine internalEngine = new InternalEngine(config);
|
|
|
+ InternalEngine internalEngine = createEngine(config);
|
|
|
int docId = 0;
|
|
|
final ParsedDocument doc = testParsedDocument(Integer.toString(docId), null,
|
|
|
testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null);
|
|
@@ -3662,53 +3495,13 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
IOUtils.close(initialEngine);
|
|
|
}
|
|
|
|
|
|
- try (Engine recoveringEngine =
|
|
|
- new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) {
|
|
|
+ try (Engine recoveringEngine = new InternalEngine(initialEngine.config())) {
|
|
|
recoveringEngine.recoverFromTranslog();
|
|
|
recoveringEngine.fillSeqNoGaps(2);
|
|
|
assertThat(recoveringEngine.getLocalCheckpointTracker().getCheckpoint(), greaterThanOrEqualTo((long) (docs - 1)));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void testSequenceNumberAdvancesToMaxSeqNoOnEngineOpenOnReplica() throws IOException {
|
|
|
- final long v = 1;
|
|
|
- final VersionType t = VersionType.EXTERNAL;
|
|
|
- final long ts = IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP;
|
|
|
- final int docs = randomIntBetween(1, 32);
|
|
|
- InternalEngine initialEngine = null;
|
|
|
- try {
|
|
|
- initialEngine = engine;
|
|
|
- for (int i = 0; i < docs; i++) {
|
|
|
- final String id = Integer.toString(i);
|
|
|
- final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null);
|
|
|
- final Term uid = newUid(doc);
|
|
|
- // create a gap at sequence number 3 * i + 1
|
|
|
- initialEngine.index(new Engine.Index(uid, doc, 3 * i, 1, v, t, REPLICA, System.nanoTime(), ts, false));
|
|
|
- initialEngine.delete(new Engine.Delete("type", id, uid, 3 * i + 2, 1, v, t, REPLICA, System.nanoTime()));
|
|
|
- }
|
|
|
-
|
|
|
- // bake the commit with the local checkpoint stuck at 0 and gaps all along the way up to the max sequence number
|
|
|
- assertThat(initialEngine.getLocalCheckpointTracker().getCheckpoint(), equalTo((long) 0));
|
|
|
- assertThat(initialEngine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo((long) (3 * (docs - 1) + 2)));
|
|
|
- initialEngine.flush(true, true);
|
|
|
-
|
|
|
- for (int i = 0; i < docs; i++) {
|
|
|
- final String id = Integer.toString(i);
|
|
|
- final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null);
|
|
|
- final Term uid = newUid(doc);
|
|
|
- initialEngine.index(new Engine.Index(uid, doc, 3 * i + 1, 1, v, t, REPLICA, System.nanoTime(), ts, false));
|
|
|
- }
|
|
|
- } finally {
|
|
|
- IOUtils.close(initialEngine);
|
|
|
- }
|
|
|
-
|
|
|
- try (Engine recoveringEngine =
|
|
|
- new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) {
|
|
|
- recoveringEngine.recoverFromTranslog();
|
|
|
- recoveringEngine.fillSeqNoGaps(1);
|
|
|
- assertThat(recoveringEngine.getLocalCheckpointTracker().getCheckpoint(), greaterThanOrEqualTo((long) (3 * (docs - 1) + 2 - 1)));
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
/** java docs */
|
|
|
public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOException {
|
|
@@ -3803,7 +3596,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
final BiFunction<Long, Long, LocalCheckpointTracker> supplier = (ms, lcp) -> new LocalCheckpointTracker(
|
|
|
maxSeqNo,
|
|
|
localCheckpoint);
|
|
|
- noOpEngine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG), supplier) {
|
|
|
+ noOpEngine = new InternalEngine(engine.config(), supplier) {
|
|
|
@Override
|
|
|
protected long doGenerateSeqNoForOperation(Operation operation) {
|
|
|
throw new UnsupportedOperationException();
|
|
@@ -3950,7 +3743,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
completedSeqNos.add(seqNo);
|
|
|
}
|
|
|
};
|
|
|
- actualEngine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG), supplier);
|
|
|
+ actualEngine = new InternalEngine(engine.config(), supplier);
|
|
|
final int operations = randomIntBetween(0, 1024);
|
|
|
final Set<Long> expectedCompletedSeqNos = new HashSet<>();
|
|
|
for (int i = 0; i < operations; i++) {
|
|
@@ -4013,15 +3806,14 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
|
|
|
boolean flushed = false;
|
|
|
- AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
+ AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
Engine recoveringEngine = null;
|
|
|
try {
|
|
|
assertEquals(docs - 1, engine.getLocalCheckpointTracker().getMaxSeqNo());
|
|
|
assertEquals(docs - 1, engine.getLocalCheckpointTracker().getCheckpoint());
|
|
|
assertEquals(maxSeqIDOnReplica, replicaEngine.getLocalCheckpointTracker().getMaxSeqNo());
|
|
|
assertEquals(checkpointOnReplica, replicaEngine.getLocalCheckpointTracker().getCheckpoint());
|
|
|
- recoveringEngine = new InternalEngine(copy(
|
|
|
- replicaEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, globalCheckpoint::get));
|
|
|
+ recoveringEngine = new InternalEngine(copy(replicaEngine.config(), globalCheckpoint::get));
|
|
|
assertEquals(numDocsOnReplica, recoveringEngine.getTranslog().uncommittedOperations());
|
|
|
recoveringEngine.recoverFromTranslog();
|
|
|
assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo());
|
|
@@ -4054,8 +3846,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
// now do it again to make sure we preserve values etc.
|
|
|
try {
|
|
|
- recoveringEngine = new InternalEngine(
|
|
|
- copy(replicaEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, globalCheckpoint::get));
|
|
|
+ recoveringEngine = new InternalEngine(copy(replicaEngine.config(), globalCheckpoint::get));
|
|
|
if (flushed) {
|
|
|
assertEquals(0, recoveringEngine.getTranslog().uncommittedOperations());
|
|
|
}
|
|
@@ -4234,10 +4025,11 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
final Path translogPath = createTempDir();
|
|
|
store = createStore();
|
|
|
- final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
|
|
|
final EngineConfig engineConfig = config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null, null,
|
|
|
() -> globalCheckpoint.get());
|
|
|
+ EngineDiskUtils.createEmpty(store.directory(), translogPath, shardId);
|
|
|
try (Engine engine = new InternalEngine(engineConfig) {
|
|
|
@Override
|
|
|
protected void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException {
|
|
@@ -4249,6 +4041,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
super.commitIndexWriter(writer, translog, syncId);
|
|
|
}
|
|
|
}) {
|
|
|
+ engine.recoverFromTranslog();
|
|
|
int numDocs = scaledRandomIntBetween(10, 100);
|
|
|
final String translogUUID = engine.getTranslog().getTranslogUUID();
|
|
|
for (int docId = 0; docId < numDocs; docId++) {
|
|
@@ -4340,7 +4133,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
public void testAcquireIndexCommit() throws Exception {
|
|
|
IOUtils.close(engine, store);
|
|
|
store = createStore();
|
|
|
- final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
try (InternalEngine engine = createEngine(store, createTempDir(), globalCheckpoint::get)) {
|
|
|
int numDocs = between(1, 20);
|
|
|
for (int i = 0; i < numDocs; i++) {
|
|
@@ -4377,10 +4170,10 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
|
|
|
public void testOpenIndexAndTranslogKeepOnlySafeCommit() throws Exception {
|
|
|
IOUtils.close(engine);
|
|
|
- final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
- final EngineConfig config = copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, globalCheckpoint::get);
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
+ final EngineConfig config = copy(engine.config(), globalCheckpoint::get);
|
|
|
final IndexCommit safeCommit;
|
|
|
- try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG))) {
|
|
|
+ try (InternalEngine engine = createEngine(config)) {
|
|
|
final int numDocs = between(5, 50);
|
|
|
for (int i = 0; i < numDocs; i++) {
|
|
|
index(engine, i);
|
|
@@ -4394,44 +4187,16 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
globalCheckpoint.set(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)));
|
|
|
engine.getTranslog().sync();
|
|
|
}
|
|
|
- try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) {
|
|
|
+ try (InternalEngine engine = new InternalEngine(config)) {
|
|
|
final List<IndexCommit> existingCommits = DirectoryReader.listCommits(engine.store.directory());
|
|
|
- assertThat("OPEN_INDEX_AND_TRANSLOG should keep only safe commit", existingCommits, contains(safeCommit));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public void testOpenIndexCreateTranslogKeepOnlyLastCommit() throws Exception {
|
|
|
- IOUtils.close(engine);
|
|
|
- final EngineConfig config = copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG);
|
|
|
- final Map<String, String> lastCommit;
|
|
|
- try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) {
|
|
|
- engine.skipTranslogRecovery();
|
|
|
- final int numDocs = between(5, 50);
|
|
|
- for (int i = 0; i < numDocs; i++) {
|
|
|
- index(engine, i);
|
|
|
- if (randomBoolean()) {
|
|
|
- engine.flush();
|
|
|
- }
|
|
|
- }
|
|
|
- final List<IndexCommit> commits = DirectoryReader.listCommits(engine.store.directory());
|
|
|
- lastCommit = commits.get(commits.size() - 1).getUserData();
|
|
|
- }
|
|
|
- try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG))) {
|
|
|
- final List<IndexCommit> existingCommits = DirectoryReader.listCommits(engine.store.directory());
|
|
|
- assertThat("OPEN_INDEX_CREATE_TRANSLOG should keep only last commit", existingCommits, hasSize(1));
|
|
|
- final Map<String, String> userData = existingCommits.get(0).getUserData();
|
|
|
- assertThat(userData.get(SequenceNumbers.MAX_SEQ_NO), equalTo(lastCommit.get(SequenceNumbers.MAX_SEQ_NO)));
|
|
|
- assertThat(userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY), equalTo(lastCommit.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)));
|
|
|
- // Translog tags should be fresh.
|
|
|
- assertThat(userData.get(Translog.TRANSLOG_UUID_KEY), not(equalTo(lastCommit.get(Translog.TRANSLOG_UUID_KEY))));
|
|
|
- assertThat(userData.get(Translog.TRANSLOG_GENERATION_KEY), equalTo("2"));
|
|
|
+ assertThat("safe commit should be kept", existingCommits, contains(safeCommit));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
public void testCleanUpCommitsWhenGlobalCheckpointAdvanced() throws Exception {
|
|
|
IOUtils.close(engine, store);
|
|
|
store = createStore();
|
|
|
- final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
try (InternalEngine engine = createEngine(store, createTempDir(), globalCheckpoint::get)) {
|
|
|
final int numDocs = scaledRandomIntBetween(10, 100);
|
|
|
for (int docId = 0; docId < numDocs; docId++) {
|
|
@@ -4456,7 +4221,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
public void testCleanupCommitsWhenReleaseSnapshot() throws Exception {
|
|
|
IOUtils.close(engine, store);
|
|
|
store = createStore();
|
|
|
- final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
try (InternalEngine engine = createEngine(store, createTempDir(), globalCheckpoint::get)) {
|
|
|
final int numDocs = scaledRandomIntBetween(10, 100);
|
|
|
for (int docId = 0; docId < numDocs; docId++) {
|