conservative timeouts on internal recovery actions
safe guards against cases where intenral recovery actions take too long (possibly due to a bug)
This commit is contained in:
parent
f5a3261e15
commit
9a90c1c3b5
|
@ -31,6 +31,7 @@ import org.elasticsearch.common.compress.CompressorFactory;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
|
||||
|
@ -60,11 +61,12 @@ public class RecoverySource extends AbstractComponent {
|
|||
}
|
||||
|
||||
private final TransportService transportService;
|
||||
|
||||
private final IndicesService indicesService;
|
||||
|
||||
private final RecoverySettings recoverySettings;
|
||||
|
||||
private final TimeValue internalActionTimeout;
|
||||
private final TimeValue internalActionLongTimeout;
|
||||
|
||||
|
||||
@Inject
|
||||
public RecoverySource(Settings settings, TransportService transportService, IndicesService indicesService,
|
||||
|
@ -76,6 +78,8 @@ public class RecoverySource extends AbstractComponent {
|
|||
this.recoverySettings = recoverySettings;
|
||||
|
||||
transportService.registerHandler(Actions.START_RECOVERY, new StartRecoveryTransportRequestHandler());
|
||||
this.internalActionTimeout = componentSettings.getAsTime("internal_action_timeout", TimeValue.timeValueMinutes(15));
|
||||
this.internalActionLongTimeout = new TimeValue(internalActionTimeout.millis() * 2);
|
||||
}
|
||||
|
||||
private RecoveryResponse recover(final StartRecoveryRequest request) {
|
||||
|
@ -123,7 +127,7 @@ public class RecoverySource extends AbstractComponent {
|
|||
|
||||
RecoveryFilesInfoRequest recoveryInfoFilesRequest = new RecoveryFilesInfoRequest(request.recoveryId(), request.shardId(), response.phase1FileNames, response.phase1FileSizes,
|
||||
response.phase1ExistingFileNames, response.phase1ExistingFileSizes, response.phase1TotalSize, response.phase1ExistingTotalSize);
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest, EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest, TransportRequestOptions.options().withTimeout(internalActionTimeout), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(response.phase1FileNames.size());
|
||||
final AtomicReference<Exception> lastException = new AtomicReference<Exception>();
|
||||
|
@ -159,10 +163,9 @@ public class RecoverySource extends AbstractComponent {
|
|||
indexInput.readBytes(buf, 0, toRead, false);
|
||||
BytesArray content = new BytesArray(buf, 0, toRead);
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILE_CHUNK, new RecoveryFileChunkRequest(request.recoveryId(), request.shardId(), name, position, len, md.checksum(), content),
|
||||
TransportRequestOptions.options().withCompress(shouldCompressRequest).withLowType(), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
TransportRequestOptions.options().withCompress(shouldCompressRequest).withLowType().withTimeout(internalActionTimeout), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
readCount += toRead;
|
||||
}
|
||||
indexInput.close();
|
||||
} catch (Exception e) {
|
||||
lastException.set(e);
|
||||
} finally {
|
||||
|
@ -187,7 +190,7 @@ public class RecoverySource extends AbstractComponent {
|
|||
|
||||
// now, set the clean files request
|
||||
Set<String> snapshotFiles = Sets.newHashSet(snapshot.getFiles());
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), snapshotFiles), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), snapshotFiles), TransportRequestOptions.options().withTimeout(internalActionTimeout), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
|
||||
stopWatch.stop();
|
||||
logger.trace("[{}][{}] recovery [phase1] to {}: took [{}]", request.shardId().index().name(), request.shardId().id(), request.targetNode(), stopWatch.totalTime());
|
||||
|
@ -204,7 +207,7 @@ public class RecoverySource extends AbstractComponent {
|
|||
}
|
||||
logger.trace("[{}][{}] recovery [phase2] to {}: start", request.shardId().index().name(), request.shardId().id(), request.targetNode());
|
||||
StopWatch stopWatch = new StopWatch().start();
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.PREPARE_TRANSLOG, new RecoveryPrepareForTranslogOperationsRequest(request.recoveryId(), request.shardId()), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.PREPARE_TRANSLOG, new RecoveryPrepareForTranslogOperationsRequest(request.recoveryId(), request.shardId()), TransportRequestOptions.options().withTimeout(internalActionTimeout), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
stopWatch.stop();
|
||||
response.startTime = stopWatch.totalTime().millis();
|
||||
logger.trace("[{}][{}] recovery [phase2] to {}: start took [{}]", request.shardId().index().name(), request.shardId().id(), request.targetNode(), stopWatch.totalTime());
|
||||
|
@ -226,7 +229,7 @@ public class RecoverySource extends AbstractComponent {
|
|||
logger.trace("[{}][{}] recovery [phase3] to {}: sending transaction log operations", request.shardId().index().name(), request.shardId().id(), request.targetNode());
|
||||
StopWatch stopWatch = new StopWatch().start();
|
||||
int totalOperations = sendSnapshot(snapshot);
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FINALIZE, new RecoveryFinalizeRecoveryRequest(request.recoveryId(), request.shardId()), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FINALIZE, new RecoveryFinalizeRecoveryRequest(request.recoveryId(), request.shardId()), TransportRequestOptions.options().withTimeout(internalActionLongTimeout), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
if (request.markAsRelocated()) {
|
||||
// TODO what happens if the recovery process fails afterwards, we need to mark this back to started
|
||||
try {
|
||||
|
@ -264,7 +267,7 @@ public class RecoverySource extends AbstractComponent {
|
|||
}
|
||||
|
||||
RecoveryTranslogOperationsRequest translogOperationsRequest = new RecoveryTranslogOperationsRequest(request.recoveryId(), request.shardId(), operations);
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest, TransportRequestOptions.options().withCompress(recoverySettings.compress()).withLowType(), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest, TransportRequestOptions.options().withCompress(recoverySettings.compress()).withLowType().withTimeout(internalActionLongTimeout), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
ops = 0;
|
||||
size = 0;
|
||||
operations.clear();
|
||||
|
@ -273,7 +276,7 @@ public class RecoverySource extends AbstractComponent {
|
|||
// send the leftover
|
||||
if (!operations.isEmpty()) {
|
||||
RecoveryTranslogOperationsRequest translogOperationsRequest = new RecoveryTranslogOperationsRequest(request.recoveryId(), request.shardId(), operations);
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest, TransportRequestOptions.options().withCompress(recoverySettings.compress()).withLowType(), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest, TransportRequestOptions.options().withCompress(recoverySettings.compress()).withLowType().withTimeout(internalActionLongTimeout), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
}
|
||||
return totalOperations;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue