|
34 | 34 |
|
35 | 35 | import org.apache.lucene.index.CorruptIndexException;
|
36 | 36 | import org.opensearch.Version;
|
| 37 | +import org.opensearch.action.admin.cluster.allocation.ClusterAllocationExplainResponse; |
37 | 38 | import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction;
|
38 | 39 | import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest;
|
39 | 40 | import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction;
|
|
55 | 56 | import org.opensearch.cluster.metadata.IndexMetadata;
|
56 | 57 | import org.opensearch.cluster.node.DiscoveryNode;
|
57 | 58 | import org.opensearch.cluster.routing.ShardRouting;
|
| 59 | +import org.opensearch.cluster.routing.ShardRoutingState; |
58 | 60 | import org.opensearch.cluster.routing.UnassignedInfo;
|
| 61 | +import org.opensearch.cluster.routing.allocation.AllocationDecision; |
59 | 62 | import org.opensearch.cluster.routing.allocation.ExistingShardsAllocator;
|
60 | 63 | import org.opensearch.cluster.service.ClusterService;
|
61 | 64 | import org.opensearch.common.settings.Settings;
|
|
98 | 101 | import java.util.Map;
|
99 | 102 | import java.util.Set;
|
100 | 103 | import java.util.concurrent.ExecutionException;
|
| 104 | +import java.util.stream.Collectors; |
101 | 105 | import java.util.stream.IntStream;
|
102 | 106 |
|
103 | 107 | import static java.util.Collections.emptyMap;
|
104 | 108 | import static java.util.Collections.emptySet;
|
105 | 109 | import static org.opensearch.cluster.coordination.ClusterBootstrapService.INITIAL_CLUSTER_MANAGER_NODES_SETTING;
|
106 | 110 | import static org.opensearch.cluster.health.ClusterHealthStatus.GREEN;
|
107 | 111 | import static org.opensearch.cluster.health.ClusterHealthStatus.RED;
|
| 112 | +import static org.opensearch.cluster.health.ClusterHealthStatus.YELLOW; |
108 | 113 | import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
|
109 | 114 | import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS;
|
| 115 | +import static org.opensearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; |
110 | 116 | import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder;
|
111 | 117 | import static org.opensearch.gateway.GatewayRecoveryTestUtils.corruptShard;
|
112 | 118 | import static org.opensearch.gateway.GatewayRecoveryTestUtils.getDiscoveryNodes;
|
@@ -753,6 +759,7 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception {
|
753 | 759 | Settings.builder()
|
754 | 760 | .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
|
755 | 761 | .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
|
| 762 | + .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms") |
756 | 763 | .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms")
|
757 | 764 | .build()
|
758 | 765 | );
|
@@ -843,6 +850,87 @@ public void testBatchModeDisabled() throws Exception {
|
843 | 850 | ensureGreen("test");
|
844 | 851 | }
|
845 | 852 |
|
| 853 | + public void testMultipleReplicaShardAssignmentWithDelayedAllocationAndDifferentNodeStartTimeInBatchMode() throws Exception { |
| 854 | + internalCluster().startClusterManagerOnlyNodes( |
| 855 | + 1, |
| 856 | + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() |
| 857 | + ); |
| 858 | + internalCluster().startDataOnlyNodes(6); |
| 859 | + createIndex( |
| 860 | + "test", |
| 861 | + Settings.builder() |
| 862 | + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) |
| 863 | + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 3) |
| 864 | + .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "60m") |
| 865 | + .build() |
| 866 | + ); |
| 867 | + ensureGreen("test"); |
| 868 | + |
| 869 | + List<String> nodesWithReplicaShards = findNodesWithShard(false); |
| 870 | + Settings replicaNode0DataPathSettings = internalCluster().dataPathSettings(nodesWithReplicaShards.get(0)); |
| 871 | + Settings replicaNode1DataPathSettings = internalCluster().dataPathSettings(nodesWithReplicaShards.get(1)); |
| 872 | + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodesWithReplicaShards.get(0))); |
| 873 | + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodesWithReplicaShards.get(1))); |
| 874 | + |
| 875 | + ensureStableCluster(5); |
| 876 | + |
| 877 | + logger.info("--> explicitly triggering reroute"); |
| 878 | + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); |
| 879 | + assertTrue(clusterRerouteResponse.isAcknowledged()); |
| 880 | + |
| 881 | + ClusterHealthResponse health = client().admin().cluster().health(Requests.clusterHealthRequest().timeout("5m")).actionGet(); |
| 882 | + assertFalse(health.isTimedOut()); |
| 883 | + assertEquals(YELLOW, health.getStatus()); |
| 884 | + assertEquals(2, health.getUnassignedShards()); |
| 885 | + // shard should be unassigned because of Allocation_Delayed |
| 886 | + ClusterAllocationExplainResponse allocationExplainResponse = client().admin() |
| 887 | + .cluster() |
| 888 | + .prepareAllocationExplain() |
| 889 | + .setIndex("test") |
| 890 | + .setShard(0) |
| 891 | + .setPrimary(false) |
| 892 | + .get(); |
| 893 | + assertEquals( |
| 894 | + AllocationDecision.ALLOCATION_DELAYED, |
| 895 | + allocationExplainResponse.getExplanation().getShardAllocationDecision().getAllocateDecision().getAllocationDecision() |
| 896 | + ); |
| 897 | + |
| 898 | + logger.info("--> restarting the node 1"); |
| 899 | + internalCluster().startDataOnlyNode( |
| 900 | + Settings.builder().put("node.name", nodesWithReplicaShards.get(0)).put(replicaNode0DataPathSettings).build() |
| 901 | + ); |
| 902 | + clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); |
| 903 | + assertTrue(clusterRerouteResponse.isAcknowledged()); |
| 904 | + ensureStableCluster(6); |
| 905 | + waitUntil( |
| 906 | + () -> client().admin().cluster().health(Requests.clusterHealthRequest().timeout("5m")).actionGet().getInitializingShards() == 0 |
| 907 | + ); |
| 908 | + |
| 909 | + health = client().admin().cluster().health(Requests.clusterHealthRequest().timeout("5m")).actionGet(); |
| 910 | + assertFalse(health.isTimedOut()); |
| 911 | + assertEquals(YELLOW, health.getStatus()); |
| 912 | + assertEquals(1, health.getUnassignedShards()); |
| 913 | + assertEquals(1, health.getDelayedUnassignedShards()); |
| 914 | + allocationExplainResponse = client().admin() |
| 915 | + .cluster() |
| 916 | + .prepareAllocationExplain() |
| 917 | + .setIndex("test") |
| 918 | + .setShard(0) |
| 919 | + .setPrimary(false) |
| 920 | + .get(); |
| 921 | + assertEquals( |
| 922 | + AllocationDecision.ALLOCATION_DELAYED, |
| 923 | + allocationExplainResponse.getExplanation().getShardAllocationDecision().getAllocateDecision().getAllocationDecision() |
| 924 | + ); |
| 925 | + |
| 926 | + logger.info("--> restarting the node 0"); |
| 927 | + internalCluster().startDataOnlyNode( |
| 928 | + Settings.builder().put("node.name", nodesWithReplicaShards.get(1)).put(replicaNode1DataPathSettings).build() |
| 929 | + ); |
| 930 | + ensureStableCluster(7); |
| 931 | + ensureGreen("test"); |
| 932 | + } |
| 933 | + |
846 | 934 | public void testNBatchesCreationAndAssignment() throws Exception {
|
847 | 935 | // we will reduce batch size to 5 to make sure we have enough batches to test assignment
|
848 | 936 | // Total number of primary shards = 50 (50 indices*1)
|
@@ -1293,4 +1381,14 @@ private void prepareIndex(String indexName, int numberOfPrimaryShards) {
|
1293 | 1381 | index(indexName, "type", "1", Collections.emptyMap());
|
1294 | 1382 | flush(indexName);
|
1295 | 1383 | }
|
| 1384 | + |
| 1385 | + private List<String> findNodesWithShard(final boolean primary) { |
| 1386 | + ClusterState state = client().admin().cluster().prepareState().get().getState(); |
| 1387 | + List<ShardRouting> startedShards = state.routingTable().shardsWithState(ShardRoutingState.STARTED); |
| 1388 | + List<ShardRouting> requiredStartedShards = startedShards.stream() |
| 1389 | + .filter(startedShard -> startedShard.primary() == primary) |
| 1390 | + .collect(Collectors.toList()); |
| 1391 | + Collections.shuffle(requiredStartedShards, random()); |
| 1392 | + return requiredStartedShards.stream().map(shard -> state.nodes().get(shard.currentNodeId()).getName()).collect(Collectors.toList()); |
| 1393 | + } |
1296 | 1394 | }
|
0 commit comments