Skip to content

Commit

Permalink
[Zen2] Best-effort cluster formation if unconfigured (elastic#36215)
Browse files Browse the repository at this point in the history
In real deployments it is important that clusters are properly configured to
avoid accidentally forming multiple independent clusters at cluster
bootstrapping time. However we also expect to be able to unpack Elasticsearch
and start up one or more nodes without any up-front configuration, and have
them do their best to find each other and form a cluster after a few seconds.

This change adds a delayed automatic bootstrapping process to nodes that start
up with no relevant settings set to support the desired out-of-the-box
experience without compromising safety in properly-configured deployments.
  • Loading branch information
DaveCTurner authored Dec 7, 2018
1 parent 879397d commit c32e4fb
Show file tree
Hide file tree
Showing 9 changed files with 211 additions and 8 deletions.
11 changes: 10 additions & 1 deletion docs/reference/migration/migrate_7_0/cluster.asciidoc
Original file line number Diff line number Diff line change
Expand Up @@ -24,4 +24,13 @@ These shard preferences are removed in favour of the `_prefer_nodes` and `_only_
Clusters now have soft limits on the total number of open shards in the cluster
based on the number of nodes and the `cluster.max_shards_per_node` cluster
setting, to prevent accidental operations that would destabilize the cluster.
More information can be found in the <<misc-cluster,documentation for that setting>>.
More information can be found in the <<misc-cluster,documentation for that setting>>.

[float]
==== Discovery configuration is required in production
Production deployments of Elasticsearch now require at least one of the following settings
to be specified in the `elasticsearch.yml` configuration file:

- `discovery.zen.ping.unicast.hosts`
- `discovery.zen.hosts_provider`
- `cluster.initial_master_nodes`
18 changes: 18 additions & 0 deletions docs/reference/setup/bootstrap-checks.asciidoc
Original file line number Diff line number Diff line change
Expand Up @@ -236,3 +236,21 @@ versions of the HotSpot JVM.
The all permission check ensures that the security policy used during bootstrap
does not grant the `java.security.AllPermission` to Elasticsearch. Running with
the all permission granted is equivalent to disabling the security manager.

=== Discovery configuration check

By default, when Elasticsearch first starts up it will try and discover other
nodes running on the same host. If no elected master can be discovered within a
few seconds then Elasticsearch will form a cluster that includes any other
nodes that were discovered. It is useful to be able to form this cluster
without any extra configuration in development mode, but this is unsuitable for
production because it's possible to form multiple clusters and lose data as a
result.

This bootstrap check ensures that discovery is not running with the default
configuration. It can be satisfied by setting at least one of the following
properties:

- `discovery.zen.ping.unicast.hosts`
- `discovery.zen.hosts_provider`
- `cluster.initial_master_nodes`
2 changes: 1 addition & 1 deletion qa/unconfigured-node-name/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ apply plugin: 'elasticsearch.rest-test'

integTestCluster {
setting 'node.name', null
// TODO: Run this using zen2
// TODO: Run this using zen2, with no discovery configuration at all, demonstrating that the node forms a cluster on its own without help
setting 'discovery.type', 'zen'
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,10 @@
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.util.Constants;
import org.elasticsearch.cluster.coordination.ClusterBootstrapService;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.transport.BoundTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.discovery.DiscoveryModule;
Expand All @@ -46,6 +48,12 @@
import java.util.function.Predicate;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING;
import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_HOSTS_PROVIDER_SETTING;
import static org.elasticsearch.discovery.zen.SettingsBasedHostsProvider.DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING;

/**
* We enforce bootstrap checks once a node has the transport protocol bound to a non-loopback interface or if the system property {@code
Expand Down Expand Up @@ -207,6 +215,7 @@ static List<BootstrapCheck> checks() {
checks.add(new EarlyAccessCheck());
checks.add(new G1GCCheck());
checks.add(new AllPermissionCheck());
checks.add(new DiscoveryConfiguredCheck());
return Collections.unmodifiableList(checks);
}

Expand Down Expand Up @@ -713,4 +722,21 @@ boolean isAllPermissionGranted() {

}

static class DiscoveryConfiguredCheck implements BootstrapCheck {
@Override
public BootstrapCheckResult check(BootstrapContext context) {
if (DiscoveryModule.ZEN2_DISCOVERY_TYPE.equals(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(context.settings)) == false) {
return BootstrapCheckResult.success();
}
if (ClusterBootstrapService.discoveryIsConfigured(context.settings)) {
return BootstrapCheckResult.success();
}

return BootstrapCheckResult.failure(String.format(
Locale.ROOT,
"the default discovery settings are unsuitable for production use; at least one of [%s] must be configured",
Stream.of(DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING, DISCOVERY_HOSTS_PROVIDER_SETTING, INITIAL_MASTER_NODES_SETTING)
.map(Setting::getKey).collect(Collectors.joining(", "))));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.elasticsearch.action.admin.cluster.bootstrap.GetDiscoveredNodesRequest;
import org.elasticsearch.action.admin.cluster.bootstrap.GetDiscoveredNodesResponse;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
Expand All @@ -44,6 +45,10 @@
import java.util.Collections;
import java.util.List;
import java.util.function.Function;
import java.util.stream.Stream;

import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_HOSTS_PROVIDER_SETTING;
import static org.elasticsearch.discovery.zen.SettingsBasedHostsProvider.DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING;

public class ClusterBootstrapService {

Expand All @@ -57,22 +62,82 @@ public class ClusterBootstrapService {
public static final Setting<List<String>> INITIAL_MASTER_NODES_SETTING =
Setting.listSetting("cluster.initial_master_nodes", Collections.emptyList(), Function.identity(), Property.NodeScope);

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

private final int initialMasterNodeCount;
private final List<String> initialMasterNodes;
@Nullable
private final TimeValue unconfiguredBootstrapTimeout;
private final TransportService transportService;
private volatile boolean running;

public ClusterBootstrapService(Settings settings, TransportService transportService) {
initialMasterNodeCount = INITIAL_MASTER_NODE_COUNT_SETTING.get(settings);
initialMasterNodes = INITIAL_MASTER_NODES_SETTING.get(settings);
unconfiguredBootstrapTimeout = discoveryIsConfigured(settings) ? null : UNCONFIGURED_BOOTSTRAP_TIMEOUT_SETTING.get(settings);
this.transportService = transportService;
}

public static boolean discoveryIsConfigured(Settings settings) {
return Stream.of(DISCOVERY_HOSTS_PROVIDER_SETTING, DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING,
INITIAL_MASTER_NODE_COUNT_SETTING, INITIAL_MASTER_NODES_SETTING).anyMatch(s -> s.exists(settings));
}

public void start() {
assert running == false;
running = true;

if ((initialMasterNodeCount > 0 || initialMasterNodes.isEmpty() == false) && transportService.getLocalNode().isMasterNode()) {
if (transportService.getLocalNode().isMasterNode() == false) {
return;
}

if (unconfiguredBootstrapTimeout != null) {
logger.info("no discovery configuration found, will perform best-effort cluster bootstrapping after [{}] " +
"unless existing master is discovered", unconfiguredBootstrapTimeout);
final ThreadContext threadContext = transportService.getThreadPool().getThreadContext();
try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
threadContext.markAsSystemContext();

transportService.getThreadPool().scheduleUnlessShuttingDown(unconfiguredBootstrapTimeout, Names.SAME, new Runnable() {
@Override
public void run() {
final GetDiscoveredNodesRequest request = new GetDiscoveredNodesRequest();
logger.trace("sending {}", request);
transportService.sendRequest(transportService.getLocalNode(), GetDiscoveredNodesAction.NAME, request,
new TransportResponseHandler<GetDiscoveredNodesResponse>() {
@Override
public void handleResponse(GetDiscoveredNodesResponse response) {
logger.debug("discovered {}, starting to bootstrap", response.getNodes());
awaitBootstrap(response.getBootstrapConfiguration());
}

@Override
public void handleException(TransportException exp) {
logger.warn("discovery attempt failed", exp);
}

@Override
public String executor() {
return Names.SAME;
}

@Override
public GetDiscoveredNodesResponse read(StreamInput in) throws IOException {
return new GetDiscoveredNodesResponse(in);
}
});
}

@Override
public String toString() {
return "unconfigured-discovery delayed bootstrap";
}
});

}
} else if (initialMasterNodeCount > 0 || initialMasterNodes.isEmpty() == false) {
logger.debug("unsafely waiting for discovery of [{}] master-eligible nodes", initialMasterNodeCount);

final ThreadContext threadContext = transportService.getThreadPool().getThreadContext();
Expand Down Expand Up @@ -116,7 +181,6 @@ public GetDiscoveredNodesResponse read(StreamInput in) throws IOException {
}

public void stop() {
assert running == true;
running = false;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -501,7 +501,9 @@ public void startInitialJoin() {
becomeCandidate("startInitialJoin");
}

clusterBootstrapService.start();
if (isInitialConfigurationSet() == false) {
clusterBootstrapService.start();
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -473,6 +473,7 @@ public void apply(Settings value, Settings current, Settings previous) {
TransportAddVotingConfigExclusionsAction.MAXIMUM_VOTING_CONFIG_EXCLUSIONS_SETTING,
ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING,
ClusterBootstrapService.INITIAL_MASTER_NODE_COUNT_SETTING,
ClusterBootstrapService.UNCONFIGURED_BOOTSTRAP_TIMEOUT_SETTING,
LagDetector.CLUSTER_FOLLOWER_LAG_TIMEOUT_SETTING
)));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,14 @@

import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.Constants;
import org.elasticsearch.cluster.coordination.ClusterBootstrapService;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.BoundTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.zen.SettingsBasedHostsProvider;
import org.elasticsearch.monitor.jvm.JvmInfo;
import org.elasticsearch.node.NodeValidationException;
import org.elasticsearch.test.ESTestCase;
Expand Down Expand Up @@ -700,4 +704,34 @@ public boolean alwaysEnforce() {
assertThat(alwaysEnforced, hasToString(containsString("error")));
}

public void testDiscoveryConfiguredCheck() throws NodeValidationException {
final List<BootstrapCheck> checks = Collections.singletonList(new BootstrapChecks.DiscoveryConfiguredCheck());

final BootstrapContext zen2Context = new BootstrapContext(Settings.builder()
.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), ZEN2_DISCOVERY_TYPE).build(), MetaData.EMPTY_META_DATA);

// not always enforced
BootstrapChecks.check(zen2Context, false, checks);

// not enforced for non-zen2 discovery
BootstrapChecks.check(new BootstrapContext(Settings.builder().put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(),
randomFrom(ZEN_DISCOVERY_TYPE, "single-node", randomAlphaOfLength(5))).build(), MetaData.EMPTY_META_DATA), true, checks);

final NodeValidationException e = expectThrows(NodeValidationException.class,
() -> BootstrapChecks.check(zen2Context, true, checks));
assertThat(e, hasToString(containsString("the default discovery settings are unsuitable for production use; at least one " +
"of [discovery.zen.ping.unicast.hosts, discovery.zen.hosts_provider, cluster.initial_master_nodes] must be configured")));

CheckedConsumer<Settings.Builder, NodeValidationException> ensureChecksPass = b ->
{
final BootstrapContext context = new BootstrapContext(b
.put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), ZEN2_DISCOVERY_TYPE).build(), MetaData.EMPTY_META_DATA);
BootstrapChecks.check(context, true, checks);
};

ensureChecksPass.accept(Settings.builder().putList(DiscoveryModule.DISCOVERY_HOSTS_PROVIDER_SETTING.getKey()));
ensureChecksPass.accept(Settings.builder().putList(SettingsBasedHostsProvider.DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey()));
ensureChecksPass.accept(Settings.builder().put(ClusterBootstrapService.INITIAL_MASTER_NODE_COUNT_SETTING.getKey(), 0));
ensureChecksPass.accept(Settings.builder().putList(ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.getKey()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,14 @@
import org.elasticsearch.action.admin.cluster.bootstrap.BootstrapClusterAction;
import org.elasticsearch.action.admin.cluster.bootstrap.BootstrapClusterRequest;
import org.elasticsearch.action.admin.cluster.bootstrap.BootstrapClusterResponse;
import org.elasticsearch.action.admin.cluster.bootstrap.BootstrapConfiguration.NodeDescription;
import org.elasticsearch.action.admin.cluster.bootstrap.GetDiscoveredNodesAction;
import org.elasticsearch.action.admin.cluster.bootstrap.GetDiscoveredNodesRequest;
import org.elasticsearch.action.admin.cluster.bootstrap.GetDiscoveredNodesResponse;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNode.Role;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.Settings.Builder;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.MockTransport;
Expand All @@ -50,7 +52,11 @@
import static java.util.Collections.singleton;
import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING;
import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODE_COUNT_SETTING;
import static org.elasticsearch.common.settings.Settings.builder;
import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_HOSTS_PROVIDER_SETTING;
import static org.elasticsearch.discovery.zen.SettingsBasedHostsProvider.DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING;
import static org.elasticsearch.node.Node.NODE_NAME_SETTING;
import static org.hamcrest.Matchers.equalTo;

public class ClusterBootstrapServiceTests extends ESTestCase {

Expand All @@ -65,7 +71,7 @@ public void createServices() {
otherNode1 = newDiscoveryNode("other1");
otherNode2 = newDiscoveryNode("other2");

deterministicTaskQueue = new DeterministicTaskQueue(Settings.builder().put(NODE_NAME_SETTING.getKey(), "node").build(), random());
deterministicTaskQueue = new DeterministicTaskQueue(builder().put(NODE_NAME_SETTING.getKey(), "node").build(), random());

final MockTransport transport = new MockTransport() {
@Override
Expand All @@ -77,6 +83,9 @@ protected void onSendRequest(long requestId, String action, TransportRequest req
transportService = transport.createTransportService(Settings.EMPTY, deterministicTaskQueue.getThreadPool(),
TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundTransportAddress -> localNode, null, emptySet());

clusterBootstrapService = new ClusterBootstrapService(builder().put(INITIAL_MASTER_NODE_COUNT_SETTING.getKey(), 3).build(),
transportService);

final Settings settings;
if (randomBoolean()) {
settings = Settings.builder().put(INITIAL_MASTER_NODE_COUNT_SETTING.getKey(), 3).build();
Expand Down Expand Up @@ -109,8 +118,24 @@ public void testDoesNothingOnNonMasterNodes() {
deterministicTaskQueue.runAllTasks();
}

public void testDoesNothingIfSettingIsUnset() {
clusterBootstrapService = new ClusterBootstrapService(Settings.EMPTY, transportService);
public void testDoesNothingByDefaultIfHostsProviderConfigured() {
testConfiguredIfSettingSet(builder().putList(DISCOVERY_HOSTS_PROVIDER_SETTING.getKey()));
}

public void testDoesNothingByDefaultIfUnicastHostsConfigured() {
testConfiguredIfSettingSet(builder().putList(DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey()));
}

public void testDoesNothingByDefaultIfMasterNodeCountConfigured() {
testConfiguredIfSettingSet(builder().put(INITIAL_MASTER_NODE_COUNT_SETTING.getKey(), 0));
}

public void testDoesNothingByDefaultIfMasterNodesConfigured() {
testConfiguredIfSettingSet(builder().putList(INITIAL_MASTER_NODES_SETTING.getKey()));
}

private void testConfiguredIfSettingSet(Builder builder) {
clusterBootstrapService = new ClusterBootstrapService(builder.build(), transportService);
transportService.registerRequestHandler(GetDiscoveredNodesAction.NAME, Names.SAME, GetDiscoveredNodesRequest::new,
(request, channel, task) -> {
throw new AssertionError("should not make a discovery request");
Expand All @@ -119,6 +144,30 @@ public void testDoesNothingIfSettingIsUnset() {
deterministicTaskQueue.runAllTasks();
}

public void testBootstrapsAutomaticallyWithDefaultConfiguration() {
clusterBootstrapService = new ClusterBootstrapService(Settings.EMPTY, transportService);

final Set<DiscoveryNode> discoveredNodes = Stream.of(localNode, otherNode1, otherNode2).collect(Collectors.toSet());
transportService.registerRequestHandler(GetDiscoveredNodesAction.NAME, Names.SAME, GetDiscoveredNodesRequest::new,
(request, channel, task) -> channel.sendResponse(new GetDiscoveredNodesResponse(discoveredNodes)));

final AtomicBoolean bootstrapped = new AtomicBoolean();
transportService.registerRequestHandler(BootstrapClusterAction.NAME, Names.SAME, BootstrapClusterRequest::new,
(request, channel, task) -> {
assertThat(request.getBootstrapConfiguration().getNodeDescriptions().stream()
.map(NodeDescription::getId).collect(Collectors.toSet()),
equalTo(discoveredNodes.stream().map(DiscoveryNode::getId).collect(Collectors.toSet())));

channel.sendResponse(new BootstrapClusterResponse(randomBoolean()));
assertTrue(bootstrapped.compareAndSet(false, true));
});

startServices();
deterministicTaskQueue.runAllTasks();

assertTrue(bootstrapped.get());
}

public void testDoesNotRetryOnDiscoveryFailure() {
transportService.registerRequestHandler(GetDiscoveredNodesAction.NAME, Names.SAME, GetDiscoveredNodesRequest::new,
new TransportRequestHandler<GetDiscoveredNodesRequest>() {
Expand Down

0 comments on commit c32e4fb

Please sign in to comment.