|
@@ -148,6 +148,7 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper;
|
|
|
import org.elasticsearch.index.mapper.SourceFieldMapper;
|
|
|
import org.elasticsearch.index.seqno.LocalCheckpointTracker;
|
|
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
|
|
+import org.elasticsearch.index.seqno.SeqNoStats;
|
|
|
import org.elasticsearch.index.seqno.SequenceNumbers;
|
|
|
import org.elasticsearch.index.shard.IndexSearcherWrapper;
|
|
|
import org.elasticsearch.index.shard.ShardId;
|
|
@@ -4047,56 +4048,52 @@ public class InternalEngineTests extends EngineTestCase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void testRestoreLocalCheckpointFromTranslog() throws IOException {
|
|
|
- engine.close();
|
|
|
- InternalEngine actualEngine = null;
|
|
|
- try {
|
|
|
- final Set<Long> completedSeqNos = new HashSet<>();
|
|
|
- final BiFunction<Long, Long, LocalCheckpointTracker> supplier = (maxSeqNo, localCheckpoint) -> new LocalCheckpointTracker(
|
|
|
- maxSeqNo,
|
|
|
- localCheckpoint) {
|
|
|
- @Override
|
|
|
- public void markSeqNoAsCompleted(long seqNo) {
|
|
|
- super.markSeqNoAsCompleted(seqNo);
|
|
|
- completedSeqNos.add(seqNo);
|
|
|
- }
|
|
|
- };
|
|
|
- trimUnsafeCommits(engine.config());
|
|
|
- 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++) {
|
|
|
- if (rarely() && i < operations - 1) {
|
|
|
+ public void testRestoreLocalHistoryFromTranslog() throws IOException {
|
|
|
+ final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
|
|
+ try (Store store = createStore()) {
|
|
|
+ final ArrayList<Long> seqNos = new ArrayList<>();
|
|
|
+ final int numOps = randomIntBetween(0, 1024);
|
|
|
+ for (int i = 0; i < numOps; i++) {
|
|
|
+ if (rarely()) {
|
|
|
continue;
|
|
|
}
|
|
|
- expectedCompletedSeqNos.add((long) i);
|
|
|
+ seqNos.add((long) i);
|
|
|
}
|
|
|
-
|
|
|
- final ArrayList<Long> seqNos = new ArrayList<>(expectedCompletedSeqNos);
|
|
|
Randomness.shuffle(seqNos);
|
|
|
- for (final long seqNo : seqNos) {
|
|
|
- final String id = Long.toString(seqNo);
|
|
|
- final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null);
|
|
|
- final Term uid = newUid(doc);
|
|
|
- final long time = System.nanoTime();
|
|
|
- actualEngine.index(new Engine.Index(uid, doc, seqNo, 1, 1, null, REPLICA, time, time, false));
|
|
|
- if (rarely()) {
|
|
|
- actualEngine.rollTranslogGeneration();
|
|
|
+ final EngineConfig engineConfig;
|
|
|
+ final SeqNoStats prevSeqNoStats;
|
|
|
+ final List<DocIdSeqNoAndTerm> prevDocs;
|
|
|
+ final int totalTranslogOps;
|
|
|
+ try (InternalEngine engine = createEngine(store, createTempDir(), globalCheckpoint::get)) {
|
|
|
+ engineConfig = engine.config();
|
|
|
+ for (final long seqNo : seqNos) {
|
|
|
+ final String id = Long.toString(seqNo);
|
|
|
+ final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null);
|
|
|
+ engine.index(replicaIndexForDoc(doc, 1, seqNo, false));
|
|
|
+ if (rarely()) {
|
|
|
+ engine.rollTranslogGeneration();
|
|
|
+ }
|
|
|
+ if (rarely()) {
|
|
|
+ engine.flush();
|
|
|
+ }
|
|
|
}
|
|
|
- }
|
|
|
- final long currentLocalCheckpoint = actualEngine.getLocalCheckpoint();
|
|
|
- final long resetLocalCheckpoint =
|
|
|
- randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), Math.toIntExact(currentLocalCheckpoint));
|
|
|
- actualEngine.getLocalCheckpointTracker().resetCheckpoint(resetLocalCheckpoint);
|
|
|
- completedSeqNos.clear();
|
|
|
- actualEngine.restoreLocalCheckpointFromTranslog();
|
|
|
- final Set<Long> intersection = new HashSet<>(expectedCompletedSeqNos);
|
|
|
- intersection.retainAll(LongStream.range(resetLocalCheckpoint + 1, operations).boxed().collect(Collectors.toSet()));
|
|
|
- assertThat(completedSeqNos, equalTo(intersection));
|
|
|
- assertThat(actualEngine.getLocalCheckpoint(), equalTo(currentLocalCheckpoint));
|
|
|
- assertThat(generateNewSeqNo(actualEngine), equalTo((long) operations));
|
|
|
- } finally {
|
|
|
- IOUtils.close(actualEngine);
|
|
|
+ globalCheckpoint.set(randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, engine.getLocalCheckpoint()));
|
|
|
+ engine.syncTranslog();
|
|
|
+ prevSeqNoStats = engine.getSeqNoStats(globalCheckpoint.get());
|
|
|
+ prevDocs = getDocIds(engine, true);
|
|
|
+ totalTranslogOps = engine.getTranslog().totalOperations();
|
|
|
+ }
|
|
|
+ trimUnsafeCommits(engineConfig);
|
|
|
+ try (InternalEngine engine = new InternalEngine(engineConfig)) {
|
|
|
+ engine.recoverFromTranslog(translogHandler, globalCheckpoint.get());
|
|
|
+ engine.restoreLocalHistoryFromTranslog(translogHandler);
|
|
|
+ assertThat(getDocIds(engine, true), equalTo(prevDocs));
|
|
|
+ SeqNoStats seqNoStats = engine.getSeqNoStats(globalCheckpoint.get());
|
|
|
+ assertThat(seqNoStats.getLocalCheckpoint(), equalTo(prevSeqNoStats.getLocalCheckpoint()));
|
|
|
+ assertThat(seqNoStats.getMaxSeqNo(), equalTo(prevSeqNoStats.getMaxSeqNo()));
|
|
|
+ assertThat(engine.getTranslog().totalOperations(), equalTo(totalTranslogOps));
|
|
|
+ }
|
|
|
+ assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, createMapperService("test"));
|
|
|
}
|
|
|
}
|
|
|
|