|
| 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 | + |
| 20 | +package org.elasticsearch.cluster.coordination; |
| 21 | + |
| 22 | +import org.apache.logging.log4j.message.ParameterizedMessage; |
| 23 | +import org.elasticsearch.cluster.node.DiscoveryNode; |
| 24 | +import org.elasticsearch.cluster.node.DiscoveryNodes; |
| 25 | +import org.elasticsearch.common.component.AbstractComponent; |
| 26 | +import org.elasticsearch.common.io.stream.StreamInput; |
| 27 | +import org.elasticsearch.common.io.stream.StreamOutput; |
| 28 | +import org.elasticsearch.common.lease.Releasable; |
| 29 | +import org.elasticsearch.common.settings.Setting; |
| 30 | +import org.elasticsearch.common.settings.Settings; |
| 31 | +import org.elasticsearch.common.unit.TimeValue; |
| 32 | +import org.elasticsearch.tasks.Task; |
| 33 | +import org.elasticsearch.threadpool.ThreadPool.Names; |
| 34 | +import org.elasticsearch.transport.ConnectTransportException; |
| 35 | +import org.elasticsearch.transport.TransportChannel; |
| 36 | +import org.elasticsearch.transport.TransportException; |
| 37 | +import org.elasticsearch.transport.TransportRequest; |
| 38 | +import org.elasticsearch.transport.TransportRequestOptions; |
| 39 | +import org.elasticsearch.transport.TransportRequestOptions.Type; |
| 40 | +import org.elasticsearch.transport.TransportResponse; |
| 41 | +import org.elasticsearch.transport.TransportResponse.Empty; |
| 42 | +import org.elasticsearch.transport.TransportResponseHandler; |
| 43 | +import org.elasticsearch.transport.TransportService; |
| 44 | + |
| 45 | +import java.io.IOException; |
| 46 | +import java.util.Objects; |
| 47 | +import java.util.concurrent.atomic.AtomicBoolean; |
| 48 | +import java.util.concurrent.atomic.AtomicLong; |
| 49 | + |
| 50 | +/** |
| 51 | + * The LeaderChecker is responsible for allowing followers to check that the currently elected leader is still connected and healthy. We are |
| 52 | + * fairly lenient, possibly allowing multiple checks to fail before considering the leader to be faulty, to allow for the leader to |
| 53 | + * temporarily stand down on occasion, e.g. if it needs to move to a higher term. On deciding that the leader has failed a follower will |
| 54 | + * become a candidate and attempt to become a leader itself. |
| 55 | + */ |
| 56 | +public class LeaderChecker extends AbstractComponent { |
| 57 | + |
| 58 | + public static final String LEADER_CHECK_ACTION_NAME = "internal:coordination/fault_detection/leader_check"; |
| 59 | + |
| 60 | + // the time between checks sent to the leader |
| 61 | + public static final Setting<TimeValue> LEADER_CHECK_INTERVAL_SETTING = |
| 62 | + Setting.timeSetting("cluster.fault_detection.leader_check.interval", |
| 63 | + TimeValue.timeValueMillis(10000), TimeValue.timeValueMillis(100), Setting.Property.NodeScope); |
| 64 | + |
| 65 | + // the timeout for each check sent to the leader |
| 66 | + public static final Setting<TimeValue> LEADER_CHECK_TIMEOUT_SETTING = |
| 67 | + Setting.timeSetting("cluster.fault_detection.leader_check.timeout", |
| 68 | + TimeValue.timeValueMillis(10000), TimeValue.timeValueMillis(1), Setting.Property.NodeScope); |
| 69 | + |
| 70 | + // the number of failed checks that must happen before the leader is considered to have failed. |
| 71 | + public static final Setting<Integer> LEADER_CHECK_RETRY_COUNT_SETTING = |
| 72 | + Setting.intSetting("cluster.fault_detection.leader_check.retry_count", 3, 1, Setting.Property.NodeScope); |
| 73 | + |
| 74 | + private final TimeValue leaderCheckInterval; |
| 75 | + private final TimeValue leaderCheckTimeout; |
| 76 | + private final int leaderCheckRetryCount; |
| 77 | + private final TransportService transportService; |
| 78 | + private final Runnable onLeaderFailure; |
| 79 | + |
| 80 | + private volatile DiscoveryNodes lastPublishedDiscoveryNodes; |
| 81 | + |
| 82 | + public LeaderChecker(final Settings settings, final TransportService transportService, final Runnable onLeaderFailure) { |
| 83 | + super(settings); |
| 84 | + leaderCheckInterval = LEADER_CHECK_INTERVAL_SETTING.get(settings); |
| 85 | + leaderCheckTimeout = LEADER_CHECK_TIMEOUT_SETTING.get(settings); |
| 86 | + leaderCheckRetryCount = LEADER_CHECK_RETRY_COUNT_SETTING.get(settings); |
| 87 | + this.transportService = transportService; |
| 88 | + this.onLeaderFailure = onLeaderFailure; |
| 89 | + |
| 90 | + transportService.registerRequestHandler(LEADER_CHECK_ACTION_NAME, Names.SAME, LeaderCheckRequest::new, this::handleLeaderCheck); |
| 91 | + } |
| 92 | + |
| 93 | + /** |
| 94 | + * Start a leader checker for the given leader. Should only be called after successfully joining this leader. |
| 95 | + * |
| 96 | + * @param leader the node to be checked as leader |
| 97 | + * @return a `Releasable` that can be used to stop this checker. |
| 98 | + */ |
| 99 | + public Releasable startLeaderChecker(final DiscoveryNode leader) { |
| 100 | + assert transportService.getLocalNode().equals(leader) == false; |
| 101 | + CheckScheduler checkScheduler = new CheckScheduler(leader); |
| 102 | + checkScheduler.handleWakeUp(); |
| 103 | + return checkScheduler; |
| 104 | + } |
| 105 | + |
| 106 | + /** |
| 107 | + * Update the "known" discovery nodes. Should be called on the leader before a new cluster state is published to reflect the new |
| 108 | + * publication targets, and also called if a leader becomes a non-leader. |
| 109 | + * TODO if heartbeats can make nodes become followers then this needs to be called before a heartbeat is sent to a new node too. |
| 110 | + * |
| 111 | + * isLocalNodeElectedMaster() should reflect whether this node is a leader, and nodeExists() |
| 112 | + * should indicate whether nodes are known publication targets or not. |
| 113 | + */ |
| 114 | + public void setLastPublishedDiscoveryNodes(DiscoveryNodes discoveryNodes) { |
| 115 | + logger.trace("updating last-published nodes: {}", discoveryNodes); |
| 116 | + lastPublishedDiscoveryNodes = discoveryNodes; |
| 117 | + } |
| 118 | + |
| 119 | + private void handleLeaderCheck(LeaderCheckRequest request, TransportChannel transportChannel, Task task) throws IOException { |
| 120 | + final DiscoveryNodes lastPublishedDiscoveryNodes = this.lastPublishedDiscoveryNodes; |
| 121 | + assert lastPublishedDiscoveryNodes != null; |
| 122 | + |
| 123 | + if (lastPublishedDiscoveryNodes.isLocalNodeElectedMaster() == false) { |
| 124 | + logger.debug("non-master handling {}", request); |
| 125 | + transportChannel.sendResponse(new CoordinationStateRejectedException("non-leader rejecting leader check")); |
| 126 | + } else if (lastPublishedDiscoveryNodes.nodeExists(request.getSender()) == false) { |
| 127 | + logger.debug("leader check from unknown node: {}", request); |
| 128 | + transportChannel.sendResponse(new CoordinationStateRejectedException("leader check from unknown node")); |
| 129 | + } else { |
| 130 | + logger.trace("handling {}", request); |
| 131 | + transportChannel.sendResponse(Empty.INSTANCE); |
| 132 | + } |
| 133 | + } |
| 134 | + |
| 135 | + private class CheckScheduler implements Releasable { |
| 136 | + |
| 137 | + private final AtomicBoolean isClosed = new AtomicBoolean(); |
| 138 | + private final AtomicLong failureCountSinceLastSuccess = new AtomicLong(); |
| 139 | + private final DiscoveryNode leader; |
| 140 | + |
| 141 | + CheckScheduler(final DiscoveryNode leader) { |
| 142 | + this.leader = leader; |
| 143 | + } |
| 144 | + |
| 145 | + @Override |
| 146 | + public void close() { |
| 147 | + if (isClosed.compareAndSet(false, true) == false) { |
| 148 | + logger.debug("already closed"); |
| 149 | + } else { |
| 150 | + logger.debug("closed"); |
| 151 | + } |
| 152 | + } |
| 153 | + |
| 154 | + void handleWakeUp() { |
| 155 | + if (isClosed.get()) { |
| 156 | + logger.debug("closed check scheduler woken up, doing nothing"); |
| 157 | + return; |
| 158 | + } |
| 159 | + |
| 160 | + logger.trace("checking {} with [{}] = {}", leader, LEADER_CHECK_TIMEOUT_SETTING.getKey(), leaderCheckTimeout); |
| 161 | + |
| 162 | + // TODO lag detection: |
| 163 | + // In the PoC, the leader sent its current version to the follower in the response to a LeaderCheck, so the follower |
| 164 | + // could detect if it was lagging. We'd prefer this to be implemented on the leader, so the response is just |
| 165 | + // TransportResponse.Empty here. |
| 166 | + transportService.sendRequest(leader, LEADER_CHECK_ACTION_NAME, new LeaderCheckRequest(transportService.getLocalNode()), |
| 167 | + TransportRequestOptions.builder().withTimeout(leaderCheckTimeout).withType(Type.PING).build(), |
| 168 | + |
| 169 | + new TransportResponseHandler<TransportResponse.Empty>() { |
| 170 | + @Override |
| 171 | + public void handleResponse(Empty response) { |
| 172 | + if (isClosed.get()) { |
| 173 | + logger.debug("closed check scheduler received a response, doing nothing"); |
| 174 | + return; |
| 175 | + } |
| 176 | + |
| 177 | + failureCountSinceLastSuccess.set(0); |
| 178 | + scheduleNextWakeUp(); // logs trace message indicating success |
| 179 | + } |
| 180 | + |
| 181 | + @Override |
| 182 | + public void handleException(TransportException exp) { |
| 183 | + if (isClosed.get()) { |
| 184 | + logger.debug("closed check scheduler received a response, doing nothing"); |
| 185 | + return; |
| 186 | + } |
| 187 | + |
| 188 | + if (exp instanceof ConnectTransportException || exp.getCause() instanceof ConnectTransportException) { |
| 189 | + logger.debug(new ParameterizedMessage("leader [{}] disconnected, failing immediately", leader), exp); |
| 190 | + leaderFailed(); |
| 191 | + return; |
| 192 | + } |
| 193 | + |
| 194 | + long failureCount = failureCountSinceLastSuccess.incrementAndGet(); |
| 195 | + if (failureCount >= leaderCheckRetryCount) { |
| 196 | + logger.debug(new ParameterizedMessage("{} consecutive failures (limit [{}] is {}) so leader [{}] has failed", |
| 197 | + failureCount, LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), leaderCheckRetryCount, leader), exp); |
| 198 | + leaderFailed(); |
| 199 | + return; |
| 200 | + } |
| 201 | + |
| 202 | + logger.debug(new ParameterizedMessage("{} consecutive failures (limit [{}] is {}) with leader [{}]", |
| 203 | + failureCount, LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), leaderCheckRetryCount, leader), exp); |
| 204 | + scheduleNextWakeUp(); |
| 205 | + } |
| 206 | + |
| 207 | + @Override |
| 208 | + public String executor() { |
| 209 | + return Names.SAME; |
| 210 | + } |
| 211 | + }); |
| 212 | + } |
| 213 | + |
| 214 | + private void leaderFailed() { |
| 215 | + if (isClosed.compareAndSet(false, true)) { |
| 216 | + transportService.getThreadPool().generic().execute(onLeaderFailure); |
| 217 | + } else { |
| 218 | + logger.debug("already closed, not failing leader"); |
| 219 | + } |
| 220 | + } |
| 221 | + |
| 222 | + private void scheduleNextWakeUp() { |
| 223 | + logger.trace("scheduling next check of {} for [{}] = {}", leader, LEADER_CHECK_INTERVAL_SETTING.getKey(), leaderCheckInterval); |
| 224 | + transportService.getThreadPool().schedule(leaderCheckInterval, Names.SAME, new Runnable() { |
| 225 | + @Override |
| 226 | + public void run() { |
| 227 | + handleWakeUp(); |
| 228 | + } |
| 229 | + |
| 230 | + @Override |
| 231 | + public String toString() { |
| 232 | + return "scheduled check of leader " + leader; |
| 233 | + } |
| 234 | + }); |
| 235 | + } |
| 236 | + } |
| 237 | + |
| 238 | + public static class LeaderCheckRequest extends TransportRequest { |
| 239 | + |
| 240 | + private final DiscoveryNode sender; |
| 241 | + |
| 242 | + public LeaderCheckRequest(final DiscoveryNode sender) { |
| 243 | + this.sender = sender; |
| 244 | + } |
| 245 | + |
| 246 | + public LeaderCheckRequest(final StreamInput in) throws IOException { |
| 247 | + super(in); |
| 248 | + sender = new DiscoveryNode(in); |
| 249 | + } |
| 250 | + |
| 251 | + @Override |
| 252 | + public void writeTo(final StreamOutput out) throws IOException { |
| 253 | + super.writeTo(out); |
| 254 | + sender.writeTo(out); |
| 255 | + } |
| 256 | + |
| 257 | + public DiscoveryNode getSender() { |
| 258 | + return sender; |
| 259 | + } |
| 260 | + |
| 261 | + @Override |
| 262 | + public boolean equals(final Object o) { |
| 263 | + if (this == o) return true; |
| 264 | + if (o == null || getClass() != o.getClass()) return false; |
| 265 | + final LeaderCheckRequest that = (LeaderCheckRequest) o; |
| 266 | + return Objects.equals(sender, that.sender); |
| 267 | + } |
| 268 | + |
| 269 | + @Override |
| 270 | + public int hashCode() { |
| 271 | + return Objects.hash(sender); |
| 272 | + } |
| 273 | + |
| 274 | + @Override |
| 275 | + public String toString() { |
| 276 | + return "LeaderCheckRequest{" + |
| 277 | + "sender=" + sender + |
| 278 | + '}'; |
| 279 | + } |
| 280 | + } |
| 281 | +} |
| 282 | + |
0 commit comments