|
|
@@ -24,6 +24,7 @@ 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.Randomness;
|
|
|
import org.elasticsearch.common.io.stream.StreamInput;
|
|
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
|
|
import org.elasticsearch.common.io.stream.Writeable;
|
|
|
@@ -50,6 +51,7 @@ import java.util.List;
|
|
|
import java.util.Locale;
|
|
|
import java.util.Map;
|
|
|
import java.util.Objects;
|
|
|
+import java.util.Random;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
@@ -91,6 +93,9 @@ public class CoordinationDiagnosticsService implements ClusterStateListener {
|
|
|
*/
|
|
|
private final int unacceptableIdentityChanges;
|
|
|
|
|
|
+ // ThreadLocal because our unit testing framework does not like sharing Randoms across threads
|
|
|
+ private final ThreadLocal<Random> random = ThreadLocal.withInitial(Randomness::get);
|
|
|
+
|
|
|
/*
|
|
|
* This is a Map of tasks that are periodically reaching out to other master eligible nodes to get their ClusterFormationStates for
|
|
|
* diagnosis. The key is the DisoveryNode for the master eligible node being polled, and the value is a Cancellable.
|
|
|
@@ -722,6 +727,20 @@ public class CoordinationDiagnosticsService implements ClusterStateListener {
|
|
|
return masterEligibleNodes;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Returns a random master eligible node, or null if this node does not know about any master eligible nodes
|
|
|
+ * @return A random master eligible node or null
|
|
|
+ */
|
|
|
+ // Non-private for unit testing
|
|
|
+ @Nullable
|
|
|
+ DiscoveryNode getRandomMasterEligibleNode() {
|
|
|
+ Collection<DiscoveryNode> masterEligibleNodes = getMasterEligibleNodes();
|
|
|
+ if (masterEligibleNodes.isEmpty()) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ return masterEligibleNodes.toArray(new DiscoveryNode[0])[random.get().nextInt(masterEligibleNodes.size())];
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* This returns true if this node has seen a master node within the last few seconds
|
|
|
* @return true if this node has seen a master node within the last few seconds, false otherwise
|
|
|
@@ -958,7 +977,7 @@ public class CoordinationDiagnosticsService implements ClusterStateListener {
|
|
|
Consumer<RemoteMasterHealthResult> responseConsumer,
|
|
|
AtomicReference<Scheduler.Cancellable> cancellableReference
|
|
|
) {
|
|
|
- DiscoveryNode masterEligibleNode = getMasterEligibleNodes().stream().findAny().orElse(null);
|
|
|
+ DiscoveryNode masterEligibleNode = getRandomMasterEligibleNode();
|
|
|
try {
|
|
|
cancellableReference.set(
|
|
|
fetchCoordinationDiagnostics(
|
|
|
@@ -1002,7 +1021,7 @@ public class CoordinationDiagnosticsService implements ClusterStateListener {
|
|
|
* cancellableReference, so it will not be run again.
|
|
|
*/
|
|
|
try {
|
|
|
- DiscoveryNode masterEligibleNode = getMasterEligibleNodes().stream().findAny().orElse(null);
|
|
|
+ DiscoveryNode masterEligibleNode = getRandomMasterEligibleNode();
|
|
|
cancellableReference.set(
|
|
|
fetchCoordinationDiagnostics(
|
|
|
masterEligibleNode,
|