-
Notifications
You must be signed in to change notification settings - Fork 25.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Polling cluster formation state for master-is-stable health indicator #88397
Merged
masseyke
merged 29 commits into
elastic:master
from
masseyke:feature/polling-cluster-formation-state
Jul 14, 2022
Merged
Changes from 27 commits
Commits
Show all changes
29 commits
Select commit
Hold shift + click to select a range
85eaacf
Polling cluster formation state
masseyke 85323a4
cancelling all cancellables
masseyke dc3b581
fixing compilation errors
masseyke 5d215f3
adding unit test
masseyke 9cff8c7
spotlessApply
masseyke 6eabf6d
simplifying
masseyke 3e38a50
more cleanup
masseyke 7ee8a03
more cleanup
masseyke 6605e87
fixing ConcurrentModificationError
masseyke c5e7260
braking apart huge nested callbacks
masseyke 27ebfd7
using StepListener
masseyke e14af5b
Unit testing
masseyke 919b5f7
checkstyle
masseyke 630a092
cleaning up
masseyke f407efc
cleaning up
masseyke be04f87
Update docs/changelog/88397.yaml
masseyke f760226
code review feedback
masseyke 4c127d9
removing MultipleCancellablesWrapper(
masseyke 0168cfb
improving comments
masseyke a282a0d
code review feedback
masseyke b7937a9
Merge branch 'master' into feature/polling-cluster-formation-state
elasticmachine 9ef64e5
cleaning up
masseyke 69099bb
cleaning up
masseyke 963279f
committing working code
masseyke c86f450
committing working code
masseyke fb77e27
code review feedback
masseyke 8cea8a5
cleanup
masseyke 0144e60
code review feedback
masseyke e7e8949
Merge branch 'master' into feature/polling-cluster-formation-state
elasticmachine File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,5 @@ | ||
pr: 88397 | ||
summary: Polling cluster formation state for master-is-stable health indicator | ||
area: Health | ||
type: enhancement | ||
issues: [] |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -10,16 +10,29 @@ | |
|
||
import org.apache.logging.log4j.LogManager; | ||
import org.apache.logging.log4j.Logger; | ||
import org.elasticsearch.Version; | ||
import org.elasticsearch.action.ActionListener; | ||
import org.elasticsearch.action.ActionListenerResponseHandler; | ||
import org.elasticsearch.action.StepListener; | ||
import org.elasticsearch.action.admin.cluster.coordination.ClusterFormationInfoAction; | ||
import org.elasticsearch.cluster.ClusterChangedEvent; | ||
import org.elasticsearch.cluster.ClusterStateListener; | ||
import org.elasticsearch.cluster.node.DiscoveryNode; | ||
import org.elasticsearch.cluster.service.ClusterApplierService; | ||
import org.elasticsearch.cluster.service.ClusterService; | ||
import org.elasticsearch.common.io.stream.StreamInput; | ||
import org.elasticsearch.common.io.stream.StreamOutput; | ||
import org.elasticsearch.common.io.stream.Writeable; | ||
import org.elasticsearch.common.settings.Setting; | ||
import org.elasticsearch.core.Nullable; | ||
import org.elasticsearch.core.Releasable; | ||
import org.elasticsearch.core.Releasables; | ||
import org.elasticsearch.core.TimeValue; | ||
import org.elasticsearch.threadpool.Scheduler; | ||
import org.elasticsearch.threadpool.ThreadPool; | ||
import org.elasticsearch.transport.ConnectionProfile; | ||
import org.elasticsearch.transport.TransportRequestOptions; | ||
import org.elasticsearch.transport.TransportService; | ||
|
||
import java.io.IOException; | ||
import java.io.PrintWriter; | ||
|
@@ -30,7 +43,12 @@ | |
import java.util.Locale; | ||
import java.util.Objects; | ||
import java.util.Set; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.ConcurrentMap; | ||
import java.util.concurrent.CopyOnWriteArrayList; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.function.BiConsumer; | ||
import java.util.function.Consumer; | ||
import java.util.stream.Collectors; | ||
|
||
/** | ||
|
@@ -47,6 +65,7 @@ | |
*/ | ||
public class CoordinationDiagnosticsService implements ClusterStateListener { | ||
private final ClusterService clusterService; | ||
private final TransportService transportService; | ||
private final Coordinator coordinator; | ||
private final MasterHistoryService masterHistoryService; | ||
/** | ||
|
@@ -63,6 +82,19 @@ public class CoordinationDiagnosticsService implements ClusterStateListener { | |
*/ | ||
private final int unacceptableIdentityChanges; | ||
|
||
/* | ||
* This is a list of tasks that are periodically reaching out to other master eligible nodes to get their ClusterFormationStates for | ||
* diagnosis. | ||
* The field is accessed (reads/writes) from multiple threads, but the reference itself is only ever changed on the cluster change | ||
* event thread. | ||
*/ | ||
private volatile List<Scheduler.Cancellable> clusterFormationInfoTasks = null; | ||
/* | ||
* This field holds the results of the tasks in the clusterFormationInfoTasks field above. The field is accessed (reads/writes) from | ||
* multiple threads, but the reference itself is only ever changed on the cluster change event thread. | ||
*/ | ||
private volatile ConcurrentMap<DiscoveryNode, ClusterFormationStateOrException> clusterFormationResponses = null; | ||
|
||
private static final Logger logger = LogManager.getLogger(CoordinationDiagnosticsService.class); | ||
|
||
/** | ||
|
@@ -98,10 +130,12 @@ public class CoordinationDiagnosticsService implements ClusterStateListener { | |
|
||
public CoordinationDiagnosticsService( | ||
ClusterService clusterService, | ||
TransportService transportService, | ||
Coordinator coordinator, | ||
MasterHistoryService masterHistoryService | ||
) { | ||
this.clusterService = clusterService; | ||
this.transportService = transportService; | ||
this.coordinator = coordinator; | ||
this.masterHistoryService = masterHistoryService; | ||
this.nodeHasMasterLookupTimeframe = NODE_HAS_MASTER_LOOKUP_TIMEFRAME_SETTING.get(clusterService.getSettings()); | ||
|
@@ -410,6 +444,168 @@ public void clusterChanged(ClusterChangedEvent event) { | |
} | ||
} | ||
} | ||
if (currentMaster == null && clusterService.localNode().isMasterNode()) { | ||
/* | ||
* This begins polling all master-eligible nodes for cluster formation information. However there's a 10-second delay before it | ||
* starts, so in the normal situation where during a master transition it flips from master1 -> null -> master2, it the | ||
* polling tasks will be canceled before any requests are actually made. | ||
*/ | ||
beginPollingClusterFormationInfo(); | ||
} else { | ||
cancelPollingClusterFormationInfo(); | ||
} | ||
} | ||
|
||
/** | ||
* This method begins polling all known master-eligible nodes for cluster formation information. After a 10-second initial delay, it | ||
* polls each node every 10 seconds until cancelPollingClusterFormationInfo() is called. | ||
*/ | ||
private void beginPollingClusterFormationInfo() { | ||
assert ThreadPool.assertCurrentThreadPool(ClusterApplierService.CLUSTER_UPDATE_THREAD_NAME); | ||
cancelPollingClusterFormationInfo(); | ||
ConcurrentMap<DiscoveryNode, ClusterFormationStateOrException> responses = new ConcurrentHashMap<>(); | ||
List<Scheduler.Cancellable> cancellables = new CopyOnWriteArrayList<>(); | ||
beginPollingClusterFormationInfo(getMasterEligibleNodes(), responses::put, cancellables::add); | ||
clusterFormationResponses = responses; | ||
clusterFormationInfoTasks = cancellables; | ||
} | ||
|
||
/** | ||
* This method returns quickly, but in the background schedules to query the remote node's cluster formation state in 10 seconds, and | ||
* repeats doing that until cancel() is called on all of the Cancellable that this method inserts into cancellables. This method | ||
* exists (rather than being just part of the beginPollingClusterFormationInfo() above) in order to facilitate unit testing. | ||
* @param nodeResponseConsumer A consumer for any results produced for a node by this method | ||
* @param cancellableConsumer A consumer for any Cancellable tasks produced by this method | ||
*/ | ||
// Non-private for testing | ||
void beginPollingClusterFormationInfo( | ||
Collection<DiscoveryNode> masterEligibleNodes, | ||
BiConsumer<DiscoveryNode, ClusterFormationStateOrException> nodeResponseConsumer, | ||
Consumer<Scheduler.Cancellable> cancellableConsumer | ||
) { | ||
masterEligibleNodes.forEach(masterEligibleNode -> { | ||
Consumer<ClusterFormationStateOrException> responseConsumer = result -> nodeResponseConsumer.accept(masterEligibleNode, result); | ||
cancellableConsumer.accept( | ||
fetchClusterFormationInfo( | ||
masterEligibleNode, | ||
responseConsumer.andThen(rescheduleFetchConsumer(masterEligibleNode, responseConsumer, cancellableConsumer)) | ||
) | ||
); | ||
}); | ||
} | ||
|
||
/** | ||
* This wraps the responseConsumer in a Consumer that will run beginPollingClusterFormationInfo() after responseConsumer has | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it will run |
||
* completed, adding the resulting Cancellable to cancellableConsumer. | ||
* @param masterEligibleNode The node being polled | ||
* @param responseConsumer The response consumer to be wrapped | ||
* @param cancellableConsumer The list of Cancellables | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is not accurate anymore |
||
* @return | ||
*/ | ||
private Consumer<CoordinationDiagnosticsService.ClusterFormationStateOrException> rescheduleFetchConsumer( | ||
DiscoveryNode masterEligibleNode, | ||
Consumer<CoordinationDiagnosticsService.ClusterFormationStateOrException> responseConsumer, | ||
Consumer<Scheduler.Cancellable> cancellableConsumer | ||
) { | ||
return response -> { | ||
cancellableConsumer.accept( | ||
fetchClusterFormationInfo( | ||
masterEligibleNode, | ||
responseConsumer.andThen(rescheduleFetchConsumer(masterEligibleNode, responseConsumer, cancellableConsumer)) | ||
) | ||
); | ||
}; | ||
} | ||
|
||
private void cancelPollingClusterFormationInfo() { | ||
assert ThreadPool.assertCurrentThreadPool(ClusterApplierService.CLUSTER_UPDATE_THREAD_NAME); | ||
if (clusterFormationResponses != null) { | ||
clusterFormationInfoTasks.forEach(Scheduler.Cancellable::cancel); | ||
clusterFormationResponses = null; | ||
clusterFormationInfoTasks = null; | ||
} | ||
} | ||
|
||
/** | ||
* This method returns quickly, but in the background schedules to query the remote node's cluster formation state in 10 seconds | ||
* unless cancel() is called on the Cancellable that this method returns. | ||
* @param node The node to poll for cluster formation information | ||
* @param responseConsumer The consumer of the cluster formation info for the node, or the exception encountered while contacting it | ||
* @return A Cancellable for the task that is scheduled to fetch cluster formation information | ||
*/ | ||
private Scheduler.Cancellable fetchClusterFormationInfo( | ||
DiscoveryNode node, | ||
Consumer<ClusterFormationStateOrException> responseConsumer | ||
) { | ||
StepListener<Releasable> connectionListener = new StepListener<>(); | ||
StepListener<ClusterFormationInfoAction.Response> fetchClusterInfoListener = new StepListener<>(); | ||
long startTime = System.nanoTime(); | ||
connectionListener.whenComplete(releasable -> { | ||
logger.trace("Opened connection to {}, making cluster coordination info request", node); | ||
// If we don't get a response in 10 seconds that is a failure worth capturing on its own: | ||
final TimeValue transportTimeout = TimeValue.timeValueSeconds(10); | ||
transportService.sendRequest( | ||
node, | ||
ClusterFormationInfoAction.NAME, | ||
new ClusterFormationInfoAction.Request(), | ||
TransportRequestOptions.timeout(transportTimeout), | ||
new ActionListenerResponseHandler<>( | ||
ActionListener.runBefore(fetchClusterInfoListener, () -> Releasables.close(releasable)), | ||
ClusterFormationInfoAction.Response::new | ||
) | ||
); | ||
}, e -> { | ||
logger.warn("Exception connecting to master node", e); | ||
responseConsumer.accept(new ClusterFormationStateOrException(e)); | ||
}); | ||
|
||
fetchClusterInfoListener.whenComplete(response -> { | ||
long endTime = System.nanoTime(); | ||
logger.trace("Received cluster coordination info from {} in {}", node, TimeValue.timeValueNanos(endTime - startTime)); | ||
responseConsumer.accept(new ClusterFormationStateOrException(response.getClusterFormationState())); | ||
}, e -> { | ||
logger.warn("Exception in cluster coordination info request to master node", e); | ||
responseConsumer.accept(new ClusterFormationStateOrException(e)); | ||
}); | ||
|
||
return transportService.getThreadPool().schedule(() -> { | ||
Version minSupportedVersion = Version.V_8_4_0; | ||
if (node.getVersion().onOrAfter(minSupportedVersion) == false) { // This was introduced in 8.4.0 | ||
logger.trace( | ||
"Cannot get cluster coordination info for {} because it is at version {} and {} is required", | ||
node, | ||
node.getVersion(), | ||
minSupportedVersion | ||
); | ||
} else { | ||
transportService.connectToNode( | ||
// Note: This connection must be explicitly closed in the connectionListener | ||
node, | ||
ConnectionProfile.buildDefaultConnectionProfile(clusterService.getSettings()), | ||
connectionListener | ||
); | ||
} | ||
}, new TimeValue(10, TimeUnit.SECONDS), ThreadPool.Names.SAME); | ||
} | ||
|
||
// Non-private for testing | ||
record ClusterFormationStateOrException( | ||
ClusterFormationFailureHelper.ClusterFormationState clusterFormationState, | ||
Exception exception | ||
) { | ||
ClusterFormationStateOrException { | ||
if (clusterFormationState != null && exception != null) { | ||
throw new IllegalArgumentException("Cluster formation state and exception cannot both be non-null"); | ||
} | ||
} | ||
|
||
ClusterFormationStateOrException(ClusterFormationFailureHelper.ClusterFormationState clusterFormationState) { | ||
this(clusterFormationState, null); | ||
} | ||
|
||
ClusterFormationStateOrException(Exception exception) { | ||
this(null, exception); | ||
} | ||
} | ||
|
||
public record CoordinationDiagnosticsResult( | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
IMO we shouldn't be constantly calling this
cancel...
when the cluster is healthy, but only when re-gaining a master.@DaveCTurner what do you think?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's not a huge deal. Maybe we could use
null
instead ofnew ConcurrentHashMap<>()
to distinguish "we're not polling" from "we're polling but have no entries"?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My thinking was that it's basically a no-op (there won't be any scheduled tasks most of the time), and the miniscule performance hit was worth the risk of complicating the code and/or accidentally not calling it when it needed to be called.
I can have
cancelPollingClusterFormationInfo
setclusterFormationResponses
to null and havebeginPollingClusterFormationInfo
create anew ConcurrentHashMap<>()
(I think that's what you're suggesting?) -- that would save a little garbage collection.