forked from elastic/elasticsearch
-
Notifications
You must be signed in to change notification settings - Fork 1
/
MinimumMasterNodesTests.java
311 lines (256 loc) · 16.2 KB
/
MinimumMasterNodesTests.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster;
import com.google.common.base.Predicate;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.junit.Test;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.client.Requests.clusterHealthRequest;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope;
import static org.hamcrest.Matchers.*;
@ClusterScope(scope = Scope.TEST, numDataNodes =0)
public class MinimumMasterNodesTests extends ElasticsearchIntegrationTest {
@Test
public void simpleMinimumMasterNodes() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "zen")
.put("discovery.zen.minimum_master_nodes", 2)
.put("discovery.zen.ping_timeout", "200ms")
.put("discovery.initial_state_timeout", "500ms")
.put("gateway.type", "local")
.build();
logger.info("--> start first node");
internalCluster().startNode(settings);
logger.info("--> should be blocked, no master...");
ClusterState state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(true));
assertThat(state.nodes().size(), equalTo(1)); // verify that we still see the local node in the cluster state
logger.info("--> start second node, cluster should be formed");
internalCluster().startNode(settings);
ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForNodes("2").execute().actionGet();
assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.nodes().size(), equalTo(2));
assertThat(state.metaData().indices().containsKey("test"), equalTo(false));
createIndex("test");
NumShards numShards = getNumShards("test");
logger.info("--> indexing some data");
for (int i = 0; i < 100; i++) {
client().prepareIndex("test", "type1", Integer.toString(i)).setSource("field", "value").execute().actionGet();
}
// make sure that all shards recovered before trying to flush
assertThat(client().admin().cluster().prepareHealth("test").setWaitForActiveShards(numShards.totalNumShards).execute().actionGet().getActiveShards(), equalTo(numShards.totalNumShards));
// flush for simpler debugging
flushAndRefresh();
logger.info("--> verify we the data back");
for (int i = 0; i < 10; i++) {
assertThat(client().prepareCount().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getCount(), equalTo(100l));
}
internalCluster().stopCurrentMasterNode();
awaitBusy(new Predicate<Object>() {
public boolean apply(Object obj) {
ClusterState state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
return state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID);
}
});
state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(true));
assertThat(state.nodes().size(), equalTo(1)); // verify that we still see the local node in the cluster state
logger.info("--> starting the previous master node again...");
internalCluster().startNode(settings);
clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForYellowStatus().setWaitForNodes("2").execute().actionGet();
assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.nodes().size(), equalTo(2));
assertThat(state.metaData().indices().containsKey("test"), equalTo(true));
logger.info("Running Cluster Health");
ClusterHealthResponse clusterHealth = client().admin().cluster().health(clusterHealthRequest().waitForGreenStatus()).actionGet();
logger.info("Done Cluster Health, status " + clusterHealth.getStatus());
assertThat(clusterHealth.isTimedOut(), equalTo(false));
assertThat(clusterHealth.getStatus(), equalTo(ClusterHealthStatus.GREEN));
logger.info("--> verify we the data back");
for (int i = 0; i < 10; i++) {
assertThat(client().prepareCount().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getCount(), equalTo(100l));
}
internalCluster().stopRandomNonMasterNode();
assertThat(awaitBusy(new Predicate<Object>() {
public boolean apply(Object obj) {
ClusterState state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
return state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID);
}
}), equalTo(true));
logger.info("--> starting the previous master node again...");
internalCluster().startNode(settings);
clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForNodes("2").setWaitForGreenStatus().execute().actionGet();
assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(false));
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.nodes().size(), equalTo(2));
assertThat(state.metaData().indices().containsKey("test"), equalTo(true));
logger.info("Running Cluster Health");
ensureGreen();
logger.info("--> verify we the data back");
for (int i = 0; i < 10; i++) {
assertThat(client().prepareCount().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getCount(), equalTo(100l));
}
}
@Test
@TestLogging("cluster.service:TRACE,discovery:TRACE,indices.cluster:TRACE")
public void multipleNodesShutdownNonMasterNodes() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "zen")
.put("discovery.zen.minimum_master_nodes", 3)
.put("discovery.zen.ping_timeout", "1s")
.put("discovery.initial_state_timeout", "500ms")
.put("gateway.type", "local")
.build();
logger.info("--> start first 2 nodes");
internalCluster().startNode(settings);
internalCluster().startNode(settings);
ClusterState state;
awaitBusy(new Predicate<Object>() {
public boolean apply(Object obj) {
ClusterState state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
return state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID);
}
});
awaitBusy(new Predicate<Object>() {
public boolean apply(Object obj) {
ClusterState state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
return state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID);
}
});
state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(true));
state = client().admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
assertThat(state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), equalTo(true));
logger.info("--> start two more nodes");
internalCluster().startNodesAsync(2, settings).get();
ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForNodes("4").execute().actionGet();
assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.nodes().size(), equalTo(4));
createIndex("test");
NumShards numShards = getNumShards("test");
logger.info("--> indexing some data");
for (int i = 0; i < 100; i++) {
client().prepareIndex("test", "type1", Integer.toString(i)).setSource("field", "value").execute().actionGet();
}
// make sure that all shards recovered before trying to flush
assertThat(client().admin().cluster().prepareHealth("test").setWaitForActiveShards(numShards.totalNumShards).execute().actionGet().isTimedOut(), equalTo(false));
// flush for simpler debugging
client().admin().indices().prepareFlush().execute().actionGet();
client().admin().indices().prepareRefresh().execute().actionGet();
logger.info("--> verify we the data back");
for (int i = 0; i < 10; i++) {
assertThat(client().prepareCount().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getCount(), equalTo(100l));
}
internalCluster().stopRandomNonMasterNode();
internalCluster().stopRandomNonMasterNode();
logger.info("--> verify that there is no master anymore on remaining nodes");
// spin here to wait till the state is set
assertNoMasterBlockOnAllNodes();
logger.info("--> start back the 2 nodes ");
String[] newNodes = internalCluster().startNodesAsync(2, settings).get().toArray(Strings.EMPTY_ARRAY);
clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForNodes("4").execute().actionGet();
assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
logger.info("--> running Cluster Health");
ClusterHealthResponse clusterHealth = client().admin().cluster().health(clusterHealthRequest().waitForGreenStatus()).actionGet();
logger.info("--> done Cluster Health, status " + clusterHealth.getStatus());
assertThat(clusterHealth.isTimedOut(), equalTo(false));
assertThat(clusterHealth.getStatus(), equalTo(ClusterHealthStatus.GREEN));
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.nodes().size(), equalTo(4));
// we prefer to elect up and running nodes
assertThat(state.nodes().masterNodeId(), not(isOneOf(newNodes)));
logger.info("--> verify we the data back");
for (int i = 0; i < 10; i++) {
assertThat(client().prepareCount().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getCount(), equalTo(100l));
}
}
@Test
public void dynamicUpdateMinimumMasterNodes() throws InterruptedException, IOException {
Settings settings = settingsBuilder()
.put("discovery.type", "zen")
.put("discovery.zen.ping_timeout", "400ms")
.put("discovery.initial_state_timeout", "500ms")
.put("gateway.type", "local")
.build();
logger.info("--> start 2 nodes");
internalCluster().startNode(settings);
internalCluster().startNode(settings);
// wait until second node join the cluster
ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForNodes("2").get();
assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
logger.info("--> setting minimum master node to 2");
setMinimumMasterNodes(2);
// make sure it has been processed on all nodes (master node spawns a secondary cluster state update task)
for (Client client : internalCluster()) {
assertThat(client.admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setLocal(true).get().isTimedOut(),
equalTo(false));
}
logger.info("--> stopping a node");
internalCluster().stopRandomDataNode();
logger.info("--> verifying min master node has effect");
assertNoMasterBlockOnAllNodes();
logger.info("--> bringing another node up");
internalCluster().startNode(settingsBuilder().put(settings).put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, 2).build());
clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForNodes("2").get();
assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
}
private void assertNoMasterBlockOnAllNodes() throws InterruptedException {
assertThat(awaitBusy(new Predicate<Object>() {
public boolean apply(Object obj) {
boolean success = true;
for (Client client : internalCluster()) {
ClusterState state = client.admin().cluster().prepareState().setLocal(true).execute().actionGet().getState();
success &= state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID);
if (logger.isDebugEnabled()) {
logger.debug("Checking for NO_MASTER_BLOCK on client: {} NO_MASTER_BLOCK: [{}]", client, state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID));
}
}
return success;
}
}, 20, TimeUnit.SECONDS), equalTo(true));
}
}