Skip to content

Commit ebc4681

Browse files
authored
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 b85b12d commit ebc4681

23 files changed

+316
-531
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
@@ -10,6 +10,13 @@
1010
cluster.health:
1111
wait_for_no_initializing_shards: true
1212
wait_for_events: languid
13+
# Before 8.0, an empty shard has two empty translog files as we used the translog_generation commit tag as the minimum required
14+
# translog generation for recovery. Here we force-flush to have a consistent translog stats for both old and new indices.
15+
- do:
16+
indices.flush:
17+
index: test
18+
force: true
19+
wait_if_ongoing: true
1320
- do:
1421
indices.stats:
1522
metric: [ translog ]
@@ -37,10 +44,9 @@
3744
- do:
3845
indices.stats:
3946
metric: [ translog ]
40-
# after flushing we have one empty translog file while an empty index before flushing has two empty translog files.
41-
- lt: { indices.test.primaries.translog.size_in_bytes: $creation_size }
47+
- match: { indices.test.primaries.translog.size_in_bytes: $creation_size }
4248
- match: { indices.test.primaries.translog.operations: 0 }
43-
- lt: { indices.test.primaries.translog.uncommitted_size_in_bytes: $creation_size }
49+
- match: { indices.test.primaries.translog.uncommitted_size_in_bytes: $creation_size }
4450
- match: { indices.test.primaries.translog.uncommitted_operations: 0 }
4551

4652
---

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

+20-34
Original file line numberDiff line numberDiff line change
@@ -385,7 +385,7 @@ public int restoreLocalHistoryFromTranslog(TranslogRecoveryRunner translogRecove
385385
try (ReleasableLock ignored = readLock.acquire()) {
386386
ensureOpen();
387387
final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
388-
try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(localCheckpoint + 1)) {
388+
try (Translog.Snapshot snapshot = getTranslog().newSnapshot(localCheckpoint + 1, Long.MAX_VALUE)) {
389389
return translogRecoveryRunner.run(this, snapshot);
390390
}
391391
}
@@ -458,23 +458,24 @@ public void skipTranslogRecovery() {
458458
}
459459

460460
private void recoverFromTranslogInternal(TranslogRecoveryRunner translogRecoveryRunner, long recoverUpToSeqNo) throws IOException {
461-
Translog.TranslogGeneration translogGeneration = translog.getGeneration();
462461
final int opsRecovered;
463-
final long translogFileGen = Long.parseLong(lastCommittedSegmentInfos.getUserData().get(Translog.TRANSLOG_GENERATION_KEY));
464-
try (Translog.Snapshot snapshot = translog.newSnapshotFromGen(
465-
new Translog.TranslogGeneration(translog.getTranslogUUID(), translogFileGen), recoverUpToSeqNo)) {
466-
opsRecovered = translogRecoveryRunner.run(this, snapshot);
467-
} catch (Exception e) {
468-
throw new EngineException(shardId, "failed to recover from translog", e);
462+
final long localCheckpoint = getProcessedLocalCheckpoint();
463+
if (localCheckpoint < recoverUpToSeqNo) {
464+
try (Translog.Snapshot snapshot = translog.newSnapshot(localCheckpoint + 1, recoverUpToSeqNo)) {
465+
opsRecovered = translogRecoveryRunner.run(this, snapshot);
466+
} catch (Exception e) {
467+
throw new EngineException(shardId, "failed to recover from translog", e);
468+
}
469+
} else {
470+
opsRecovered = 0;
469471
}
470472
// flush if we recovered something or if we have references to older translogs
471473
// note: if opsRecovered == 0 and we have older translogs it means they are corrupted or 0 length.
472474
assert pendingTranslogRecovery.get() : "translogRecovery is not pending but should be";
473475
pendingTranslogRecovery.set(false); // we are good - now we can commit
474476
if (opsRecovered > 0) {
475-
logger.trace("flushing post recovery from translog. ops recovered [{}]. committed translog id [{}]. current id [{}]",
476-
opsRecovered, translogGeneration == null ? null :
477-
translogGeneration.translogFileGeneration, translog.currentFileGeneration());
477+
logger.trace("flushing post recovery from translog: ops recovered [{}], current translog generation [{}]",
478+
opsRecovered, translog.currentFileGeneration());
478479
commitIndexWriter(indexWriter, translog);
479480
refreshLastCommittedSegmentInfos();
480481
refresh("translog_recovery");
@@ -486,7 +487,8 @@ private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy
486487
LongSupplier globalCheckpointSupplier, LongConsumer persistedSequenceNumberConsumer) throws IOException {
487488

488489
final TranslogConfig translogConfig = engineConfig.getTranslogConfig();
489-
final String translogUUID = loadTranslogUUIDFromLastCommit();
490+
final Map<String, String> userData = store.readLastCommittedSegmentsInfo().getUserData();
491+
final String translogUUID = Objects.requireNonNull(userData.get(Translog.TRANSLOG_UUID_KEY));
490492
// We expect that this shard already exists, so it must already have an existing translog else something is badly wrong!
491493
return new Translog(translogConfig, translogUUID, translogDeletionPolicy, globalCheckpointSupplier,
492494
engineConfig.getPrimaryTermSupplier(), persistedSequenceNumberConsumer);
@@ -551,18 +553,6 @@ public long getWritingBytes() {
551553
return indexWriter.getFlushingBytes() + versionMap.getRefreshingBytes();
552554
}
553555

554-
/**
555-
* Reads the current stored translog ID from the last commit data.
556-
*/
557-
@Nullable
558-
private String loadTranslogUUIDFromLastCommit() throws IOException {
559-
final Map<String, String> commitUserData = store.readLastCommittedSegmentsInfo().getUserData();
560-
if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY) == false) {
561-
throw new IllegalStateException("commit doesn't contain translog generation id");
562-
}
563-
return commitUserData.get(Translog.TRANSLOG_UUID_KEY);
564-
}
565-
566556
/**
567557
* Reads the current stored history ID from the IW commit data.
568558
*/
@@ -1588,8 +1578,10 @@ public boolean shouldPeriodicallyFlush() {
15881578
if (shouldPeriodicallyFlushAfterBigMerge.get()) {
15891579
return true;
15901580
}
1581+
final long localCheckpointOfLastCommit =
1582+
Long.parseLong(lastCommittedSegmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY));
15911583
final long translogGenerationOfLastCommit =
1592-
Long.parseLong(lastCommittedSegmentInfos.userData.get(Translog.TRANSLOG_GENERATION_KEY));
1584+
translog.getMinGenerationForSeqNo(localCheckpointOfLastCommit + 1).translogFileGeneration;
15931585
final long flushThreshold = config().getIndexSettings().getFlushThresholdSize().getBytes();
15941586
if (translog.sizeInBytesByMinGen(translogGenerationOfLastCommit) < flushThreshold) {
15951587
return false;
@@ -2281,11 +2273,6 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl
22812273
ensureCanFlush();
22822274
try {
22832275
final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
2284-
final Translog.TranslogGeneration translogGeneration = translog.getMinGenerationForSeqNo(localCheckpoint + 1);
2285-
final String translogFileGeneration = Long.toString(translogGeneration.translogFileGeneration);
2286-
final String translogUUID = translogGeneration.translogUUID;
2287-
final String localCheckpointValue = Long.toString(localCheckpoint);
2288-
22892276
writer.setLiveCommitData(() -> {
22902277
/*
22912278
* The user data captured above (e.g. local checkpoint) contains data that must be evaluated *before* Lucene flushes
@@ -2296,10 +2283,9 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl
22962283
* {@link IndexWriter#commit()} call flushes all documents, we defer computation of the maximum sequence number to the time
22972284
* of invocation of the commit data iterator (which occurs after all documents have been flushed to Lucene).
22982285
*/
2299-
final Map<String, String> commitData = new HashMap<>(7);
2300-
commitData.put(Translog.TRANSLOG_GENERATION_KEY, translogFileGeneration);
2301-
commitData.put(Translog.TRANSLOG_UUID_KEY, translogUUID);
2302-
commitData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, localCheckpointValue);
2286+
final Map<String, String> commitData = new HashMap<>(6);
2287+
commitData.put(Translog.TRANSLOG_UUID_KEY, translog.getTranslogUUID());
2288+
commitData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(localCheckpoint));
23032289
commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(localCheckpointTracker.getMaxSeqNo()));
23042290
commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get()));
23052291
commitData.put(HISTORY_UUID_KEY, historyUUID);

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();
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();
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
@@ -220,11 +220,10 @@ private static TranslogStats translogStats(final EngineConfig config, final Segm
220220
if (translogUuid == null) {
221221
throw new IllegalStateException("commit doesn't contain translog unique id");
222222
}
223-
final long translogGenOfLastCommit = Long.parseLong(infos.getUserData().get(Translog.TRANSLOG_GENERATION_KEY));
224223
final TranslogConfig translogConfig = config.getTranslogConfig();
225224
final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy();
226-
translogDeletionPolicy.setTranslogGenerationOfLastCommit(translogGenOfLastCommit);
227-
225+
final long localCheckpoint = Long.parseLong(infos.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY));
226+
translogDeletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpoint);
228227
try (Translog translog = new Translog(translogConfig, translogUuid, translogDeletionPolicy, config.getGlobalCheckpointSupplier(),
229228
config.getPrimaryTermSupplier(), seqNo -> {})
230229
) {

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

+1-4
Original file line numberDiff line numberDiff line change
@@ -1437,10 +1437,7 @@ public void associateIndexWithNewTranslog(final String translogUUID) throws IOEx
14371437
if (translogUUID.equals(getUserData(writer).get(Translog.TRANSLOG_UUID_KEY))) {
14381438
throw new IllegalArgumentException("a new translog uuid can't be equal to existing one. got [" + translogUUID + "]");
14391439
}
1440-
final Map<String, String> map = new HashMap<>();
1441-
map.put(Translog.TRANSLOG_GENERATION_KEY, "1");
1442-
map.put(Translog.TRANSLOG_UUID_KEY, translogUUID);
1443-
updateCommitData(writer, map);
1440+
updateCommitData(writer, Map.of(Translog.TRANSLOG_UUID_KEY, translogUUID));
14441441
} finally {
14451442
metadataLock.writeLock().unlock();
14461443
}

0 commit comments

Comments
 (0)