Skip to content

Commit db6b9c2

Browse files
committed
Use local checkpoint to calculate min translog gen for recovery (#51905)
Today we use the translog_generation of the safe commit as the minimum required translog generation for recovery. This approach has a limitation, where we won't be able to clean up translog unless we flush. Reopening an already recovered engine will create a new empty translog, and we leave it there until we force flush. This commit removes the translog_generation commit tag and uses the local checkpoint of the safe commit to calculate the minimum required translog generation for recovery instead. Closes #49970
1 parent b4179a8 commit db6b9c2

24 files changed

+341
-550
lines changed

rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml

+9-3
Original file line numberDiff line numberDiff line change
@@ -88,6 +88,13 @@
8888
cluster.health:
8989
wait_for_no_initializing_shards: true
9090
wait_for_events: languid
91+
# Before 8.0, an empty shard has two empty translog files as we used the translog_generation commit tag as the minimum required
92+
# translog generation for recovery. Here we force-flush to have a consistent translog stats for both old and new indices.
93+
- do:
94+
indices.flush:
95+
index: test
96+
force: true
97+
wait_if_ongoing: true
9198
- do:
9299
indices.stats:
93100
metric: [ translog ]
@@ -115,10 +122,9 @@
115122
- do:
116123
indices.stats:
117124
metric: [ translog ]
118-
# after flushing we have one empty translog file while an empty index before flushing has two empty translog files.
119-
- lt: { indices.test.primaries.translog.size_in_bytes: $creation_size }
125+
- match: { indices.test.primaries.translog.size_in_bytes: $creation_size }
120126
- match: { indices.test.primaries.translog.operations: 0 }
121-
- lt: { indices.test.primaries.translog.uncommitted_size_in_bytes: $creation_size }
127+
- match: { indices.test.primaries.translog.uncommitted_size_in_bytes: $creation_size }
122128
- match: { indices.test.primaries.translog.uncommitted_operations: 0 }
123129

124130
---

server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java

+3-9
Original file line numberDiff line numberDiff line change
@@ -121,16 +121,10 @@ private void updateRetentionPolicy() throws IOException {
121121
assert Thread.holdsLock(this);
122122
logger.debug("Safe commit [{}], last commit [{}]", commitDescription(safeCommit), commitDescription(lastCommit));
123123
assert safeCommit.isDeleted() == false : "The safe commit must not be deleted";
124-
final long minRequiredGen = Long.parseLong(safeCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY));
125124
assert lastCommit.isDeleted() == false : "The last commit must not be deleted";
126-
final long lastGen = Long.parseLong(lastCommit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY));
127-
128-
assert minRequiredGen <= lastGen : "minRequiredGen must not be greater than lastGen";
129-
translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastGen);
130-
translogDeletionPolicy.setMinTranslogGenerationForRecovery(minRequiredGen);
131-
132-
softDeletesPolicy.setLocalCheckpointOfSafeCommit(
133-
Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)));
125+
final long localCheckpointOfSafeCommit = Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY));
126+
softDeletesPolicy.setLocalCheckpointOfSafeCommit(localCheckpointOfSafeCommit);
127+
translogDeletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpointOfSafeCommit);
134128
}
135129

