Skip to content

Commit 9d0e0eb

Browse files
authored
[Zen2] Remove initial master node count setting (#37150)
The `cluster.unsafe_initial_master_node_count` setting was introduced as a temporary measure while the design of `cluster.initial_master_nodes` was being finalised. This commit removes this temporary setting, replacing it with usages of `cluster.initial_master_nodes` where appropriate.
1 parent c0f8c89 commit 9d0e0eb

File tree

12 files changed

+200
-283
lines changed

12 files changed

+200
-283
lines changed

server/src/main/java/org/elasticsearch/action/admin/cluster/bootstrap/GetDiscoveredNodesRequest.java

Lines changed: 1 addition & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -35,8 +35,6 @@
3535
*/
3636
public class GetDiscoveredNodesRequest extends ActionRequest {
3737

38-
private int waitForNodes = 1;
39-
4038
@Nullable // if the request should wait indefinitely
4139
private TimeValue timeout = TimeValue.timeValueSeconds(30);
4240

@@ -47,35 +45,10 @@ public GetDiscoveredNodesRequest() {
4745

4846
public GetDiscoveredNodesRequest(StreamInput in) throws IOException {
4947
super(in);
50-
waitForNodes = in.readInt();
5148
timeout = in.readOptionalTimeValue();
5249
requiredNodes = in.readList(StreamInput::readString);
5350
}
5451

55-
/**
56-
* Sometimes it is useful only to receive a successful response after discovering a certain number of master-eligible nodes. This
57-
* parameter controls this behaviour.
58-
*
59-
* @param waitForNodes the minimum number of nodes to have discovered before this request will receive a successful response. Must
60-
* be at least 1, because we always discover the local node.
61-
*/
62-
public void setWaitForNodes(int waitForNodes) {
63-
if (waitForNodes < 1) {
64-
throw new IllegalArgumentException("always finds at least one node, waiting for [" + waitForNodes + "] is not allowed");
65-
}
66-
this.waitForNodes = waitForNodes;
67-
}
68-
69-
/**
70-
* Sometimes it is useful only to receive a successful response after discovering a certain number of master-eligible nodes. This
71-
* parameter controls this behaviour.
72-
*
73-
* @return the minimum number of nodes to have discovered before this request will receive a successful response.
74-
*/
75-
public int getWaitForNodes() {
76-
return waitForNodes;
77-
}
78-
7952
/**
8053
* Sometimes it is useful to wait until enough nodes have been discovered, rather than failing immediately. This parameter controls how
8154
* long to wait, and defaults to 30s.
@@ -133,16 +106,14 @@ public void readFrom(StreamInput in) throws IOException {
133106
@Override
134107
public void writeTo(StreamOutput out) throws IOException {
135108
super.writeTo(out);
136-
out.writeInt(waitForNodes);
137109
out.writeOptionalTimeValue(timeout);
138110
out.writeStringList(requiredNodes);
139111
}
140112

141113
@Override
142114
public String toString() {
143115
return "GetDiscoveredNodesRequest{" +
144-
"waitForNodes=" + waitForNodes +
145-
", timeout=" + timeout +
116+
"timeout=" + timeout +
146117
", requiredNodes=" + requiredNodes + "}";
147118
}
148119
}

server/src/main/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportGetDiscoveredNodesAction.java

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -152,10 +152,6 @@ private static boolean matchesRequirement(DiscoveryNode discoveryNode, String re
152152
}
153153

154154
private static boolean checkWaitRequirements(GetDiscoveredNodesRequest request, Set<DiscoveryNode> nodes) {
155-
if (nodes.size() < request.getWaitForNodes()) {
156-
return false;
157-
}
158-
159155
List<String> requirements = request.getRequiredNodes();
160156
final Set<DiscoveryNode> selectedNodes = new HashSet<>();
161157
for (final String requirement : requirements) {

server/src/main/java/org/elasticsearch/cluster/coordination/ClusterBootstrapService.java

Lines changed: 4 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -54,35 +54,28 @@ public class ClusterBootstrapService {
5454

5555
private static final Logger logger = LogManager.getLogger(ClusterBootstrapService.class);
5656

57-
// The number of master-eligible nodes which, if discovered, can be used to bootstrap the cluster. This setting is unsafe in the event
58-
// that more master nodes are started than expected.
59-
public static final Setting<Integer> INITIAL_MASTER_NODE_COUNT_SETTING =
60-
Setting.intSetting("cluster.unsafe_initial_master_node_count", 0, 0, Property.NodeScope);
61-
6257
public static final Setting<List<String>> INITIAL_MASTER_NODES_SETTING =
6358
Setting.listSetting("cluster.initial_master_nodes", Collections.emptyList(), Function.identity(), Property.NodeScope);
6459

6560
public static final Setting<TimeValue> UNCONFIGURED_BOOTSTRAP_TIMEOUT_SETTING =
6661
Setting.timeSetting("discovery.unconfigured_bootstrap_timeout",
6762
TimeValue.timeValueSeconds(3), TimeValue.timeValueMillis(1), Property.NodeScope);
6863

69-
private final int initialMasterNodeCount;
7064
private final List<String> initialMasterNodes;
7165
@Nullable
7266
private final TimeValue unconfiguredBootstrapTimeout;
7367
private final TransportService transportService;
7468
private volatile boolean running;
7569

7670
public ClusterBootstrapService(Settings settings, TransportService transportService) {
77-
initialMasterNodeCount = INITIAL_MASTER_NODE_COUNT_SETTING.get(settings);
7871
initialMasterNodes = INITIAL_MASTER_NODES_SETTING.get(settings);
7972
unconfiguredBootstrapTimeout = discoveryIsConfigured(settings) ? null : UNCONFIGURED_BOOTSTRAP_TIMEOUT_SETTING.get(settings);
8073
this.transportService = transportService;
8174
}
8275

8376
public static boolean discoveryIsConfigured(Settings settings) {
84-
return Stream.of(DISCOVERY_HOSTS_PROVIDER_SETTING, DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING,
85-
INITIAL_MASTER_NODE_COUNT_SETTING, INITIAL_MASTER_NODES_SETTING).anyMatch(s -> s.exists(settings));
77+
return Stream.of(DISCOVERY_HOSTS_PROVIDER_SETTING, DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING, INITIAL_MASTER_NODES_SETTING)
78+
.anyMatch(s -> s.exists(settings));
8679
}
8780

8881
public void start() {
@@ -144,25 +137,21 @@ public String toString() {
144137
});
145138

146139
}
147-
} else if (initialMasterNodeCount > 0 || initialMasterNodes.isEmpty() == false) {
148-
logger.debug("unsafely waiting for discovery of [{}] master-eligible nodes", initialMasterNodeCount);
140+
} else if (initialMasterNodes.isEmpty() == false) {
141+
logger.debug("waiting for discovery of master-eligible nodes matching {}", initialMasterNodes);
149142

150143
final ThreadContext threadContext = transportService.getThreadPool().getThreadContext();
151144
try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
152145
threadContext.markAsSystemContext();
153146

154147
final GetDiscoveredNodesRequest request = new GetDiscoveredNodesRequest();
155-
if (initialMasterNodeCount > 0) {
156-
request.setWaitForNodes(initialMasterNodeCount);
157-
}
158148
request.setRequiredNodes(initialMasterNodes);
159149
request.setTimeout(null);
160150
logger.trace("sending {}", request);
161151
transportService.sendRequest(transportService.getLocalNode(), GetDiscoveredNodesAction.NAME, request,
162152
new TransportResponseHandler<GetDiscoveredNodesResponse>() {
163153
@Override
164154
public void handleResponse(GetDiscoveredNodesResponse response) {
165-
assert response.getNodes().size() >= initialMasterNodeCount;
166155
assert response.getNodes().stream().allMatch(DiscoveryNode::isMasterNode);
167156
logger.debug("discovered {}, starting to bootstrap", response.getNodes());
168157
awaitBootstrap(response.getBootstrapConfiguration());

server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java

Lines changed: 2 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,6 @@
4242
import java.util.stream.StreamSupport;
4343

4444
import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING;
45-
import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODE_COUNT_SETTING;
4645

4746
public class ClusterFormationFailureHelper {
4847
private static final Logger logger = LogManager.getLogger(ClusterFormationFailureHelper.class);
@@ -148,23 +147,13 @@ String getDescription() {
148147

149148
final String bootstrappingDescription;
150149

151-
if (INITIAL_MASTER_NODE_COUNT_SETTING.get(Settings.EMPTY).equals(INITIAL_MASTER_NODE_COUNT_SETTING.get(settings))
152-
&& INITIAL_MASTER_NODES_SETTING.get(Settings.EMPTY).equals(INITIAL_MASTER_NODES_SETTING.get(settings))) {
150+
if (INITIAL_MASTER_NODES_SETTING.get(Settings.EMPTY).equals(INITIAL_MASTER_NODES_SETTING.get(settings))) {
153151
bootstrappingDescription = "[" + INITIAL_MASTER_NODES_SETTING.getKey() + "] is empty on this node";
154-
} else if (INITIAL_MASTER_NODES_SETTING.get(Settings.EMPTY).equals(INITIAL_MASTER_NODES_SETTING.get(settings))) {
155-
bootstrappingDescription = String.format(Locale.ROOT,
156-
"this node must discover at least [%d] master-eligible nodes to bootstrap a cluster",
157-
INITIAL_MASTER_NODE_COUNT_SETTING.get(settings));
158-
} else if (INITIAL_MASTER_NODE_COUNT_SETTING.get(settings) <= INITIAL_MASTER_NODES_SETTING.get(settings).size()) {
152+
} else {
159153
// TODO update this when we can bootstrap on only a quorum of the initial nodes
160154
bootstrappingDescription = String.format(Locale.ROOT,
161155
"this node must discover master-eligible nodes %s to bootstrap a cluster",
162156
INITIAL_MASTER_NODES_SETTING.get(settings));
163-
} else {
164-
// TODO update this when we can bootstrap on only a quorum of the initial nodes
165-
bootstrappingDescription = String.format(Locale.ROOT,
166-
"this node must discover at least [%d] master-eligible nodes, including %s, to bootstrap a cluster",
167-
INITIAL_MASTER_NODE_COUNT_SETTING.get(settings), INITIAL_MASTER_NODES_SETTING.get(settings));
168157
}
169158

170159
return String.format(Locale.ROOT,

server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -485,7 +485,6 @@ public void apply(Settings value, Settings current, Settings previous) {
485485
Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION,
486486
TransportAddVotingConfigExclusionsAction.MAXIMUM_VOTING_CONFIG_EXCLUSIONS_SETTING,
487487
ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING,
488-
ClusterBootstrapService.INITIAL_MASTER_NODE_COUNT_SETTING,
489488
ClusterBootstrapService.UNCONFIGURED_BOOTSTRAP_TIMEOUT_SETTING,
490489
LagDetector.CLUSTER_FOLLOWER_LAG_TIMEOUT_SETTING,
491490
DiscoveryUpgradeService.BWC_PING_TIMEOUT_SETTING,

server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/GetDiscoveredNodesRequestTests.java

Lines changed: 0 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -31,20 +31,6 @@
3131

3232
public class GetDiscoveredNodesRequestTests extends ESTestCase {
3333

34-
public void testWaitForNodesValidation() {
35-
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
36-
assertThat("default value is 1", getDiscoveredNodesRequest.getWaitForNodes(), is(1));
37-
38-
final int newWaitForNodes = randomIntBetween(1, 10);
39-
getDiscoveredNodesRequest.setWaitForNodes(newWaitForNodes);
40-
assertThat("value updated", getDiscoveredNodesRequest.getWaitForNodes(), is(newWaitForNodes));
41-
42-
final IllegalArgumentException exception
43-
= expectThrows(IllegalArgumentException.class, () -> getDiscoveredNodesRequest.setWaitForNodes(randomIntBetween(-10, 0)));
44-
assertThat(exception.getMessage(), startsWith("always finds at least one node, waiting for "));
45-
assertThat(exception.getMessage(), endsWith(" is not allowed"));
46-
}
47-
4834
public void testTimeoutValidation() {
4935
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
5036
assertThat("default value is 30s", getDiscoveredNodesRequest.getTimeout(), is(TimeValue.timeValueSeconds(30)));
@@ -65,10 +51,6 @@ public void testTimeoutValidation() {
6551
public void testSerialization() throws IOException {
6652
final GetDiscoveredNodesRequest originalRequest = new GetDiscoveredNodesRequest();
6753

68-
if (randomBoolean()) {
69-
originalRequest.setWaitForNodes(randomIntBetween(1, 10));
70-
}
71-
7254
if (randomBoolean()) {
7355
originalRequest.setTimeout(TimeValue.parseTimeValue(randomTimeValue(), "timeout"));
7456
} else if (randomBoolean()) {
@@ -77,7 +59,6 @@ public void testSerialization() throws IOException {
7759

7860
final GetDiscoveredNodesRequest deserialized = copyWriteable(originalRequest, writableRegistry(), GetDiscoveredNodesRequest::new);
7961

80-
assertThat(deserialized.getWaitForNodes(), equalTo(originalRequest.getWaitForNodes()));
8162
assertThat(deserialized.getTimeout(), equalTo(originalRequest.getTimeout()));
8263
}
8364
}

server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportGetDiscoveredNodesActionTests.java

Lines changed: 6 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -72,6 +72,7 @@
7272
import static java.util.Collections.emptyMap;
7373
import static java.util.Collections.emptySet;
7474
import static java.util.Collections.singleton;
75+
import static java.util.Collections.singletonList;
7576
import static org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING;
7677
import static org.elasticsearch.discovery.PeerFinder.REQUEST_PEERS_ACTION_NAME;
7778
import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME;
@@ -204,8 +205,8 @@ public void testFailsQuicklyWithZeroTimeoutAndAcceptsNullTimeout() throws Interr
204205

205206
{
206207
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
207-
getDiscoveredNodesRequest.setWaitForNodes(2);
208208
getDiscoveredNodesRequest.setTimeout(null);
209+
getDiscoveredNodesRequest.setRequiredNodes(singletonList("not-a-node"));
209210
transportService.sendRequest(localNode, GetDiscoveredNodesAction.NAME, getDiscoveredNodesRequest, new ResponseHandler() {
210211
@Override
211212
public void handleResponse(GetDiscoveredNodesResponse response) {
@@ -221,8 +222,8 @@ public void handleException(TransportException exp) {
221222

222223
{
223224
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
224-
getDiscoveredNodesRequest.setWaitForNodes(2);
225225
getDiscoveredNodesRequest.setTimeout(TimeValue.ZERO);
226+
getDiscoveredNodesRequest.setRequiredNodes(singletonList("not-a-node"));
226227

227228
final CountDownLatch countDownLatch = new CountDownLatch(1);
228229
transportService.sendRequest(localNode, GetDiscoveredNodesAction.NAME, getDiscoveredNodesRequest, new ResponseHandler() {
@@ -254,7 +255,6 @@ public void testFailsIfAlreadyBootstrapped() throws InterruptedException {
254255

255256
final CountDownLatch countDownLatch = new CountDownLatch(1);
256257
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
257-
getDiscoveredNodesRequest.setWaitForNodes(2);
258258
getDiscoveredNodesRequest.setTimeout(null);
259259
transportService.sendRequest(localNode, GetDiscoveredNodesAction.NAME, getDiscoveredNodesRequest, new ResponseHandler() {
260260
@Override
@@ -283,8 +283,8 @@ public void testFailsIfAcceptsClusterStateWithNonemptyConfiguration() throws Int
283283

284284
final CountDownLatch countDownLatch = new CountDownLatch(1);
285285
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
286-
getDiscoveredNodesRequest.setWaitForNodes(3);
287286
getDiscoveredNodesRequest.setTimeout(null);
287+
getDiscoveredNodesRequest.setRequiredNodes(singletonList("not-a-node"));
288288
transportService.sendRequest(localNode, GetDiscoveredNodesAction.NAME, getDiscoveredNodesRequest, new ResponseHandler() {
289289
@Override
290290
public void handleResponse(GetDiscoveredNodesResponse response) {
@@ -342,7 +342,6 @@ public PublishWithJoinResponse read(StreamInput in) throws IOException {
342342
public void testGetsDiscoveredNodesWithZeroTimeout() throws InterruptedException {
343343
setupGetDiscoveredNodesAction();
344344
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
345-
getDiscoveredNodesRequest.setWaitForNodes(2);
346345
getDiscoveredNodesRequest.setTimeout(TimeValue.ZERO);
347346
assertWaitConditionMet(getDiscoveredNodesRequest);
348347
}
@@ -377,7 +376,6 @@ public void testGetsDiscoveredNodesDuplicateName() throws InterruptedException {
377376
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
378377
String name = localNode.getName();
379378
getDiscoveredNodesRequest.setRequiredNodes(Arrays.asList(name, name));
380-
getDiscoveredNodesRequest.setWaitForNodes(1);
381379
getDiscoveredNodesRequest.setTimeout(TimeValue.ZERO);
382380
assertWaitConditionFailedOnDuplicate(getDiscoveredNodesRequest, "[" + localNode + "] matches [" + name + ", " + name + ']');
383381
}
@@ -396,7 +394,6 @@ public void testGetsDiscoveredNodesTimeoutOnMissing() throws InterruptedExceptio
396394
final CountDownLatch latch = new CountDownLatch(1);
397395
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
398396
getDiscoveredNodesRequest.setRequiredNodes(Arrays.asList(localNode.getAddress().toString(), "_missing"));
399-
getDiscoveredNodesRequest.setWaitForNodes(1);
400397
getDiscoveredNodesRequest.setTimeout(TimeValue.ZERO);
401398
transportService.sendRequest(localNode, GetDiscoveredNodesAction.NAME, getDiscoveredNodesRequest, new ResponseHandler() {
402399
@Override
@@ -423,8 +420,7 @@ public void testThrowsExceptionIfDuplicateDiscoveredLater() throws InterruptedEx
423420

424421
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
425422
final String ip = localNode.getAddress().getAddress();
426-
getDiscoveredNodesRequest.setRequiredNodes(Collections.singletonList(ip));
427-
getDiscoveredNodesRequest.setWaitForNodes(2);
423+
getDiscoveredNodesRequest.setRequiredNodes(Arrays.asList(ip, "not-a-node"));
428424

429425
final CountDownLatch countDownLatch = new CountDownLatch(1);
430426
transportService.sendRequest(localNode, GetDiscoveredNodesAction.NAME, getDiscoveredNodesRequest, new ResponseHandler() {
@@ -480,7 +476,7 @@ private void setupGetDiscoveredNodesAction() throws InterruptedException {
480476
executeRequestPeersAction();
481477

482478
final GetDiscoveredNodesRequest getDiscoveredNodesRequest = new GetDiscoveredNodesRequest();
483-
getDiscoveredNodesRequest.setWaitForNodes(2);
479+
getDiscoveredNodesRequest.setRequiredNodes(Arrays.asList(localNode.getName(), otherNode.getName()));
484480
assertWaitConditionMet(getDiscoveredNodesRequest);
485481
}
486482

server/src/test/java/org/elasticsearch/bootstrap/BootstrapChecksTests.java

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -729,7 +729,6 @@ public void testDiscoveryConfiguredCheck() throws NodeValidationException {
729729

730730
ensureChecksPass.accept(Settings.builder().putList(DiscoveryModule.DISCOVERY_HOSTS_PROVIDER_SETTING.getKey()));
731731
ensureChecksPass.accept(Settings.builder().putList(SettingsBasedHostsProvider.DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey()));
732-
ensureChecksPass.accept(Settings.builder().put(ClusterBootstrapService.INITIAL_MASTER_NODE_COUNT_SETTING.getKey(), 0));
733732
ensureChecksPass.accept(Settings.builder().putList(ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.getKey()));
734733
}
735734
}

0 commit comments

Comments
 (0)