Skip to content
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,176 @@
/*
* 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.coordination;

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.ElasticsearchTimeoutException;
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;
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.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.threadpool.ThreadPool.Names;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportService;

import java.io.IOException;

public class ClusterBootstrapService {

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

// The number of master-eligible nodes which, if discovered, can be used to bootstrap the cluster. This setting is unsafe in the event
// that more master nodes are started than expected.
public static final Setting<Integer> INITIAL_MASTER_NODE_COUNT_SETTING =
Setting.intSetting("cluster.unsafe_initial_master_node_count", 0, 0, Property.NodeScope);

private final int initialMasterNodeCount;
private final TransportService transportService;
private volatile boolean running;

public ClusterBootstrapService(Settings settings, TransportService transportService) {
initialMasterNodeCount = INITIAL_MASTER_NODE_COUNT_SETTING.get(settings);
this.transportService = transportService;
}

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

if (initialMasterNodeCount > 0 && transportService.getLocalNode().isMasterNode()) {
logger.debug("unsafely waiting for discovery of [{}] master-eligible nodes", initialMasterNodeCount);

final ThreadContext threadContext = transportService.getThreadPool().getThreadContext();
try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
threadContext.markAsSystemContext();
awaitDiscovery();
}
}
}

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

private void awaitDiscovery() {
if (running == false) {
logger.debug("awaitDiscovery: not running");
return;
}

final GetDiscoveredNodesRequest request = new GetDiscoveredNodesRequest();
request.setWaitForNodes(initialMasterNodeCount);
logger.trace("sending {}", request);
transportService.sendRequest(transportService.getLocalNode(), GetDiscoveredNodesAction.NAME, request,
new TransportResponseHandler<GetDiscoveredNodesResponse>() {
@Override
public void handleResponse(GetDiscoveredNodesResponse response) {
assert response.getNodes().size() >= initialMasterNodeCount;
assert response.getNodes().stream().allMatch(DiscoveryNode::isMasterNode);

logger.debug("discovered {}, starting to bootstrap", response.getNodes());
awaitBootstrap(response.getBootstrapConfiguration());
}

@Override
public void handleException(TransportException exp) {
if (exp.getRootCause() instanceof ElasticsearchTimeoutException) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we simply set the timeout to a very high value instead of adding this retry logic here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How high is high enough? I'd prefer to retry forever rather than have to remember that this might time out and stop in a future debugging session.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can also make the timeout optional, so that setting it to null makes it unbounded (i.e. does not schedule a timeout)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, introduced nullability to the timeout in 9f7e951.

logger.debug(new ParameterizedMessage("discovery attempt timed out, retrying, request={}", request), exp);
awaitDiscovery();
} else {
// exceptions other than a timeout are fatal
logger.warn("discovery attempt failed, not retrying", exp);
}
}

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

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

private void awaitBootstrap(final BootstrapConfiguration bootstrapConfiguration) {
if (running == false) {
logger.debug("awaitBootstrap: not running");
return;
}

BootstrapClusterRequest request = new BootstrapClusterRequest(bootstrapConfiguration);
logger.trace("sending {}", request);
transportService.sendRequest(transportService.getLocalNode(), BootstrapClusterAction.NAME, request,
new TransportResponseHandler<BootstrapClusterResponse>() {
@Override
public void handleResponse(BootstrapClusterResponse response) {
logger.debug("bootstrapped successful: received {}", response);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

successfully?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed the message in 12d90d2 - I meant "bootstrapping" not "bootstrapped".

}

@Override
public void handleException(TransportException exp) {
// log a warning since a failure here indicates a bad problem, such as:
// - bootstrap configuration resolution failed (e.g. discovered nodes no longer match those in the bootstrap config)
// - discovered nodes no longer form a quorum in the bootstrap config
logger.warn(new ParameterizedMessage("automatic cluster bootstrapping failed, retrying [{}]",
bootstrapConfiguration.getNodeDescriptions()), exp);

// There's not really much else we can do apart from retry and hope that the problem goes away. The retry is delayed
// since a tight loop here is unlikely to help.
transportService.getThreadPool().scheduleUnlessShuttingDown(TimeValue.timeValueSeconds(10), Names.SAME, new Runnable() {
@Override
public void run() {
awaitBootstrap(bootstrapConfiguration);
}

@Override
public String toString() {
return "retry bootstrapping with " + bootstrapConfiguration.getNodeDescriptions();
}
});
}

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

@Override
public BootstrapClusterResponse read(StreamInput in) throws IOException {
return new BootstrapClusterResponse(in);
}
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
private Releasable prevotingRound;
private long maxTermSeen;
private final Reconfigurator reconfigurator;
private final ClusterBootstrapService clusterBootstrapService;

private Mode mode;
private Optional<DiscoveryNode> lastKnownLeader;
Expand Down Expand Up @@ -151,6 +152,7 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe
this.clusterApplier = clusterApplier;
masterService.setClusterStateSupplier(this::getStateForMasterService);
this.reconfigurator = new Reconfigurator(settings, clusterSettings);
this.clusterBootstrapService = new ClusterBootstrapService(settings, transportService);
}

private Runnable getOnLeaderFailure() {
Expand Down Expand Up @@ -483,11 +485,14 @@ public void startInitialJoin() {
synchronized (mutex) {
becomeCandidate("startInitialJoin");
}

clusterBootstrapService.start();
}

@Override
protected void doStop() {
configuredHostsResolver.stop();
clusterBootstrapService.stop();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.elasticsearch.cluster.InternalClusterInfoService;
import org.elasticsearch.cluster.NodeConnectionsService;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.coordination.ClusterBootstrapService;
import org.elasticsearch.cluster.coordination.Coordinator;
import org.elasticsearch.cluster.coordination.ElectionSchedulerFactory;
import org.elasticsearch.cluster.coordination.JoinHelper;
Expand Down Expand Up @@ -459,7 +460,8 @@ public void apply(Settings value, Settings current, Settings previous) {
Coordinator.PUBLISH_TIMEOUT_SETTING,
JoinHelper.JOIN_TIMEOUT_SETTING,
Reconfigurator.CLUSTER_AUTO_SHRINK_VOTING_CONFIGURATION,
TransportAddVotingTombstonesAction.MAXIMUM_VOTING_TOMBSTONES_SETTING
TransportAddVotingTombstonesAction.MAXIMUM_VOTING_TOMBSTONES_SETTING,
ClusterBootstrapService.INITIAL_MASTER_NODE_COUNT_SETTING
)));

public static List<SettingUpgrader<?>> BUILT_IN_SETTING_UPGRADERS = Collections.unmodifiableList(Arrays.asList(
Expand Down
Loading