void sendFiles(
     Store store,
     StoreFileMetaData[] files,
     Function<StoreFileMetaData, OutputStream> outputStreamFactory)
     throws Exception {
   store.incRef();
   try {
     ArrayUtil.timSort(
         files, (a, b) -> Long.compare(a.length(), b.length())); // send smallest first
     for (int i = 0; i < files.length; i++) {
       final StoreFileMetaData md = files[i];
       try (final IndexInput indexInput =
           store.directory().openInput(md.name(), IOContext.READONCE)) {
         // it's fine that we are only having the indexInput in the try/with block. The copy
         // methods handles
         // exceptions during close correctly and doesn't hide the original exception.
         Streams.copy(
             new InputStreamIndexInput(indexInput, md.length()), outputStreamFactory.apply(md));
       } catch (Exception e) {
         final IOException corruptIndexException;
         if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) {
           if (store.checkIntegrityNoException(md)
               == false) { // we are corrupted on the primary -- fail!
             logger.warn("{} Corrupted file detected {} checksum mismatch", shardId, md);
             failEngine(corruptIndexException);
             throw corruptIndexException;
           } 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);
             logger.warn(
                 (org.apache.logging.log4j.util.Supplier<?>)
                     () ->
                         new ParameterizedMessage(
                             "{} Remote file corruption on node {}, recovering {}. local checksum OK",
                             shardId,
                             request.targetNode(),
                             md),
                 corruptIndexException);
             throw exception;
           }
         } else {
           throw e;
         }
       }
     }
   } finally {
     store.decRef();
   }
 }
  public void testHandleExceptinoOnSendSendFiles() 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());
    Path tempDir = createTempDir();
    Store store = newStore(tempDir, false);
    AtomicBoolean failedEngine = new AtomicBoolean(false);
    RecoverySourceHandler handler =
        new RecoverySourceHandler(null, request, recoverySettings, null, logger) {
          @Override
          protected void failEngine(IOException cause) {
            assertFalse(failedEngine.get());
            failedEngine.set(true);
          }
        };
    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();
    writer.close();

    Store.MetadataSnapshot metadata = store.getMetadata();
    List<StoreFileMetaData> metas = new ArrayList<>();
    for (StoreFileMetaData md : metadata) {
      metas.add(md);
    }
    final boolean throwCorruptedIndexException = randomBoolean();
    Store targetStore = newStore(createTempDir(), false);
    try {
      handler.sendFiles(
          store,
          metas.toArray(new StoreFileMetaData[0]),
          (md) -> {
            if (throwCorruptedIndexException) {
              throw new RuntimeException(new CorruptIndexException("foo", "bar"));
            } else {
              throw new RuntimeException("boom");
            }
          });
      fail("exception index");
    } catch (RuntimeException ex) {
      assertNull(ExceptionsHelper.unwrapCorruption(ex));
      if (throwCorruptedIndexException) {
        assertEquals(
            ex.getMessage(), "[File corruption occurred on recovery but checksums are ok]");
      } else {
        assertEquals(ex.getMessage(), "boom");
      }
    } catch (CorruptIndexException ex) {
      fail("not expected here");
    }
    assertFalse(failedEngine.get());
    IOUtils.close(store, targetStore);
  }
  public void testHandleCorruptedIndexOnSendSendFiles() 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());
    Path tempDir = createTempDir();
    Store store = newStore(tempDir, false);
    AtomicBoolean failedEngine = new AtomicBoolean(false);
    RecoverySourceHandler handler =
        new RecoverySourceHandler(null, request, recoverySettings, null, logger) {
          @Override
          protected void failEngine(IOException cause) {
            assertFalse(failedEngine.get());
            failedEngine.set(true);
          }
        };
    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();
    writer.close();

    Store.MetadataSnapshot metadata = store.getMetadata();
    List<StoreFileMetaData> metas = new ArrayList<>();
    for (StoreFileMetaData md : metadata) {
      metas.add(md);
    }

    CorruptionUtils.corruptFile(
        getRandom(),
        FileSystemUtils.files(
            tempDir,
            (p) ->
                (p.getFileName().toString().equals("write.lock")
                        || p.getFileName().toString().startsWith("extra"))
                    == false));
    Store targetStore = newStore(createTempDir(), false);
    try {
      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);
            }
          });
      fail("corrupted index");
    } catch (IOException ex) {
      assertNotNull(ExceptionsHelper.unwrapCorruption(ex));
    }
    assertTrue(failedEngine.get());
    IOUtils.close(store, targetStore);
  }
  /**
   * 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();
    }
  }