@Override
 public void writeTo(StreamOutput out) throws IOException {
   super.writeTo(out);
   out.writeLong(recoveryId);
   shardId.writeTo(out);
   if (out.getVersion().onOrAfter(Version.V_1_5_0)) {
     snapshotFiles.writeTo(out);
   } else {
     out.writeVInt(snapshotFiles.size());
     for (StoreFileMetaData snapshotFile : snapshotFiles) {
       out.writeString(snapshotFile.name());
     }
   }
 }
 @Override
 public void readFrom(StreamInput in) throws IOException {
   super.readFrom(in);
   recoveryId = in.readLong();
   shardId = ShardId.readShardId(in);
   if (in.getVersion().onOrAfter(Version.V_1_5_0)) {
     snapshotFiles = Store.MetadataSnapshot.read(in);
   } else {
     int size = in.readVInt();
     legacySnapshotFiles = Sets.newHashSetWithExpectedSize(size);
     for (int i = 0; i < size; i++) {
       legacySnapshotFiles.add(in.readString());
     }
   }
 }
  private void doRecovery(final RecoveryTarget recoveryTarget) {
    assert recoveryTarget.sourceNode() != null : "can't do a recovery without a source node";

    logger.trace("collecting local files for {}", recoveryTarget);
    Store.MetadataSnapshot metadataSnapshot = null;
    try {
      if (recoveryTarget.indexShard().indexSettings().isOnSharedFilesystem()) {
        // we are not going to copy any files, so don't bother listing files, potentially running
        // into concurrency issues with the primary changing files underneath us.
        metadataSnapshot = Store.MetadataSnapshot.EMPTY;
      } else {
        metadataSnapshot = recoveryTarget.indexShard().snapshotStoreMetadata();
      }
    } catch (org.apache.lucene.index.IndexNotFoundException e) {
      // happens on an empty folder. no need to log
      metadataSnapshot = Store.MetadataSnapshot.EMPTY;
    } catch (IOException e) {
      logger.warn("error while listing local files, recover as if there are none", e);
      metadataSnapshot = Store.MetadataSnapshot.EMPTY;
    } catch (Exception e) {
      // this will be logged as warning later on...
      logger.trace("unexpected error while listing local files, failing recovery", e);
      onGoingRecoveries.failRecovery(
          recoveryTarget.recoveryId(),
          new RecoveryFailedException(recoveryTarget.state(), "failed to list local files", e),
          true);
      return;
    }
    logger.trace("{} local file count: [{}]", recoveryTarget, metadataSnapshot.size());
    final StartRecoveryRequest request =
        new StartRecoveryRequest(
            recoveryTarget.shardId(),
            recoveryTarget.sourceNode(),
            clusterService.localNode(),
            metadataSnapshot,
            recoveryTarget.state().getPrimary(),
            recoveryTarget.recoveryId());

    final AtomicReference<RecoveryResponse> responseHolder = new AtomicReference<>();
    try {
      logger.trace(
          "[{}][{}] starting recovery from {}",
          request.shardId().getIndex().getName(),
          request.shardId().id(),
          request.sourceNode());
      recoveryTarget.indexShard().prepareForIndexRecovery();
      recoveryTarget
          .CancellableThreads()
          .execute(
              () ->
                  responseHolder.set(
                      transportService
                          .submitRequest(
                              request.sourceNode(),
                              PeerRecoverySourceService.Actions.START_RECOVERY,
                              request,
                              new FutureTransportResponseHandler<RecoveryResponse>() {
                                @Override
                                public RecoveryResponse newInstance() {
                                  return new RecoveryResponse();
                                }
                              })
                          .txGet()));
      final RecoveryResponse recoveryResponse = responseHolder.get();
      assert responseHolder != null;
      final TimeValue recoveryTime = new TimeValue(recoveryTarget.state().getTimer().time());
      // do this through ongoing recoveries to remove it from the collection
      onGoingRecoveries.markRecoveryAsDone(recoveryTarget.recoveryId());
      if (logger.isTraceEnabled()) {
        StringBuilder sb = new StringBuilder();
        sb.append('[')
            .append(request.shardId().getIndex().getName())
            .append(']')
            .append('[')
            .append(request.shardId().id())
            .append("] ");
        sb.append("recovery completed from ")
            .append(request.sourceNode())
            .append(", took[")
            .append(recoveryTime)
            .append("]\n");
        sb.append("   phase1: recovered_files [")
            .append(recoveryResponse.phase1FileNames.size())
            .append("]")
            .append(" with " + "total_size of [")
            .append(new ByteSizeValue(recoveryResponse.phase1TotalSize))
            .append("]")
            .append(", took [")
            .append(timeValueMillis(recoveryResponse.phase1Time))
            .append("], throttling_wait [")
            .append(timeValueMillis(recoveryResponse.phase1ThrottlingWaitTime))
            .append(']')
            .append("\n");
        sb.append("         : reusing_files   [")
            .append(recoveryResponse.phase1ExistingFileNames.size())
            .append("] with " + "total_size of [")
            .append(new ByteSizeValue(recoveryResponse.phase1ExistingTotalSize))
            .append("]\n");
        sb.append("   phase2: start took [")
            .append(timeValueMillis(recoveryResponse.startTime))
            .append("]\n");
        sb.append("         : recovered [")
            .append(recoveryResponse.phase2Operations)
            .append("]")
            .append(" transaction log " + "operations")
            .append(", took [")
            .append(timeValueMillis(recoveryResponse.phase2Time))
            .append("]")
            .append("\n");
        logger.trace("{}", sb);
      } else {
        logger.debug(
            "{} recovery done from [{}], took [{}]",
            request.shardId(),
            recoveryTarget.sourceNode(),
            recoveryTime);
      }
    } catch (CancellableThreads.ExecutionCancelledException e) {
      logger.trace("recovery cancelled", e);
    } catch (Exception e) {
      if (logger.isTraceEnabled()) {
        logger.trace(
            (Supplier<?>)
                () ->
                    new ParameterizedMessage(
                        "[{}][{}] Got exception on recovery",
                        request.shardId().getIndex().getName(),
                        request.shardId().id()),
            e);
      }
      Throwable cause = ExceptionsHelper.unwrapCause(e);
      if (cause instanceof CancellableThreads.ExecutionCancelledException) {
        // this can also come from the source wrapped in a RemoteTransportException
        onGoingRecoveries.failRecovery(
            recoveryTarget.recoveryId(),
            new RecoveryFailedException(request, "source has canceled the" + " recovery", cause),
            false);
        return;
      }
      if (cause instanceof RecoveryEngineException) {
        // unwrap an exception that was thrown as part of the recovery
        cause = cause.getCause();
      }
      // do it twice, in case we have double transport exception
      cause = ExceptionsHelper.unwrapCause(cause);
      if (cause instanceof RecoveryEngineException) {
        // unwrap an exception that was thrown as part of the recovery
        cause = cause.getCause();
      }

      // here, we would add checks against exception that need to be retried (and not removeAndClean
      // in this case)

      if (cause instanceof IllegalIndexShardStateException
          || cause instanceof IndexNotFoundException
          || cause instanceof ShardNotFoundException) {
        // if the target is not ready yet, retry
        retryRecovery(
            recoveryTarget,
            "remote shard not ready",
            recoverySettings.retryDelayStateSync(),
            request);
        return;
      }

      if (cause instanceof DelayRecoveryException) {
        retryRecovery(recoveryTarget, cause, recoverySettings.retryDelayStateSync(), request);
        return;
      }

      if (cause instanceof ConnectTransportException) {
        logger.debug(
            "delaying recovery of {} for [{}] due to networking error [{}]",
            recoveryTarget.shardId(),
            recoverySettings.retryDelayNetwork(),
            cause.getMessage());
        retryRecovery(
            recoveryTarget, cause.getMessage(), recoverySettings.retryDelayNetwork(), request);
        return;
      }

      if (cause instanceof AlreadyClosedException) {
        onGoingRecoveries.failRecovery(
            recoveryTarget.recoveryId(),
            new RecoveryFailedException(request, "source shard is " + "closed", cause),
            false);
        return;
      }
      onGoingRecoveries.failRecovery(
          recoveryTarget.recoveryId(), new RecoveryFailedException(request, e), true);
    }
  }
 public void testSendFiles() throws Throwable {
   Settings settings =
       Settings.builder()
           .put("indices.recovery.concurrent_streams", 1)
           .put("indices.recovery.concurrent_small_file_streams", 1)
           .build();
   final RecoverySettings recoverySettings = new RecoverySettings(settings, service);
   StartRecoveryRequest request =
       new StartRecoveryRequest(
           shardId,
           new DiscoveryNode("b", DummyTransportAddress.INSTANCE, Version.CURRENT),
           new DiscoveryNode("b", DummyTransportAddress.INSTANCE, Version.CURRENT),
           null,
           RecoveryState.Type.STORE,
           randomLong());
   Store store = newStore(createTempDir());
   RecoverySourceHandler handler =
       new RecoverySourceHandler(null, request, recoverySettings, null, logger);
   Directory dir = store.directory();
   RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
   int numDocs = randomIntBetween(10, 100);
   for (int i = 0; i < numDocs; i++) {
     Document document = new Document();
     document.add(new StringField("id", Integer.toString(i), Field.Store.YES));
     document.add(
         newField("field", randomUnicodeOfCodepointLengthBetween(1, 10), TextField.TYPE_STORED));
     writer.addDocument(document);
   }
   writer.commit();
   Store.MetadataSnapshot metadata = store.getMetadata();
   List<StoreFileMetaData> metas = new ArrayList<>();
   for (StoreFileMetaData md : metadata) {
     metas.add(md);
   }
   Store targetStore = newStore(createTempDir());
   handler.sendFiles(
       store,
       metas.toArray(new StoreFileMetaData[0]),
       (md) -> {
         try {
           return new IndexOutputOutputStream(
               targetStore.createVerifyingOutput(md.name(), md, IOContext.DEFAULT)) {
             @Override
             public void close() throws IOException {
               super.close();
               store
                   .directory()
                   .sync(Collections.singleton(md.name())); // sync otherwise MDW will mess with it
             }
           };
         } catch (IOException e) {
           throw new RuntimeException(e);
         }
       });
   Store.MetadataSnapshot targetStoreMetadata = targetStore.getMetadata();
   Store.RecoveryDiff recoveryDiff = targetStoreMetadata.recoveryDiff(metadata);
   assertEquals(metas.size(), recoveryDiff.identical.size());
   assertEquals(0, recoveryDiff.different.size());
   assertEquals(0, recoveryDiff.missing.size());
   IndexReader reader = DirectoryReader.open(targetStore.directory());
   assertEquals(numDocs, reader.maxDoc());
   IOUtils.close(reader, writer, store, targetStore);
 }
  /**
   * Perform phase1 of the recovery operations. Once this {@link SnapshotIndexCommit} snapshot has
   * been performed no commit operations (files being fsync'd) are effectively allowed on this index
   * until all recovery phases are done
   *
   * <p>Phase1 examines the segment files on the target node and copies over the segments that are
   * missing. Only segments that have the same size and checksum can be reused
   *
   * <p>{@code InternalEngine#recover} is responsible for snapshotting the index and releasing the
   * snapshot once all 3 phases of recovery are complete
   */
  @Override
  public void phase1(final SnapshotIndexCommit snapshot) throws ElasticsearchException {
    cancellableThreads.checkForCancel();
    // Total size of segment files that are recovered
    long totalSize = 0;
    // Total size of segment files that were able to be re-used
    long existingTotalSize = 0;
    final Store store = shard.store();
    store.incRef();
    try {
      StopWatch stopWatch = new StopWatch().start();
      final Store.MetadataSnapshot recoverySourceMetadata = store.getMetadata(snapshot);
      for (String name : snapshot.getFiles()) {
        final StoreFileMetaData md = recoverySourceMetadata.get(name);
        if (md == null) {
          logger.info(
              "Snapshot differs from actual index for file: {} meta: {}",
              name,
              recoverySourceMetadata.asMap());
          throw new CorruptIndexException(
              "Snapshot differs from actual index - maybe index was removed metadata has "
                  + recoverySourceMetadata.asMap().size()
                  + " files");
        }
      }
      String recoverySourceSyncId = recoverySourceMetadata.getSyncId();
      String recoveryTargetSyncId = request.metadataSnapshot().getSyncId();
      final boolean recoverWithSyncId =
          recoverySourceSyncId != null && recoverySourceSyncId.equals(recoveryTargetSyncId);
      if (recoverWithSyncId) {
        final long numDocsTarget = request.metadataSnapshot().getNumDocs();
        final long numDocsSource = recoverySourceMetadata.getNumDocs();
        if (numDocsTarget != numDocsSource) {
          throw new IllegalStateException(
              "try to recover "
                  + request.shardId()
                  + " from primary shard with sync id but number of docs differ: "
                  + numDocsTarget
                  + " ("
                  + request.sourceNode().getName()
                  + ", primary) vs "
                  + numDocsSource
                  + "("
                  + request.targetNode().getName()
                  + ")");
        }
        // we shortcut recovery here because we have nothing to copy. but we must still start the
        // engine on the target.
        // so we don't return here
        logger.trace(
            "[{}][{}] skipping [phase1] to {} - identical sync id [{}] found on both source and target",
            indexName,
            shardId,
            request.targetNode(),
            recoverySourceSyncId);
      } else {

        // Generate a "diff" of all the identical, different, and missing
        // segment files on the target node, using the existing files on
        // the source node
        final Store.RecoveryDiff diff =
            recoverySourceMetadata.recoveryDiff(request.metadataSnapshot());
        for (StoreFileMetaData md : diff.identical) {
          response.phase1ExistingFileNames.add(md.name());
          response.phase1ExistingFileSizes.add(md.length());
          existingTotalSize += md.length();
          if (logger.isTraceEnabled()) {
            logger.trace(
                "[{}][{}] recovery [phase1] to {}: not recovering [{}], exists in local store and has checksum [{}], size [{}]",
                indexName,
                shardId,
                request.targetNode(),
                md.name(),
                md.checksum(),
                md.length());
          }
          totalSize += md.length();
        }
        for (StoreFileMetaData md : Iterables.concat(diff.different, diff.missing)) {
          if (request.metadataSnapshot().asMap().containsKey(md.name())) {
            logger.trace(
                "[{}][{}] recovery [phase1] to {}: recovering [{}], exists in local store, but is different: remote [{}], local [{}]",
                indexName,
                shardId,
                request.targetNode(),
                md.name(),
                request.metadataSnapshot().get(md.name()),
                md);
          } else {
            logger.trace(
                "[{}][{}] recovery [phase1] to {}: recovering [{}], does not exists in remote",
                indexName,
                shardId,
                request.targetNode(),
                md.name());
          }
          response.phase1FileNames.add(md.name());
          response.phase1FileSizes.add(md.length());
          totalSize += md.length();
        }
        response.phase1TotalSize = totalSize;
        response.phase1ExistingTotalSize = existingTotalSize;

        logger.trace(
            "[{}][{}] recovery [phase1] to {}: recovering_files [{}] with total_size [{}], reusing_files [{}] with total_size [{}]",
            indexName,
            shardId,
            request.targetNode(),
            response.phase1FileNames.size(),
            new ByteSizeValue(totalSize),
            response.phase1ExistingFileNames.size(),
            new ByteSizeValue(existingTotalSize));
        cancellableThreads.execute(
            new Interruptable() {
              @Override
              public void run() throws InterruptedException {
                RecoveryFilesInfoRequest recoveryInfoFilesRequest =
                    new RecoveryFilesInfoRequest(
                        request.recoveryId(),
                        request.shardId(),
                        response.phase1FileNames,
                        response.phase1FileSizes,
                        response.phase1ExistingFileNames,
                        response.phase1ExistingFileSizes,
                        shard.translog().estimatedNumberOfOperations(),
                        response.phase1TotalSize,
                        response.phase1ExistingTotalSize);
                transportService
                    .submitRequest(
                        request.targetNode(),
                        RecoveryTarget.Actions.FILES_INFO,
                        recoveryInfoFilesRequest,
                        TransportRequestOptions.options()
                            .withTimeout(recoverySettings.internalActionTimeout()),
                        EmptyTransportResponseHandler.INSTANCE_SAME)
                    .txGet();
              }
            });

        // This latch will be used to wait until all files have been transferred to the target node
        final CountDownLatch latch = new CountDownLatch(response.phase1FileNames.size());
        final CopyOnWriteArrayList<Throwable> exceptions = new CopyOnWriteArrayList<>();
        final AtomicReference<Throwable> corruptedEngine = new AtomicReference<>();
        int fileIndex = 0;
        ThreadPoolExecutor pool;

        // How many bytes we've copied since we last called RateLimiter.pause
        final AtomicLong bytesSinceLastPause = new AtomicLong();

        for (final String name : response.phase1FileNames) {
          long fileSize = response.phase1FileSizes.get(fileIndex);

          // Files are split into two categories, files that are "small"
          // (under 5mb) and other files. Small files are transferred
          // using a separate thread pool dedicated to small files.
          //
          // The idea behind this is that while we are transferring an
          // older, large index, a user may create a new index, but that
          // index will not be able to recover until the large index
          // finishes, by using two different thread pools we can allow
          // tiny files (like segments for a brand new index) to be
          // recovered while ongoing large segment recoveries are
          // happening. It also allows these pools to be configured
          // separately.
          if (fileSize > RecoverySettings.SMALL_FILE_CUTOFF_BYTES) {
            pool = recoverySettings.concurrentStreamPool();
          } else {
            pool = recoverySettings.concurrentSmallFileStreamPool();
          }

          pool.execute(
              new AbstractRunnable() {
                @Override
                public void onFailure(Throwable t) {
                  // we either got rejected or the store can't be incremented / we are canceled
                  logger.debug("Failed to transfer file [" + name + "] on recovery");
                }

                public void onAfter() {
                  // Signify this file has completed by decrementing the latch
                  latch.countDown();
                }

                @Override
                protected void doRun() {
                  cancellableThreads.checkForCancel();
                  store.incRef();
                  final StoreFileMetaData md = recoverySourceMetadata.get(name);
                  try (final IndexInput indexInput =
                      store.directory().openInput(name, IOContext.READONCE)) {
                    final int BUFFER_SIZE = (int) recoverySettings.fileChunkSize().bytes();
                    final byte[] buf = new byte[BUFFER_SIZE];
                    boolean shouldCompressRequest = recoverySettings.compress();
                    if (CompressorFactory.isCompressed(indexInput)) {
                      shouldCompressRequest = false;
                    }

                    final long len = indexInput.length();
                    long readCount = 0;
                    final TransportRequestOptions requestOptions =
                        TransportRequestOptions.options()
                            .withCompress(shouldCompressRequest)
                            .withType(TransportRequestOptions.Type.RECOVERY)
                            .withTimeout(recoverySettings.internalActionTimeout());

                    while (readCount < len) {
                      if (shard.state()
                          == IndexShardState.CLOSED) { // check if the shard got closed on us
                        throw new IndexShardClosedException(shard.shardId());
                      }
                      int toRead =
                          readCount + BUFFER_SIZE > len ? (int) (len - readCount) : BUFFER_SIZE;
                      final long position = indexInput.getFilePointer();

                      // Pause using the rate limiter, if desired, to throttle the recovery
                      RateLimiter rl = recoverySettings.rateLimiter();
                      long throttleTimeInNanos = 0;
                      if (rl != null) {
                        long bytes = bytesSinceLastPause.addAndGet(toRead);
                        if (bytes > rl.getMinPauseCheckBytes()) {
                          // Time to pause
                          bytesSinceLastPause.addAndGet(-bytes);
                          throttleTimeInNanos = rl.pause(bytes);
                          shard.recoveryStats().addThrottleTime(throttleTimeInNanos);
                        }
                      }
                      indexInput.readBytes(buf, 0, toRead, false);
                      final BytesArray content = new BytesArray(buf, 0, toRead);
                      readCount += toRead;
                      final boolean lastChunk = readCount == len;
                      final RecoveryFileChunkRequest fileChunkRequest =
                          new RecoveryFileChunkRequest(
                              request.recoveryId(),
                              request.shardId(),
                              md,
                              position,
                              content,
                              lastChunk,
                              shard.translog().estimatedNumberOfOperations(),
                              throttleTimeInNanos);
                      cancellableThreads.execute(
                          new Interruptable() {
                            @Override
                            public void run() throws InterruptedException {
                              // Actually send the file chunk to the target node, waiting for it to
                              // complete
                              transportService
                                  .submitRequest(
                                      request.targetNode(),
                                      RecoveryTarget.Actions.FILE_CHUNK,
                                      fileChunkRequest,
                                      requestOptions,
                                      EmptyTransportResponseHandler.INSTANCE_SAME)
                                  .txGet();
                            }
                          });
                    }
                  } catch (Throwable e) {
                    final Throwable corruptIndexException;
                    if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) {
                      if (store.checkIntegrity(md)
                          == false) { // we are corrupted on the primary -- fail!
                        logger.warn(
                            "{} Corrupted file detected {} checksum mismatch", shard.shardId(), md);
                        if (corruptedEngine.compareAndSet(null, corruptIndexException) == false) {
                          // if we are not the first exception, add ourselves as suppressed to the
                          // main one:
                          corruptedEngine.get().addSuppressed(e);
                        }
                      } else { // corruption has happened on the way to replica
                        RemoteTransportException exception =
                            new RemoteTransportException(
                                "File corruption occurred on recovery but checksums are ok", null);
                        exception.addSuppressed(e);
                        exceptions.add(0, exception); // last exception first
                        logger.warn(
                            "{} Remote file corruption on node {}, recovering {}. local checksum OK",
                            corruptIndexException,
                            shard.shardId(),
                            request.targetNode(),
                            md);
                      }
                    } else {
                      exceptions.add(0, e); // last exceptions first
                    }
                  } finally {
                    store.decRef();
                  }
                }
              });
          fileIndex++;
        }

        cancellableThreads.execute(
            new Interruptable() {
              @Override
              public void run() throws InterruptedException {
                // Wait for all files that need to be transferred to finish transferring
                latch.await();
              }
            });

        if (corruptedEngine.get() != null) {
          throw corruptedEngine.get();
        } else {
          ExceptionsHelper.rethrowAndSuppress(exceptions);
        }

        cancellableThreads.execute(
            new Interruptable() {
              @Override
              public void run() throws InterruptedException {
                // Send the CLEAN_FILES request, which takes all of the files that
                // were transferred and renames them from their temporary file
                // names to the actual file names. It also writes checksums for
                // the files after they have been renamed.
                //
                // Once the files have been renamed, any other files that are not
                // related to this recovery (out of date segments, for example)
                // are deleted
                try {
                  transportService
                      .submitRequest(
                          request.targetNode(),
                          RecoveryTarget.Actions.CLEAN_FILES,
                          new RecoveryCleanFilesRequest(
                              request.recoveryId(),
                              shard.shardId(),
                              recoverySourceMetadata,
                              shard.translog().estimatedNumberOfOperations()),
                          TransportRequestOptions.options()
                              .withTimeout(recoverySettings.internalActionTimeout()),
                          EmptyTransportResponseHandler.INSTANCE_SAME)
                      .txGet();
                } catch (RemoteTransportException remoteException) {
                  final IOException corruptIndexException;
                  // we realized that after the index was copied and we wanted to finalize the
                  // recovery
                  // the index was corrupted:
                  //   - maybe due to a broken segments file on an empty index (transferred with no
                  // checksum)
                  //   - maybe due to old segments without checksums or length only checks
                  if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(remoteException))
                      != null) {
                    try {
                      final Store.MetadataSnapshot recoverySourceMetadata =
                          store.getMetadata(snapshot);
                      StoreFileMetaData[] metadata =
                          Iterables.toArray(recoverySourceMetadata, StoreFileMetaData.class);
                      ArrayUtil.timSort(
                          metadata,
                          new Comparator<StoreFileMetaData>() {
                            @Override
                            public int compare(StoreFileMetaData o1, StoreFileMetaData o2) {
                              return Long.compare(
                                  o1.length(), o2.length()); // check small files first
                            }
                          });
                      for (StoreFileMetaData md : metadata) {
                        logger.debug(
                            "{} checking integrity for file {} after remove corruption exception",
                            shard.shardId(),
                            md);
                        if (store.checkIntegrity(md)
                            == false) { // we are corrupted on the primary -- fail!
                          logger.warn(
                              "{} Corrupted file detected {} checksum mismatch",
                              shard.shardId(),
                              md);
                          throw corruptIndexException;
                        }
                      }
                    } catch (IOException ex) {
                      remoteException.addSuppressed(ex);
                      throw remoteException;
                    }
                    // corruption has happened on the way to replica
                    RemoteTransportException exception =
                        new RemoteTransportException(
                            "File corruption occurred on recovery but checksums are ok", null);
                    exception.addSuppressed(remoteException);
                    logger.warn(
                        "{} Remote file corruption during finalization on node {}, recovering {}. local checksum OK",
                        corruptIndexException,
                        shard.shardId(),
                        request.targetNode());
                    throw exception;
                  } else {
                    throw remoteException;
                  }
                }
              }
            });
      }
      stopWatch.stop();
      logger.trace(
          "[{}][{}] recovery [phase1] to {}: took [{}]",
          indexName,
          shardId,
          request.targetNode(),
          stopWatch.totalTime());
      response.phase1Time = stopWatch.totalTime().millis();
    } catch (Throwable e) {
      throw new RecoverFilesRecoveryException(
          request.shardId(), response.phase1FileNames.size(), new ByteSizeValue(totalSize), e);
    } finally {
      store.decRef();
    }
  }
  /**
   * Perform phase1 of the recovery operations. Once this {@link IndexCommit} snapshot has been
   * performed no commit operations (files being fsync'd) are effectively allowed on this index
   * until all recovery phases are done
   *
   * <p>Phase1 examines the segment files on the target node and copies over the segments that are
   * missing. Only segments that have the same size and checksum can be reused
   */
  public void phase1(final IndexCommit snapshot, final Translog.View translogView) {
    cancellableThreads.checkForCancel();
    // Total size of segment files that are recovered
    long totalSize = 0;
    // Total size of segment files that were able to be re-used
    long existingTotalSize = 0;
    final Store store = shard.store();
    store.incRef();
    try {
      StopWatch stopWatch = new StopWatch().start();
      final Store.MetadataSnapshot recoverySourceMetadata;
      try {
        recoverySourceMetadata = store.getMetadata(snapshot);
      } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) {
        shard.failShard("recovery", ex);
        throw ex;
      }
      for (String name : snapshot.getFileNames()) {
        final StoreFileMetaData md = recoverySourceMetadata.get(name);
        if (md == null) {
          logger.info(
              "Snapshot differs from actual index for file: {} meta: {}",
              name,
              recoverySourceMetadata.asMap());
          throw new CorruptIndexException(
              "Snapshot differs from actual index - maybe index was removed metadata has "
                  + recoverySourceMetadata.asMap().size()
                  + " files",
              name);
        }
      }
      // Generate a "diff" of all the identical, different, and missing
      // segment files on the target node, using the existing files on
      // the source node
      String recoverySourceSyncId = recoverySourceMetadata.getSyncId();
      String recoveryTargetSyncId = request.metadataSnapshot().getSyncId();
      final boolean recoverWithSyncId =
          recoverySourceSyncId != null && recoverySourceSyncId.equals(recoveryTargetSyncId);
      if (recoverWithSyncId) {
        final long numDocsTarget = request.metadataSnapshot().getNumDocs();
        final long numDocsSource = recoverySourceMetadata.getNumDocs();
        if (numDocsTarget != numDocsSource) {
          throw new IllegalStateException(
              "try to recover "
                  + request.shardId()
                  + " from primary shard with sync id but number "
                  + "of docs differ: "
                  + numDocsTarget
                  + " ("
                  + request.sourceNode().getName()
                  + ", primary) vs "
                  + numDocsSource
                  + "("
                  + request.targetNode().getName()
                  + ")");
        }
        // we shortcut recovery here because we have nothing to copy. but we must still start the
        // engine on the target.
        // so we don't return here
        logger.trace(
            "[{}][{}] skipping [phase1] to {} - identical sync id [{}] found on both source and target",
            indexName,
            shardId,
            request.targetNode(),
            recoverySourceSyncId);
      } else {
        final Store.RecoveryDiff diff =
            recoverySourceMetadata.recoveryDiff(request.metadataSnapshot());
        for (StoreFileMetaData md : diff.identical) {
          response.phase1ExistingFileNames.add(md.name());
          response.phase1ExistingFileSizes.add(md.length());
          existingTotalSize += md.length();
          if (logger.isTraceEnabled()) {
            logger.trace(
                "[{}][{}] recovery [phase1] to {}: not recovering [{}], exists in local store and has checksum [{}],"
                    + " size [{}]",
                indexName,
                shardId,
                request.targetNode(),
                md.name(),
                md.checksum(),
                md.length());
          }
          totalSize += md.length();
        }
        List<StoreFileMetaData> phase1Files =
            new ArrayList<>(diff.different.size() + diff.missing.size());
        phase1Files.addAll(diff.different);
        phase1Files.addAll(diff.missing);
        for (StoreFileMetaData md : phase1Files) {
          if (request.metadataSnapshot().asMap().containsKey(md.name())) {
            logger.trace(
                "[{}][{}] recovery [phase1] to {}: recovering [{}], exists in local store, but is different: remote "
                    + "[{}], local [{}]",
                indexName,
                shardId,
                request.targetNode(),
                md.name(),
                request.metadataSnapshot().asMap().get(md.name()),
                md);
          } else {
            logger.trace(
                "[{}][{}] recovery [phase1] to {}: recovering [{}], does not exists in remote",
                indexName,
                shardId,
                request.targetNode(),
                md.name());
          }
          response.phase1FileNames.add(md.name());
          response.phase1FileSizes.add(md.length());
          totalSize += md.length();
        }

        response.phase1TotalSize = totalSize;
        response.phase1ExistingTotalSize = existingTotalSize;

        logger.trace(
            "[{}][{}] recovery [phase1] to {}: recovering_files [{}] with total_size [{}], reusing_files [{}] with "
                + "total_size [{}]",
            indexName,
            shardId,
            request.targetNode(),
            response.phase1FileNames.size(),
            new ByteSizeValue(totalSize),
            response.phase1ExistingFileNames.size(),
            new ByteSizeValue(existingTotalSize));
        cancellableThreads.execute(
            () ->
                recoveryTarget.receiveFileInfo(
                    response.phase1FileNames,
                    response.phase1FileSizes,
                    response.phase1ExistingFileNames,
                    response.phase1ExistingFileSizes,
                    translogView.totalOperations()));
        // How many bytes we've copied since we last called RateLimiter.pause
        final Function<StoreFileMetaData, OutputStream> outputStreamFactories =
            md ->
                new BufferedOutputStream(
                    new RecoveryOutputStream(md, translogView), chunkSizeInBytes);
        sendFiles(
            store,
            phase1Files.toArray(new StoreFileMetaData[phase1Files.size()]),
            outputStreamFactories);
        // Send the CLEAN_FILES request, which takes all of the files that
        // were transferred and renames them from their temporary file
        // names to the actual file names. It also writes checksums for
        // the files after they have been renamed.
        //
        // Once the files have been renamed, any other files that are not
        // related to this recovery (out of date segments, for example)
        // are deleted
        try {
          cancellableThreads.executeIO(
              () ->
                  recoveryTarget.cleanFiles(
                      translogView.totalOperations(), recoverySourceMetadata));
        } catch (RemoteTransportException | IOException targetException) {
          final IOException corruptIndexException;
          // we realized that after the index was copied and we wanted to finalize the recovery
          // the index was corrupted:
          //   - maybe due to a broken segments file on an empty index (transferred with no
          // checksum)
          //   - maybe due to old segments without checksums or length only checks
          if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(targetException))
              != null) {
            try {
              final Store.MetadataSnapshot recoverySourceMetadata1 = store.getMetadata(snapshot);
              StoreFileMetaData[] metadata =
                  StreamSupport.stream(recoverySourceMetadata1.spliterator(), false)
                      .toArray(size -> new StoreFileMetaData[size]);
              ArrayUtil.timSort(
                  metadata,
                  (o1, o2) -> {
                    return Long.compare(o1.length(), o2.length()); // check small files first
                  });
              for (StoreFileMetaData md : metadata) {
                cancellableThreads.checkForCancel();
                logger.debug(
                    "{} checking integrity for file {} after remove corruption exception",
                    shard.shardId(),
                    md);
                if (store.checkIntegrityNoException(md)
                    == false) { // we are corrupted on the primary -- fail!
                  shard.failShard("recovery", corruptIndexException);
                  logger.warn(
                      "{} Corrupted file detected {} checksum mismatch", shard.shardId(), md);
                  throw corruptIndexException;
                }
              }
            } catch (IOException ex) {
              targetException.addSuppressed(ex);
              throw targetException;
            }
            // corruption has happened on the way to replica
            RemoteTransportException exception =
                new RemoteTransportException(
                    "File corruption occurred on recovery but " + "checksums are ok", null);
            exception.addSuppressed(targetException);
            logger.warn(
                (org.apache.logging.log4j.util.Supplier<?>)
                    () ->
                        new ParameterizedMessage(
                            "{} Remote file corruption during finalization of recovery on node {}. local checksum OK",
                            shard.shardId(),
                            request.targetNode()),
                corruptIndexException);
            throw exception;
          } else {
            throw targetException;
          }
        }
      }

      prepareTargetForTranslog(translogView.totalOperations());

      logger.trace(
          "[{}][{}] recovery [phase1] to {}: took [{}]",
          indexName,
          shardId,
          request.targetNode(),
          stopWatch.totalTime());
      response.phase1Time = stopWatch.totalTime().millis();
    } catch (Exception e) {
      throw new RecoverFilesRecoveryException(
          request.shardId(), response.phase1FileNames.size(), new ByteSizeValue(totalSize), e);
    } finally {
      store.decRef();
    }
  }
 /** @return commit sync id if exists, else null */
 public String syncId() {
   return metadataSnapshot.getSyncId();
 }
 @Override
 public void writeTo(StreamOutput out) throws IOException {
   out.writeBoolean(allocated);
   shardId.writeTo(out);
   metadataSnapshot.writeTo(out);
 }
 public StoreFileMetaData file(String name) {
   return metadataSnapshot.asMap().get(name);
 }
 public boolean fileExists(String name) {
   return metadataSnapshot.asMap().containsKey(name);
 }
 @Override
 public Iterator<StoreFileMetaData> iterator() {
   return metadataSnapshot.iterator();
 }