|
23 | 23 | import com.carrotsearch.hppc.procedures.IntProcedure;
|
24 | 24 | import org.apache.lucene.index.IndexFileNames;
|
25 | 25 | import org.apache.lucene.util.English;
|
| 26 | +import org.elasticsearch.action.ActionFuture; |
26 | 27 | import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
| 28 | +import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse; |
27 | 29 | import org.elasticsearch.action.index.IndexRequestBuilder;
|
28 | 30 | import org.elasticsearch.action.search.SearchResponse;
|
| 31 | +import org.elasticsearch.action.support.WriteRequest; |
29 | 32 | import org.elasticsearch.client.Client;
|
30 | 33 | import org.elasticsearch.cluster.ClusterState;
|
31 | 34 | import org.elasticsearch.cluster.metadata.IndexMetaData;
|
@@ -487,6 +490,97 @@ public void testIndexAndRelocateConcurrently() throws ExecutionException, Interr
|
487 | 490 |
|
488 | 491 | }
|
489 | 492 |
|
| 493 | + public void testRelocateWhileWaitingForRefresh() { |
| 494 | + logger.info("--> starting [node1] ..."); |
| 495 | + final String node1 = internalCluster().startNode(); |
| 496 | + |
| 497 | + logger.info("--> creating test index ..."); |
| 498 | + prepareCreate("test", Settings.builder() |
| 499 | + .put("index.number_of_shards", 1) |
| 500 | + .put("index.number_of_replicas", 0) |
| 501 | + .put("index.refresh_interval", -1) // we want to control refreshes |
| 502 | + ).get(); |
| 503 | + |
| 504 | + logger.info("--> index 10 docs"); |
| 505 | + for (int i = 0; i < 10; i++) { |
| 506 | + client().prepareIndex("test", "type", Integer.toString(i)).setSource("field", "value" + i).execute().actionGet(); |
| 507 | + } |
| 508 | + logger.info("--> flush so we have an actual index"); |
| 509 | + client().admin().indices().prepareFlush().execute().actionGet(); |
| 510 | + logger.info("--> index more docs so we have something in the translog"); |
| 511 | + for (int i = 10; i < 20; i++) { |
| 512 | + client().prepareIndex("test", "type", Integer.toString(i)).setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) |
| 513 | + .setSource("field", "value" + i).execute(); |
| 514 | + } |
| 515 | + |
| 516 | + logger.info("--> start another node"); |
| 517 | + final String node2 = internalCluster().startNode(); |
| 518 | + ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID) |
| 519 | + .setWaitForNodes("2").execute().actionGet(); |
| 520 | + assertThat(clusterHealthResponse.isTimedOut(), equalTo(false)); |
| 521 | + |
| 522 | + logger.info("--> relocate the shard from node1 to node2"); |
| 523 | + client().admin().cluster().prepareReroute() |
| 524 | + .add(new MoveAllocationCommand("test", 0, node1, node2)) |
| 525 | + .execute().actionGet(); |
| 526 | + |
| 527 | + clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID) |
| 528 | + .setWaitForNoRelocatingShards(true).setTimeout(ACCEPTABLE_RELOCATION_TIME).execute().actionGet(); |
| 529 | + assertThat(clusterHealthResponse.isTimedOut(), equalTo(false)); |
| 530 | + |
| 531 | + logger.info("--> verifying count"); |
| 532 | + client().admin().indices().prepareRefresh().execute().actionGet(); |
| 533 | + assertThat(client().prepareSearch("test").setSize(0).execute().actionGet().getHits().getTotalHits(), equalTo(20L)); |
| 534 | + } |
| 535 | + |
| 536 | + public void testRelocateWhileContinuouslyIndexingAndWaitingForRefresh() { |
| 537 | + logger.info("--> starting [node1] ..."); |
| 538 | + final String node1 = internalCluster().startNode(); |
| 539 | + |
| 540 | + logger.info("--> creating test index ..."); |
| 541 | + prepareCreate("test", Settings.builder() |
| 542 | + .put("index.number_of_shards", 1) |
| 543 | + .put("index.number_of_replicas", 0) |
| 544 | + .put("index.refresh_interval", -1) // we want to control refreshes |
| 545 | + ).get(); |
| 546 | + |
| 547 | + logger.info("--> index 10 docs"); |
| 548 | + for (int i = 0; i < 10; i++) { |
| 549 | + client().prepareIndex("test", "type", Integer.toString(i)).setSource("field", "value" + i).execute().actionGet(); |
| 550 | + } |
| 551 | + logger.info("--> flush so we have an actual index"); |
| 552 | + client().admin().indices().prepareFlush().execute().actionGet(); |
| 553 | + logger.info("--> index more docs so we have something in the translog"); |
| 554 | + for (int i = 10; i < 20; i++) { |
| 555 | + client().prepareIndex("test", "type", Integer.toString(i)).setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) |
| 556 | + .setSource("field", "value" + i).execute(); |
| 557 | + } |
| 558 | + |
| 559 | + logger.info("--> start another node"); |
| 560 | + final String node2 = internalCluster().startNode(); |
| 561 | + ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID) |
| 562 | + .setWaitForNodes("2").execute().actionGet(); |
| 563 | + assertThat(clusterHealthResponse.isTimedOut(), equalTo(false)); |
| 564 | + |
| 565 | + logger.info("--> relocate the shard from node1 to node2"); |
| 566 | + ActionFuture<ClusterRerouteResponse> relocationListener = client().admin().cluster().prepareReroute() |
| 567 | + .add(new MoveAllocationCommand("test", 0, node1, node2)) |
| 568 | + .execute(); |
| 569 | + logger.info("--> index 100 docs while relocating"); |
| 570 | + for (int i = 20; i < 120; i++) { |
| 571 | + client().prepareIndex("test", "type", Integer.toString(i)).setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) |
| 572 | + .setSource("field", "value" + i).execute(); |
| 573 | + } |
| 574 | + relocationListener.actionGet(); |
| 575 | + clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID) |
| 576 | + .setWaitForNoRelocatingShards(true).setTimeout(ACCEPTABLE_RELOCATION_TIME).execute().actionGet(); |
| 577 | + assertThat(clusterHealthResponse.isTimedOut(), equalTo(false)); |
| 578 | + |
| 579 | + logger.info("--> verifying count"); |
| 580 | + client().admin().indices().prepareRefresh().execute().actionGet(); |
| 581 | + assertThat(client().prepareSearch("test").setSize(0).execute().actionGet().getHits().getTotalHits(), equalTo(120L)); |
| 582 | + } |
| 583 | + |
490 | 584 | class RecoveryCorruption implements StubbableTransport.SendRequestBehavior {
|
491 | 585 |
|
492 | 586 | private final CountDownLatch corruptionCount;
|
|
0 commit comments