Skip to content

Commit 0d99fad

Browse files
committed
Put back lastSyncedGlobalCheckpoint in deletion policy
The PR #27837 unintentionally changed to an in memory global checkpoint.
1 parent fa1159a commit 0d99fad

File tree

2 files changed

+9
-3
lines changed

2 files changed

+9
-3
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -184,7 +184,7 @@ public InternalEngine(EngineConfig engineConfig) {
184184
assert translog.getGeneration() != null;
185185
this.translog = translog;
186186
this.snapshotDeletionPolicy = new SnapshotDeletionPolicy(
187-
new CombinedDeletionPolicy(openMode, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier())
187+
new CombinedDeletionPolicy(openMode, translogDeletionPolicy, translog::getLastSyncedGlobalCheckpoint)
188188
);
189189
writer = createWriter(openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG);
190190
updateMaxUnsafeAutoIdTimestampFromWriter(writer);

core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -4242,8 +4242,14 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s
42424242
final List<IndexCommit> commits = DirectoryReader.listCommits(store.directory());
42434243
// Keep only one safe commit as the oldest commit.
42444244
final IndexCommit safeCommit = commits.get(0);
4245-
assertThat(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)),
4246-
lessThanOrEqualTo(lastSyncedGlobalCheckpoint));
4245+
if (lastSyncedGlobalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) {
4246+
// If the global checkpoint is still unassigned, we keep an empty(eg. initial) commit as a safe commit.
4247+
assertThat(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)),
4248+
equalTo(SequenceNumbers.NO_OPS_PERFORMED));
4249+
} else {
4250+
assertThat(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)),
4251+
lessThanOrEqualTo(lastSyncedGlobalCheckpoint));
4252+
}
42474253
for (int i = 1; i < commits.size(); i++) {
42484254
assertThat(Long.parseLong(commits.get(i).getUserData().get(SequenceNumbers.MAX_SEQ_NO)),
42494255
greaterThan(lastSyncedGlobalCheckpoint));

0 commit comments

Comments
 (0)