Remove DiscoveryNodes#masterNodeId in favour of existing DiscoveryNodes#getMasterNodeId

This commit is contained in:
javanna 2016-03-30 15:15:49 +02:00 committed by Luca Cavanna
parent 97f7aef776
commit f8b5d1f5b0
21 changed files with 42 additions and 51 deletions

View File

@ -317,7 +317,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
* In essence that means that all the changes from the other cluster state are also reflected by the current one * In essence that means that all the changes from the other cluster state are also reflected by the current one
*/ */
public boolean supersedes(ClusterState other) { public boolean supersedes(ClusterState other) {
return this.nodes().masterNodeId() != null && this.nodes().masterNodeId().equals(other.nodes().masterNodeId()) && this.version() > other.version(); return this.nodes().getMasterNodeId() != null && this.nodes().getMasterNodeId().equals(other.nodes().getMasterNodeId()) && this.version() > other.version();
} }
@ -382,7 +382,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
} }
if (metrics.contains(Metric.MASTER_NODE)) { if (metrics.contains(Metric.MASTER_NODE)) {
builder.field("master_node", nodes().masterNodeId()); builder.field("master_node", nodes().getMasterNodeId());
} }
if (metrics.contains(Metric.BLOCKS)) { if (metrics.contains(Metric.BLOCKS)) {

View File

@ -30,7 +30,7 @@ public enum MasterNodeChangePredicate implements ClusterStateObserver.ChangePred
ClusterState.ClusterStateStatus newStatus) { ClusterState.ClusterStateStatus newStatus) {
// checking if the masterNodeId changed is insufficient as the // checking if the masterNodeId changed is insufficient as the
// same master node might get re-elected after a disruption // same master node might get re-elected after a disruption
return newState.nodes().masterNodeId() != null && newState != previousState; return newState.nodes().getMasterNodeId() != null && newState != previousState;
} }
@Override @Override

View File

