|  | @@ -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));
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 |