Skip to content

Commit b8c9dc9

Browse files
authored
Disable recovery monitor before recovery start (#93551)
We do nontrivial amounts of work before we start a peer recovery, particularly recovering from the local translog up to its global checkpoint. Today the recovery monitor is running during this time, and will (repeatedly) fail the recovery if it takes more than 30 minutes to complete. With this commit we disable the recovery monitor until this local process has completed. Closes #93542
1 parent 052a4f1 commit b8c9dc9

File tree

4 files changed

+20
-10
lines changed

4 files changed

+20
-10
lines changed

docs/changelog/93551.yaml

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,6 @@
1+
pr: 93551
2+
summary: Disable recovery monitor before recovery start
3+
area: Recovery
4+
type: bug
5+
issues:
6+
- 93542

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import org.elasticsearch.core.CheckedFunction;
3333
import org.elasticsearch.core.Nullable;
3434
import org.elasticsearch.core.Releasable;
35+
import org.elasticsearch.core.Releasables;
3536
import org.elasticsearch.core.TimeValue;
3637
import org.elasticsearch.index.IndexNotFoundException;
3738
import org.elasticsearch.index.engine.RecoveryEngineException;
@@ -219,6 +220,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi
219220
final RecoveryState recoveryState = recoveryTarget.state();
220221
final RecoveryState.Timer timer = recoveryState.getTimer();
221222
final IndexShard indexShard = recoveryTarget.indexShard();
223+
final Releasable onCompletion = Releasables.wrap(recoveryTarget.disableRecoveryMonitor(), recoveryRef);
222224

223225
final var failureHandler = ActionListener.notifyOnce(ActionListener.runBefore(ActionListener.noop().delegateResponse((l, e) -> {
224226
// this will be logged as warning later on...
@@ -228,7 +230,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi
228230
new RecoveryFailedException(recoveryTarget.state(), "failed to prepare shard for recovery", e),
229231
true
230232
);
231-
}), recoveryRef::close));
233+
}), onCompletion::close));
232234

233235
if (indexShard.routingEntry().isPromotableToPrimary() == false) {
234236
assert preExistingRequest == null;

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -255,11 +255,11 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) {
255255
}
256256

257257
/**
258-
* a reference to {@link RecoveryTarget}, which implements {@link AutoCloseable}. closing the reference
258+
* a reference to {@link RecoveryTarget}, which implements {@link Releasable}. closing the reference
259259
* causes {@link RecoveryTarget#decRef()} to be called. This makes sure that the underlying resources
260260
* will not be freed until {@link RecoveryRef#close()} is called.
261261
*/
262-
public static class RecoveryRef implements AutoCloseable {
262+
public static class RecoveryRef implements Releasable {
263263

264264
private final RecoveryTarget status;
265265
private final AtomicBoolean closed = new AtomicBoolean(false);

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

Lines changed: 9 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import org.elasticsearch.core.AbstractRefCounted;
2727
import org.elasticsearch.core.Nullable;
2828
import org.elasticsearch.core.Releasable;
29+
import org.elasticsearch.core.Releasables;
2930
import org.elasticsearch.index.engine.Engine;
3031
import org.elasticsearch.index.mapper.MapperException;
3132
import org.elasticsearch.index.seqno.ReplicationTracker;
@@ -47,6 +48,7 @@
4748
import java.util.List;
4849
import java.util.concurrent.CountDownLatch;
4950
import java.util.concurrent.atomic.AtomicBoolean;
51+
import java.util.concurrent.atomic.AtomicInteger;
5052
import java.util.concurrent.atomic.AtomicLong;
5153

5254
import static org.elasticsearch.core.Strings.format;
@@ -80,7 +82,7 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
8082
// last time this status was accessed
8183
private volatile long lastAccessTime = System.nanoTime();
8284

83-
private volatile boolean recoveryMonitorEnabled = true;
85+
private final AtomicInteger recoveryMonitorBlocks = new AtomicInteger();
8486

8587
@Nullable // if we're not downloading files from snapshots in this recovery or we're retrying
8688
private volatile Releasable snapshotFileDownloadsPermit;
@@ -185,7 +187,7 @@ public boolean hasPermitToDownloadSnapshotFiles() {
185187

186188
/** return the last time this RecoveryStatus was used (based on System.nanoTime() */
187189
public long lastAccessTime() {
188-
if (recoveryMonitorEnabled) {
190+
if (recoveryMonitorBlocks.get() == 0) {
189191
return lastAccessTime;
190192
}
191193
return System.nanoTime();
@@ -204,12 +206,11 @@ public void setLastAccessTime() {
204206
* @return releasable that once closed will re-enable liveness checks by the recovery monitor
205207
*/
206208
public Releasable disableRecoveryMonitor() {
207-
assert recoveryMonitorEnabled : "recovery monitor already disabled";
208-
recoveryMonitorEnabled = false;
209-
return () -> {
209+
recoveryMonitorBlocks.incrementAndGet();
210+
return Releasables.releaseOnce(() -> {
210211
setLastAccessTime();
211-
recoveryMonitorEnabled = true;
212-
};
212+
recoveryMonitorBlocks.decrementAndGet();
213+
});
213214
}
214215

215216
public Store store() {
@@ -316,6 +317,7 @@ public void markAsDone() {
316317

317318
@Override
318319
protected void closeInternal() {
320+
assert recoveryMonitorBlocks.get() == 0;
319321
try {
320322
multiFileWriter.close();
321323
} finally {

0 commit comments

Comments
 (0)