@ -170,22 +170,13 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
return nodes.containsKey(nodeId); return nodes.containsKey(nodeId);
} }
/**
* Get the id of the master node
*
* @return id of the master
*/
public String masterNodeId() {
return this.masterNodeId;
}
/** /**
* Get the id of the master node * Get the id of the master node
* *
* @return id of the master * @return id of the master
*/ */
public String getMasterNodeId() { public String getMasterNodeId() {
return masterNodeId(); return this.masterNodeId;
} }
/** /**
@ -316,7 +307,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
resolvedNodesIds.add(localNodeId); resolvedNodesIds.add(localNodeId);
} }
} else if (nodeId.equals("_master")) { } else if (nodeId.equals("_master")) {
String masterNodeId = masterNodeId(); String masterNodeId = getMasterNodeId();
if (masterNodeId != null) { if (masterNodeId != null) {
resolvedNodesIds.add(masterNodeId); resolvedNodesIds.add(masterNodeId);
} }
@ -603,7 +594,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
} }
public Builder(DiscoveryNodes nodes) { public Builder(DiscoveryNodes nodes) {
this.masterNodeId = nodes.masterNodeId(); this.masterNodeId = nodes.getMasterNodeId();
this.localNodeId = nodes.localNodeId(); this.localNodeId = nodes.localNodeId();
this.nodes = ImmutableOpenMap.builder(nodes.getNodes()); this.nodes = ImmutableOpenMap.builder(nodes.getNodes());
} }

View File

@ -357,7 +357,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) { if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) {
// its a fresh update from the master as we transition from a start of not having a master to having one // its a fresh update from the master as we transition from a start of not having a master to having one
logger.debug("got first state from fresh master [{}]", nodeSpecificClusterState.nodes().masterNodeId()); logger.debug("got first state from fresh master [{}]", nodeSpecificClusterState.nodes().getMasterNodeId());
return nodeSpecificClusterState; return nodeSpecificClusterState;
} }

View File

@ -634,7 +634,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
@Override @Override
public ClusterState execute(ClusterState currentState) { public ClusterState execute(ClusterState currentState) {
if (!masterNode.getId().equals(currentState.nodes().masterNodeId())) { if (!masterNode.getId().equals(currentState.nodes().getMasterNodeId())) {
// master got switched on us, no need to send anything // master got switched on us, no need to send anything
return currentState; return currentState;
} }
@ -698,7 +698,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) { if (currentState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock())) {
// its a fresh update from the master as we transition from a start of not having a master to having one // its a fresh update from the master as we transition from a start of not having a master to having one
logger.debug("got first state from fresh master [{}]", newClusterState.nodes().masterNodeId()); logger.debug("got first state from fresh master [{}]", newClusterState.nodes().getMasterNodeId());
long count = clusterJoinsCounter.incrementAndGet(); long count = clusterJoinsCounter.incrementAndGet();
logger.trace("updated cluster join cluster to [{}]", count); logger.trace("updated cluster join cluster to [{}]", count);
@ -783,10 +783,10 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
*/ */
public static void validateStateIsFromCurrentMaster(ESLogger logger, DiscoveryNodes currentNodes, ClusterState newClusterState) { public static void validateStateIsFromCurrentMaster(ESLogger logger, DiscoveryNodes currentNodes, ClusterState newClusterState) {
if (currentNodes.masterNodeId() == null) { if (currentNodes.getMasterNodeId() == null) {
return; return;
} }
if (!currentNodes.masterNodeId().equals(newClusterState.nodes().masterNodeId())) { if (!currentNodes.getMasterNodeId().equals(newClusterState.nodes().getMasterNodeId())) {
logger.warn("received a cluster state from a different master than the current one, rejecting (received {}, current {})", newClusterState.nodes().masterNode(), currentNodes.masterNode()); logger.warn("received a cluster state from a different master than the current one, rejecting (received {}, current {})", newClusterState.nodes().masterNode(), currentNodes.masterNode());
throw new IllegalStateException("cluster state from a different master than the current one, rejecting (received " + newClusterState.nodes().masterNode() + ", current " + currentNodes.masterNode() + ")"); throw new IllegalStateException("cluster state from a different master than the current one, rejecting (received " + newClusterState.nodes().masterNode() + ", current " + currentNodes.masterNode() + ")");
} }

View File

@ -278,7 +278,7 @@ public class PendingClusterStatesQueue {
"[uuid[%s], v[%d], m[%s]]", "[uuid[%s], v[%d], m[%s]]",
stateUUID(), stateUUID(),
state.version(), state.version(),
state.nodes().masterNodeId() state.nodes().getMasterNodeId()
); );
} }
} }

View File

@ -163,7 +163,7 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
} }
DiscoveryNodes nodes = state.nodes(); DiscoveryNodes nodes = state.nodes();
if (state.nodes().masterNodeId() == null) { if (state.nodes().getMasterNodeId() == null) {
logger.debug("not recovering from gateway, no master elected yet"); logger.debug("not recovering from gateway, no master elected yet");
} else if (recoverAfterNodes != -1 && (nodes.getMasterAndDataNodes().size()) < recoverAfterNodes) { } else if (recoverAfterNodes != -1 && (nodes.getMasterAndDataNodes().size()) < recoverAfterNodes) {
logger.debug("not recovering from gateway, nodes_size (data+master) [{}] < recover_after_nodes [{}]", logger.debug("not recovering from gateway, nodes_size (data+master) [{}] < recover_after_nodes [{}]",

View File

@ -337,7 +337,7 @@ public class Node implements Closeable {
if (DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings).millis() > 0) { if (DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings).millis() > 0) {
final ThreadPool thread = injector.getInstance(ThreadPool.class); final ThreadPool thread = injector.getInstance(ThreadPool.class);
ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, thread.getThreadContext()); ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, thread.getThreadContext());
if (observer.observedState().nodes().masterNodeId() == null) { if (observer.observedState().nodes().getMasterNodeId() == null) {
final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch latch = new CountDownLatch(1);
observer.waitForNextChange(new ClusterStateObserver.Listener() { observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override @Override

View File

@ -81,7 +81,7 @@ public class RestMasterAction extends AbstractCatAction {
DiscoveryNodes nodes = state.getState().nodes(); DiscoveryNodes nodes = state.getState().nodes();
table.startRow(); table.startRow();
DiscoveryNode master = nodes.get(nodes.masterNodeId()); DiscoveryNode master = nodes.get(nodes.getMasterNodeId());
if (master == null) { if (master == null) {
table.addCell("-"); table.addCell("-");
table.addCell("-"); table.addCell("-");

View File

@ -223,7 +223,7 @@ public class RestNodesAction extends AbstractCatAction {
boolean fullId = req.paramAsBoolean("full_id", false); boolean fullId = req.paramAsBoolean("full_id", false);
DiscoveryNodes nodes = state.getState().nodes(); DiscoveryNodes nodes = state.getState().nodes();
String masterId = nodes.masterNodeId(); String masterId = nodes.getMasterNodeId();
Table table = getTableWithHeader(req); Table table = getTableWithHeader(req);
for (DiscoveryNode node : nodes) { for (DiscoveryNode node : nodes) {

View File

@ -159,8 +159,8 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
} else if (prev.equals(curr) == false) { } else if (prev.equals(curr) == false) {
processIndexShardSnapshots(event); processIndexShardSnapshots(event);
} }
String masterNodeId = event.state().nodes().masterNodeId(); String masterNodeId = event.state().nodes().getMasterNodeId();
if (masterNodeId != null && masterNodeId.equals(event.previousState().nodes().masterNodeId()) == false) { if (masterNodeId != null && masterNodeId.equals(event.previousState().nodes().getMasterNodeId()) == false) {
syncShardStatsOnNewMaster(event); syncShardStatsOnNewMaster(event);
} }

View File

@ -254,7 +254,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
state = client().admin().cluster().prepareState().execute().actionGet().getState(); state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.nodes().getSize(), equalTo(4)); assertThat(state.nodes().getSize(), equalTo(4));
// we prefer to elect up and running nodes // we prefer to elect up and running nodes
assertThat(state.nodes().masterNodeId(), not(isOneOf(newNodes))); assertThat(state.nodes().getMasterNodeId(), not(isOneOf(newNodes)));
logger.info("--> verify we the data back"); logger.info("--> verify we the data back");
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {

View File

@ -46,7 +46,7 @@ public class SpecificMasterNodesIT extends ESIntegTestCase {
logger.info("--> start data node / non master node"); logger.info("--> start data node / non master node");
internalCluster().startNode(settingsBuilder().put(Node.NODE_DATA_SETTING.getKey(), true).put(Node.NODE_MASTER_SETTING.getKey(), false).put("discovery.initial_state_timeout", "1s")); internalCluster().startNode(settingsBuilder().put(Node.NODE_DATA_SETTING.getKey(), true).put(Node.NODE_MASTER_SETTING.getKey(), false).put("discovery.initial_state_timeout", "1s"));
try { try {
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet().getState().nodes().masterNodeId(), nullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet().getState().nodes().getMasterNodeId(), nullValue());
fail("should not be able to find master"); fail("should not be able to find master");
} catch (MasterNotDiscoveredException e) { } catch (MasterNotDiscoveredException e) {
// all is well, no master elected // all is well, no master elected
@ -60,7 +60,7 @@ public class SpecificMasterNodesIT extends ESIntegTestCase {
internalCluster().stopCurrentMasterNode(); internalCluster().stopCurrentMasterNode();
try { try {
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet().getState().nodes().masterNodeId(), nullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet().getState().nodes().getMasterNodeId(), nullValue());
fail("should not be able to find master"); fail("should not be able to find master");
} catch (MasterNotDiscoveredException e) { } catch (MasterNotDiscoveredException e) {
// all is well, no master elected // all is well, no master elected
@ -76,7 +76,7 @@ public class SpecificMasterNodesIT extends ESIntegTestCase {
logger.info("--> start data node / non master node"); logger.info("--> start data node / non master node");
internalCluster().startNode(settingsBuilder().put(Node.NODE_DATA_SETTING.getKey(), true).put(Node.NODE_MASTER_SETTING.getKey(), false).put("discovery.initial_state_timeout", "1s")); internalCluster().startNode(settingsBuilder().put(Node.NODE_DATA_SETTING.getKey(), true).put(Node.NODE_MASTER_SETTING.getKey(), false).put("discovery.initial_state_timeout", "1s"));
try { try {
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet().getState().nodes().masterNodeId(), nullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet().getState().nodes().getMasterNodeId(), nullValue());
fail("should not be able to find master"); fail("should not be able to find master");
} catch (MasterNotDiscoveredException e) { } catch (MasterNotDiscoveredException e) {
// all is well, no master elected // all is well, no master elected

View File

@ -129,7 +129,7 @@ public class DiscoveryNodesTests extends ESTestCase {
}, ELECTED_MASTER("_master") { }, ELECTED_MASTER("_master") {
@Override @Override
Set<String> matchingNodeIds(DiscoveryNodes nodes) { Set<String> matchingNodeIds(DiscoveryNodes nodes) {
return Collections.singleton(nodes.masterNodeId()); return Collections.singleton(nodes.getMasterNodeId());
} }
}, MASTER_ELIGIBLE(DiscoveryNode.Role.MASTER.getRoleName() + ":true") { }, MASTER_ELIGIBLE(DiscoveryNode.Role.MASTER.getRoleName() + ":true") {
@Override @Override

View File

@ -402,7 +402,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
try { try {
assertEquals("unequal versions", state.version(), nodeState.version()); assertEquals("unequal versions", state.version(), nodeState.version());
assertEquals("unequal node count", state.nodes().getSize(), nodeState.nodes().getSize()); assertEquals("unequal node count", state.nodes().getSize(), nodeState.nodes().getSize());
assertEquals("different masters ", state.nodes().masterNodeId(), nodeState.nodes().masterNodeId()); assertEquals("different masters ", state.nodes().getMasterNodeId(), nodeState.nodes().getMasterNodeId());
assertEquals("different meta data version", state.metaData().version(), nodeState.metaData().version()); assertEquals("different meta data version", state.metaData().version(), nodeState.metaData().version());
if (!state.routingTable().prettyPrint().equals(nodeState.routingTable().prettyPrint())) { if (!state.routingTable().prettyPrint().equals(nodeState.routingTable().prettyPrint())) {
fail("different routing"); fail("different routing");
@ -638,7 +638,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
internalCluster().getInstance(ClusterService.class, oldMasterNode).add(new ClusterStateListener() { internalCluster().getInstance(ClusterService.class, oldMasterNode).add(new ClusterStateListener() {
@Override @Override
public void clusterChanged(ClusterChangedEvent event) { public void clusterChanged(ClusterChangedEvent event) {
if (event.state().nodes().masterNodeId() == null) { if (event.state().nodes().getMasterNodeId() == null) {
oldMasterNodeSteppedDown.countDown(); oldMasterNodeSteppedDown.countDown();
} }
} }

View File

@ -108,18 +108,18 @@ public class PendingClusterStatesQueueTests extends ESTestCase {
List<ClusterStateContext> committedContexts = randomCommitStates(queue); List<ClusterStateContext> committedContexts = randomCommitStates(queue);
ClusterState randomCommitted = randomFrom(committedContexts).state; ClusterState randomCommitted = randomFrom(committedContexts).state;
queue.markAsProcessed(randomCommitted); queue.markAsProcessed(randomCommitted);
final String processedMaster = randomCommitted.nodes().masterNodeId(); final String processedMaster = randomCommitted.nodes().getMasterNodeId();
// now check that queue doesn't contain anything pending from another master // now check that queue doesn't contain anything pending from another master
for (ClusterStateContext context : queue.pendingStates) { for (ClusterStateContext context : queue.pendingStates) {
final String pendingMaster = context.state.nodes().masterNodeId(); final String pendingMaster = context.state.nodes().getMasterNodeId();
assertThat("found a cluster state from [" + pendingMaster assertThat("found a cluster state from [" + pendingMaster
+ "], after a state from [" + processedMaster + "] was processed", + "], after a state from [" + processedMaster + "] was processed",
pendingMaster, equalTo(processedMaster)); pendingMaster, equalTo(processedMaster));
} }
// and check all committed contexts from another master were failed // and check all committed contexts from another master were failed
for (ClusterStateContext context : committedContexts) { for (ClusterStateContext context : committedContexts) {
if (context.state.nodes().masterNodeId().equals(processedMaster) == false) { if (context.state.nodes().getMasterNodeId().equals(processedMaster) == false) {
assertThat(((MockListener) context.listener).failure, notNullValue()); assertThat(((MockListener) context.listener).failure, notNullValue());
} }
} }

View File

@ -240,7 +240,7 @@ public class RareClusterStateIT extends ESIntegTestCase {
for (ShardRouting shard : shards) { for (ShardRouting shard : shards) {
if (shard.primary()) { if (shard.primary()) {
// primary must not be on the master node // primary must not be on the master node
assertFalse(state.nodes().masterNodeId().equals(shard.currentNodeId())); assertFalse(state.nodes().getMasterNodeId().equals(shard.currentNodeId()));
} else { } else {
fail(); // only primaries fail(); // only primaries
} }
@ -358,7 +358,7 @@ public class RareClusterStateIT extends ESIntegTestCase {
for (ShardRouting shard : shards) { for (ShardRouting shard : shards) {
if (shard.primary()) { if (shard.primary()) {
// primary must be on the master // primary must be on the master
assertEquals(state.nodes().masterNodeId(), shard.currentNodeId()); assertEquals(state.nodes().getMasterNodeId(), shard.currentNodeId());
} else { } else {
assertTrue(shard.active()); assertTrue(shard.active());
} }

View File

@ -63,20 +63,20 @@ public class AzureMinimumMasterNodesTests extends AbstractAzureComputeServiceTes
logger.info("--> start data node / non master node"); logger.info("--> start data node / non master node");
internalCluster().startNode(); internalCluster().startNode();
try { try {
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet().getState().nodes().masterNodeId(), nullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("100ms").execute().actionGet().getState().nodes().getMasterNodeId(), nullValue());
fail("should not be able to find master"); fail("should not be able to find master");
} catch (MasterNotDiscoveredException e) { } catch (MasterNotDiscoveredException e) {
// all is well, no master elected // all is well, no master elected
} }
logger.info("--> start another node"); logger.info("--> start another node");
internalCluster().startNode(); internalCluster().startNode();
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), notNullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue());
logger.info("--> stop master node"); logger.info("--> stop master node");
internalCluster().stopCurrentMasterNode(); internalCluster().stopCurrentMasterNode();
try { try {
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), nullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), nullValue());
fail("should not be able to find master"); fail("should not be able to find master");
} catch (MasterNotDiscoveredException e) { } catch (MasterNotDiscoveredException e) {
// all is well, no master elected // all is well, no master elected
@ -84,6 +84,6 @@ public class AzureMinimumMasterNodesTests extends AbstractAzureComputeServiceTes
logger.info("--> start another node"); logger.info("--> start another node");
internalCluster().startNode(); internalCluster().startNode();
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), notNullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue());
} }
} }

View File

@ -45,7 +45,7 @@ public class AzureSimpleTests extends AbstractAzureComputeServiceTestCase {
logger.info("--> start one node"); logger.info("--> start one node");
internalCluster().startNode(settings); internalCluster().startNode(settings);
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), notNullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue());
// We expect having 1 node as part of the cluster, let's test that // We expect having 1 node as part of the cluster, let's test that
checkNumberOfNodes(1); checkNumberOfNodes(1);
@ -58,7 +58,7 @@ public class AzureSimpleTests extends AbstractAzureComputeServiceTestCase {
logger.info("--> start one node"); logger.info("--> start one node");
internalCluster().startNode(settings); internalCluster().startNode(settings);
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), notNullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue());
// We expect having 1 node as part of the cluster, let's test that // We expect having 1 node as part of the cluster, let's test that
checkNumberOfNodes(1); checkNumberOfNodes(1);

View File

@ -46,11 +46,11 @@ public class AzureTwoStartedNodesTests extends AbstractAzureComputeServiceTestCa
logger.info("--> start first node"); logger.info("--> start first node");
internalCluster().startNode(settings); internalCluster().startNode(settings);
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), notNullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue());
logger.info("--> start another node"); logger.info("--> start another node");
internalCluster().startNode(settings); internalCluster().startNode(settings);
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), notNullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue());
// We expect having 2 nodes as part of the cluster, let's test that // We expect having 2 nodes as part of the cluster, let's test that
checkNumberOfNodes(2); checkNumberOfNodes(2);
@ -64,11 +64,11 @@ public class AzureTwoStartedNodesTests extends AbstractAzureComputeServiceTestCa
logger.info("--> start first node"); logger.info("--> start first node");
internalCluster().startNode(settings); internalCluster().startNode(settings);
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), notNullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue());
logger.info("--> start another node"); logger.info("--> start another node");
internalCluster().startNode(settings); internalCluster().startNode(settings);
assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().masterNodeId(), notNullValue()); assertThat(client().admin().cluster().prepareState().setMasterNodeTimeout("1s").execute().actionGet().getState().nodes().getMasterNodeId(), notNullValue());
// We expect having 2 nodes as part of the cluster, let's test that // We expect having 2 nodes as part of the cluster, let's test that
checkNumberOfNodes(2); checkNumberOfNodes(2);

View File

@ -1046,7 +1046,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
masterClusterState = ClusterState.Builder.fromBytes(masterClusterStateBytes, null); masterClusterState = ClusterState.Builder.fromBytes(masterClusterStateBytes, null);
Map<String, Object> masterStateMap = convertToMap(masterClusterState); Map<String, Object> masterStateMap = convertToMap(masterClusterState);
int masterClusterStateSize = ClusterState.Builder.toBytes(masterClusterState).length; int masterClusterStateSize = ClusterState.Builder.toBytes(masterClusterState).length;
String masterId = masterClusterState.nodes().masterNodeId(); String masterId = masterClusterState.nodes().getMasterNodeId();
for (Client client : cluster().getClients()) { for (Client client : cluster().getClients()) {
ClusterState localClusterState = client.admin().cluster().prepareState().all().setLocal(true).get().getState(); ClusterState localClusterState = client.admin().cluster().prepareState().all().setLocal(true).get().getState();
byte[] localClusterStateBytes = ClusterState.Builder.toBytes(localClusterState); byte[] localClusterStateBytes = ClusterState.Builder.toBytes(localClusterState);
@ -1056,7 +1056,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
final int localClusterStateSize = ClusterState.Builder.toBytes(localClusterState).length; final int localClusterStateSize = ClusterState.Builder.toBytes(localClusterState).length;
// Check that the non-master node has the same version of the cluster state as the master and // Check that the non-master node has the same version of the cluster state as the master and
// that the master node matches the master (otherwise there is no requirement for the cluster state to match) // that the master node matches the master (otherwise there is no requirement for the cluster state to match)
if (masterClusterState.version() == localClusterState.version() && masterId.equals(localClusterState.nodes().masterNodeId())) { if (masterClusterState.version() == localClusterState.version() && masterId.equals(localClusterState.nodes().getMasterNodeId())) {
try { try {
assertEquals("clusterstate UUID does not match", masterClusterState.stateUUID(), localClusterState.stateUUID()); assertEquals("clusterstate UUID does not match", masterClusterState.stateUUID(), localClusterState.stateUUID());
// We cannot compare serialization bytes since serialization order of maps is not guaranteed // We cannot compare serialization bytes since serialization order of maps is not guaranteed