From d382a3d87eafffdcb7db9803fb98769fd206c2e9 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 7 Feb 2023 09:05:40 +0000 Subject: [PATCH] Disable recovery monitor before recovery start 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 --- .../indices/recovery/PeerRecoveryTargetService.java | 4 +++- .../elasticsearch/indices/recovery/RecoveriesCollection.java | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index abd1ef4aaf958..8a2b594c9af59 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -32,6 +32,7 @@ import org.elasticsearch.core.CheckedFunction; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.engine.RecoveryEngineException; @@ -219,6 +220,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi final RecoveryState recoveryState = recoveryTarget.state(); final RecoveryState.Timer timer = recoveryState.getTimer(); final IndexShard indexShard = recoveryTarget.indexShard(); + final Releasable onCompletion = Releasables.wrap(recoveryTarget.disableRecoveryMonitor(), recoveryRef); final var failureHandler = ActionListener.notifyOnce(ActionListener.runBefore(ActionListener.noop().delegateResponse((l, e) -> { // this will be logged as warning later on... @@ -228,7 +230,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi new RecoveryFailedException(recoveryTarget.state(), "failed to prepare shard for recovery", e), true ); - }), recoveryRef::close)); + }), onCompletion::close)); if (indexShard.routingEntry().isPromotableToPrimary() == false) { assert preExistingRequest == null; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java index 59ed1ba2b871f..54310659a39ff 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java @@ -255,11 +255,11 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { } /** - * a reference to {@link RecoveryTarget}, which implements {@link AutoCloseable}. closing the reference + * a reference to {@link RecoveryTarget}, which implements {@link Releasable}. closing the reference * causes {@link RecoveryTarget#decRef()} to be called. This makes sure that the underlying resources * will not be freed until {@link RecoveryRef#close()} is called. */ - public static class RecoveryRef implements AutoCloseable { + public static class RecoveryRef implements Releasable { private final RecoveryTarget status; private final AtomicBoolean closed = new AtomicBoolean(false);