Rename DiscoveryNodes#localNodeMaster to isLocalNodeElectedMaster
This commit is contained in:
parent
2075c7f0a7
commit
30b57be8f4
|
@ -114,7 +114,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
|
||||||
// We're about to send a second update task, so we need to check if we're still the elected master
|
// We're about to send a second update task, so we need to check if we're still the elected master
|
||||||
// For example the minimum_master_node could have been breached and we're no longer elected master,
|
// For example the minimum_master_node could have been breached and we're no longer elected master,
|
||||||
// so we should *not* execute the reroute.
|
// so we should *not* execute the reroute.
|
||||||
if (!clusterService.state().nodes().localNodeMaster()) {
|
if (!clusterService.state().nodes().isLocalNodeElectedMaster()) {
|
||||||
logger.debug("Skipping reroute after cluster update settings, because node is no longer master");
|
logger.debug("Skipping reroute after cluster update settings, because node is no longer master");
|
||||||
listener.onResponse(new ClusterUpdateSettingsResponse(updateSettingsAcked, updater.getTransientUpdates(), updater.getPersistentUpdate()));
|
listener.onResponse(new ClusterUpdateSettingsResponse(updateSettingsAcked, updater.getTransientUpdates(), updater.getPersistentUpdate()));
|
||||||
return;
|
return;
|
||||||
|
|
|
@ -111,7 +111,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
|
||||||
}
|
}
|
||||||
|
|
||||||
ClusterHealthStatus clusterStatus = null;
|
ClusterHealthStatus clusterStatus = null;
|
||||||
if (clusterService.state().nodes().localNodeMaster()) {
|
if (clusterService.state().nodes().isLocalNodeElectedMaster()) {
|
||||||
clusterStatus = new ClusterStateHealth(clusterService.state()).getStatus();
|
clusterStatus = new ClusterStateHealth(clusterService.state()).getStatus();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -131,7 +131,7 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
|
||||||
protected void doStart() {
|
protected void doStart() {
|
||||||
final ClusterState clusterState = observer.observedState();
|
final ClusterState clusterState = observer.observedState();
|
||||||
final DiscoveryNodes nodes = clusterState.nodes();
|
final DiscoveryNodes nodes = clusterState.nodes();
|
||||||
if (nodes.localNodeMaster() || localExecute(request)) {
|
if (nodes.isLocalNodeElectedMaster() || localExecute(request)) {
|
||||||
// check for block, if blocked, retry, else, execute locally
|
// check for block, if blocked, retry, else, execute locally
|
||||||
final ClusterBlockException blockException = checkBlock(request, clusterState);
|
final ClusterBlockException blockException = checkBlock(request, clusterState);
|
||||||
if (blockException != null) {
|
if (blockException != null) {
|
||||||
|
|
|
@ -186,7 +186,7 @@ public class ClusterChangedEvent {
|
||||||
* Returns <code>true</code> iff the local node is the mater node of the cluster.
|
* Returns <code>true</code> iff the local node is the mater node of the cluster.
|
||||||
*/
|
*/
|
||||||
public boolean localNodeMaster() {
|
public boolean localNodeMaster() {
|
||||||
return state.nodes().localNodeMaster();
|
return state.nodes().isLocalNodeElectedMaster();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -79,7 +79,7 @@ public class MetaDataUpdateSettingsService extends AbstractComponent implements
|
||||||
@Override
|
@Override
|
||||||
public void clusterChanged(ClusterChangedEvent event) {
|
public void clusterChanged(ClusterChangedEvent event) {
|
||||||
// update an index with number of replicas based on data nodes if possible
|
// update an index with number of replicas based on data nodes if possible
|
||||||
if (!event.state().nodes().localNodeMaster()) {
|
if (!event.state().nodes().isLocalNodeElectedMaster()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
// we will want to know this for translating "all" to a number
|
// we will want to know this for translating "all" to a number
|
||||||
|
|
|
@ -86,9 +86,9 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns <tt>true</tt> if the local node is the master node.
|
* Returns <tt>true</tt> if the local node is the elected master node.
|
||||||
*/
|
*/
|
||||||
public boolean localNodeMaster() {
|
public boolean isLocalNodeElectedMaster() {
|
||||||
if (localNodeId == null) {
|
if (localNodeId == null) {
|
||||||
// we don't know yet the local node id, return false
|
// we don't know yet the local node id, return false
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -100,7 +100,7 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void clusterChanged(ClusterChangedEvent event) {
|
public void clusterChanged(ClusterChangedEvent event) {
|
||||||
if (event.state().nodes().localNodeMaster()) {
|
if (event.state().nodes().isLocalNodeElectedMaster()) {
|
||||||
// Figure out if an existing scheduled reroute is good enough or whether we need to cancel and reschedule.
|
// Figure out if an existing scheduled reroute is good enough or whether we need to cancel and reschedule.
|
||||||
// If the minimum of the currently relevant delay settings is larger than something we scheduled in the past,
|
// If the minimum of the currently relevant delay settings is larger than something we scheduled in the past,
|
||||||
// we are guaranteed that the planned schedule will happen before any of the current shard delays are expired.
|
// we are guaranteed that the planned schedule will happen before any of the current shard delays are expired.
|
||||||
|
|
|
@ -494,7 +494,7 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
|
||||||
}
|
}
|
||||||
logger.debug("processing [{}]: execute", source);
|
logger.debug("processing [{}]: execute", source);
|
||||||
ClusterState previousClusterState = clusterState;
|
ClusterState previousClusterState = clusterState;
|
||||||
if (!previousClusterState.nodes().localNodeMaster() && executor.runOnlyOnMaster()) {
|
if (!previousClusterState.nodes().isLocalNodeElectedMaster() && executor.runOnlyOnMaster()) {
|
||||||
logger.debug("failing [{}]: local node is no longer master", source);
|
logger.debug("failing [{}]: local node is no longer master", source);
|
||||||
toExecute.stream().forEach(task -> task.listener.onNoLongerMaster(task.source));
|
toExecute.stream().forEach(task -> task.listener.onNoLongerMaster(task.source));
|
||||||
return;
|
return;
|
||||||
|
@ -561,7 +561,7 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ArrayList<Discovery.AckListener> ackListeners = new ArrayList<>();
|
ArrayList<Discovery.AckListener> ackListeners = new ArrayList<>();
|
||||||
if (newClusterState.nodes().localNodeMaster()) {
|
if (newClusterState.nodes().isLocalNodeElectedMaster()) {
|
||||||
// only the master controls the version numbers
|
// only the master controls the version numbers
|
||||||
Builder builder = ClusterState.builder(newClusterState).incrementVersion();
|
Builder builder = ClusterState.builder(newClusterState).incrementVersion();
|
||||||
if (previousClusterState.routingTable() != newClusterState.routingTable()) {
|
if (previousClusterState.routingTable() != newClusterState.routingTable()) {
|
||||||
|
@ -617,7 +617,7 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
|
||||||
// if we are the master, publish the new state to all nodes
|
// if we are the master, publish the new state to all nodes
|
||||||
// we publish here before we send a notification to all the listeners, since if it fails
|
// we publish here before we send a notification to all the listeners, since if it fails
|
||||||
// we don't want to notify
|
// we don't want to notify
|
||||||
if (newClusterState.nodes().localNodeMaster()) {
|
if (newClusterState.nodes().isLocalNodeElectedMaster()) {
|
||||||
logger.debug("publishing cluster state version [{}]", newClusterState.version());
|
logger.debug("publishing cluster state version [{}]", newClusterState.version());
|
||||||
try {
|
try {
|
||||||
clusterStatePublisher.accept(clusterChangedEvent, ackListener);
|
clusterStatePublisher.accept(clusterChangedEvent, ackListener);
|
||||||
|
@ -661,7 +661,7 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
|
||||||
}
|
}
|
||||||
|
|
||||||
//manual ack only from the master at the end of the publish
|
//manual ack only from the master at the end of the publish
|
||||||
if (newClusterState.nodes().localNodeMaster()) {
|
if (newClusterState.nodes().isLocalNodeElectedMaster()) {
|
||||||
try {
|
try {
|
||||||
ackListener.onNodeAck(newClusterState.nodes().getLocalNode(), null);
|
ackListener.onNodeAck(newClusterState.nodes().getLocalNode(), null);
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
|
|
|
@ -322,7 +322,7 @@ public class NodeJoinController extends AbstractComponent {
|
||||||
public void onElectedAsMaster(ClusterState state) {
|
public void onElectedAsMaster(ClusterState state) {
|
||||||
assert pendingSetAsMasterTask.get() : "onElectedAsMaster called but pendingSetAsMasterTask is not set";
|
assert pendingSetAsMasterTask.get() : "onElectedAsMaster called but pendingSetAsMasterTask is not set";
|
||||||
assertClusterStateThread();
|
assertClusterStateThread();
|
||||||
assert state.nodes().localNodeMaster() : "onElectedAsMaster called but local node is not master";
|
assert state.nodes().isLocalNodeElectedMaster() : "onElectedAsMaster called but local node is not master";
|
||||||
if (closed.compareAndSet(false, true)) {
|
if (closed.compareAndSet(false, true)) {
|
||||||
try {
|
try {
|
||||||
onClose();
|
onClose();
|
||||||
|
|
|
@ -244,7 +244,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
||||||
if (sendLeaveRequest) {
|
if (sendLeaveRequest) {
|
||||||
if (nodes.getMasterNode() == null) {
|
if (nodes.getMasterNode() == null) {
|
||||||
// if we don't know who the master is, nothing to do here
|
// if we don't know who the master is, nothing to do here
|
||||||
} else if (!nodes.localNodeMaster()) {
|
} else if (!nodes.isLocalNodeElectedMaster()) {
|
||||||
try {
|
try {
|
||||||
membership.sendLeaveRequestBlocking(nodes.getMasterNode(), nodes.getLocalNode(), TimeValue.timeValueSeconds(1));
|
membership.sendLeaveRequestBlocking(nodes.getMasterNode(), nodes.getLocalNode(), TimeValue.timeValueSeconds(1));
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
@ -302,7 +302,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) {
|
public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) {
|
||||||
if (!clusterChangedEvent.state().getNodes().localNodeMaster()) {
|
if (!clusterChangedEvent.state().getNodes().isLocalNodeElectedMaster()) {
|
||||||
throw new IllegalStateException("Shouldn't publish state when not master");
|
throw new IllegalStateException("Shouldn't publish state when not master");
|
||||||
}
|
}
|
||||||
nodesFD.updateNodesAndPing(clusterChangedEvent.state());
|
nodesFD.updateNodesAndPing(clusterChangedEvent.state());
|
||||||
|
@ -683,7 +683,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
||||||
assert newClusterState.nodes().getMasterNode() != null : "received a cluster state without a master";
|
assert newClusterState.nodes().getMasterNode() != null : "received a cluster state without a master";
|
||||||
assert !newClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) : "received a cluster state with a master block";
|
assert !newClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) : "received a cluster state with a master block";
|
||||||
|
|
||||||
if (currentState.nodes().localNodeMaster()) {
|
if (currentState.nodes().isLocalNodeElectedMaster()) {
|
||||||
return handleAnotherMaster(currentState, newClusterState.nodes().getMasterNode(), newClusterState.version(), "via a new cluster state");
|
return handleAnotherMaster(currentState, newClusterState.nodes().getMasterNode(), newClusterState.version(), "via a new cluster state");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -951,11 +951,11 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean localNodeMaster() {
|
private boolean localNodeMaster() {
|
||||||
return nodes().localNodeMaster();
|
return nodes().isLocalNodeElectedMaster();
|
||||||
}
|
}
|
||||||
|
|
||||||
private ClusterState handleAnotherMaster(ClusterState localClusterState, final DiscoveryNode otherMaster, long otherClusterStateVersion, String reason) {
|
private ClusterState handleAnotherMaster(ClusterState localClusterState, final DiscoveryNode otherMaster, long otherClusterStateVersion, String reason) {
|
||||||
assert localClusterState.nodes().localNodeMaster() : "handleAnotherMaster called but current node is not a master";
|
assert localClusterState.nodes().isLocalNodeElectedMaster() : "handleAnotherMaster called but current node is not a master";
|
||||||
assert Thread.currentThread().getName().contains(ClusterService.UPDATE_THREAD_NAME) : "not called from the cluster state update thread";
|
assert Thread.currentThread().getName().contains(ClusterService.UPDATE_THREAD_NAME) : "not called from the cluster state update thread";
|
||||||
|
|
||||||
if (otherClusterStateVersion > localClusterState.version()) {
|
if (otherClusterStateVersion > localClusterState.version()) {
|
||||||
|
|
|
@ -346,7 +346,7 @@ public class MasterFaultDetection extends FaultDetection {
|
||||||
// all processing is finished.
|
// all processing is finished.
|
||||||
//
|
//
|
||||||
|
|
||||||
if (!nodes.localNodeMaster() || !nodes.nodeExists(request.nodeId)) {
|
if (!nodes.isLocalNodeElectedMaster() || !nodes.nodeExists(request.nodeId)) {
|
||||||
logger.trace("checking ping from [{}] under a cluster state thread", request.nodeId);
|
logger.trace("checking ping from [{}] under a cluster state thread", request.nodeId);
|
||||||
clusterService.submitStateUpdateTask("master ping (from: [" + request.nodeId + "])", new ClusterStateUpdateTask() {
|
clusterService.submitStateUpdateTask("master ping (from: [" + request.nodeId + "])", new ClusterStateUpdateTask() {
|
||||||
|
|
||||||
|
|
|
@ -153,7 +153,7 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
|
||||||
|
|
||||||
final ClusterState state = event.state();
|
final ClusterState state = event.state();
|
||||||
|
|
||||||
if (state.nodes().localNodeMaster() == false) {
|
if (state.nodes().isLocalNodeElectedMaster() == false) {
|
||||||
// not our job to recover
|
// not our job to recover
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -477,7 +477,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
|
||||||
public void updateIndexShardSnapshotStatus(SnapshotId snapshotId, ShardId shardId, SnapshotsInProgress.ShardSnapshotStatus status) {
|
public void updateIndexShardSnapshotStatus(SnapshotId snapshotId, ShardId shardId, SnapshotsInProgress.ShardSnapshotStatus status) {
|
||||||
UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshotId, shardId, status);
|
UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshotId, shardId, status);
|
||||||
try {
|
try {
|
||||||
if (clusterService.state().nodes().localNodeMaster()) {
|
if (clusterService.state().nodes().isLocalNodeElectedMaster()) {
|
||||||
innerUpdateSnapshotState(request);
|
innerUpdateSnapshotState(request);
|
||||||
} else {
|
} else {
|
||||||
transportService.sendRequest(clusterService.state().nodes().getMasterNode(),
|
transportService.sendRequest(clusterService.state().nodes().getMasterNode(),
|
||||||
|
|
|
@ -540,7 +540,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
||||||
private void processSnapshotsOnRemovedNodes(ClusterChangedEvent event) {
|
private void processSnapshotsOnRemovedNodes(ClusterChangedEvent event) {
|
||||||
if (removedNodesCleanupNeeded(event)) {
|
if (removedNodesCleanupNeeded(event)) {
|
||||||
// Check if we just became the master
|
// Check if we just became the master
|
||||||
final boolean newMaster = !event.previousState().nodes().localNodeMaster();
|
final boolean newMaster = !event.previousState().nodes().isLocalNodeElectedMaster();
|
||||||
clusterService.submitStateUpdateTask("update snapshot state after node removal", new ClusterStateUpdateTask() {
|
clusterService.submitStateUpdateTask("update snapshot state after node removal", new ClusterStateUpdateTask() {
|
||||||
@Override
|
@Override
|
||||||
public ClusterState execute(ClusterState currentState) throws Exception {
|
public ClusterState execute(ClusterState currentState) throws Exception {
|
||||||
|
@ -719,7 +719,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
|
||||||
|
|
||||||
private boolean removedNodesCleanupNeeded(ClusterChangedEvent event) {
|
private boolean removedNodesCleanupNeeded(ClusterChangedEvent event) {
|
||||||
// Check if we just became the master
|
// Check if we just became the master
|
||||||
boolean newMaster = !event.previousState().nodes().localNodeMaster();
|
boolean newMaster = !event.previousState().nodes().isLocalNodeElectedMaster();
|
||||||
SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE);
|
SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE);
|
||||||
if (snapshotsInProgress == null) {
|
if (snapshotsInProgress == null) {
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -519,7 +519,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
|
||||||
|
|
||||||
// the first node should be a master as the minimum required is 1
|
// the first node should be a master as the minimum required is 1
|
||||||
assertThat(clusterService.state().nodes().getMasterNode(), notNullValue());
|
assertThat(clusterService.state().nodes().getMasterNode(), notNullValue());
|
||||||
assertThat(clusterService.state().nodes().localNodeMaster(), is(true));
|
assertThat(clusterService.state().nodes().isLocalNodeElectedMaster(), is(true));
|
||||||
assertThat(testService.master(), is(true));
|
assertThat(testService.master(), is(true));
|
||||||
|
|
||||||
String node_1 = internalCluster().startNode(settings);
|
String node_1 = internalCluster().startNode(settings);
|
||||||
|
@ -530,7 +530,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
|
||||||
assertThat(clusterHealth.isTimedOut(), equalTo(false));
|
assertThat(clusterHealth.isTimedOut(), equalTo(false));
|
||||||
|
|
||||||
// the second node should not be the master as node1 is already the master.
|
// the second node should not be the master as node1 is already the master.
|
||||||
assertThat(clusterService1.state().nodes().localNodeMaster(), is(false));
|
assertThat(clusterService1.state().nodes().isLocalNodeElectedMaster(), is(false));
|
||||||
assertThat(testService1.master(), is(false));
|
assertThat(testService1.master(), is(false));
|
||||||
|
|
||||||
internalCluster().stopCurrentMasterNode();
|
internalCluster().stopCurrentMasterNode();
|
||||||
|
@ -538,7 +538,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
|
||||||
assertThat(clusterHealth.isTimedOut(), equalTo(false));
|
assertThat(clusterHealth.isTimedOut(), equalTo(false));
|
||||||
|
|
||||||
// now that node0 is closed, node1 should be elected as master
|
// now that node0 is closed, node1 should be elected as master
|
||||||
assertThat(clusterService1.state().nodes().localNodeMaster(), is(true));
|
assertThat(clusterService1.state().nodes().isLocalNodeElectedMaster(), is(true));
|
||||||
assertThat(testService1.master(), is(true));
|
assertThat(testService1.master(), is(true));
|
||||||
|
|
||||||
// start another node and set min_master_node
|
// start another node and set min_master_node
|
||||||
|
|
|
@ -208,7 +208,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
||||||
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() {
|
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void onElectedAsMaster(ClusterState state) {
|
public void onElectedAsMaster(ClusterState state) {
|
||||||
assertThat("callback called with elected as master, but state disagrees", state.nodes().localNodeMaster(), equalTo(true));
|
assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), equalTo(true));
|
||||||
electionFuture.markAsDone();
|
electionFuture.markAsDone();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -256,7 +256,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
||||||
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() {
|
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void onElectedAsMaster(ClusterState state) {
|
public void onElectedAsMaster(ClusterState state) {
|
||||||
assertThat("callback called with elected as master, but state disagrees", state.nodes().localNodeMaster(), equalTo(true));
|
assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), equalTo(true));
|
||||||
electionFuture.markAsDone();
|
electionFuture.markAsDone();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -374,7 +374,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
||||||
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueMillis(1), new NodeJoinController.ElectionCallback() {
|
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueMillis(1), new NodeJoinController.ElectionCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void onElectedAsMaster(ClusterState state) {
|
public void onElectedAsMaster(ClusterState state) {
|
||||||
assertThat("callback called with elected as master, but state disagrees", state.nodes().localNodeMaster(), equalTo(true));
|
assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), equalTo(true));
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -492,7 +492,7 @@ public class NodeJoinControllerTests extends ESTestCase {
|
||||||
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() {
|
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void onElectedAsMaster(ClusterState state) {
|
public void onElectedAsMaster(ClusterState state) {
|
||||||
assertThat("callback called with elected as master, but state disagrees", state.nodes().localNodeMaster(), equalTo(true));
|
assertThat("callback called with elected as master, but state disagrees", state.nodes().isLocalNodeElectedMaster(), equalTo(true));
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue