mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-08 03:49:38 +00:00
Make recovery source partially non-blocking (#37291)
Today a peer-recovery may run into a deadlock if the value of node_concurrent_recoveries is too high. This happens because the peer-recovery is executed in a blocking fashion. This commit attempts to make the recovery source partially non-blocking. I will make three follow-ups to make it fully non-blocking: (1) send translog operations, (2) primary relocation, (3) send commit files. Relates #36195
This commit is contained in:
parent
63fe3c6ed6
commit
44a1071018
@ -18,6 +18,7 @@
|
||||
*/
|
||||
package org.elasticsearch.common.util;
|
||||
|
||||
import org.apache.lucene.util.SetOnce;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
@ -38,34 +39,35 @@ public class CancellableThreads {
|
||||
private final Set<Thread> threads = new HashSet<>();
|
||||
// needs to be volatile as it is also read outside of synchronized blocks.
|
||||
private volatile boolean cancelled = false;
|
||||
private final SetOnce<OnCancel> onCancel = new SetOnce<>();
|
||||
private String reason;
|
||||
|
||||
public synchronized boolean isCancelled() {
|
||||
return cancelled;
|
||||
}
|
||||
|
||||
|
||||
/** call this will throw an exception if operation was cancelled.
|
||||
* Override {@link #onCancel(String, Exception)} for custom failure logic */
|
||||
public synchronized void checkForCancel() {
|
||||
if (isCancelled()) {
|
||||
onCancel(reason, null);
|
||||
}
|
||||
public void checkForCancel() {
|
||||
checkForCancel(null);
|
||||
}
|
||||
|
||||
/**
|
||||
* called if {@link #checkForCancel()} was invoked after the operation was cancelled.
|
||||
* the default implementation always throws an {@link ExecutionCancelledException}, suppressing
|
||||
* any other exception that occurred before cancellation
|
||||
* @param reason reason for failure supplied by the caller of {@link #cancel}
|
||||
* @param suppressedException any error that was encountered during the execution before the operation was cancelled.
|
||||
*/
|
||||
protected void onCancel(String reason, @Nullable Exception suppressedException) {
|
||||
RuntimeException e = new ExecutionCancelledException("operation was cancelled reason [" + reason + "]");
|
||||
if (suppressedException != null) {
|
||||
e.addSuppressed(suppressedException);
|
||||
private void checkForCancel(Exception beforeCancelException) {
|
||||
if (isCancelled()) {
|
||||
final String reason;
|
||||
final OnCancel onCancel;
|
||||
synchronized (this) {
|
||||
reason = this.reason;
|
||||
onCancel = this.onCancel.get();
|
||||
}
|
||||
if (onCancel != null) {
|
||||
onCancel.onCancel(reason, beforeCancelException);
|
||||
}
|
||||
// fallback to the default exception
|
||||
final RuntimeException cancelExp = new ExecutionCancelledException("operation was cancelled reason [" + reason + "]");
|
||||
if (beforeCancelException != null) {
|
||||
cancelExp.addSuppressed(beforeCancelException);
|
||||
}
|
||||
throw cancelExp;
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
|
||||
private synchronized boolean add() {
|
||||
@ -125,17 +127,14 @@ public class CancellableThreads {
|
||||
// clear the flag interrupted flag as we are checking for failure..
|
||||
Thread.interrupted();
|
||||
}
|
||||
synchronized (this) {
|
||||
if (isCancelled()) {
|
||||
onCancel(reason, ioException != null ? ioException : runtimeException);
|
||||
} else if (ioException != null) {
|
||||
// if we're not canceling, we throw the original exception
|
||||
throw ioException;
|
||||
}
|
||||
if (runtimeException != null) {
|
||||
// if we're not canceling, we throw the original exception
|
||||
throw runtimeException;
|
||||
}
|
||||
checkForCancel(ioException != null ? ioException : runtimeException);
|
||||
if (ioException != null) {
|
||||
// if we're not canceling, we throw the original exception
|
||||
throw ioException;
|
||||
}
|
||||
if (runtimeException != null) {
|
||||
// if we're not canceling, we throw the original exception
|
||||
throw runtimeException;
|
||||
}
|
||||
if (cancelledByExternalInterrupt) {
|
||||
// restore interrupt flag to at least adhere to expected behavior
|
||||
@ -185,4 +184,26 @@ public class CancellableThreads {
|
||||
super(in);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Registers a callback that will be invoked when some running operations are cancelled or {@link #checkForCancel()} is called.
|
||||
*/
|
||||
public synchronized void setOnCancel(OnCancel onCancel) {
|
||||
this.onCancel.set(onCancel);
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
public interface OnCancel {
|
||||
/**
|
||||
* Called when some running operations are cancelled or {@link #checkForCancel()} is explicitly called.
|
||||
* If this method throws an exception, cancelling tasks will fail with that exception; otherwise they
|
||||
* will fail with the default exception {@link ExecutionCancelledException}.
|
||||
*
|
||||
* @param reason the reason of the cancellation
|
||||
* @param beforeCancelException any error that was encountered during the execution before the operations were cancelled.
|
||||
* @see #checkForCancel()
|
||||
* @see #setOnCancel(OnCancel)
|
||||
*/
|
||||
void onCancel(String reason, @Nullable Exception beforeCancelException);
|
||||
}
|
||||
}
|
||||
|
@ -22,6 +22,8 @@ package org.elasticsearch.indices.recovery;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
@ -81,7 +83,7 @@ public class PeerRecoverySourceService implements IndexEventListener {
|
||||
}
|
||||
}
|
||||
|
||||
private RecoveryResponse recover(final StartRecoveryRequest request) throws IOException {
|
||||
private void recover(StartRecoveryRequest request, ActionListener<RecoveryResponse> listener) throws IOException {
|
||||
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
||||
final IndexShard shard = indexService.getShard(request.shardId().id());
|
||||
|
||||
@ -101,18 +103,13 @@ public class PeerRecoverySourceService implements IndexEventListener {
|
||||
RecoverySourceHandler handler = ongoingRecoveries.addNewRecovery(request, shard);
|
||||
logger.trace("[{}][{}] starting recovery to {}", request.shardId().getIndex().getName(), request.shardId().id(),
|
||||
request.targetNode());
|
||||
try {
|
||||
return handler.recoverToTarget();
|
||||
} finally {
|
||||
ongoingRecoveries.remove(shard, handler);
|
||||
}
|
||||
handler.recoverToTarget(ActionListener.runAfter(listener, () -> ongoingRecoveries.remove(shard, handler)));
|
||||
}
|
||||
|
||||
class StartRecoveryTransportRequestHandler implements TransportRequestHandler<StartRecoveryRequest> {
|
||||
@Override
|
||||
public void messageReceived(final StartRecoveryRequest request, final TransportChannel channel, Task task) throws Exception {
|
||||
RecoveryResponse response = recover(request);
|
||||
channel.sendResponse(response);
|
||||
recover(request, new HandledTransportAction.ChannelActionListener<>(channel, Actions.START_RECOVERY, request));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -26,7 +26,7 @@ import org.elasticsearch.transport.TransportResponse;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
final class RecoveryResponse extends TransportResponse {
|
||||
public final class RecoveryResponse extends TransportResponse {
|
||||
|
||||
final List<String> phase1FileNames;
|
||||
final List<Long> phase1FileSizes;
|
||||
|
@ -34,7 +34,6 @@ import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.StopWatch;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
@ -70,6 +69,7 @@ import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Supplier;
|
||||
@ -96,22 +96,7 @@ public class RecoverySourceHandler {
|
||||
private final StartRecoveryRequest request;
|
||||
private final int chunkSizeInBytes;
|
||||
private final RecoveryTargetHandler recoveryTarget;
|
||||
|
||||
private final CancellableThreads cancellableThreads = new CancellableThreads() {
|
||||
@Override
|
||||
protected void onCancel(String reason, @Nullable Exception suppressedException) {
|
||||
RuntimeException e;
|
||||
if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us
|
||||
e = new IndexShardClosedException(shard.shardId(), "shard is closed and recovery was canceled reason [" + reason + "]");
|
||||
} else {
|
||||
e = new ExecutionCancelledException("recovery was canceled reason [" + reason + "]");
|
||||
}
|
||||
if (suppressedException != null) {
|
||||
e.addSuppressed(suppressedException);
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
};
|
||||
private final CancellableThreads cancellableThreads = new CancellableThreads();
|
||||
|
||||
public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recoveryTarget,
|
||||
final StartRecoveryRequest request,
|
||||
@ -131,19 +116,37 @@ public class RecoverySourceHandler {
|
||||
/**
|
||||
* performs the recovery from the local engine to the target
|
||||
*/
|
||||
public RecoveryResponse recoverToTarget() throws IOException {
|
||||
runUnderPrimaryPermit(() -> {
|
||||
final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable();
|
||||
ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId());
|
||||
if (targetShardRouting == null) {
|
||||
logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(),
|
||||
request.targetNode());
|
||||
throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node");
|
||||
}
|
||||
assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting;
|
||||
}, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger);
|
||||
|
||||
try (Closeable ignored = shard.acquireRetentionLockForPeerRecovery()) {
|
||||
public void recoverToTarget(ActionListener<RecoveryResponse> listener) {
|
||||
final List<Closeable> resources = new CopyOnWriteArrayList<>();
|
||||
final Closeable releaseResources = () -> IOUtils.close(resources);
|
||||
final ActionListener<RecoveryResponse> wrappedListener = ActionListener.notifyOnce(listener);
|
||||
try {
|
||||
cancellableThreads.setOnCancel((reason, beforeCancelEx) -> {
|
||||
final RuntimeException e;
|
||||
if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us
|
||||
e = new IndexShardClosedException(shard.shardId(), "shard is closed and recovery was canceled reason [" + reason + "]");
|
||||
} else {
|
||||
e = new CancellableThreads.ExecutionCancelledException("recovery was canceled reason [" + reason + "]");
|
||||
}
|
||||
if (beforeCancelEx != null) {
|
||||
e.addSuppressed(beforeCancelEx);
|
||||
}
|
||||
IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e));
|
||||
throw e;
|
||||
});
|
||||
runUnderPrimaryPermit(() -> {
|
||||
final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable();
|
||||
ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId());
|
||||
if (targetShardRouting == null) {
|
||||
logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(),
|
||||
request.targetNode());
|
||||
throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node");
|
||||
}
|
||||
assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting;
|
||||
}, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ",
|
||||
shard, cancellableThreads, logger);
|
||||
final Closeable retentionLock = shard.acquireRetentionLockForPeerRecovery();
|
||||
resources.add(retentionLock);
|
||||
final long startingSeqNo;
|
||||
final long requiredSeqNoRangeStart;
|
||||
final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO &&
|
||||
@ -217,6 +220,8 @@ public class RecoverySourceHandler {
|
||||
}
|
||||
final SendSnapshotResult sendSnapshotResult;
|
||||
try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) {
|
||||
// we can release the retention lock here because the snapshot itself will retain the required operations.
|
||||
IOUtils.close(retentionLock, () -> resources.remove(retentionLock));
|
||||
// we have to capture the max_seen_auto_id_timestamp and the max_seq_no_of_updates to make sure that these values
|
||||
// are at least as high as the corresponding values on the primary when any of these operations were executed on it.
|
||||
final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp();
|
||||
@ -229,10 +234,16 @@ public class RecoverySourceHandler {
|
||||
|
||||
finalizeRecovery(sendSnapshotResult.targetLocalCheckpoint);
|
||||
final long phase1ThrottlingWaitTime = 0L; // TODO: return the actual throttle time
|
||||
return new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes,
|
||||
sendFileResult.phase1ExistingFileNames, sendFileResult.phase1ExistingFileSizes, sendFileResult.totalSize,
|
||||
sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, prepareEngineTime.millis(),
|
||||
sendSnapshotResult.totalOperations, sendSnapshotResult.tookTime.millis());
|
||||
assert resources.isEmpty() : "not every resource is released [" + resources + "]";
|
||||
IOUtils.close(resources);
|
||||
wrappedListener.onResponse(
|
||||
new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes,
|
||||
sendFileResult.phase1ExistingFileNames, sendFileResult.phase1ExistingFileSizes, sendFileResult.totalSize,
|
||||
sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, prepareEngineTime.millis(),
|
||||
sendSnapshotResult.totalOperations, sendSnapshotResult.tookTime.millis())
|
||||
);
|
||||
} catch (Exception e) {
|
||||
IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,12 @@ import org.elasticsearch.test.ESTestCase;
|
||||
import org.hamcrest.Matchers;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class CancellableThreadsTests extends ESTestCase {
|
||||
public static class CustomException extends RuntimeException {
|
||||
@ -39,6 +44,8 @@ public class CancellableThreadsTests extends ESTestCase {
|
||||
}
|
||||
}
|
||||
|
||||
static class ThrowOnCancelException extends RuntimeException {
|
||||
}
|
||||
|
||||
private class TestPlan {
|
||||
public final int id;
|
||||
@ -167,6 +174,19 @@ public class CancellableThreadsTests extends ESTestCase {
|
||||
}
|
||||
|
||||
readyForCancel.await();
|
||||
final boolean throwInOnCancel = randomBoolean();
|
||||
final AtomicInteger invokeTimes = new AtomicInteger();
|
||||
cancellableThreads.setOnCancel((reason, beforeCancelException) -> {
|
||||
invokeTimes.getAndIncrement();
|
||||
if (throwInOnCancel) {
|
||||
ThrowOnCancelException e = new ThrowOnCancelException();
|
||||
if (beforeCancelException != null) {
|
||||
e.addSuppressed(beforeCancelException);
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
});
|
||||
|
||||
cancellableThreads.cancel("test");
|
||||
for (Thread thread : threads) {
|
||||
thread.join(20000);
|
||||
@ -181,7 +201,11 @@ public class CancellableThreadsTests extends ESTestCase {
|
||||
assertNull(exceptions[i]);
|
||||
} else {
|
||||
// in all other cases, we expect a cancellation exception.
|
||||
assertThat(exceptions[i], Matchers.instanceOf(CancellableThreads.ExecutionCancelledException.class));
|
||||
if (throwInOnCancel) {
|
||||
assertThat(exceptions[i], Matchers.instanceOf(ThrowOnCancelException.class));
|
||||
} else {
|
||||
assertThat(exceptions[i], Matchers.instanceOf(ExecutionCancelledException.class));
|
||||
}
|
||||
if (plan.exceptAfterCancel) {
|
||||
assertThat(exceptions[i].getSuppressed(),
|
||||
Matchers.arrayContaining(
|
||||
@ -191,8 +215,17 @@ public class CancellableThreadsTests extends ESTestCase {
|
||||
assertThat(exceptions[i].getSuppressed(), Matchers.emptyArray());
|
||||
}
|
||||
}
|
||||
assertThat(interrupted[plan.id], Matchers.equalTo(plan.presetInterrupt));
|
||||
assertThat(interrupted[plan.id], equalTo(plan.presetInterrupt));
|
||||
}
|
||||
assertThat(invokeTimes.longValue(),
|
||||
equalTo(Arrays.stream(plans).filter(p -> p.exceptBeforeCancel == false && p.exitBeforeCancel == false).count()));
|
||||
if (throwInOnCancel) {
|
||||
expectThrows(ThrowOnCancelException.class, cancellableThreads::checkForCancel);
|
||||
} else {
|
||||
expectThrows(ExecutionCancelledException.class, cancellableThreads::checkForCancel);
|
||||
}
|
||||
assertThat(invokeTimes.longValue(),
|
||||
equalTo(Arrays.stream(plans).filter(p -> p.exceptBeforeCancel == false && p.exitBeforeCancel == false).count() + 1));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -35,6 +35,7 @@ import org.apache.lucene.store.IOContext;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
@ -433,7 +434,11 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||
}
|
||||
|
||||
};
|
||||
expectThrows(IndexShardRelocatedException.class, handler::recoverToTarget);
|
||||
PlainActionFuture<RecoveryResponse> future = new PlainActionFuture<>();
|
||||
expectThrows(IndexShardRelocatedException.class, () -> {
|
||||
handler.recoverToTarget(future);
|
||||
future.actionGet();
|
||||
});
|
||||
assertFalse(phase1Called.get());
|
||||
assertFalse(prepareTargetForTranslogCalled.get());
|
||||
assertFalse(phase2Called.get());
|
||||
|
@ -23,6 +23,7 @@ import org.apache.lucene.store.Directory;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.action.support.replication.TransportReplicationAction;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
@ -67,6 +68,7 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
||||
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
|
||||
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
||||
import org.elasticsearch.indices.recovery.RecoveryFailedException;
|
||||
import org.elasticsearch.indices.recovery.RecoveryResponse;
|
||||
import org.elasticsearch.indices.recovery.RecoverySourceHandler;
|
||||
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
||||
@ -604,7 +606,10 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||
(int) ByteSizeUnit.MB.toBytes(1));
|
||||
primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null,
|
||||
currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable, Collections.emptySet());
|
||||
recovery.recoverToTarget();
|
||||
|
||||
PlainActionFuture<RecoveryResponse> future = new PlainActionFuture<>();
|
||||
recovery.recoverToTarget(future);
|
||||
future.actionGet();
|
||||
recoveryTarget.markAsDone();
|
||||
}
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user