@@ -400,7 +400,7 @@ public int restoreLocalHistoryFromTranslog(TranslogRecoveryRunner translogRecove
400
400
try (ReleasableLock ignored = readLock .acquire ()) {
401
401
ensureOpen ();
402
402
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 )) {
404
404
return translogRecoveryRunner .run (this , snapshot );
405
405
}
406
406
}
@@ -473,23 +473,24 @@ public void skipTranslogRecovery() {
473
473
}
474
474
475
475
private void recoverFromTranslogInternal (TranslogRecoveryRunner translogRecoveryRunner , long recoverUpToSeqNo ) throws IOException {
476
- Translog .TranslogGeneration translogGeneration = translog .getGeneration ();
477
476
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 ;
484
486
}
485
487
// flush if we recovered something or if we have references to older translogs
486
488
// note: if opsRecovered == 0 and we have older translogs it means they are corrupted or 0 length.
487
489
assert pendingTranslogRecovery .get () : "translogRecovery is not pending but should be" ;
488
490
pendingTranslogRecovery .set (false ); // we are good - now we can commit
489
491
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 ());
493
494
commitIndexWriter (indexWriter , translog , null );
494
495
refreshLastCommittedSegmentInfos ();
495
496
refresh ("translog_recovery" );
@@ -501,7 +502,8 @@ private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy
501
502
LongSupplier globalCheckpointSupplier , LongConsumer persistedSequenceNumberConsumer ) throws IOException {
502
503
503
504
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 ));
505
507
// We expect that this shard already exists, so it must already have an existing translog else something is badly wrong!
506
508
return new Translog (translogConfig , translogUUID , translogDeletionPolicy , globalCheckpointSupplier ,
507
509
engineConfig .getPrimaryTermSupplier (), persistedSequenceNumberConsumer );
@@ -549,7 +551,7 @@ public Translog.Snapshot readHistoryOperations(String reason, HistorySource hist
549
551
ensureSoftDeletesEnabled ();
550
552
return newChangesSnapshot (reason , mapperService , Math .max (0 , startingSeqNo ), Long .MAX_VALUE , false );
551
553
} else {
552
- return getTranslog ().newSnapshotFromMinSeqNo (startingSeqNo );
554
+ return getTranslog ().newSnapshot (startingSeqNo , Long . MAX_VALUE );
553
555
}
554
556
}
555
557
@@ -598,18 +600,6 @@ public long getWritingBytes() {
598
600
return indexWriter .getFlushingBytes () + versionMap .getRefreshingBytes ();
599
601
}
600
602
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
-
613
603
/**
614
604
* Reads the current stored history ID from the IW commit data.
615
605
*/
@@ -1688,8 +1678,9 @@ final boolean tryRenewSyncCommit() {
1688
1678
ensureOpen ();
1689
1679
ensureCanFlush ();
1690
1680
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 ) {
1693
1684
logger .trace ("start renewing sync commit [{}]" , syncId );
1694
1685
commitIndexWriter (indexWriter , translog , syncId );
1695
1686
logger .debug ("successfully sync committed. sync id [{}]." , syncId );
@@ -1714,8 +1705,10 @@ public boolean shouldPeriodicallyFlush() {
1714
1705
if (shouldPeriodicallyFlushAfterBigMerge .get ()) {
1715
1706
return true ;
1716
1707
}
1708
+ final long localCheckpointOfLastCommit =
1709
+ Long .parseLong (lastCommittedSegmentInfos .userData .get (SequenceNumbers .LOCAL_CHECKPOINT_KEY ));
1717
1710
final long translogGenerationOfLastCommit =
1718
- Long . parseLong ( lastCommittedSegmentInfos . userData . get ( Translog . TRANSLOG_GENERATION_KEY )) ;
1711
+ translog . getMinGenerationForSeqNo ( localCheckpointOfLastCommit + 1 ). translogFileGeneration ;
1719
1712
final long flushThreshold = config ().getIndexSettings ().getFlushThresholdSize ().getBytes ();
1720
1713
if (translog .sizeInBytesByMinGen (translogGenerationOfLastCommit ) < flushThreshold ) {
1721
1714
return false ;
@@ -2423,11 +2416,6 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl
2423
2416
ensureCanFlush ();
2424
2417
try {
2425
2418
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
-
2431
2419
writer .setLiveCommitData (() -> {
2432
2420
/*
2433
2421
* 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
2438
2426
* {@link IndexWriter#commit()} call flushes all documents, we defer computation of the maximum sequence number to the time
2439
2427
* of invocation of the commit data iterator (which occurs after all documents have been flushed to Lucene).
2440
2428
*/
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 ));
2445
2432
if (syncId != null ) {
2446
2433
commitData .put (Engine .SYNC_COMMIT_ID , syncId );
2447
2434
}
@@ -2657,7 +2644,7 @@ public boolean hasCompleteOperationHistory(String reason, HistorySource historyS
2657
2644
return true ;
2658
2645
}
2659
2646
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 )) {
2661
2648
Translog .Operation operation ;
2662
2649
while ((operation = snapshot .next ()) != null ) {
2663
2650
if (operation .seqNo () != SequenceNumbers .UNASSIGNED_SEQ_NO ) {
0 commit comments