Skip to content

Commit d15684d

Browse files
authored
Use global checkpoint as starting seq in ops-based recovery (#43463)
Today we use the local checkpoint of the safe commit on replicas as the starting sequence number of operation-based peer recovery. While this is a good choice due to its simplicity, we need to share this information between copies if we use retention leases in peer recovery. We can avoid this extra work if we use the global checkpoint as the starting sequence number. With this change, we will try to recover replica locally up to the global checkpoint before performing peer recovery. This commit should also increase the chance of operation-based recovery.
1 parent 69c94f4 commit d15684d

File tree

9 files changed

+343
-205
lines changed

9 files changed

+343
-205
lines changed

server/src/main/java/org/elasticsearch/index/shard/IndexShard.java

+106-17
Original file line numberDiff line numberDiff line change
@@ -159,6 +159,7 @@
159159
import java.util.Locale;
160160
import java.util.Map;
161161
import java.util.Objects;
162+
import java.util.Optional;
162163
import java.util.Set;
163164
import java.util.concurrent.CopyOnWriteArrayList;
164165
import java.util.concurrent.CountDownLatch;
@@ -1359,6 +1360,81 @@ public void prepareForIndexRecovery() {
13591360
assert currentEngineReference.get() == null;
13601361
}
13611362

1363+
/**
1364+
* A best effort to bring up this shard to the global checkpoint using the local translog before performing a peer recovery.
1365+
*
1366+
* @return a sequence number that an operation-based peer recovery can start with.
1367+
* This is the first operation after the local checkpoint of the safe commit if exists.
1368+
*/
1369+
public long recoverLocallyUpToGlobalCheckpoint() {
1370+
if (state != IndexShardState.RECOVERING) {
1371+
throw new IndexShardNotRecoveringException(shardId, state);
1372+
}
1373+
assert recoveryState.getStage() == RecoveryState.Stage.INDEX : "unexpected recovery stage [" + recoveryState.getStage() + "]";
1374+
assert routingEntry().recoverySource().getType() == RecoverySource.Type.PEER : "not a peer recovery [" + routingEntry() + "]";
1375+
final Optional<SequenceNumbers.CommitInfo> safeCommit;
1376+
final long globalCheckpoint;
1377+
try {
1378+
final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY);
1379+
globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID);
1380+
safeCommit = store.findSafeIndexCommit(globalCheckpoint);
1381+
} catch (org.apache.lucene.index.IndexNotFoundException e) {
1382+
logger.trace("skip local recovery as no index commit found");
1383+
return UNASSIGNED_SEQ_NO;
1384+
} catch (Exception e) {
1385+
logger.debug("skip local recovery as failed to find the safe commit", e);
1386+
return UNASSIGNED_SEQ_NO;
1387+
}
1388+
if (safeCommit.isPresent() == false) {
1389+
logger.trace("skip local recovery as no safe commit found");
1390+
return UNASSIGNED_SEQ_NO;
1391+
}
1392+
assert safeCommit.get().localCheckpoint <= globalCheckpoint : safeCommit.get().localCheckpoint + " > " + globalCheckpoint;
1393+
try {
1394+
maybeCheckIndex(); // check index here and won't do it again if ops-based recovery occurs
1395+
recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
1396+
if (safeCommit.get().localCheckpoint == globalCheckpoint) {
1397+
logger.trace("skip local recovery as the safe commit is up to date; safe commit {} global checkpoint {}",
1398+
safeCommit.get(), globalCheckpoint);
1399+
recoveryState.getTranslog().totalLocal(0);
1400+
return globalCheckpoint + 1;
1401+
}
1402+
try {
1403+
final Engine.TranslogRecoveryRunner translogRecoveryRunner = (engine, snapshot) -> {
1404+
recoveryState.getTranslog().totalLocal(snapshot.totalOperations());
1405+
final int recoveredOps = runTranslogRecovery(engine, snapshot, Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY,
1406+
recoveryState.getTranslog()::incrementRecoveredOperations);
1407+
recoveryState.getTranslog().totalLocal(recoveredOps); // adjust the total local to reflect the actual count
1408+
return recoveredOps;
1409+
};
1410+
innerOpenEngineAndTranslog();
1411+
getEngine().recoverFromTranslog(translogRecoveryRunner, globalCheckpoint);
1412+
logger.trace("shard locally recovered up to {}", getEngine().getSeqNoStats(globalCheckpoint));
1413+
} finally {
1414+
synchronized (mutex) {
1415+
IOUtils.close(currentEngineReference.getAndSet(null));
1416+
}
1417+
}
1418+
} catch (Exception e) {
1419+
logger.debug(new ParameterizedMessage("failed to recover shard locally up to global checkpoint {}", globalCheckpoint), e);
1420+
return UNASSIGNED_SEQ_NO;
1421+
}
1422+
try {
1423+
// we need to find the safe commit again as we should have created a new one during the local recovery
1424+
final Optional<SequenceNumbers.CommitInfo> newSafeCommit = store.findSafeIndexCommit(globalCheckpoint);
1425+
assert newSafeCommit.isPresent() : "no safe commit found after local recovery";
1426+
return newSafeCommit.get().localCheckpoint + 1;
1427+
} catch (Exception e) {
1428+
if (Assertions.ENABLED) {
1429+
throw new AssertionError(
1430+
"failed to find the safe commit after recovering shard locally up to global checkpoint " + globalCheckpoint, e);
1431+
}
1432+
logger.debug(new ParameterizedMessage(
1433+
"failed to find the safe commit after recovering shard locally up to global checkpoint {}", globalCheckpoint), e);
1434+
return UNASSIGNED_SEQ_NO;
1435+
}
1436+
}
1437+
13621438
public void trimOperationOfPreviousPrimaryTerms(long aboveSeqNo) {
13631439
getEngine().trimOperationsFromTranslog(getOperationPrimaryTerm(), aboveSeqNo);
13641440
}
@@ -1462,6 +1538,9 @@ int runTranslogRecovery(Engine engine, Translog.Snapshot snapshot, Engine.Operat
14621538
* Operations from the translog will be replayed to bring lucene up to date.
14631539
**/
14641540
public void openEngineAndRecoverFromTranslog() throws IOException {
1541+
assert recoveryState.getStage() == RecoveryState.Stage.INDEX : "unexpected recovery stage [" + recoveryState.getStage() + "]";
1542+
maybeCheckIndex();
1543+
recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
14651544
final RecoveryState.Translog translogRecoveryStats = recoveryState.getTranslog();
14661545
final Engine.TranslogRecoveryRunner translogRecoveryRunner = (engine, snapshot) -> {
14671546
translogRecoveryStats.totalOperations(snapshot.totalOperations());
@@ -1478,6 +1557,8 @@ public void openEngineAndRecoverFromTranslog() throws IOException {
14781557
* The translog is kept but its operations won't be replayed.
14791558
*/
14801559
public void openEngineAndSkipTranslogRecovery() throws IOException {
1560+
assert routingEntry().recoverySource().getType() == RecoverySource.Type.PEER : "not a peer recovery [" + routingEntry() + "]";
1561+
assert recoveryState.getStage() == RecoveryState.Stage.TRANSLOG : "unexpected recovery stage [" + recoveryState.getStage() + "]";
14811562
innerOpenEngineAndTranslog();
14821563
getEngine().skipTranslogRecovery();
14831564
}
@@ -1486,17 +1567,6 @@ private void innerOpenEngineAndTranslog() throws IOException {
14861567
if (state != IndexShardState.RECOVERING) {
14871568
throw new IndexShardNotRecoveringException(shardId, state);
14881569
}
1489-
recoveryState.setStage(RecoveryState.Stage.VERIFY_INDEX);
1490-
// also check here, before we apply the translog
1491-
if (Booleans.isTrue(checkIndexOnStartup) || "checksum".equals(checkIndexOnStartup)) {
1492-
try {
1493-
checkIndex();
1494-
} catch (IOException ex) {
1495-
throw new RecoveryFailedException(recoveryState, "check index failed", ex);
1496-
}
1497-
}
1498-
recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
1499-
15001570
final EngineConfig config = newEngineConfig();
15011571

15021572
// we disable deletes since we allow for operations to be executed against the shard while recovering
@@ -1552,14 +1622,22 @@ private void onNewEngine(Engine newEngine) {
15521622
*/
15531623
public void performRecoveryRestart() throws IOException {
15541624
synchronized (mutex) {
1555-
if (state != IndexShardState.RECOVERING) {
1556-
throw new IndexShardNotRecoveringException(shardId, state);
1557-
}
15581625
assert refreshListeners.pendingCount() == 0 : "we can't restart with pending listeners";
1559-
final Engine engine = this.currentEngineReference.getAndSet(null);
1560-
IOUtils.close(engine);
1561-
recoveryState().setStage(RecoveryState.Stage.INIT);
1626+
IOUtils.close(currentEngineReference.getAndSet(null));
1627+
resetRecoveryStage();
1628+
}
1629+
}
1630+
1631+
/**
1632+
* If a file-based recovery occurs, a recovery target calls this method to reset the recovery stage.
1633+
*/
1634+
public void resetRecoveryStage() {
1635+
assert routingEntry().recoverySource().getType() == RecoverySource.Type.PEER : "not a peer recovery [" + routingEntry() + "]";
1636+
assert currentEngineReference.get() == null;
1637+
if (state != IndexShardState.RECOVERING) {
1638+
throw new IndexShardNotRecoveringException(shardId, state);
15621639
}
1640+
recoveryState().setStage(RecoveryState.Stage.INIT);
15631641
}
15641642

15651643
/**
@@ -2296,6 +2374,17 @@ public void noopUpdate(String type) {
22962374
internalIndexingStats.noopUpdate(type);
22972375
}
22982376

2377+
public void maybeCheckIndex() {
2378+
recoveryState.setStage(RecoveryState.Stage.VERIFY_INDEX);
2379+
if (Booleans.isTrue(checkIndexOnStartup) || "checksum".equals(checkIndexOnStartup)) {
2380+
try {
2381+
checkIndex();
2382+
} catch (IOException ex) {
2383+
throw new RecoveryFailedException(recoveryState, "check index failed", ex);
2384+
}
2385+
}
2386+
}
2387+
22992388
void checkIndex() throws IOException {
23002389
if (store.tryIncRef()) {
23012390
try {

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

+17
Original file line numberDiff line numberDiff line change
@@ -96,6 +96,7 @@
9696
import java.util.Iterator;
9797
import java.util.List;
9898
import java.util.Map;
99+
import java.util.Optional;
99100
import java.util.concurrent.TimeUnit;
100101
import java.util.concurrent.atomic.AtomicBoolean;
101102
import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -1560,6 +1561,22 @@ public void trimUnsafeCommits(final long lastSyncedGlobalCheckpoint, final long
15601561
}
15611562
}
15621563

1564+
/**
1565+
* Returns a {@link org.elasticsearch.index.seqno.SequenceNumbers.CommitInfo} of the safe commit if exists.
1566+
*/
1567+
public Optional<SequenceNumbers.CommitInfo> findSafeIndexCommit(long globalCheckpoint) throws IOException {
1568+
final List<IndexCommit> commits = DirectoryReader.listCommits(directory);
1569+
assert commits.isEmpty() == false : "no commit found";
1570+
final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commits, globalCheckpoint);
1571+
final SequenceNumbers.CommitInfo commitInfo = SequenceNumbers.loadSeqNoInfoFromLuceneCommit(safeCommit.getUserData().entrySet());
1572+
// all operations of the safe commit must be at most the global checkpoint.
1573+
if (commitInfo.maxSeqNo <= globalCheckpoint) {
1574+
return Optional.of(commitInfo);
1575+
} else {
1576+
return Optional.empty();
1577+
}
1578+
}
1579+
15631580
private static void updateCommitData(IndexWriter writer, Map<String, String> keysToUpdate) throws IOException {
15641581
final Map<String, String> userData = getUserData(writer);
15651582
userData.putAll(keysToUpdate);

server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java

+14-75
Original file line numberDiff line numberDiff line change
@@ -22,8 +22,6 @@
2222
import org.apache.logging.log4j.LogManager;
2323
import org.apache.logging.log4j.Logger;
2424
import org.apache.logging.log4j.message.ParameterizedMessage;
25-
import org.apache.lucene.index.DirectoryReader;
26-
import org.apache.lucene.index.IndexCommit;
2725
import org.apache.lucene.store.AlreadyClosedException;
2826
import org.apache.lucene.store.RateLimiter;
2927
import org.elasticsearch.ElasticsearchException;
@@ -44,18 +42,14 @@
4442
import org.elasticsearch.common.util.CancellableThreads;
4543
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
4644
import org.elasticsearch.index.IndexNotFoundException;
47-
import org.elasticsearch.index.engine.CombinedDeletionPolicy;
4845
import org.elasticsearch.index.engine.RecoveryEngineException;
4946
import org.elasticsearch.index.mapper.MapperException;
50-
import org.elasticsearch.index.seqno.SequenceNumbers;
5147
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
5248
import org.elasticsearch.index.shard.IndexEventListener;
5349
import org.elasticsearch.index.shard.IndexShard;
5450
import org.elasticsearch.index.shard.ShardId;
5551
import org.elasticsearch.index.shard.ShardNotFoundException;
5652
import org.elasticsearch.index.store.Store;
57-
import org.elasticsearch.index.translog.Translog;
58-
import org.elasticsearch.index.translog.TranslogCorruptedException;
5953
import org.elasticsearch.indices.recovery.RecoveriesCollection.RecoveryRef;
6054
import org.elasticsearch.tasks.Task;
6155
import org.elasticsearch.threadpool.ThreadPool;
@@ -68,12 +62,11 @@
6862
import org.elasticsearch.transport.TransportService;
6963

7064
import java.io.IOException;
71-
import java.util.List;
72-
import java.util.StringJoiner;
7365
import java.util.concurrent.atomic.AtomicLong;
7466
import java.util.function.Consumer;
7567

7668
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
69+
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
7770

7871
/**
7972
* The recovery target handles recoveries of peer shards of the shard+node to recover to.
@@ -178,9 +171,12 @@ private void doRecovery(final long recoveryId) {
178171
cancellableThreads = recoveryTarget.cancellableThreads();
179172
try {
180173
assert recoveryTarget.sourceNode() != null : "can not do a recovery without a source node";
181-
request = getStartRecoveryRequest(recoveryTarget);
182174
logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId());
183175
recoveryTarget.indexShard().prepareForIndexRecovery();
176+
final long startingSeqNo = recoveryTarget.indexShard().recoverLocallyUpToGlobalCheckpoint();
177+
assert startingSeqNo == UNASSIGNED_SEQ_NO || recoveryTarget.state().getStage() == RecoveryState.Stage.TRANSLOG :
178+
"unexpected recovery stage [" + recoveryTarget.state().getStage() + "] starting seqno [ " + startingSeqNo + "]";
179+
request = getStartRecoveryRequest(logger, clusterService.localNode(), recoveryTarget, startingSeqNo);
184180
} catch (final Exception e) {
185181
// this will be logged as warning later on...
186182
logger.trace("unexpected error while preparing shard for peer recovery, failing recovery", e);
@@ -319,7 +315,7 @@ public RecoveryResponse read(StreamInput in) throws IOException {
319315
* @param recoveryTarget the target of the recovery
320316
* @return a snapshot of the store metadata
321317
*/
322-
private Store.MetadataSnapshot getStoreMetadataSnapshot(final RecoveryTarget recoveryTarget) {
318+
private static Store.MetadataSnapshot getStoreMetadataSnapshot(final Logger logger, final RecoveryTarget recoveryTarget) {
323319
try {
324320
return recoveryTarget.indexShard().snapshotStoreMetadata();
325321
} catch (final org.apache.lucene.index.IndexNotFoundException e) {
@@ -335,89 +331,32 @@ private Store.MetadataSnapshot getStoreMetadataSnapshot(final RecoveryTarget rec
335331
/**
336332
* Prepare the start recovery request.
337333
*
334+
* @param logger the logger
335+
* @param localNode the local node of the recovery target
338336
* @param recoveryTarget the target of the recovery
337+
* @param startingSeqNo a sequence number that an operation-based peer recovery can start with.
338+
* This is the first operation after the local checkpoint of the safe commit if exists.
339339
* @return a start recovery request
340340
*/
341-
private StartRecoveryRequest getStartRecoveryRequest(final RecoveryTarget recoveryTarget) {
341+
public static StartRecoveryRequest getStartRecoveryRequest(Logger logger, DiscoveryNode localNode,
342+
RecoveryTarget recoveryTarget, long startingSeqNo) {
342343
final StartRecoveryRequest request;
343344
logger.trace("{} collecting local files for [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode());
344345

345-
final Store.MetadataSnapshot metadataSnapshot = getStoreMetadataSnapshot(recoveryTarget);
346+
final Store.MetadataSnapshot metadataSnapshot = getStoreMetadataSnapshot(logger, recoveryTarget);
346347
logger.trace("{} local file count [{}]", recoveryTarget.shardId(), metadataSnapshot.size());
347-
348-
final long startingSeqNo;
349-
if (metadataSnapshot.size() > 0) {
350-
startingSeqNo = getStartingSeqNo(logger, recoveryTarget);
351-
} else {
352-
startingSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO;
353-
}
354-
355-
if (startingSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO) {
356-
logger.trace("{} preparing for file-based recovery from [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode());
357-
} else {
358-
logger.trace(
359-
"{} preparing for sequence-number-based recovery starting at sequence number [{}] from [{}]",
360-
recoveryTarget.shardId(),
361-
startingSeqNo,
362-
recoveryTarget.sourceNode());
363-
}
364-
365348
request = new StartRecoveryRequest(
366349
recoveryTarget.shardId(),
367350
recoveryTarget.indexShard().routingEntry().allocationId().getId(),
368351
recoveryTarget.sourceNode(),
369-
clusterService.localNode(),
352+
localNode,
370353
metadataSnapshot,
371354
recoveryTarget.state().getPrimary(),
372355
recoveryTarget.recoveryId(),
373356
startingSeqNo);
374357
return request;
375358
}
376359

377-
/**
378-
* Get the starting sequence number for a sequence-number-based request.
379-
*
380-
* @param recoveryTarget the target of the recovery
381-
* @return the starting sequence number or {@link SequenceNumbers#UNASSIGNED_SEQ_NO} if obtaining the starting sequence number
382-
* failed
383-
*/
384-
public static long getStartingSeqNo(final Logger logger, final RecoveryTarget recoveryTarget) {
385-
try {
386-
final Store store = recoveryTarget.store();
387-
final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY);
388-
final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.translogLocation(), translogUUID);
389-
final List<IndexCommit> existingCommits = DirectoryReader.listCommits(store.directory());
390-
final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(existingCommits, globalCheckpoint);
391-
final SequenceNumbers.CommitInfo seqNoStats = Store.loadSeqNoInfo(safeCommit);
392-
if (logger.isTraceEnabled()) {
393-
final StringJoiner descriptionOfExistingCommits = new StringJoiner(",");
394-
for (IndexCommit commit : existingCommits) {
395-
descriptionOfExistingCommits.add(CombinedDeletionPolicy.commitDescription(commit));
396-
}
397-
logger.trace("Calculate starting seqno based on global checkpoint [{}], safe commit [{}], existing commits [{}]",
398-
globalCheckpoint, CombinedDeletionPolicy.commitDescription(safeCommit), descriptionOfExistingCommits);
399-
}
400-
if (seqNoStats.maxSeqNo <= globalCheckpoint) {
401-
assert seqNoStats.localCheckpoint <= globalCheckpoint;
402-
/*
403-
* Commit point is good for sequence-number based recovery as the maximum sequence number included in it is below the global
404-
* checkpoint (i.e., it excludes any operations that may not be on the primary). Recovery will start at the first operation
405-
* after the local checkpoint stored in the commit.
406-
*/
407-
return seqNoStats.localCheckpoint + 1;
408-
} else {
409-
return SequenceNumbers.UNASSIGNED_SEQ_NO;
410-
}
411-
} catch (final TranslogCorruptedException | IOException e) {
412-
/*
413-
* This can happen, for example, if a phase one of the recovery completed successfully, a network partition happens before the
414-
* translog on the recovery target is opened, the recovery enters a retry loop seeing now that the index files are on disk and
415-
* proceeds to attempt a sequence-number-based recovery.
416-
*/
417-
return SequenceNumbers.UNASSIGNED_SEQ_NO;
418-
}
419-
}
420-
421360
public interface RecoveryListener {
422361
void onRecoveryDone(RecoveryState state);
423362

0 commit comments

Comments
 (0)