public static void main(String[] args) throws Exception {
    System.setProperty("es.logger.prefix", "");
    Natives.tryMlockall();

    Settings settings =
        settingsBuilder()
            .put("gateway.type", "local")
            .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, "false")
            .put(SETTING_NUMBER_OF_SHARDS, 1)
            .put(SETTING_NUMBER_OF_REPLICAS, 0)
            .put(TransportModule.TRANSPORT_TYPE_KEY, "local")
            .build();

    String clusterName = ReplicaRecoveryBenchmark.class.getSimpleName();
    Node node1 =
        nodeBuilder().clusterName(clusterName).settings(settingsBuilder().put(settings)).node();

    final ESLogger logger = ESLoggerFactory.getLogger("benchmark");

    final Client client1 = node1.client();
    client1
        .admin()
        .cluster()
        .prepareUpdateSettings()
        .setPersistentSettings("logger.indices.recovery: TRACE")
        .get();
    final BackgroundIndexer indexer =
        new BackgroundIndexer(
            INDEX_NAME, TYPE_NAME, client1, 0, CONCURRENT_INDEXERS, false, new Random());
    indexer.setMinFieldSize(10);
    indexer.setMaxFieldSize(150);
    try {
      client1.admin().indices().prepareDelete(INDEX_NAME).get();
    } catch (IndexMissingException e) {
    }
    client1.admin().indices().prepareCreate(INDEX_NAME).get();
    indexer.start(DOC_COUNT / 2);
    while (indexer.totalIndexedDocs() < DOC_COUNT / 2) {
      Thread.sleep(5000);
      logger.info("--> indexed {} of {}", indexer.totalIndexedDocs(), DOC_COUNT);
    }
    client1.admin().indices().prepareFlush().get();
    indexer.continueIndexing(DOC_COUNT / 2);
    while (indexer.totalIndexedDocs() < DOC_COUNT) {
      Thread.sleep(5000);
      logger.info("--> indexed {} of {}", indexer.totalIndexedDocs(), DOC_COUNT);
    }

    logger.info("--> starting another node and allocating a shard on it");

    Node node2 =
        nodeBuilder().clusterName(clusterName).settings(settingsBuilder().put(settings)).node();

    client1
        .admin()
        .indices()
        .prepareUpdateSettings(INDEX_NAME)
        .setSettings(IndexMetaData.SETTING_NUMBER_OF_REPLICAS + ": 1")
        .get();

    final AtomicBoolean end = new AtomicBoolean(false);

    final Thread backgroundLogger =
        new Thread(
            new Runnable() {

              long lastTime = System.currentTimeMillis();
              long lastDocs = indexer.totalIndexedDocs();
              long lastBytes = 0;
              long lastTranslogOps = 0;

              @Override
              public void run() {
                while (true) {
                  try {
                    Thread.sleep(5000);
                  } catch (InterruptedException e) {

                  }
                  if (end.get()) {
                    return;
                  }
                  long currentTime = System.currentTimeMillis();
                  long currentDocs = indexer.totalIndexedDocs();
                  RecoveryResponse recoveryResponse =
                      client1
                          .admin()
                          .indices()
                          .prepareRecoveries(INDEX_NAME)
                          .setActiveOnly(true)
                          .get();
                  List<ShardRecoveryResponse> indexRecoveries =
                      recoveryResponse.shardResponses().get(INDEX_NAME);
                  long translogOps;
                  long bytes;
                  if (indexRecoveries.size() > 0) {
                    translogOps =
                        indexRecoveries.get(0).recoveryState().getTranslog().recoveredOperations();
                    bytes =
                        recoveryResponse
                            .shardResponses()
                            .get(INDEX_NAME)
                            .get(0)
                            .recoveryState()
                            .getIndex()
                            .recoveredBytes();
                  } else {
                    bytes = lastBytes = 0;
                    translogOps = lastTranslogOps = 0;
                  }
                  float seconds = (currentTime - lastTime) / 1000.0F;
                  logger.info(
                      "--> indexed [{}];[{}] doc/s, recovered [{}] MB/s , translog ops [{}]/s ",
                      currentDocs,
                      (currentDocs - lastDocs) / seconds,
                      (bytes - lastBytes) / 1024.0F / 1024F / seconds,
                      (translogOps - lastTranslogOps) / seconds);
                  lastBytes = bytes;
                  lastTranslogOps = translogOps;
                  lastTime = currentTime;
                  lastDocs = currentDocs;
                }
              }
            });

    backgroundLogger.start();

    client1.admin().cluster().prepareHealth().setWaitForGreenStatus().get();

    logger.info("--> green. starting relocation cycles");

    long startDocIndexed = indexer.totalIndexedDocs();
    indexer.continueIndexing(DOC_COUNT * 50);

    long totalRecoveryTime = 0;
    long startTime = System.currentTimeMillis();
    long[] recoveryTimes = new long[3];
    for (int iteration = 0; iteration < 3; iteration++) {
      logger.info("--> removing replicas");
      client1
          .admin()
          .indices()
          .prepareUpdateSettings(INDEX_NAME)
          .setSettings(IndexMetaData.SETTING_NUMBER_OF_REPLICAS + ": 0")
          .get();
      logger.info("--> adding replica again");
      long recoveryStart = System.currentTimeMillis();
      client1
          .admin()
          .indices()
          .prepareUpdateSettings(INDEX_NAME)
          .setSettings(IndexMetaData.SETTING_NUMBER_OF_REPLICAS + ": 1")
          .get();
      client1
          .admin()
          .cluster()
          .prepareHealth(INDEX_NAME)
          .setWaitForGreenStatus()
          .setTimeout("15m")
          .get();
      long recoveryTime = System.currentTimeMillis() - recoveryStart;
      totalRecoveryTime += recoveryTime;
      recoveryTimes[iteration] = recoveryTime;
      logger.info("--> recovery done in [{}]", new TimeValue(recoveryTime));

      // sleep some to let things clean up
      Thread.sleep(10000);
    }

    long endDocIndexed = indexer.totalIndexedDocs();
    long totalTime = System.currentTimeMillis() - startTime;
    indexer.stop();

    end.set(true);

    backgroundLogger.interrupt();

    backgroundLogger.join();

    logger.info(
        "average doc/s [{}], average relocation time [{}], taking [{}], [{}], [{}]",
        (endDocIndexed - startDocIndexed) * 1000.0 / totalTime,
        new TimeValue(totalRecoveryTime / 3),
        TimeValue.timeValueMillis(recoveryTimes[0]),
        TimeValue.timeValueMillis(recoveryTimes[1]),
        TimeValue.timeValueMillis(recoveryTimes[2]));

    client1.close();
    node1.close();
    node2.close();
  }
  @Test
  public void testRelocationWhileIndexingRandom() throws Exception {
    int numberOfRelocations = scaledRandomIntBetween(1, rarely() ? 10 : 4);
    int numberOfReplicas = randomBoolean() ? 0 : 1;
    int numberOfNodes = numberOfReplicas == 0 ? 2 : 3;

    logger.info(
        "testRelocationWhileIndexingRandom(numRelocations={}, numberOfReplicas={}, numberOfNodes={})",
        numberOfRelocations,
        numberOfReplicas,
        numberOfNodes);

    String[] nodes = new String[numberOfNodes];
    logger.info("--> starting [node1] ...");
    nodes[0] = internalCluster().startNode();

    logger.info("--> creating test index ...");
    client()
        .admin()
        .indices()
        .prepareCreate("test")
        .setSettings(
            settingsBuilder()
                .put("index.number_of_shards", 1)
                .put("index.number_of_replicas", numberOfReplicas))
        .execute()
        .actionGet();

    for (int i = 1; i < numberOfNodes; i++) {
      logger.info("--> starting [node{}] ...", i + 1);
      nodes[i] = internalCluster().startNode();
      if (i != numberOfNodes - 1) {
        ClusterHealthResponse healthResponse =
            client()
                .admin()
                .cluster()
                .prepareHealth()
                .setWaitForEvents(Priority.LANGUID)
                .setWaitForNodes(Integer.toString(i + 1))
                .setWaitForGreenStatus()
                .execute()
                .actionGet();
        assertThat(healthResponse.isTimedOut(), equalTo(false));
      }
    }

    int numDocs = scaledRandomIntBetween(200, 2500);
    try (BackgroundIndexer indexer = new BackgroundIndexer("test", "type1", client(), numDocs)) {
      logger.info("--> waiting for {} docs to be indexed ...", numDocs);
      waitForDocs(numDocs, indexer);
      logger.info("--> {} docs indexed", numDocs);

      logger.info("--> starting relocations...");
      int nodeShiftBased = numberOfReplicas; // if we have replicas shift those
      for (int i = 0; i < numberOfRelocations; i++) {
        int fromNode = (i % 2);
        int toNode = fromNode == 0 ? 1 : 0;
        fromNode += nodeShiftBased;
        toNode += nodeShiftBased;
        numDocs = scaledRandomIntBetween(200, 1000);
        logger.debug("--> Allow indexer to index [{}] documents", numDocs);
        indexer.continueIndexing(numDocs);
        logger.info("--> START relocate the shard from {} to {}", nodes[fromNode], nodes[toNode]);
        client()
            .admin()
            .cluster()
            .prepareReroute()
            .add(new MoveAllocationCommand(new ShardId("test", 0), nodes[fromNode], nodes[toNode]))
            .get();
        if (rarely()) {
          logger.debug("--> flushing");
          client().admin().indices().prepareFlush().get();
        }
        ClusterHealthResponse clusterHealthResponse =
            client()
                .admin()
                .cluster()
                .prepareHealth()
                .setWaitForEvents(Priority.LANGUID)
                .setWaitForRelocatingShards(0)
                .setTimeout(ACCEPTABLE_RELOCATION_TIME)
                .execute()
                .actionGet();
        assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
        clusterHealthResponse =
            client()
                .admin()
                .cluster()
                .prepareHealth()
                .setWaitForEvents(Priority.LANGUID)
                .setWaitForRelocatingShards(0)
                .setTimeout(ACCEPTABLE_RELOCATION_TIME)
                .execute()
                .actionGet();
        assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
        indexer.pauseIndexing();
        logger.info("--> DONE relocate the shard from {} to {}", fromNode, toNode);
      }
      logger.info("--> done relocations");
      logger.info("--> waiting for indexing threads to stop ...");
      indexer.stop();
      logger.info("--> indexing threads stopped");

      logger.info("--> refreshing the index");
      client().admin().indices().prepareRefresh("test").execute().actionGet();
      logger.info("--> searching the index");
      boolean ranOnce = false;
      for (int i = 0; i < 10; i++) {
        try {
          logger.info("--> START search test round {}", i + 1);
          SearchHits hits =
              client()
                  .prepareSearch("test")
                  .setQuery(matchAllQuery())
                  .setSize((int) indexer.totalIndexedDocs())
                  .setNoFields()
                  .execute()
                  .actionGet()
                  .getHits();
          ranOnce = true;
          if (hits.totalHits() != indexer.totalIndexedDocs()) {
            int[] hitIds = new int[(int) indexer.totalIndexedDocs()];
            for (int hit = 0; hit < indexer.totalIndexedDocs(); hit++) {
              hitIds[hit] = hit + 1;
            }
            IntHashSet set = IntHashSet.from(hitIds);
            for (SearchHit hit : hits.hits()) {
              int id = Integer.parseInt(hit.id());
              if (!set.remove(id)) {
                logger.error("Extra id [{}]", id);
              }
            }
            set.forEach(
                new IntProcedure() {

                  @Override
                  public void apply(int value) {
                    logger.error("Missing id [{}]", value);
                  }
                });
          }
          assertThat(hits.totalHits(), equalTo(indexer.totalIndexedDocs()));
          logger.info("--> DONE search test round {}", i + 1);
        } catch (SearchPhaseExecutionException ex) {
          // TODO: the first run fails with this failure, waiting for relocating nodes set to 0 is
          // not enough?
          logger.warn("Got exception while searching.", ex);
        }
      }
      if (!ranOnce) {
        fail();
      }
    }
  }