Skip to content

Commit 9d41798

Browse files
authored
[Zen2] Add warning if cluster fails to form fast enough (#35993)
* Add warning if cluster fails to form fast enough Today if a leader is not discovered or elected then nodes are essentially silent at INFO and above, and log copiously at DEBUG and below. A short delay when electing a leader is not unusual, for instance if other nodes have not yet started, but a persistent failure to elect a leader is a problem worthy of log messages in the default configuration. With this change, while there is no leader each node outputs a WARN-level log message every 10 seconds (by default) indicating as such, describing the current discovery state and the current quorum(s). * Add note about whether the discovered nodes form a quorum or not * Introduce separate ClusterFormationFailureHelper ... and back out the unnecessary changes elsewhere * It can be volatile
1 parent f2df0a5 commit 9d41798

File tree

5 files changed

+532
-5
lines changed

5 files changed

+532
-5
lines changed
Lines changed: 209 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,209 @@
1+
/*
2+
* Licensed to Elasticsearch under one or more contributor
3+
* license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright
5+
* ownership. Elasticsearch licenses this file to you under
6+
* the Apache License, Version 2.0 (the "License"); you may
7+
* not use this file except in compliance with the License.
8+
* You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.elasticsearch.cluster.coordination;
20+
21+
import org.apache.logging.log4j.LogManager;
22+
import org.apache.logging.log4j.Logger;
23+
import org.elasticsearch.Version;
24+
import org.elasticsearch.cluster.ClusterState;
25+
import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration;
26+
import org.elasticsearch.cluster.coordination.CoordinationState.VoteCollection;
27+
import org.elasticsearch.cluster.node.DiscoveryNode;
28+
import org.elasticsearch.common.Nullable;
29+
import org.elasticsearch.common.settings.Setting;
30+
import org.elasticsearch.common.settings.Settings;
31+
import org.elasticsearch.common.transport.TransportAddress;
32+
import org.elasticsearch.common.unit.TimeValue;
33+
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
34+
import org.elasticsearch.threadpool.ThreadPool;
35+
import org.elasticsearch.threadpool.ThreadPool.Names;
36+
37+
import java.util.List;
38+
import java.util.Locale;
39+
import java.util.Set;
40+
import java.util.function.Supplier;
41+
import java.util.stream.Collectors;
42+
import java.util.stream.StreamSupport;
43+
44+
import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING;
45+
import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODE_COUNT_SETTING;
46+
47+
public class ClusterFormationFailureHelper {
48+
private static final Logger logger = LogManager.getLogger(ClusterFormationFailureHelper.class);
49+
50+
public static final Setting<TimeValue> DISCOVERY_CLUSTER_FORMATION_WARNING_TIMEOUT_SETTING =
51+
Setting.timeSetting("discovery.cluster_formation_warning_timeout",
52+
TimeValue.timeValueMillis(10000), TimeValue.timeValueMillis(1), Setting.Property.NodeScope);
53+
54+
private final Supplier<ClusterFormationState> clusterFormationStateSupplier;
55+
private final ThreadPool threadPool;
56+
private final TimeValue clusterFormationWarningTimeout;
57+
@Nullable // if no warning is scheduled
58+
private volatile WarningScheduler warningScheduler;
59+
60+
public ClusterFormationFailureHelper(Settings settings, Supplier<ClusterFormationState> clusterFormationStateSupplier,
61+
ThreadPool threadPool) {
62+
this.clusterFormationStateSupplier = clusterFormationStateSupplier;
63+
this.threadPool = threadPool;
64+
this.clusterFormationWarningTimeout = DISCOVERY_CLUSTER_FORMATION_WARNING_TIMEOUT_SETTING.get(settings);
65+
}
66+
67+
public boolean isRunning() {
68+
return warningScheduler != null;
69+
}
70+
71+
public void start() {
72+
assert warningScheduler == null;
73+
warningScheduler = new WarningScheduler();
74+
warningScheduler.scheduleNextWarning();
75+
}
76+
77+
public void stop() {
78+
warningScheduler = null;
79+
}
80+
81+
private class WarningScheduler {
82+
83+
private boolean isActive() {
84+
return warningScheduler == this;
85+
}
86+
87+
void scheduleNextWarning() {
88+
threadPool.scheduleUnlessShuttingDown(clusterFormationWarningTimeout, Names.GENERIC, new AbstractRunnable() {
89+
@Override
90+
public void onFailure(Exception e) {
91+
logger.debug("unexpected exception scheduling cluster formation warning", e);
92+
}
93+
94+
@Override
95+
protected void doRun() {
96+
if (isActive()) {
97+
logger.warn(clusterFormationStateSupplier.get().getDescription());
98+
}
99+
}
100+
101+
@Override
102+
public void onAfter() {
103+
if (isActive()) {
104+
scheduleNextWarning();
105+
}
106+
}
107+
108+
@Override
109+
public String toString() {
110+
return "emit warning if cluster not formed";
111+
}
112+
});
113+
}
114+
}
115+
116+
static class ClusterFormationState {
117+
private final Settings settings;
118+
private final ClusterState clusterState;
119+
private final List<TransportAddress> resolvedAddresses;
120+
private final List<DiscoveryNode> foundPeers;
121+
122+
ClusterFormationState(Settings settings, ClusterState clusterState, List<TransportAddress> resolvedAddresses,
123+
List<DiscoveryNode> foundPeers) {
124+
this.settings = settings;
125+
this.clusterState = clusterState;
126+
this.resolvedAddresses = resolvedAddresses;
127+
this.foundPeers = foundPeers;
128+
}
129+
130+
String getDescription() {
131+
final List<String> clusterStateNodes
132+
= StreamSupport.stream(clusterState.nodes().spliterator(), false).map(DiscoveryNode::toString).collect(Collectors.toList());
133+
134+
final String discoveryWillContinueDescription = String.format(Locale.ROOT,
135+
"discovery will continue using %s from hosts providers and %s from last-known cluster state",
136+
resolvedAddresses, clusterStateNodes);
137+
138+
final String discoveryStateIgnoringQuorum = String.format(Locale.ROOT, "have discovered %s; %s",
139+
foundPeers, discoveryWillContinueDescription);
140+
141+
if (clusterState.nodes().getLocalNode().isMasterNode() == false) {
142+
return String.format(Locale.ROOT, "master not discovered yet: %s", discoveryStateIgnoringQuorum);
143+
}
144+
145+
if (clusterState.getLastAcceptedConfiguration().isEmpty()) {
146+
147+
// TODO handle the case that there is a 6.x node around here, when rolling upgrades are supported
148+
149+
final String bootstrappingDescription;
150+
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))) {
153+
bootstrappingDescription = "cluster bootstrapping is disabled 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()) {
159+
// TODO update this when we can bootstrap on only a quorum of the initial nodes
160+
bootstrappingDescription = String.format(Locale.ROOT,
161+
"this node must discover master-eligible nodes %s to bootstrap a cluster",
162+
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));
168+
}
169+
170+
return String.format(Locale.ROOT,
171+
"master not discovered yet, this node has not previously joined a bootstrapped (v%d+) cluster, and %s: %s",
172+
Version.V_6_6_0.major + 1, bootstrappingDescription, discoveryStateIgnoringQuorum);
173+
}
174+
175+
assert clusterState.getLastCommittedConfiguration().isEmpty() == false;
176+
177+
final String quorumDescription;
178+
if (clusterState.getLastAcceptedConfiguration().equals(clusterState.getLastCommittedConfiguration())) {
179+
quorumDescription = describeQuorum(clusterState.getLastAcceptedConfiguration());
180+
} else {
181+
quorumDescription = describeQuorum(clusterState.getLastAcceptedConfiguration())
182+
+ " and "
183+
+ describeQuorum(clusterState.getLastCommittedConfiguration());
184+
}
185+
186+
final VoteCollection voteCollection = new VoteCollection();
187+
foundPeers.forEach(voteCollection::addVote);
188+
final String isQuorumOrNot
189+
= CoordinationState.isElectionQuorum(voteCollection, clusterState) ? "is a quorum" : "is not a quorum";
190+
191+
return String.format(Locale.ROOT,
192+
"master not discovered or elected yet, an election requires %s, have discovered %s which %s; %s",
193+
quorumDescription, foundPeers, isQuorumOrNot, discoveryWillContinueDescription);
194+
}
195+
196+
private String describeQuorum(VotingConfiguration votingConfiguration) {
197+
final Set<String> nodeIds = votingConfiguration.getNodeIds();
198+
assert nodeIds.isEmpty() == false;
199+
200+
if (nodeIds.size() == 1) {
201+
return "a node with id " + nodeIds;
202+
} else if (nodeIds.size() == 2) {
203+
return "two nodes with ids " + nodeIds;
204+
} else {
205+
return "at least " + (nodeIds.size() / 2 + 1) + " nodes with ids from " + nodeIds;
206+
}
207+
}
208+
}
209+
}

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

Lines changed: 17 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -32,8 +32,9 @@
3232
import org.elasticsearch.cluster.ClusterStateTaskConfig;
3333
import org.elasticsearch.cluster.ClusterStateUpdateTask;
3434
import org.elasticsearch.cluster.block.ClusterBlocks;
35-
import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration;
35+
import org.elasticsearch.cluster.coordination.ClusterFormationFailureHelper.ClusterFormationState;
3636
import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfigExclusion;
37+
import org.elasticsearch.cluster.coordination.CoordinationMetaData.VotingConfiguration;
3738
import org.elasticsearch.cluster.coordination.FollowersChecker.FollowerCheckRequest;
3839
import org.elasticsearch.cluster.coordination.JoinHelper.InitialJoinAccumulator;
3940
import org.elasticsearch.cluster.metadata.MetaData;
@@ -121,6 +122,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery
121122
private final Reconfigurator reconfigurator;
122123
private final ClusterBootstrapService clusterBootstrapService;
123124
private final LagDetector lagDetector;
125+
private final ClusterFormationFailureHelper clusterFormationFailureHelper;
124126

125127
private Mode mode;
126128
private Optional<DiscoveryNode> lastKnownLeader;
@@ -161,6 +163,13 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe
161163
this.clusterBootstrapService = new ClusterBootstrapService(settings, transportService);
162164
this.lagDetector = new LagDetector(settings, transportService.getThreadPool(), n -> removeNode(n, "lagging"),
163165
transportService::getLocalNode);
166+
this.clusterFormationFailureHelper = new ClusterFormationFailureHelper(settings, this::getClusterFormationState,
167+
transportService.getThreadPool());
168+
}
169+
170+
private ClusterFormationState getClusterFormationState() {
171+
return new ClusterFormationState(settings, getStateForMasterService(), peerFinder.getLastResolvedAddresses(),
172+
StreamSupport.stream(peerFinder.getFoundPeers().spliterator(), false).collect(Collectors.toList()));
164173
}
165174

166175
private Runnable getOnLeaderFailure() {
@@ -374,6 +383,7 @@ void becomeCandidate(String method) {
374383
joinAccumulator = joinHelper.new CandidateJoinAccumulator();
375384

376385
peerFinder.activate(coordinationState.get().getLastAcceptedState().nodes());
386+
clusterFormationFailureHelper.start();
377387
leaderChecker.setCurrentNodes(DiscoveryNodes.EMPTY_NODES);
378388
leaderChecker.updateLeader(null);
379389

@@ -404,6 +414,7 @@ void becomeLeader(String method) {
404414

405415
lastKnownLeader = Optional.of(getLocalNode());
406416
peerFinder.deactivate(getLocalNode());
417+
clusterFormationFailureHelper.stop();
407418
closePrevotingAndElectionScheduler();
408419
preVoteCollector.update(getPreVoteResponse(), getLocalNode());
409420

@@ -428,6 +439,7 @@ void becomeFollower(String method, DiscoveryNode leaderNode) {
428439

429440
lastKnownLeader = Optional.of(leaderNode);
430441
peerFinder.deactivate(leaderNode);
442+
clusterFormationFailureHelper.stop();
431443
closePrevotingAndElectionScheduler();
432444
cancelActivePublication();
433445
preVoteCollector.update(getPreVoteResponse(), leaderNode);
@@ -543,6 +555,7 @@ public void invariant() {
543555
assert leaderChecker.leader() == null : leaderChecker.leader();
544556
assert applierState.nodes().getMasterNodeId() == null || getLocalNode().equals(applierState.nodes().getMasterNode());
545557
assert preVoteCollector.getLeader() == getLocalNode() : preVoteCollector;
558+
assert clusterFormationFailureHelper.isRunning() == false;
546559

547560
final boolean activePublication = currentPublication.map(CoordinatorPublication::isActiveForCurrentLeader).orElse(false);
548561
if (becomingMaster && activePublication == false) {
@@ -582,6 +595,7 @@ public void invariant() {
582595
assert followersChecker.getKnownFollowers().isEmpty();
583596
assert currentPublication.map(Publication::isCommitted).orElse(true);
584597
assert preVoteCollector.getLeader().equals(lastKnownLeader.get()) : preVoteCollector;
598+
assert clusterFormationFailureHelper.isRunning() == false;
585599
} else {
586600
assert mode == Mode.CANDIDATE;
587601
assert joinAccumulator instanceof JoinHelper.CandidateJoinAccumulator;
@@ -594,6 +608,7 @@ public void invariant() {
594608
assert applierState.nodes().getMasterNodeId() == null;
595609
assert currentPublication.map(Publication::isCommitted).orElse(true);
596610
assert preVoteCollector.getLeader() == null : preVoteCollector;
611+
assert clusterFormationFailureHelper.isRunning();
597612
}
598613
}
599614
}
@@ -823,7 +838,7 @@ public void publish(ClusterChangedEvent clusterChangedEvent, ActionListener<Void
823838
Strings.toString(clusterChangedEvent.previousState()).equals(
824839
Strings.toString(clusterStateWithNoMasterBlock(coordinationState.get().getLastAcceptedState())))
825840
: Strings.toString(clusterChangedEvent.previousState()) + " vs "
826-
+ Strings.toString(clusterStateWithNoMasterBlock(coordinationState.get().getLastAcceptedState()));
841+
+ Strings.toString(clusterStateWithNoMasterBlock(coordinationState.get().getLastAcceptedState()));
827842

828843
final ClusterState clusterState = clusterChangedEvent.state();
829844

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,11 +33,12 @@
3333
import org.elasticsearch.cluster.NodeConnectionsService;
3434
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
3535
import org.elasticsearch.cluster.coordination.ClusterBootstrapService;
36-
import org.elasticsearch.cluster.coordination.LagDetector;
36+
import org.elasticsearch.cluster.coordination.ClusterFormationFailureHelper;
3737
import org.elasticsearch.cluster.coordination.Coordinator;
3838
import org.elasticsearch.cluster.coordination.ElectionSchedulerFactory;
3939
import org.elasticsearch.cluster.coordination.FollowersChecker;
4040
import org.elasticsearch.cluster.coordination.JoinHelper;
41+
import org.elasticsearch.cluster.coordination.LagDetector;
4142
import org.elasticsearch.cluster.coordination.LeaderChecker;
4243
import org.elasticsearch.cluster.coordination.Reconfigurator;
4344
import org.elasticsearch.cluster.metadata.IndexGraveyard;
@@ -457,6 +458,7 @@ public void apply(Settings value, Settings current, Settings previous) {
457458
EnableAssignmentDecider.CLUSTER_TASKS_ALLOCATION_ENABLE_SETTING,
458459
PeerFinder.DISCOVERY_FIND_PEERS_INTERVAL_SETTING,
459460
PeerFinder.DISCOVERY_REQUEST_PEERS_TIMEOUT_SETTING,
461+
ClusterFormationFailureHelper.DISCOVERY_CLUSTER_FORMATION_WARNING_TIMEOUT_SETTING,
460462
ElectionSchedulerFactory.ELECTION_INITIAL_TIMEOUT_SETTING,
461463
ElectionSchedulerFactory.ELECTION_BACK_OFF_TIME_SETTING,
462464
ElectionSchedulerFactory.ELECTION_MAX_TIMEOUT_SETTING,

server/src/main/java/org/elasticsearch/discovery/PeerFinder.java

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -62,6 +62,7 @@
6262
import java.util.function.Consumer;
6363
import java.util.stream.Collectors;
6464

65+
import static java.util.Collections.emptyList;
6566
import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap;
6667

6768
public abstract class PeerFinder {
@@ -94,6 +95,7 @@ public abstract class PeerFinder {
9495
private DiscoveryNodes lastAcceptedNodes;
9596
private final Map<TransportAddress, Peer> peersByAddress = newConcurrentMap();
9697
private Optional<DiscoveryNode> leader = Optional.empty();
98+
private volatile List<TransportAddress> lastResolvedAddresses = emptyList();
9799

98100
public PeerFinder(Settings settings, TransportService transportService, TransportAddressConnector transportAddressConnector,
99101
ConfiguredHostsResolver configuredHostsResolver) {
@@ -164,7 +166,7 @@ PeersResponse handlePeersRequest(PeersRequest peersRequest) {
164166
knownPeers = getFoundPeersUnderLock();
165167
} else {
166168
assert leader.isPresent() || lastAcceptedNodes == null;
167-
knownPeers = Collections.emptyList();
169+
knownPeers = emptyList();
168170
}
169171
return new PeersResponse(leader, knownPeers, currentTerm);
170172
}
@@ -207,6 +209,10 @@ private DiscoveryNode getLocalNode() {
207209
*/
208210
protected abstract void onFoundPeersUpdated();
209211

212+
public List<TransportAddress> getLastResolvedAddresses() {
213+
return lastResolvedAddresses;
214+
}
215+
210216
public interface TransportAddressConnector {
211217
/**
212218
* Identify the node at the given address and, if it is a master node and not the local node then establish a full connection to it.
@@ -266,6 +272,7 @@ private boolean handleWakeUp() {
266272

267273
configuredHostsResolver.resolveConfiguredHosts(providedAddresses -> {
268274
synchronized (mutex) {
275+
lastResolvedAddresses = providedAddresses;
269276
logger.trace("probing resolved transport addresses {}", providedAddresses);
270277
providedAddresses.forEach(this::startProbe);
271278
}
@@ -495,7 +502,7 @@ private class Zen1UnicastPingRequestHandler implements TransportRequestHandler<U
495502
@Override
496503
public void messageReceived(UnicastZenPing.UnicastPingRequest request, TransportChannel channel, Task task) throws Exception {
497504
final PeersRequest peersRequest = new PeersRequest(request.pingResponse.node(),
498-
Optional.ofNullable(request.pingResponse.master()).map(Collections::singletonList).orElse(Collections.emptyList()));
505+
Optional.ofNullable(request.pingResponse.master()).map(Collections::singletonList).orElse(emptyList()));
499506
final PeersResponse peersResponse = handlePeersRequest(peersRequest);
500507
final List<ZenPing.PingResponse> pingResponses = new ArrayList<>();
501508
final ClusterName clusterName = ClusterName.CLUSTER_NAME_SETTING.get(settings);

0 commit comments

Comments
 (0)