Rename DiscoveryNodes#localNodeMaster to isLocalNodeElectedMaster

This commit is contained in:
javanna 2016-03-30 15:22:39 +02:00 committed by Luca Cavanna
parent 2075c7f0a7
commit 30b57be8f4
16 changed files with 30 additions and 30 deletions

View File

@ -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
// For example the minimum_master_node could have been breached and we're no longer elected master,
// 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");
listener.onResponse(new ClusterUpdateSettingsResponse(updateSettingsAcked, updater.getTransientUpdates(), updater.getPersistentUpdate()));
return;

View File

@ -111,7 +111,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
}
ClusterHealthStatus clusterStatus = null;
if (clusterService.state().nodes().localNodeMaster()) {
if (clusterService.state().nodes().isLocalNodeElectedMaster()) {
clusterStatus = new ClusterStateHealth(clusterService.state()).getStatus();
}

View File

@ -131,7 +131,7 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
protected void doStart() {
final ClusterState clusterState = observer.observedState();
final DiscoveryNodes nodes = clusterState.nodes();
if (nodes.localNodeMaster() || localExecute(request)) {
if (nodes.isLocalNodeElectedMaster() || localExecute(request)) {
// check for block, if blocked, retry, else, execute locally
final ClusterBlockException blockException = checkBlock(request, clusterState);
if (blockException != null) {

View File

@ -186,7 +186,7 @@ public class ClusterChangedEvent {
* Returns <code>true</code> iff the local node is the mater node of the cluster.
*/
public boolean localNodeMaster() {
return state.nodes().localNodeMaster();
return state.nodes().isLocalNodeElectedMaster();
}
/**

View File

@ -79,7 +79,7 @@ public class MetaDataUpdateSettingsService extends AbstractComponent implements
@Override
public void clusterChanged(ClusterChangedEvent event) {
// update an index with number of replicas based on data nodes if possible
if (!event.state().nodes().localNodeMaster()) {
if (!event.state().nodes().isLocalNodeElectedMaster()) {
return;
}
// we will want to know this for translating "all" to a number

View File

@ -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) {
// we don't know yet the local node id, return false
return false;

View File

@ -100,7 +100,7 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
@Override
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.
// 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.

View File

@ -494,7 +494,7 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
}
logger.debug("processing [{}]: execute", source);
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);
toExecute.stream().forEach(task -> task.listener.onNoLongerMaster(task.source));
return;
@ -561,7 +561,7 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
try {
ArrayList<Discovery.AckListener> ackListeners = new ArrayList<>();
if (newClusterState.nodes().localNodeMaster()) {
if (newClusterState.nodes().isLocalNodeElectedMaster()) {
// only the master controls the version numbers
Builder builder = ClusterState.builder(newClusterState).incrementVersion();
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
// we publish here before we send a notification to all the listeners, since if it fails
// we don't want to notify
if (newClusterState.nodes().localNodeMaster()) {
if (newClusterState.nodes().isLocalNodeElectedMaster()) {
logger.debug("publishing cluster state version [{}]", newClusterState.version());
try {
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
if (newClusterState.nodes().localNodeMaster()) {
if (newClusterState.nodes().isLocalNodeElectedMaster()) {
try {
ackListener.onNodeAck(newClusterState.nodes().getLocalNode(), null);
} catch (Throwable t) {

View File

@ -322,7 +322,7 @@ public class NodeJoinController extends AbstractComponent {
public void onElectedAsMaster(ClusterState state) {
assert pendingSetAsMasterTask.get() : "onElectedAsMaster called but pendingSetAsMasterTask is not set";
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)) {
try {
onClose();

View File

@ -244,7 +244,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
if (sendLeaveRequest) {
if (nodes.getMasterNode() == null) {
// if we don't know who the master is, nothing to do here
} else if (!nodes.localNodeMaster()) {
} else if (!nodes.isLocalNodeElectedMaster()) {
try {
membership.sendLeaveRequestBlocking(nodes.getMasterNode(), nodes.getLocalNode(), TimeValue.timeValueSeconds(1));
} catch (Exception e) {
@ -302,7 +302,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
@Override
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");
}
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.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");
}
@ -951,11 +951,11 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
}
private boolean localNodeMaster() {
return nodes().localNodeMaster();
return nodes().isLocalNodeElectedMaster();
}
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";
if (otherClusterStateVersion > localClusterState.version()) {

View File

@ -346,7 +346,7 @@ public class MasterFaultDetection extends FaultDetection {
// 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);
clusterService.submitStateUpdateTask("master ping (from: [" + request.nodeId + "])", new ClusterStateUpdateTask() {

View File

@ -153,7 +153,7 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
final ClusterState state = event.state();
if (state.nodes().localNodeMaster() == false) {
if (state.nodes().isLocalNodeElectedMaster() == false) {
// not our job to recover
return;
}

View File

@ -477,7 +477,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
public void updateIndexShardSnapshotStatus(SnapshotId snapshotId, ShardId shardId, SnapshotsInProgress.ShardSnapshotStatus status) {
UpdateIndexShardSnapshotStatusRequest request = new UpdateIndexShardSnapshotStatusRequest(snapshotId, shardId, status);
try {
if (clusterService.state().nodes().localNodeMaster()) {
if (clusterService.state().nodes().isLocalNodeElectedMaster()) {
innerUpdateSnapshotState(request);
} else {
transportService.sendRequest(clusterService.state().nodes().getMasterNode(),

View File

@ -540,7 +540,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
private void processSnapshotsOnRemovedNodes(ClusterChangedEvent event) {
if (removedNodesCleanupNeeded(event)) {
// 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() {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
@ -719,7 +719,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
private boolean removedNodesCleanupNeeded(ClusterChangedEvent event) {
// 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);
if (snapshotsInProgress == null) {
return false;

View File

@ -519,7 +519,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
// the first node should be a master as the minimum required is 1
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));
String node_1 = internalCluster().startNode(settings);
@ -530,7 +530,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
assertThat(clusterHealth.isTimedOut(), equalTo(false));
// 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));
internalCluster().stopCurrentMasterNode();
@ -538,7 +538,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
assertThat(clusterHealth.isTimedOut(), equalTo(false));
// 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));
// start another node and set min_master_node

View File

@ -208,7 +208,7 @@ public class NodeJoinControllerTests extends ESTestCase {
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() {
@Override
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();
}
@ -256,7 +256,7 @@ public class NodeJoinControllerTests extends ESTestCase {
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() {
@Override
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();
}
@ -374,7 +374,7 @@ public class NodeJoinControllerTests extends ESTestCase {
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueMillis(1), new NodeJoinController.ElectionCallback() {
@Override
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();
}
@ -492,7 +492,7 @@ public class NodeJoinControllerTests extends ESTestCase {
nodeJoinController.waitToBeElectedAsMaster(requiredJoins, TimeValue.timeValueHours(30), new NodeJoinController.ElectionCallback() {
@Override
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();
}