@@ -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 );
0 commit comments