136130
protected int getDocCountOfCommit(IndexCommit indexCommit) throws IOException {

server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java

+25-38
Original file line numberDiff line numberDiff line change
@@ -400,7 +400,7 @@ public int restoreLocalHistoryFromTranslog(TranslogRecoveryRunner translogRecove
400400
try (ReleasableLock ignored = readLock.acquire()) {
401401
ensureOpen();
402402
final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
403-
try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(localCheckpoint + 1)) {
403+
try (Translog.Snapshot snapshot = getTranslog().newSnapshot(localCheckpoint + 1, Long.MAX_VALUE)) {
404404
return translogRecoveryRunner.run(this, snapshot);
405405
}
406406
}
@@ -473,23 +473,24 @@ public void skipTranslogRecovery() {
473473
}
474474

475475
private void recoverFromTranslogInternal(TranslogRecoveryRunner translogRecoveryRunner, long recoverUpToSeqNo) throws IOException {
476-
Translog.TranslogGeneration translogGeneration = translog.getGeneration();
477476
final int opsRecovered;
478-
final long translogFileGen = Long.parseLong(lastCommittedSegmentInfos.getUserData().get(Translog.TRANSLOG_GENERATION_KEY));
479-
try (Translog.Snapshot snapshot = translog.newSnapshotFromGen(
480-
new Translog.TranslogGeneration(translog.getTranslogUUID(), translogFileGen), recoverUpToSeqNo)) {
481-
opsRecovered = translogRecoveryRunner.run(this, snapshot);
482-
} catch (Exception e) {
483-
throw new EngineException(shardId, "failed to recover from translog", e);
477+
final long localCheckpoint = getProcessedLocalCheckpoint();
478+
if (localCheckpoint < recoverUpToSeqNo) {
479+
try (Translog.Snapshot snapshot = translog.newSnapshot(localCheckpoint + 1, recoverUpToSeqNo)) {
480+
opsRecovered = translogRecoveryRunner.run(this, snapshot);
481+
} catch (Exception e) {
482+
throw new EngineException(shardId, "failed to recover from translog", e);
483+
}
484+
} else {
485+
opsRecovered = 0;
484486
}
485487
// flush if we recovered something or if we have references to older translogs
486488
// note: if opsRecovered == 0 and we have older translogs it means they are corrupted or 0 length.
487489
assert pendingTranslogRecovery.get() : "translogRecovery is not pending but should be";
488490
pendingTranslogRecovery.set(false); // we are good - now we can commit
489491
if (opsRecovered > 0) {
490-
logger.trace("flushing post recovery from translog. ops recovered [{}]. committed translog id [{}]. current id [{}]",
491-
opsRecovered, translogGeneration == null ? null :
492-
translogGeneration.translogFileGeneration, translog.currentFileGeneration());
492+
logger.trace("flushing post recovery from translog: ops recovered [{}], current translog generation [{}]",
493+
opsRecovered, translog.currentFileGeneration());
493494
commitIndexWriter(indexWriter, translog, null);
494495
refreshLastCommittedSegmentInfos();
495496
refresh("translog_recovery");
@@ -501,7 +502,8 @@ private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy
501502
LongSupplier globalCheckpointSupplier, LongConsumer persistedSequenceNumberConsumer) throws IOException {
502503

503504
final TranslogConfig translogConfig = engineConfig.getTranslogConfig();
504-
final String translogUUID = loadTranslogUUIDFromLastCommit();
505+
final Map<String, String> userData = store.readLastCommittedSegmentsInfo().getUserData();
506+
final String translogUUID = Objects.requireNonNull(userData.get(Translog.TRANSLOG_UUID_KEY));
505507
// We expect that this shard already exists, so it must already have an existing translog else something is badly wrong!
506508
return new Translog(translogConfig, translogUUID, translogDeletionPolicy, globalCheckpointSupplier,
507509
engineConfig.getPrimaryTermSupplier(), persistedSequenceNumberConsumer);
@@ -549,7 +551,7 @@ public Translog.Snapshot readHistoryOperations(String reason, HistorySource hist
549551
ensureSoftDeletesEnabled();
550552
return newChangesSnapshot(reason, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false);
551553
} else {
552-
return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo);
554+
return getTranslog().newSnapshot(startingSeqNo, Long.MAX_VALUE);
553555
}
554556
}
555557

@@ -598,18 +600,6 @@ public long getWritingBytes() {
598600
return indexWriter.getFlushingBytes() + versionMap.getRefreshingBytes();
599601
}
600602

601-
/**
602-
* Reads the current stored translog ID from the last commit data.
603-
*/
604-
@Nullable
605-
private String loadTranslogUUIDFromLastCommit() throws IOException {
606-
final Map<String, String> commitUserData = store.readLastCommittedSegmentsInfo().getUserData();
607-
if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY) == false) {
608-
throw new IllegalStateException("commit doesn't contain translog generation id");
609-
}
610-
return commitUserData.get(Translog.TRANSLOG_UUID_KEY);
611-
}
612-
613603
/**
614604
* Reads the current stored history ID from the IW commit data.
615605
*/
@@ -1688,8 +1678,9 @@ final boolean tryRenewSyncCommit() {
16881678
ensureOpen();
16891679
ensureCanFlush();
16901680
String syncId = lastCommittedSegmentInfos.getUserData().get(SYNC_COMMIT_ID);
1691-
long translogGenOfLastCommit = Long.parseLong(lastCommittedSegmentInfos.userData.get(Translog.TRANSLOG_GENERATION_KEY));
1692-
if (syncId != null && indexWriter.hasUncommittedChanges() && translog.totalOperationsByMinGen(translogGenOfLastCommit) == 0) {
1681+
long localCheckpointOfLastCommit = Long.parseLong(lastCommittedSegmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY));
1682+
if (syncId != null && indexWriter.hasUncommittedChanges() &&
1683+
translog.estimateTotalOperationsFromMinSeq(localCheckpointOfLastCommit + 1) == 0) {
16931684
logger.trace("start renewing sync commit [{}]", syncId);
16941685
commitIndexWriter(indexWriter, translog, syncId);
16951686
logger.debug("successfully sync committed. sync id [{}].", syncId);
@@ -1714,8 +1705,10 @@ public boolean shouldPeriodicallyFlush() {
17141705
if (shouldPeriodicallyFlushAfterBigMerge.get()) {
17151706
return true;
17161707
}
1708+
final long localCheckpointOfLastCommit =
1709+
Long.parseLong(lastCommittedSegmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY));
17171710
final long translogGenerationOfLastCommit =
1718-
Long.parseLong(lastCommittedSegmentInfos.userData.get(Translog.TRANSLOG_GENERATION_KEY));
1711+
translog.getMinGenerationForSeqNo(localCheckpointOfLastCommit + 1).translogFileGeneration;
17191712
final long flushThreshold = config().getIndexSettings().getFlushThresholdSize().getBytes();
17201713
if (translog.sizeInBytesByMinGen(translogGenerationOfLastCommit) < flushThreshold) {
17211714
return false;
@@ -2423,11 +2416,6 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl
24232416
ensureCanFlush();
24242417
try {
24252418
final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
2426-
final Translog.TranslogGeneration translogGeneration = translog.getMinGenerationForSeqNo(localCheckpoint + 1);
2427-
final String translogFileGeneration = Long.toString(translogGeneration.translogFileGeneration);
2428-
final String translogUUID = translogGeneration.translogUUID;
2429-
final String localCheckpointValue = Long.toString(localCheckpoint);
2430-
24312419
writer.setLiveCommitData(() -> {
24322420
/*
24332421
* The user data captured above (e.g. local checkpoint) contains data that must be evaluated *before* Lucene flushes
@@ -2438,10 +2426,9 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl
24382426
* {@link IndexWriter#commit()} call flushes all documents, we defer computation of the maximum sequence number to the time
24392427
* of invocation of the commit data iterator (which occurs after all documents have been flushed to Lucene).
24402428
*/
2441-
final Map<String, String> commitData = new HashMap<>(8);
2442-
commitData.put(Translog.TRANSLOG_GENERATION_KEY, translogFileGeneration);
2443-
commitData.put(Translog.TRANSLOG_UUID_KEY, translogUUID);
2444-
commitData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, localCheckpointValue);
2429+
final Map<String, String> commitData = new HashMap<>(7);
2430+
commitData.put(Translog.TRANSLOG_UUID_KEY, translog.getTranslogUUID());
2431+
commitData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint));
24452432
if (syncId != null) {
24462433
commitData.put(Engine.SYNC_COMMIT_ID, syncId);
24472434
}
@@ -2657,7 +2644,7 @@ public boolean hasCompleteOperationHistory(String reason, HistorySource historyS
26572644
return true;
26582645
}
26592646
final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1);
2660-
try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(startingSeqNo)) {
2647+
try (Translog.Snapshot snapshot = getTranslog().newSnapshot(startingSeqNo, Long.MAX_VALUE)) {
26612648
Translog.Operation operation;
26622649
while ((operation = snapshot.next()) != null) {
26632650
if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) {

server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java

+12-19
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@
2828
import org.apache.lucene.store.Directory;
2929
import org.elasticsearch.common.lucene.Lucene;
3030
import org.elasticsearch.common.util.concurrent.ReleasableLock;
31+
import org.elasticsearch.index.seqno.SequenceNumbers;
3132
import org.elasticsearch.index.store.Store;
3233
import org.elasticsearch.index.translog.Translog;
3334
import org.elasticsearch.index.translog.TranslogConfig;
@@ -137,31 +138,23 @@ public void trimUnreferencedTranslogFiles() {
137138
try (ReleasableLock lock = readLock.acquire()) {
138139
ensureOpen();
139140
final List<IndexCommit> commits = DirectoryReader.listCommits(store.directory());
140-
if (commits.size() == 1) {
141+
if (commits.size() == 1 && translogStats.getTranslogSizeInBytes() > translogStats.getUncommittedSizeInBytes()) {
141142
final Map<String, String> commitUserData = getLastCommittedSegmentInfos().getUserData();
142143
final String translogUuid = commitUserData.get(Translog.TRANSLOG_UUID_KEY);
143144
if (translogUuid == null) {
144145
throw new IllegalStateException("commit doesn't contain translog unique id");
145146
}
146-
if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY) == false) {
147-
throw new IllegalStateException("commit doesn't contain translog generation id");
148-
}
149-
final long lastCommitGeneration = Long.parseLong(commitUserData.get(Translog.TRANSLOG_GENERATION_KEY));
150147
final TranslogConfig translogConfig = engineConfig.getTranslogConfig();
151-
final long minTranslogGeneration = Translog.readMinTranslogGeneration(translogConfig.getTranslogPath(), translogUuid);
152-
153-
if (minTranslogGeneration < lastCommitGeneration) {
154-
// a translog deletion policy that retains nothing but the last translog generation from safe commit
155-
final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(-1, -1, 0);
156-
translogDeletionPolicy.setTranslogGenerationOfLastCommit(lastCommitGeneration);
157-
translogDeletionPolicy.setMinTranslogGenerationForRecovery(lastCommitGeneration);
158-
159-
try (Translog translog = new Translog(translogConfig, translogUuid, translogDeletionPolicy,
160-
engineConfig.getGlobalCheckpointSupplier(), engineConfig.getPrimaryTermSupplier(), seqNo -> {})) {
161-
translog.trimUnreferencedReaders();
162-
// refresh the translog stats
163-
this.translogStats = translog.stats();
164-
}
148+
final long localCheckpoint = Long.parseLong(commitUserData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY));
149+
final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(-1, -1, 0);
150+
translogDeletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpoint);
151+
try (Translog translog = new Translog(translogConfig, translogUuid, translogDeletionPolicy,
152+
engineConfig.getGlobalCheckpointSupplier(), engineConfig.getPrimaryTermSupplier(), seqNo -> {})) {
153+
translog.trimUnreferencedReaders();
154+
// refresh the translog stats
155+
this.translogStats = translog.stats();
156+
assert translog.currentFileGeneration() == translog.getMinFileGeneration() : "translog was not trimmed "
157+
+ " current gen " + translog.currentFileGeneration() + " != min gen " + translog.getMinFileGeneration();
165158
}
166159
}
167160
} catch (final Exception e) {

server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java

+2-3
Original file line numberDiff line numberDiff line change
@@ -223,15 +223,14 @@ private static TranslogStats translogStats(final EngineConfig config, final Segm
223223
if (translogUuid == null) {
224224
throw new IllegalStateException("commit doesn't contain translog unique id");
225225
}
226-
final long translogGenOfLastCommit = Long.parseLong(infos.getUserData().get(Translog.TRANSLOG_GENERATION_KEY));
227226
final TranslogConfig translogConfig = config.getTranslogConfig();
228227
final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(
229228
config.getIndexSettings().getTranslogRetentionSize().getBytes(),
230229
config.getIndexSettings().getTranslogRetentionAge().getMillis(),
231230
config.getIndexSettings().getTranslogRetentionTotalFiles()
232231
);
233-
translogDeletionPolicy.setTranslogGenerationOfLastCommit(translogGenOfLastCommit);
234-
232+
final long localCheckpoint = Long.parseLong(infos.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY));
233+
translogDeletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpoint);
235234
try (Translog translog = new Translog(translogConfig, translogUuid, translogDeletionPolicy, config.getGlobalCheckpointSupplier(),
236235
config.getPrimaryTermSupplier(), seqNo -> {})
237236
) {

server/src/main/java/org/elasticsearch/index/store/Store.java

+3-5
Original file line numberDiff line numberDiff line change
@@ -1453,10 +1453,7 @@ public void associateIndexWithNewTranslog(final String translogUUID) throws IOEx
14531453
if (translogUUID.equals(getUserData(writer).get(Translog.TRANSLOG_UUID_KEY))) {
14541454
throw new IllegalArgumentException("a new translog uuid can't be equal to existing one. got [" + translogUUID + "]");
14551455
}
1456-
final Map<String, String> map = new HashMap<>();
1457-
map.put(Translog.TRANSLOG_GENERATION_KEY, "1");
1458-
map.put(Translog.TRANSLOG_UUID_KEY, translogUUID);
1459-
updateCommitData(writer, map);
1456+
updateCommitData(writer, Collections.singletonMap(Translog.TRANSLOG_UUID_KEY, translogUUID));
14601457
} finally {
14611458
metadataLock.writeLock().unlock();
14621459
}
@@ -1517,7 +1514,8 @@ public void trimUnsafeCommits(final long lastSyncedGlobalCheckpoint, final long
15171514
if (indexVersionCreated.before(org.elasticsearch.Version.V_6_2_0)) {
15181515
final List<IndexCommit> recoverableCommits = new ArrayList<>();
15191516
for (IndexCommit commit : existingCommits) {
1520-
if (minRetainedTranslogGen <= Long.parseLong(commit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY))) {
1517+
final String translogGeneration = commit.getUserData().get("translog_generation");
1518+
if (translogGeneration == null || minRetainedTranslogGen <= Long.parseLong(translogGeneration)) {
15211519
recoverableCommits.add(commit);
15221520
}
15231521
}

0 commit comments

Comments
 (0)