Skip to content

Commit 46c8d00

Browse files
Remove nodes with read-only filesystems (#52680) (#59138)
Today we do not allow a node to start if its filesystem is readonly, but it is possible for a filesystem to become readonly while the node is running. We don't currently have any infrastructure in place to make sure that Elasticsearch behaves well if this happens. A node that cannot write to disk may be poisonous to the rest of the cluster. With this commit we periodically verify that nodes' filesystems are writable. If a node fails these writability checks then it is removed from the cluster and prevented from re-joining until the checks start passing again. Closes #45286 Co-authored-by: Bukhtawar Khan <[email protected]>
1 parent a8220ad commit 46c8d00

26 files changed

+1299
-101
lines changed

server/src/main/java/org/elasticsearch/ElasticsearchException.java

+5
Original file line numberDiff line numberDiff line change
@@ -1046,6 +1046,11 @@ private enum ElasticsearchExceptionHandle {
10461046
org.elasticsearch.indices.recovery.PeerRecoveryNotFound.class,
10471047
org.elasticsearch.indices.recovery.PeerRecoveryNotFound::new,
10481048
158,
1049+
Version.V_7_9_0),
1050+
NODE_HEALTH_CHECK_FAILURE_EXCEPTION(
1051+
org.elasticsearch.cluster.coordination.NodeHealthCheckFailureException.class,
1052+
org.elasticsearch.cluster.coordination.NodeHealthCheckFailureException::new,
1053+
159,
10491054
Version.V_7_9_0);
10501055

10511056
final Class<? extends ElasticsearchException> exceptionClass;

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

+9-1
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import org.elasticsearch.common.unit.TimeValue;
3333
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
3434
import org.elasticsearch.gateway.GatewayMetaState;
35+
import org.elasticsearch.monitor.StatusInfo;
3536
import org.elasticsearch.threadpool.ThreadPool;
3637
import org.elasticsearch.threadpool.ThreadPool.Names;
3738

@@ -44,6 +45,7 @@
4445
import java.util.stream.StreamSupport;
4546

4647
import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING;
48+
import static org.elasticsearch.monitor.StatusInfo.Status.UNHEALTHY;
4749

4850
public class ClusterFormationFailureHelper {
4951
private static final Logger logger = LogManager.getLogger(ClusterFormationFailureHelper.class);
@@ -124,18 +126,24 @@ static class ClusterFormationState {
124126
private final List<DiscoveryNode> foundPeers;
125127
private final long currentTerm;
126128
private final ElectionStrategy electionStrategy;
129+
private final StatusInfo statusInfo;
127130

128131
ClusterFormationState(Settings settings, ClusterState clusterState, List<TransportAddress> resolvedAddresses,
129-
List<DiscoveryNode> foundPeers, long currentTerm, ElectionStrategy electionStrategy) {
132+
List<DiscoveryNode> foundPeers, long currentTerm, ElectionStrategy electionStrategy,
133+
StatusInfo statusInfo) {
130134
this.settings = settings;
131135
this.clusterState = clusterState;
132136
this.resolvedAddresses = resolvedAddresses;
133137
this.foundPeers = foundPeers;
134138
this.currentTerm = currentTerm;
135139
this.electionStrategy = electionStrategy;
140+
this.statusInfo = statusInfo;
136141
}
137142

138143
String getDescription() {
144+
if (statusInfo.getStatus() == UNHEALTHY) {
145+
return String.format(Locale.ROOT, "this node is unhealthy: %s", statusInfo.getInfo());
146+
}
139147
final List<String> clusterStateNodes = StreamSupport.stream(clusterState.nodes().getMasterNodes().values().spliterator(), false)
140148
.map(n -> n.value.toString()).collect(Collectors.toList());
141149

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

+19-6
Original file line numberDiff line numberDiff line change
@@ -70,6 +70,8 @@
7070
import org.elasticsearch.discovery.SeedHostsProvider;
7171
import org.elasticsearch.discovery.SeedHostsResolver;
7272
import org.elasticsearch.discovery.zen.PendingClusterStateStats;
73+
import org.elasticsearch.monitor.NodeHealthService;
74+
import org.elasticsearch.monitor.StatusInfo;
7375
import org.elasticsearch.threadpool.Scheduler;
7476
import org.elasticsearch.threadpool.ThreadPool.Names;
7577
import org.elasticsearch.transport.TransportResponse.Empty;
@@ -94,6 +96,7 @@
9496
import static org.elasticsearch.cluster.coordination.NoMasterBlockService.NO_MASTER_BLOCK_ID;
9597
import static org.elasticsearch.gateway.ClusterStateUpdaters.hideStateIfNotRecovered;
9698
import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK;
99+
import static org.elasticsearch.monitor.StatusInfo.Status.UNHEALTHY;
97100

98101
public class Coordinator extends AbstractLifecycleComponent implements Discovery {
99102

@@ -153,6 +156,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
153156
private Optional<Join> lastJoin;
154157
private JoinHelper.JoinAccumulator joinAccumulator;
155158
private Optional<CoordinatorPublication> currentPublication = Optional.empty();
159+
private final NodeHealthService nodeHealthService;
156160

157161
/**
158162
* @param nodeName The name of the node, used to name the {@link java.util.concurrent.ExecutorService} of the {@link SeedHostsResolver}.
@@ -162,7 +166,7 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe
162166
NamedWriteableRegistry namedWriteableRegistry, AllocationService allocationService, MasterService masterService,
163167
Supplier<CoordinationState.PersistedState> persistedStateSupplier, SeedHostsProvider seedHostsProvider,
164168
ClusterApplier clusterApplier, Collection<BiConsumer<DiscoveryNode, ClusterState>> onJoinValidators, Random random,
165-
RerouteService rerouteService, ElectionStrategy electionStrategy) {
169+
RerouteService rerouteService, ElectionStrategy electionStrategy, NodeHealthService nodeHealthService) {
166170
this.settings = settings;
167171
this.transportService = transportService;
168172
this.masterService = masterService;
@@ -172,7 +176,7 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe
172176
this.electionStrategy = electionStrategy;
173177
this.joinHelper = new JoinHelper(settings, allocationService, masterService, transportService,
174178
this::getCurrentTerm, this::getStateForMasterService, this::handleJoinRequest, this::joinLeaderInTerm, this.onJoinValidators,
175-
rerouteService);
179+
rerouteService, nodeHealthService);
176180
this.persistedStateSupplier = persistedStateSupplier;
177181
this.noMasterBlockService = new NoMasterBlockService(settings, clusterSettings);
178182
this.lastKnownLeader = Optional.empty();
@@ -182,14 +186,16 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe
182186
this.publishInfoTimeout = PUBLISH_INFO_TIMEOUT_SETTING.get(settings);
183187
this.random = random;
184188
this.electionSchedulerFactory = new ElectionSchedulerFactory(settings, random, transportService.getThreadPool());
185-
this.preVoteCollector = new PreVoteCollector(transportService, this::startElection, this::updateMaxTermSeen, electionStrategy);
189+
this.preVoteCollector = new PreVoteCollector(transportService, this::startElection, this::updateMaxTermSeen, electionStrategy,
190+
nodeHealthService);
186191
configuredHostsResolver = new SeedHostsResolver(nodeName, settings, transportService, seedHostsProvider);
187192
this.peerFinder = new CoordinatorPeerFinder(settings, transportService,
188193
new HandshakingTransportAddressConnector(settings, transportService), configuredHostsResolver);
189194
this.publicationHandler = new PublicationTransportHandler(transportService, namedWriteableRegistry,
190195
this::handlePublishRequest, this::handleApplyCommit);
191-
this.leaderChecker = new LeaderChecker(settings, transportService, this::onLeaderFailure);
192-
this.followersChecker = new FollowersChecker(settings, transportService, this::onFollowerCheckRequest, this::removeNode);
196+
this.leaderChecker = new LeaderChecker(settings, transportService, this::onLeaderFailure, nodeHealthService);
197+
this.followersChecker = new FollowersChecker(settings, transportService, this::onFollowerCheckRequest, this::removeNode,
198+
nodeHealthService);
193199
this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, logger);
194200
this.clusterApplier = clusterApplier;
195201
masterService.setClusterStateSupplier(this::getStateForMasterService);
@@ -202,12 +208,13 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe
202208
transportService::getLocalNode);
203209
this.clusterFormationFailureHelper = new ClusterFormationFailureHelper(settings, this::getClusterFormationState,
204210
transportService.getThreadPool(), joinHelper::logLastFailedJoinAttempt);
211+
this.nodeHealthService = nodeHealthService;
205212
}
206213

207214
private ClusterFormationState getClusterFormationState() {
208215
return new ClusterFormationState(settings, getStateForMasterService(), peerFinder.getLastResolvedAddresses(),
209216
Stream.concat(Stream.of(getLocalNode()), StreamSupport.stream(peerFinder.getFoundPeers().spliterator(), false))
210-
.collect(Collectors.toList()), getCurrentTerm(), electionStrategy);
217+
.collect(Collectors.toList()), getCurrentTerm(), electionStrategy, nodeHealthService.getHealth());
211218
}
212219

213220
private void onLeaderFailure(Exception e) {
@@ -1230,6 +1237,12 @@ public void run() {
12301237
return;
12311238
}
12321239

1240+
final StatusInfo statusInfo = nodeHealthService.getHealth();
1241+
if (statusInfo.getStatus() == UNHEALTHY) {
1242+
logger.debug("skip prevoting as local node is unhealthy: [{}]", statusInfo.getInfo());
1243+
return;
1244+
}
1245+
12331246
if (prevotingRound != null) {
12341247
prevotingRound.close();
12351248
}

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

+17-3
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,8 @@
3434
import org.elasticsearch.common.unit.TimeValue;
3535
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
3636
import org.elasticsearch.discovery.zen.NodesFaultDetection;
37+
import org.elasticsearch.monitor.NodeHealthService;
38+
import org.elasticsearch.monitor.StatusInfo;
3739
import org.elasticsearch.threadpool.ThreadPool.Names;
3840
import org.elasticsearch.transport.ConnectTransportException;
3941
import org.elasticsearch.transport.Transport;
@@ -57,6 +59,7 @@
5759
import java.util.function.Predicate;
5860

5961
import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap;
62+
import static org.elasticsearch.monitor.StatusInfo.Status.UNHEALTHY;
6063

6164
/**
6265
* The FollowersChecker is responsible for allowing a leader to check that its followers are still connected and healthy. On deciding that a
@@ -97,16 +100,17 @@ public class FollowersChecker {
97100
private final Set<DiscoveryNode> faultyNodes = new HashSet<>();
98101

99102
private final TransportService transportService;
100-
103+
private final NodeHealthService nodeHealthService;
101104
private volatile FastResponseState fastResponseState;
102105

103106
public FollowersChecker(Settings settings, TransportService transportService,
104107
Consumer<FollowerCheckRequest> handleRequestAndUpdateState,
105-
BiConsumer<DiscoveryNode, String> onNodeFailure) {
108+
BiConsumer<DiscoveryNode, String> onNodeFailure, NodeHealthService nodeHealthService) {
106109
this.settings = settings;
107110
this.transportService = transportService;
108111
this.handleRequestAndUpdateState = handleRequestAndUpdateState;
109112
this.onNodeFailure = onNodeFailure;
113+
this.nodeHealthService = nodeHealthService;
110114

111115
followerCheckInterval = FOLLOWER_CHECK_INTERVAL_SETTING.get(settings);
112116
followerCheckTimeout = FOLLOWER_CHECK_TIMEOUT_SETTING.get(settings);
@@ -167,8 +171,15 @@ public void updateFastResponseState(final long term, final Mode mode) {
167171
}
168172

169173
private void handleFollowerCheck(FollowerCheckRequest request, TransportChannel transportChannel) throws IOException {
170-
FastResponseState responder = this.fastResponseState;
174+
final StatusInfo statusInfo = nodeHealthService.getHealth();
175+
if (statusInfo.getStatus() == UNHEALTHY) {
176+
final String message
177+
= "handleFollowerCheck: node is unhealthy [" + statusInfo.getInfo() + "], rejecting " + statusInfo.getInfo();
178+
logger.debug(message);
179+
throw new NodeHealthCheckFailureException(message);
180+
}
171181

182+
final FastResponseState responder = this.fastResponseState;
172183
if (responder.mode == Mode.FOLLOWER && responder.term == request.term) {
173184
logger.trace("responding to {} on fast path", request);
174185
transportChannel.sendResponse(Empty.INSTANCE);
@@ -340,6 +351,9 @@ public void handleException(TransportException exp) {
340351
|| exp.getCause() instanceof ConnectTransportException) {
341352
logger.debug(() -> new ParameterizedMessage("{} disconnected", FollowerChecker.this), exp);
342353
reason = "disconnected";
354+
} else if (exp.getCause() instanceof NodeHealthCheckFailureException) {
355+
logger.debug(() -> new ParameterizedMessage("{} health check failed", FollowerChecker.this), exp);
356+
reason = "health check failed";
343357
} else {
344358
logger.debug(() -> new ParameterizedMessage("{} failed, retrying", FollowerChecker.this), exp);
345359
scheduleNextWakeUp();

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

+13-1
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,8 @@
4444
import org.elasticsearch.discovery.zen.MembershipAction;
4545
import org.elasticsearch.discovery.zen.ZenDiscovery;
4646
import org.elasticsearch.discovery.DiscoveryModule;
47+
import org.elasticsearch.monitor.NodeHealthService;
48+
import org.elasticsearch.monitor.StatusInfo;
4749
import org.elasticsearch.threadpool.ThreadPool;
4850
import org.elasticsearch.threadpool.ThreadPool.Names;
4951
import org.elasticsearch.transport.TransportChannel;
@@ -71,6 +73,8 @@
7173
import java.util.function.LongSupplier;
7274
import java.util.function.Supplier;
7375

76+
import static org.elasticsearch.monitor.StatusInfo.Status.UNHEALTHY;
77+
7478
public class JoinHelper {
7579

7680
private static final Logger logger = LogManager.getLogger(JoinHelper.class);
@@ -90,6 +94,7 @@ public class JoinHelper {
9094

9195
@Nullable // if using single-node discovery
9296
private final TimeValue joinTimeout;
97+
private final NodeHealthService nodeHealthService;
9398

9499
private final Set<Tuple<DiscoveryNode, JoinRequest>> pendingOutgoingJoins = Collections.synchronizedSet(new HashSet<>());
95100

@@ -98,9 +103,11 @@ public class JoinHelper {
98103
JoinHelper(Settings settings, AllocationService allocationService, MasterService masterService,
99104
TransportService transportService, LongSupplier currentTermSupplier, Supplier<ClusterState> currentStateSupplier,
100105
BiConsumer<JoinRequest, JoinCallback> joinHandler, Function<StartJoinRequest, Join> joinLeaderInTerm,
101-
Collection<BiConsumer<DiscoveryNode, ClusterState>> joinValidators, RerouteService rerouteService) {
106+
Collection<BiConsumer<DiscoveryNode, ClusterState>> joinValidators, RerouteService rerouteService,
107+
NodeHealthService nodeHealthService) {
102108
this.masterService = masterService;
103109
this.transportService = transportService;
110+
this.nodeHealthService = nodeHealthService;
104111
this.joinTimeout = DiscoveryModule.isSingleNodeDiscovery(settings) ? null : JOIN_TIMEOUT_SETTING.get(settings);
105112
this.joinTaskExecutor = new JoinTaskExecutor(settings, allocationService, logger, rerouteService) {
106113

@@ -268,6 +275,11 @@ void logLastFailedJoinAttempt() {
268275

269276
public void sendJoinRequest(DiscoveryNode destination, long term, Optional<Join> optionalJoin, Runnable onCompletion) {
270277
assert destination.isMasterNode() : "trying to join master-ineligible " + destination;
278+
final StatusInfo statusInfo = nodeHealthService.getHealth();
279+
if (statusInfo.getStatus() == UNHEALTHY) {
280+
logger.debug("dropping join request to [{}]: [{}]", destination, statusInfo.getInfo());
281+
return;
282+
}
271283
final JoinRequest joinRequest = new JoinRequest(transportService.getLocalNode(), term, optionalJoin);
272284
final Tuple<DiscoveryNode, JoinRequest> dedupKey = Tuple.tuple(destination, joinRequest);
273285
if (pendingOutgoingJoins.add(dedupKey)) {

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

+20-4
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,8 @@
3434
import org.elasticsearch.common.settings.Settings;
3535
import org.elasticsearch.common.unit.TimeValue;
3636
import org.elasticsearch.discovery.zen.MasterFaultDetection;
37+
import org.elasticsearch.monitor.NodeHealthService;
38+
import org.elasticsearch.monitor.StatusInfo;
3739
import org.elasticsearch.threadpool.ThreadPool.Names;
3840
import org.elasticsearch.transport.ConnectTransportException;
3941
import org.elasticsearch.transport.NodeDisconnectedException;
@@ -55,6 +57,8 @@
5557
import java.util.concurrent.atomic.AtomicReference;
5658
import java.util.function.Consumer;
5759

60+
import static org.elasticsearch.monitor.StatusInfo.Status.UNHEALTHY;
61+
5862
/**
5963
* The LeaderChecker is responsible for allowing followers to check that the currently elected leader is still connected and healthy. We are
6064
* fairly lenient, possibly allowing multiple checks to fail before considering the leader to be faulty, to allow for the leader to
@@ -88,18 +92,21 @@ public class LeaderChecker {
8892
private final int leaderCheckRetryCount;
8993
private final TransportService transportService;
9094
private final Consumer<Exception> onLeaderFailure;
95+
private final NodeHealthService nodeHealthService;
9196

9297
private AtomicReference<CheckScheduler> currentChecker = new AtomicReference<>();
9398

9499
private volatile DiscoveryNodes discoveryNodes;
95100

96-
LeaderChecker(final Settings settings, final TransportService transportService, final Consumer<Exception> onLeaderFailure) {
101+
LeaderChecker(final Settings settings, final TransportService transportService, final Consumer<Exception> onLeaderFailure,
102+
NodeHealthService nodeHealthService) {
97103
this.settings = settings;
98104
leaderCheckInterval = LEADER_CHECK_INTERVAL_SETTING.get(settings);
99105
leaderCheckTimeout = LEADER_CHECK_TIMEOUT_SETTING.get(settings);
100106
leaderCheckRetryCount = LEADER_CHECK_RETRY_COUNT_SETTING.get(settings);
101107
this.transportService = transportService;
102108
this.onLeaderFailure = onLeaderFailure;
109+
this.nodeHealthService = nodeHealthService;
103110

104111
transportService.registerRequestHandler(LEADER_CHECK_ACTION_NAME, Names.SAME, false, false, LeaderCheckRequest::new,
105112
(request, channel, task) -> {
@@ -169,8 +176,13 @@ boolean currentNodeIsMaster() {
169176
private void handleLeaderCheck(LeaderCheckRequest request) {
170177
final DiscoveryNodes discoveryNodes = this.discoveryNodes;
171178
assert discoveryNodes != null;
172-
173-
if (discoveryNodes.isLocalNodeElectedMaster() == false) {
179+
final StatusInfo statusInfo = nodeHealthService.getHealth();
180+
if (statusInfo.getStatus() == UNHEALTHY) {
181+
final String message = "rejecting leader check from [" + request.getSender() + "] " +
182+
"since node is unhealthy [" + statusInfo.getInfo() + "]";
183+
logger.debug(message);
184+
throw new NodeHealthCheckFailureException(message);
185+
} else if (discoveryNodes.isLocalNodeElectedMaster() == false) {
174186
logger.debug("rejecting leader check on non-master {}", request);
175187
throw new CoordinationStateRejectedException(
176188
"rejecting leader check from [" + request.getSender() + "] sent to a node that is no longer the master");
@@ -266,8 +278,12 @@ public void handleException(TransportException exp) {
266278
"leader [{}] disconnected during check", leader), exp);
267279
leaderFailed(new ConnectTransportException(leader, "disconnected during check", exp));
268280
return;
281+
} else if (exp.getCause() instanceof NodeHealthCheckFailureException) {
282+
logger.debug(new ParameterizedMessage(
283+
"leader [{}] health check failed", leader), exp);
284+
leaderFailed(new NodeHealthCheckFailureException("node [" + leader + "] failed health checks", exp));
285+
return;
269286
}
270-
271287
long failureCount = failureCountSinceLastSuccess.incrementAndGet();
272288
if (failureCount >= leaderCheckRetryCount) {
273289
logger.debug(new ParameterizedMessage(

0 commit comments

Comments
 (0)