Merge pull request #17410 from javanna/enhancement/discovery_node_one_getter

Remove duplicate getters from DiscoveryNode and DiscoveryNodes
This commit is contained in:
Luca Cavanna 2016-03-31 11:45:41 +02:00
commit 6b00b4b8dc
165 changed files with 707 additions and 842 deletions

View File

@ -68,10 +68,10 @@ public class NodesInfoResponse extends BaseNodesResponse<NodeInfo> implements To
builder.startObject("nodes"); builder.startObject("nodes");
for (NodeInfo nodeInfo : this) { for (NodeInfo nodeInfo : this) {
builder.startObject(nodeInfo.getNode().id(), XContentBuilder.FieldCaseConversion.NONE); builder.startObject(nodeInfo.getNode().getId(), XContentBuilder.FieldCaseConversion.NONE);
builder.field("name", nodeInfo.getNode().name(), XContentBuilder.FieldCaseConversion.NONE); builder.field("name", nodeInfo.getNode().getName(), XContentBuilder.FieldCaseConversion.NONE);
builder.field("transport_address", nodeInfo.getNode().address().toString()); builder.field("transport_address", nodeInfo.getNode().getAddress().toString());
builder.field("host", nodeInfo.getNode().getHostName(), XContentBuilder.FieldCaseConversion.NONE); builder.field("host", nodeInfo.getNode().getHostName(), XContentBuilder.FieldCaseConversion.NONE);
builder.field("ip", nodeInfo.getNode().getHostAddress(), XContentBuilder.FieldCaseConversion.NONE); builder.field("ip", nodeInfo.getNode().getHostAddress(), XContentBuilder.FieldCaseConversion.NONE);

View File

@ -299,8 +299,8 @@ public class NodeStats extends BaseNodeResponse implements ToXContent {
@Override @Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (!params.param("node_info_format", "default").equals("none")) { if (!params.param("node_info_format", "default").equals("none")) {
builder.field("name", getNode().name(), XContentBuilder.FieldCaseConversion.NONE); builder.field("name", getNode().getName(), XContentBuilder.FieldCaseConversion.NONE);
builder.field("transport_address", getNode().address().toString(), XContentBuilder.FieldCaseConversion.NONE); builder.field("transport_address", getNode().getAddress().toString(), XContentBuilder.FieldCaseConversion.NONE);
builder.field("host", getNode().getHostName(), XContentBuilder.FieldCaseConversion.NONE); builder.field("host", getNode().getHostName(), XContentBuilder.FieldCaseConversion.NONE);
builder.field("ip", getNode().getAddress(), XContentBuilder.FieldCaseConversion.NONE); builder.field("ip", getNode().getAddress(), XContentBuilder.FieldCaseConversion.NONE);

View File

@ -65,7 +65,7 @@ public class NodesStatsResponse extends BaseNodesResponse<NodeStats> implements
builder.startObject("nodes"); builder.startObject("nodes");
for (NodeStats nodeStats : this) { for (NodeStats nodeStats : this) {
builder.startObject(nodeStats.getNode().id(), XContentBuilder.FieldCaseConversion.NONE); builder.startObject(nodeStats.getNode().getId(), XContentBuilder.FieldCaseConversion.NONE);
builder.field("timestamp", nodeStats.getTimestamp()); builder.field("timestamp", nodeStats.getTimestamp());
nodeStats.toXContent(builder, params); nodeStats.toXContent(builder, params);

View File

@ -164,8 +164,8 @@ public class ListTasksResponse extends BaseTasksResponse implements ToXContent {
for (Map.Entry<DiscoveryNode, List<TaskInfo>> entry : getPerNodeTasks().entrySet()) { for (Map.Entry<DiscoveryNode, List<TaskInfo>> entry : getPerNodeTasks().entrySet()) {
DiscoveryNode node = entry.getKey(); DiscoveryNode node = entry.getKey();
builder.startObject(node.getId(), XContentBuilder.FieldCaseConversion.NONE); builder.startObject(node.getId(), XContentBuilder.FieldCaseConversion.NONE);
builder.field("name", node.name()); builder.field("name", node.getName());
builder.field("transport_address", node.address().toString()); builder.field("transport_address", node.getAddress().toString());
builder.field("host", node.getHostName()); builder.field("host", node.getHostName());
builder.field("ip", node.getAddress()); builder.field("ip", node.getAddress());

View File

@ -86,8 +86,8 @@ public class VerifyRepositoryResponse extends ActionResponse implements ToXConte
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.NODES); builder.startObject(Fields.NODES);
for (DiscoveryNode node : nodes) { for (DiscoveryNode node : nodes) {
builder.startObject(node.id(), XContentBuilder.FieldCaseConversion.NONE); builder.startObject(node.getId(), XContentBuilder.FieldCaseConversion.NONE);
builder.field(Fields.NAME, node.name(), XContentBuilder.FieldCaseConversion.NONE); builder.field(Fields.NAME, node.getName(), XContentBuilder.FieldCaseConversion.NONE);
builder.endObject(); builder.endObject();
} }
builder.endObject(); builder.endObject();

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 // 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;

View File

@ -107,7 +107,7 @@ public class TransportNodesSnapshotsStatus extends TransportNodesAction<Transpor
protected NodeSnapshotStatus nodeOperation(NodeRequest request) { protected NodeSnapshotStatus nodeOperation(NodeRequest request) {
Map<SnapshotId, Map<ShardId, SnapshotIndexShardStatus>> snapshotMapBuilder = new HashMap<>(); Map<SnapshotId, Map<ShardId, SnapshotIndexShardStatus>> snapshotMapBuilder = new HashMap<>();
try { try {
String nodeId = clusterService.localNode().id(); String nodeId = clusterService.localNode().getId();
for (SnapshotId snapshotId : request.snapshotIds) { for (SnapshotId snapshotId : request.snapshotIds) {
Map<ShardId, IndexShardSnapshotStatus> shardsStatus = snapshotShardsService.currentSnapshotShards(snapshotId); Map<ShardId, IndexShardSnapshotStatus> shardsStatus = snapshotShardsService.currentSnapshotShards(snapshotId);
if (shardsStatus == null) { if (shardsStatus == null) {

View File

@ -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();
} }

View File

@ -194,7 +194,7 @@ public class TransportIndicesShardStoresAction extends TransportMasterNodeReadAc
} }
private IndicesShardStoresResponse.StoreStatus.AllocationStatus getAllocationStatus(String index, int shardID, DiscoveryNode node) { private IndicesShardStoresResponse.StoreStatus.AllocationStatus getAllocationStatus(String index, int shardID, DiscoveryNode node) {
for (ShardRouting shardRouting : routingNodes.node(node.id())) { for (ShardRouting shardRouting : routingNodes.node(node.getId())) {
ShardId shardId = shardRouting.shardId(); ShardId shardId = shardRouting.shardId();
if (shardId.id() == shardID && shardId.getIndexName().equals(index)) { if (shardId.id() == shardID && shardId.getIndexName().equals(index)) {
if (shardRouting.primary()) { if (shardRouting.primary()) {

View File

@ -164,7 +164,7 @@ abstract class AbstractSearchAsyncAction<FirstResult extends SearchPhaseResult>
@Override @Override
public void onFailure(Throwable t) { public void onFailure(Throwable t) {
onFirstPhaseResult(shardIndex, shard, node.id(), shardIt, t); onFirstPhaseResult(shardIndex, shard, node.getId(), shardIt, t);
} }
}); });
} }

View File

@ -75,7 +75,7 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
int expectedOps = 0; int expectedOps = 0;
this.nodes = clusterState.nodes(); this.nodes = clusterState.nodes();
if (request.getScrollIds().size() == 1 && "_all".equals(request.getScrollIds().get(0))) { if (request.getScrollIds().size() == 1 && "_all".equals(request.getScrollIds().get(0))) {
expectedOps = nodes.size(); expectedOps = nodes.getSize();
} else { } else {
for (String parsedScrollId : request.getScrollIds()) { for (String parsedScrollId : request.getScrollIds()) {
ScrollIdForNode[] context = parseScrollId(parsedScrollId).getContext(); ScrollIdForNode[] context = parseScrollId(parsedScrollId).getContext();

View File

@ -299,7 +299,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
try { try {
NodeRequest nodeRequest = new NodeRequest(node.getId(), request, shards); NodeRequest nodeRequest = new NodeRequest(node.getId(), request, shards);
if (task != null) { if (task != null) {
nodeRequest.setParentTask(clusterService.localNode().id(), task.getId()); nodeRequest.setParentTask(clusterService.localNode().getId(), task.getId());
taskManager.registerChildTask(task, node.getId()); taskManager.registerChildTask(task, node.getId());
} }
transportService.sendRequest(node, transportNodeBroadcastAction, nodeRequest, new BaseTransportResponseHandler<NodeResponse>() { transportService.sendRequest(node, transportNodeBroadcastAction, nodeRequest, new BaseTransportResponseHandler<NodeResponse>() {
@ -330,7 +330,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
protected void onNodeResponse(DiscoveryNode node, int nodeIndex, NodeResponse response) { protected void onNodeResponse(DiscoveryNode node, int nodeIndex, NodeResponse response) {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("received response for [{}] from node [{}]", actionName, node.id()); logger.trace("received response for [{}] from node [{}]", actionName, node.getId());
} }
// this is defensive to protect against the possibility of double invocation // this is defensive to protect against the possibility of double invocation
@ -344,7 +344,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
} }
protected void onNodeFailure(DiscoveryNode node, int nodeIndex, Throwable t) { protected void onNodeFailure(DiscoveryNode node, int nodeIndex, Throwable t) {
String nodeId = node.id(); String nodeId = node.getId();
if (logger.isDebugEnabled() && !(t instanceof NodeShouldNotConnectException)) { if (logger.isDebugEnabled() && !(t instanceof NodeShouldNotConnectException)) {
logger.debug("failed to execute [{}] on node [{}]", t, actionName, nodeId); logger.debug("failed to execute [{}] on node [{}]", t, actionName, nodeId);
} }

View File

@ -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) {
@ -168,12 +168,12 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
}); });
} }
} else { } else {
if (nodes.masterNode() == null) { if (nodes.getMasterNode() == null) {
logger.debug("no known master node, scheduling a retry"); logger.debug("no known master node, scheduling a retry");
retry(null, MasterNodeChangePredicate.INSTANCE); retry(null, MasterNodeChangePredicate.INSTANCE);
} else { } else {
taskManager.registerChildTask(task, nodes.masterNode().getId()); taskManager.registerChildTask(task, nodes.getMasterNode().getId());
transportService.sendRequest(nodes.masterNode(), actionName, request, new ActionListenerResponseHandler<Response>(listener) { transportService.sendRequest(nodes.getMasterNode(), actionName, request, new ActionListenerResponseHandler<Response>(listener) {
@Override @Override
public Response newInstance() { public Response newInstance() {
return newResponse(); return newResponse();
@ -185,7 +185,7 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
if (cause instanceof ConnectTransportException) { if (cause instanceof ConnectTransportException) {
// we want to retry here a bit to see if a new master is elected // we want to retry here a bit to see if a new master is elected
logger.debug("connection exception while trying to forward request with action name [{}] to master node [{}], scheduling a retry. Error: [{}]", logger.debug("connection exception while trying to forward request with action name [{}] to master node [{}], scheduling a retry. Error: [{}]",
actionName, nodes.masterNode(), exp.getDetailedMessage()); actionName, nodes.getMasterNode(), exp.getDetailedMessage());
retry(cause, MasterNodeChangePredicate.INSTANCE); retry(cause, MasterNodeChangePredicate.INSTANCE);
} else { } else {
listener.onFailure(exp); listener.onFailure(exp);

View File

@ -81,7 +81,7 @@ public abstract class BaseNodesResponse<TNodeResponse extends BaseNodeResponse>
if (nodesMap == null) { if (nodesMap == null) {
nodesMap = new HashMap<>(); nodesMap = new HashMap<>();
for (TNodeResponse nodeResponse : nodes) { for (TNodeResponse nodeResponse : nodes) {
nodesMap.put(nodeResponse.getNode().id(), nodeResponse); nodesMap.put(nodeResponse.getNode().getId(), nodeResponse);
} }
} }
return nodesMap; return nodesMap;

View File

@ -127,7 +127,7 @@ public abstract class TransportNodesAction<NodesRequest extends BaseNodesRequest
ClusterState clusterState = clusterService.state(); ClusterState clusterState = clusterService.state();
String[] nodesIds = resolveNodes(request, clusterState); String[] nodesIds = resolveNodes(request, clusterState);
this.nodesIds = filterNodeIds(clusterState.nodes(), nodesIds); this.nodesIds = filterNodeIds(clusterState.nodes(), nodesIds);
ImmutableOpenMap<String, DiscoveryNode> nodes = clusterState.nodes().nodes(); ImmutableOpenMap<String, DiscoveryNode> nodes = clusterState.nodes().getNodes();
this.nodes = new DiscoveryNode[nodesIds.length]; this.nodes = new DiscoveryNode[nodesIds.length];
for (int i = 0; i < nodesIds.length; i++) { for (int i = 0; i < nodesIds.length; i++) {
this.nodes[i] = nodes.get(nodesIds[i]); this.nodes[i] = nodes.get(nodesIds[i]);
@ -161,7 +161,7 @@ public abstract class TransportNodesAction<NodesRequest extends BaseNodesRequest
} else { } else {
ChildTaskRequest nodeRequest = newNodeRequest(nodeId, request); ChildTaskRequest nodeRequest = newNodeRequest(nodeId, request);
if (task != null) { if (task != null) {
nodeRequest.setParentTask(clusterService.localNode().id(), task.getId()); nodeRequest.setParentTask(clusterService.localNode().getId(), task.getId());
taskManager.registerChildTask(task, node.getId()); taskManager.registerChildTask(task, node.getId());
} }
@ -178,7 +178,7 @@ public abstract class TransportNodesAction<NodesRequest extends BaseNodesRequest
@Override @Override
public void handleException(TransportException exp) { public void handleException(TransportException exp) {
onFailure(idx, node.id(), exp); onFailure(idx, node.getId(), exp);
} }
@Override @Override

View File

@ -449,7 +449,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
} }
final DiscoveryNode node = state.nodes().get(primary.currentNodeId()); final DiscoveryNode node = state.nodes().get(primary.currentNodeId());
taskManager.registerChildTask(task, node.getId()); taskManager.registerChildTask(task, node.getId());
if (primary.currentNodeId().equals(state.nodes().localNodeId())) { if (primary.currentNodeId().equals(state.nodes().getLocalNodeId())) {
performLocalAction(state, primary, node); performLocalAction(state, primary, node);
} else { } else {
performRemoteAction(state, primary, node); performRemoteAction(state, primary, node);
@ -552,7 +552,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
final Throwable cause = exp.unwrapCause(); final Throwable cause = exp.unwrapCause();
if (cause instanceof ConnectTransportException || cause instanceof NodeClosedException || if (cause instanceof ConnectTransportException || cause instanceof NodeClosedException ||
(isPrimaryAction && retryPrimaryException(cause))) { (isPrimaryAction && retryPrimaryException(cause))) {
logger.trace("received an error from node [{}] for request [{}], scheduling a retry", exp, node.id(), request); logger.trace("received an error from node [{}] for request [{}], scheduling a retry", exp, node.getId(), request);
retry(exp); retry(exp);
} else { } else {
finishAsFailed(exp); finishAsFailed(exp);
@ -909,12 +909,12 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
// we never execute replication operation locally as primary operation has already completed locally // we never execute replication operation locally as primary operation has already completed locally
// hence, we ignore any local shard for replication // hence, we ignore any local shard for replication
if (nodes.localNodeId().equals(shard.currentNodeId()) == false) { if (nodes.getLocalNodeId().equals(shard.currentNodeId()) == false) {
onLocalShard.accept(shard); onLocalShard.accept(shard);
} }
// send operation to relocating shard // send operation to relocating shard
// local shard can be a relocation target of a primary that is in relocated state // local shard can be a relocation target of a primary that is in relocated state
if (shard.relocating() && nodes.localNodeId().equals(shard.relocatingNodeId()) == false) { if (shard.relocating() && nodes.getLocalNodeId().equals(shard.relocatingNodeId()) == false) {
onRelocatingShard.accept(shard); onRelocatingShard.accept(shard);
} }
} }

View File

@ -228,7 +228,7 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
@Override @Override
public void onClusterServiceClose() { public void onClusterServiceClose() {
listener.onFailure(new NodeClosedException(nodes.localNode())); listener.onFailure(new NodeClosedException(nodes.getLocalNode()));
} }
@Override @Override

View File

@ -113,10 +113,10 @@ public abstract class TransportTasksAction<
results.add(response); results.add(response);
} }
} catch (Exception ex) { } catch (Exception ex) {
exceptions.add(new TaskOperationFailure(clusterService.localNode().id(), task.getId(), ex)); exceptions.add(new TaskOperationFailure(clusterService.localNode().getId(), task.getId(), ex));
} }
}); });
return new NodeTasksResponse(clusterService.localNode().id(), results, exceptions); return new NodeTasksResponse(clusterService.localNode().getId(), results, exceptions);
} }
protected String[] filterNodeIds(DiscoveryNodes nodes, String[] nodesIds) { protected String[] filterNodeIds(DiscoveryNodes nodes, String[] nodesIds) {
@ -205,7 +205,7 @@ public abstract class TransportTasksAction<
ClusterState clusterState = clusterService.state(); ClusterState clusterState = clusterService.state();
String[] nodesIds = resolveNodes(request, clusterState); String[] nodesIds = resolveNodes(request, clusterState);
this.nodesIds = filterNodeIds(clusterState.nodes(), nodesIds); this.nodesIds = filterNodeIds(clusterState.nodes(), nodesIds);
ImmutableOpenMap<String, DiscoveryNode> nodes = clusterState.nodes().nodes(); ImmutableOpenMap<String, DiscoveryNode> nodes = clusterState.nodes().getNodes();
this.nodes = new DiscoveryNode[nodesIds.length]; this.nodes = new DiscoveryNode[nodesIds.length];
for (int i = 0; i < this.nodesIds.length; i++) { for (int i = 0; i < this.nodesIds.length; i++) {
this.nodes[i] = nodes.get(this.nodesIds[i]); this.nodes[i] = nodes.get(this.nodesIds[i]);
@ -237,7 +237,7 @@ public abstract class TransportTasksAction<
onFailure(idx, nodeId, new NoSuchNodeException(nodeId)); onFailure(idx, nodeId, new NoSuchNodeException(nodeId));
} else { } else {
NodeTaskRequest nodeRequest = new NodeTaskRequest(request); NodeTaskRequest nodeRequest = new NodeTaskRequest(request);
nodeRequest.setParentTask(clusterService.localNode().id(), task.getId()); nodeRequest.setParentTask(clusterService.localNode().getId(), task.getId());
taskManager.registerChildTask(task, node.getId()); taskManager.registerChildTask(task, node.getId());
transportService.sendRequest(node, transportNodeAction, nodeRequest, builder.build(), transportService.sendRequest(node, transportNodeAction, nodeRequest, builder.build(),
new BaseTransportResponseHandler<NodeTasksResponse>() { new BaseTransportResponseHandler<NodeTasksResponse>() {
@ -253,7 +253,7 @@ public abstract class TransportTasksAction<
@Override @Override
public void handleException(TransportException exp) { public void handleException(TransportException exp) {
onFailure(idx, node.id(), exp); onFailure(idx, node.getId(), exp);
} }
@Override @Override

View File

@ -138,7 +138,7 @@ public class TransportClientNodesService extends AbstractComponent {
public List<TransportAddress> transportAddresses() { public List<TransportAddress> transportAddresses() {
List<TransportAddress> lstBuilder = new ArrayList<>(); List<TransportAddress> lstBuilder = new ArrayList<>();
for (DiscoveryNode listedNode : listedNodes) { for (DiscoveryNode listedNode : listedNodes) {
lstBuilder.add(listedNode.address()); lstBuilder.add(listedNode.getAddress());
} }
return Collections.unmodifiableList(lstBuilder); return Collections.unmodifiableList(lstBuilder);
} }
@ -164,7 +164,7 @@ public class TransportClientNodesService extends AbstractComponent {
for (TransportAddress transportAddress : transportAddresses) { for (TransportAddress transportAddress : transportAddresses) {
boolean found = false; boolean found = false;
for (DiscoveryNode otherNode : listedNodes) { for (DiscoveryNode otherNode : listedNodes) {
if (otherNode.address().equals(transportAddress)) { if (otherNode.getAddress().equals(transportAddress)) {
found = true; found = true;
logger.debug("address [{}] already exists with [{}], ignoring...", transportAddress, otherNode); logger.debug("address [{}] already exists with [{}], ignoring...", transportAddress, otherNode);
break; break;
@ -198,7 +198,7 @@ public class TransportClientNodesService extends AbstractComponent {
} }
List<DiscoveryNode> builder = new ArrayList<>(); List<DiscoveryNode> builder = new ArrayList<>();
for (DiscoveryNode otherNode : listedNodes) { for (DiscoveryNode otherNode : listedNodes) {
if (!otherNode.address().equals(transportAddress)) { if (!otherNode.getAddress().equals(transportAddress)) {
builder.add(otherNode); builder.add(otherNode);
} else { } else {
logger.debug("removing address [{}]", otherNode); logger.debug("removing address [{}]", otherNode);
@ -383,7 +383,7 @@ public class TransportClientNodesService extends AbstractComponent {
// use discovered information but do keep the original transport address, // use discovered information but do keep the original transport address,
// so people can control which address is exactly used. // so people can control which address is exactly used.
DiscoveryNode nodeWithInfo = livenessResponse.getDiscoveryNode(); DiscoveryNode nodeWithInfo = livenessResponse.getDiscoveryNode();
newNodes.add(new DiscoveryNode(nodeWithInfo.name(), nodeWithInfo.id(), nodeWithInfo.getHostName(), newNodes.add(new DiscoveryNode(nodeWithInfo.getName(), nodeWithInfo.getId(), nodeWithInfo.getHostName(),
nodeWithInfo.getHostAddress(), listedNode.getAddress(), nodeWithInfo.getAttributes(), nodeWithInfo.getHostAddress(), listedNode.getAddress(), nodeWithInfo.getAttributes(),
nodeWithInfo.getRoles(), nodeWithInfo.getVersion())); nodeWithInfo.getRoles(), nodeWithInfo.getVersion()));
} else { } else {
@ -491,11 +491,11 @@ public class TransportClientNodesService extends AbstractComponent {
for (Map.Entry<DiscoveryNode, ClusterStateResponse> entry : clusterStateResponses.entrySet()) { for (Map.Entry<DiscoveryNode, ClusterStateResponse> entry : clusterStateResponses.entrySet()) {
if (!ignoreClusterName && !clusterName.equals(entry.getValue().getClusterName())) { if (!ignoreClusterName && !clusterName.equals(entry.getValue().getClusterName())) {
logger.warn("node {} not part of the cluster {}, ignoring...", logger.warn("node {} not part of the cluster {}, ignoring...",
entry.getValue().getState().nodes().localNode(), clusterName); entry.getValue().getState().nodes().getLocalNode(), clusterName);
newFilteredNodes.add(entry.getKey()); newFilteredNodes.add(entry.getKey());
continue; continue;
} }
for (ObjectCursor<DiscoveryNode> cursor : entry.getValue().getState().nodes().dataNodes().values()) { for (ObjectCursor<DiscoveryNode> cursor : entry.getValue().getState().nodes().getDataNodes().values()) {
newNodes.add(cursor.value); newNodes.add(cursor.value);
} }
} }

View File

@ -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();
} }
/** /**

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)) {
@ -747,7 +747,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
@Override @Override
public ClusterState readFrom(StreamInput in) throws IOException { public ClusterState readFrom(StreamInput in) throws IOException {
return readFrom(in, nodes.localNode()); return readFrom(in, nodes.getLocalNode());
} }
@Override @Override

View File

@ -167,7 +167,7 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
// Check whether it was a data node that was added // Check whether it was a data node that was added
boolean dataNodeAdded = false; boolean dataNodeAdded = false;
for (DiscoveryNode addedNode : event.nodesDelta().addedNodes()) { for (DiscoveryNode addedNode : event.nodesDelta().addedNodes()) {
if (addedNode.dataNode()) { if (addedNode.isDataNode()) {
dataNodeAdded = true; dataNodeAdded = true;
break; break;
} }
@ -182,7 +182,7 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
if (this.isMaster && event.nodesRemoved()) { if (this.isMaster && event.nodesRemoved()) {
for (DiscoveryNode removedNode : event.nodesDelta().removedNodes()) { for (DiscoveryNode removedNode : event.nodesDelta().removedNodes()) {
if (removedNode.dataNode()) { if (removedNode.isDataNode()) {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("Removing node from cluster info: {}", removedNode.getId()); logger.trace("Removing node from cluster info: {}", removedNode.getId());
} }
@ -396,7 +396,7 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
ImmutableOpenMap.Builder<String, DiskUsage> newMostAvaiableUsages) { ImmutableOpenMap.Builder<String, DiskUsage> newMostAvaiableUsages) {
for (NodeStats nodeStats : nodeStatsArray) { for (NodeStats nodeStats : nodeStatsArray) {
if (nodeStats.getFs() == null) { if (nodeStats.getFs() == null) {
logger.warn("Unable to retrieve node FS stats for {}", nodeStats.getNode().name()); logger.warn("Unable to retrieve node FS stats for {}", nodeStats.getNode().getName());
} else { } else {
FsInfo.Path leastAvailablePath = null; FsInfo.Path leastAvailablePath = null;
FsInfo.Path mostAvailablePath = null; FsInfo.Path mostAvailablePath = null;
@ -410,7 +410,7 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
mostAvailablePath = info; mostAvailablePath = info;
} }
} }
String nodeId = nodeStats.getNode().id(); String nodeId = nodeStats.getNode().getId();
String nodeName = nodeStats.getNode().getName(); String nodeName = nodeStats.getNode().getName();
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("node: [{}], most available: total disk: {}, available disk: {} / least available: total disk: {}, available disk: {}", logger.trace("node: [{}], most available: total disk: {}, available disk: {} / least available: total disk: {}, available disk: {}",

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

@ -78,9 +78,9 @@ public class NodeIndexDeletedAction extends AbstractComponent {
public void nodeIndexDeleted(final ClusterState clusterState, final Index index, final IndexSettings indexSettings, final String nodeId) { public void nodeIndexDeleted(final ClusterState clusterState, final Index index, final IndexSettings indexSettings, final String nodeId) {
final DiscoveryNodes nodes = clusterState.nodes(); final DiscoveryNodes nodes = clusterState.nodes();
transportService.sendRequest(clusterState.nodes().masterNode(), transportService.sendRequest(clusterState.nodes().getMasterNode(),
INDEX_DELETED_ACTION_NAME, new NodeIndexDeletedMessage(index, nodeId), EmptyTransportResponseHandler.INSTANCE_SAME); INDEX_DELETED_ACTION_NAME, new NodeIndexDeletedMessage(index, nodeId), EmptyTransportResponseHandler.INSTANCE_SAME);
if (nodes.localNode().isDataNode() == false) { if (nodes.getLocalNode().isDataNode() == false) {
logger.trace("[{}] not acking store deletion (not a data node)", index); logger.trace("[{}] not acking store deletion (not a data node)", index);
return; return;
} }
@ -104,7 +104,7 @@ public class NodeIndexDeletedAction extends AbstractComponent {
// due to a "currently canceled recovery" or so. The shard will delete itself BEFORE the lock is released so it's guaranteed to be // due to a "currently canceled recovery" or so. The shard will delete itself BEFORE the lock is released so it's guaranteed to be
// deleted by the time we get the lock // deleted by the time we get the lock
indicesService.processPendingDeletes(indexSettings.getIndex(), indexSettings, new TimeValue(30, TimeUnit.MINUTES)); indicesService.processPendingDeletes(indexSettings.getIndex(), indexSettings, new TimeValue(30, TimeUnit.MINUTES));
transportService.sendRequest(clusterState.nodes().masterNode(), transportService.sendRequest(clusterState.nodes().getMasterNode(),
INDEX_STORE_DELETED_ACTION_NAME, new NodeIndexStoreDeletedMessage(index, nodeId), EmptyTransportResponseHandler.INSTANCE_SAME); INDEX_STORE_DELETED_ACTION_NAME, new NodeIndexStoreDeletedMessage(index, nodeId), EmptyTransportResponseHandler.INSTANCE_SAME);
} catch (LockObtainFailedException exc) { } catch (LockObtainFailedException exc) {
logger.warn("[{}] failed to lock all shards for index - timed out after 30 seconds", index); logger.warn("[{}] failed to lock all shards for index - timed out after 30 seconds", index);

View File

@ -60,11 +60,11 @@ public class NodeMappingRefreshAction extends AbstractComponent {
public void nodeMappingRefresh(final ClusterState state, final NodeMappingRefreshRequest request) { public void nodeMappingRefresh(final ClusterState state, final NodeMappingRefreshRequest request) {
final DiscoveryNodes nodes = state.nodes(); final DiscoveryNodes nodes = state.nodes();
if (nodes.masterNode() == null) { if (nodes.getMasterNode() == null) {
logger.warn("can't send mapping refresh for [{}], no master known.", request.index()); logger.warn("can't send mapping refresh for [{}], no master known.", request.index());
return; return;
} }
transportService.sendRequest(nodes.masterNode(), ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); transportService.sendRequest(nodes.getMasterNode(), ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
} }
private class NodeMappingRefreshTransportHandler implements TransportRequestHandler<NodeMappingRefreshRequest> { private class NodeMappingRefreshTransportHandler implements TransportRequestHandler<NodeMappingRefreshRequest> {

View File

@ -92,7 +92,7 @@ public class ShardStateAction extends AbstractComponent {
} }
private void sendShardAction(final String actionName, final ClusterStateObserver observer, final ShardRoutingEntry shardRoutingEntry, final Listener listener) { private void sendShardAction(final String actionName, final ClusterStateObserver observer, final ShardRoutingEntry shardRoutingEntry, final Listener listener) {
DiscoveryNode masterNode = observer.observedState().nodes().masterNode(); DiscoveryNode masterNode = observer.observedState().nodes().getMasterNode();
if (masterNode == null) { if (masterNode == null) {
logger.warn("{} no master known for action [{}] for shard [{}]", shardRoutingEntry.getShardRouting().shardId(), actionName, shardRoutingEntry.getShardRouting()); logger.warn("{} no master known for action [{}] for shard [{}]", shardRoutingEntry.getShardRouting().shardId(), actionName, shardRoutingEntry.getShardRouting());
waitForNewMasterAndRetry(actionName, observer, shardRoutingEntry, listener); waitForNewMasterAndRetry(actionName, observer, shardRoutingEntry, listener);

View File

@ -91,8 +91,8 @@ public final class ClusterStateHealth implements Iterable<ClusterIndexHealth>, S
public ClusterStateHealth(ClusterState clusterState, String[] concreteIndices) { public ClusterStateHealth(ClusterState clusterState, String[] concreteIndices) {
RoutingTableValidation validation = clusterState.routingTable().validate(clusterState.metaData()); RoutingTableValidation validation = clusterState.routingTable().validate(clusterState.metaData());
validationFailures = validation.failures(); validationFailures = validation.failures();
numberOfNodes = clusterState.nodes().size(); numberOfNodes = clusterState.nodes().getSize();
numberOfDataNodes = clusterState.nodes().dataNodes().size(); numberOfDataNodes = clusterState.nodes().getDataNodes().size();
for (String index : concreteIndices) { for (String index : concreteIndices) {
IndexRoutingTable indexRoutingTable = clusterState.routingTable().index(index); IndexRoutingTable indexRoutingTable = clusterState.routingTable().index(index);

View File

@ -294,7 +294,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
if (indexSettingsBuilder.get(SETTING_VERSION_CREATED) == null) { if (indexSettingsBuilder.get(SETTING_VERSION_CREATED) == null) {
DiscoveryNodes nodes = currentState.nodes(); DiscoveryNodes nodes = currentState.nodes();
final Version createdVersion = Version.smallest(version, nodes.smallestNonClientNodeVersion()); final Version createdVersion = Version.smallest(version, nodes.getSmallestNonClientNodeVersion());
indexSettingsBuilder.put(SETTING_VERSION_CREATED, createdVersion); indexSettingsBuilder.put(SETTING_VERSION_CREATED, createdVersion);
} }

View File

@ -101,9 +101,9 @@ public class MetaDataDeleteIndexService extends AbstractComponent {
metaDataBuilder.remove(indexName); metaDataBuilder.remove(indexName);
} }
// wait for events from all nodes that it has been removed from their respective metadata... // wait for events from all nodes that it has been removed from their respective metadata...
int count = currentState.nodes().size(); int count = currentState.nodes().getSize();
// add the notifications that the store was deleted from *data* nodes // add the notifications that the store was deleted from *data* nodes
count += currentState.nodes().dataNodes().size(); count += currentState.nodes().getDataNodes().size();
final AtomicInteger counter = new AtomicInteger(count * indices.size()); final AtomicInteger counter = new AtomicInteger(count * indices.size());
// this listener will be notified once we get back a notification based on the cluster state change below. // this listener will be notified once we get back a notification based on the cluster state change below.

View File

@ -79,11 +79,11 @@ 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
final int dataNodeCount = event.state().nodes().dataNodes().size(); final int dataNodeCount = event.state().nodes().getDataNodes().size();
Map<Integer, List<Index>> nrReplicasChanged = new HashMap<>(); Map<Integer, List<Index>> nrReplicasChanged = new HashMap<>();
// we need to do this each time in case it was changed by update settings // we need to do this each time in case it was changed by update settings

View File

@ -35,7 +35,6 @@ import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.function.Predicate; import java.util.function.Predicate;
@ -47,7 +46,7 @@ import static org.elasticsearch.common.transport.TransportAddressSerializers.add
*/ */
public class DiscoveryNode implements Writeable<DiscoveryNode>, ToXContent { public class DiscoveryNode implements Writeable<DiscoveryNode>, ToXContent {
public static boolean localNode(Settings settings) { public static boolean isLocalNode(Settings settings) {
if (Node.NODE_LOCAL_SETTING.exists(settings)) { if (Node.NODE_LOCAL_SETTING.exists(settings)) {
return Node.NODE_LOCAL_SETTING.get(settings); return Node.NODE_LOCAL_SETTING.get(settings);
} }
@ -68,20 +67,18 @@ public class DiscoveryNode implements Writeable<DiscoveryNode>, ToXContent {
return Node.NODE_DATA_SETTING.get(settings) || Node.NODE_MASTER_SETTING.get(settings); return Node.NODE_DATA_SETTING.get(settings) || Node.NODE_MASTER_SETTING.get(settings);
} }
public static boolean masterNode(Settings settings) { public static boolean isMasterNode(Settings settings) {
return Node.NODE_MASTER_SETTING.get(settings); return Node.NODE_MASTER_SETTING.get(settings);
} }
public static boolean dataNode(Settings settings) { public static boolean isDataNode(Settings settings) {
return Node.NODE_DATA_SETTING.get(settings); return Node.NODE_DATA_SETTING.get(settings);
} }
public static boolean ingestNode(Settings settings) { public static boolean isIngestNode(Settings settings) {
return Node.NODE_INGEST_SETTING.get(settings); return Node.NODE_INGEST_SETTING.get(settings);
} }
public static final List<DiscoveryNode> EMPTY_LIST = Collections.emptyList();
private final String nodeName; private final String nodeName;
private final String nodeId; private final String nodeId;
private final String hostName; private final String hostName;
@ -207,46 +204,25 @@ public class DiscoveryNode implements Writeable<DiscoveryNode>, ToXContent {
this.roles = Collections.unmodifiableSet(rolesSet); this.roles = Collections.unmodifiableSet(rolesSet);
} }
/**
* The address that the node can be communicated with.
*/
public TransportAddress address() {
return address;
}
/** /**
* The address that the node can be communicated with. * The address that the node can be communicated with.
*/ */
public TransportAddress getAddress() { public TransportAddress getAddress() {
return address(); return address;
}
/**
* The unique id of the node.
*/
public String id() {
return nodeId;
} }
/** /**
* The unique id of the node. * The unique id of the node.
*/ */
public String getId() { public String getId() {
return id(); return nodeId;
}
/**
* The name of the node.
*/
public String name() {
return this.nodeName;
} }
/** /**
* The name of the node. * The name of the node.
*/ */
public String getName() { public String getName() {
return name(); return this.nodeName;
} }
/** /**
@ -256,32 +232,18 @@ public class DiscoveryNode implements Writeable<DiscoveryNode>, ToXContent {
return this.attributes; return this.attributes;
} }
/**
* Should this node hold data (shards) or not.
*/
public boolean dataNode() {
return roles.contains(Role.DATA);
}
/** /**
* Should this node hold data (shards) or not. * Should this node hold data (shards) or not.
*/ */
public boolean isDataNode() { public boolean isDataNode() {
return dataNode(); return roles.contains(Role.DATA);
}
/**
* Can this node become master or not.
*/
public boolean masterNode() {
return roles.contains(Role.MASTER);
} }
/** /**
* Can this node become master or not. * Can this node become master or not.
*/ */
public boolean isMasterNode() { public boolean isMasterNode() {
return masterNode(); return roles.contains(Role.MASTER);
} }
/** /**
@ -299,7 +261,7 @@ public class DiscoveryNode implements Writeable<DiscoveryNode>, ToXContent {
return roles; return roles;
} }
public Version version() { public Version getVersion() {
return this.version; return this.version;
} }
@ -311,10 +273,6 @@ public class DiscoveryNode implements Writeable<DiscoveryNode>, ToXContent {
return this.hostAddress; return this.hostAddress;
} }
public Version getVersion() {
return this.version;
}
@Override @Override
public DiscoveryNode readFrom(StreamInput in) throws IOException { public DiscoveryNode readFrom(StreamInput in) throws IOException {
return new DiscoveryNode(in); return new DiscoveryNode(in);
@ -377,9 +335,9 @@ public class DiscoveryNode implements Writeable<DiscoveryNode>, ToXContent {
@Override @Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(id(), XContentBuilder.FieldCaseConversion.NONE); builder.startObject(getId(), XContentBuilder.FieldCaseConversion.NONE);
builder.field("name", name()); builder.field("name", getName());
builder.field("transport_address", address().toString()); builder.field("transport_address", getAddress().toString());
builder.startObject("attributes"); builder.startObject("attributes");
for (Map.Entry<String, String> entry : attributes.entrySet()) { for (Map.Entry<String, String> entry : attributes.entrySet()) {

View File

@ -82,8 +82,8 @@ public class DiscoveryNodeFilters {
if ("_ip".equals(attr)) { if ("_ip".equals(attr)) {
// We check both the host_ip or the publish_ip // We check both the host_ip or the publish_ip
String publishAddress = null; String publishAddress = null;
if (node.address() instanceof InetSocketTransportAddress) { if (node.getAddress() instanceof InetSocketTransportAddress) {
publishAddress = NetworkAddress.format(((InetSocketTransportAddress) node.address()).address().getAddress()); publishAddress = NetworkAddress.format(((InetSocketTransportAddress) node.getAddress()).address().getAddress());
} }
boolean match = matchByIP(values, node.getHostAddress(), publishAddress); boolean match = matchByIP(values, node.getHostAddress(), publishAddress);
@ -116,8 +116,8 @@ public class DiscoveryNodeFilters {
} else if ("_publish_ip".equals(attr)) { } else if ("_publish_ip".equals(attr)) {
// We check explicitly only the publish_ip // We check explicitly only the publish_ip
String address = null; String address = null;
if (node.address() instanceof InetSocketTransportAddress) { if (node.getAddress() instanceof InetSocketTransportAddress) {
address = NetworkAddress.format(((InetSocketTransportAddress) node.address()).address().getAddress()); address = NetworkAddress.format(((InetSocketTransportAddress) node.getAddress()).address().getAddress());
} }
boolean match = matchByIP(values, address, null); boolean match = matchByIP(values, address, null);
@ -155,7 +155,7 @@ public class DiscoveryNodeFilters {
} }
} else if ("_id".equals(attr)) { } else if ("_id".equals(attr)) {
for (String value : values) { for (String value : values) {
if (node.id().equals(value)) { if (node.getId().equals(value)) {
if (opType == OpType.OR) { if (opType == OpType.OR) {
return true; return true;
} }
@ -167,7 +167,7 @@ public class DiscoveryNodeFilters {
} }
} else if ("_name".equals(attr) || "name".equals(attr)) { } else if ("_name".equals(attr) || "name".equals(attr)) {
for (String value : values) { for (String value : values) {
if (Regex.simpleMatch(value, node.name())) { if (Regex.simpleMatch(value, node.getName())) {
if (opType == OpType.OR) { if (opType == OpType.OR) {
return true; return true;
} }

View File

@ -78,17 +78,9 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
} }
/** /**
* Is this a valid nodes that has the minimal information set. The minimal set is defined * Returns <tt>true</tt> if the local node is the elected master node.
* by the localNodeId being set.
*/ */
public boolean valid() { public boolean isLocalNodeElectedMaster() {
return localNodeId != null;
}
/**
* Returns <tt>true</tt> if the local node is the master node.
*/
public boolean localNodeMaster() {
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;
@ -96,31 +88,13 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
return localNodeId.equals(masterNodeId); return localNodeId.equals(masterNodeId);
} }
/**
* Get the number of known nodes
*
* @return number of nodes
*/
public int size() {
return nodes.size();
}
/** /**
* Get the number of known nodes * Get the number of known nodes
* *
* @return number of nodes * @return number of nodes
*/ */
public int getSize() { public int getSize() {
return size(); return nodes.size();
}
/**
* Get a {@link Map} of the discovered nodes arranged by their ids
*
* @return {@link Map} of the discovered nodes arranged by their ids
*/
public ImmutableOpenMap<String, DiscoveryNode> nodes() {
return this.nodes;
} }
/** /**
@ -129,16 +103,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
* @return {@link Map} of the discovered nodes arranged by their ids * @return {@link Map} of the discovered nodes arranged by their ids
*/ */
public ImmutableOpenMap<String, DiscoveryNode> getNodes() { public ImmutableOpenMap<String, DiscoveryNode> getNodes() {
return nodes(); return this.nodes;
}
/**
* Get a {@link Map} of the discovered data nodes arranged by their ids
*
* @return {@link Map} of the discovered data nodes arranged by their ids
*/
public ImmutableOpenMap<String, DiscoveryNode> dataNodes() {
return this.dataNodes;
} }
/** /**
@ -147,16 +112,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
* @return {@link Map} of the discovered data nodes arranged by their ids * @return {@link Map} of the discovered data nodes arranged by their ids
*/ */
public ImmutableOpenMap<String, DiscoveryNode> getDataNodes() { public ImmutableOpenMap<String, DiscoveryNode> getDataNodes() {
return dataNodes(); return this.dataNodes;
}
/**
* Get a {@link Map} of the discovered master nodes arranged by their ids
*
* @return {@link Map} of the discovered master nodes arranged by their ids
*/
public ImmutableOpenMap<String, DiscoveryNode> masterNodes() {
return this.masterNodes;
} }
/** /**
@ -165,7 +121,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
* @return {@link Map} of the discovered master nodes arranged by their ids * @return {@link Map} of the discovered master nodes arranged by their ids
*/ */
public ImmutableOpenMap<String, DiscoveryNode> getMasterNodes() { public ImmutableOpenMap<String, DiscoveryNode> getMasterNodes() {
return masterNodes(); return this.masterNodes;
} }
/** /**
@ -180,7 +136,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
* *
* @return {@link Map} of the discovered master and data nodes arranged by their ids * @return {@link Map} of the discovered master and data nodes arranged by their ids
*/ */
public ImmutableOpenMap<String, DiscoveryNode> masterAndDataNodes() { public ImmutableOpenMap<String, DiscoveryNode> getMasterAndDataNodes() {
ImmutableOpenMap.Builder<String, DiscoveryNode> nodes = ImmutableOpenMap.builder(dataNodes); ImmutableOpenMap.Builder<String, DiscoveryNode> nodes = ImmutableOpenMap.builder(dataNodes);
nodes.putAll(masterNodes); nodes.putAll(masterNodes);
return nodes.build(); return nodes.build();
@ -206,31 +162,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;
}
/**
* Get the id of the local node
*
* @return id of the local node
*/
public String localNodeId() {
return this.localNodeId;
} }
/** /**
@ -239,16 +177,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
* @return id of the local node * @return id of the local node
*/ */
public String getLocalNodeId() { public String getLocalNodeId() {
return localNodeId(); return this.localNodeId;
}
/**
* Get the local node
*
* @return local node
*/
public DiscoveryNode localNode() {
return nodes.get(localNodeId);
} }
/** /**
@ -257,16 +186,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
* @return local node * @return local node
*/ */
public DiscoveryNode getLocalNode() { public DiscoveryNode getLocalNode() {
return localNode(); return nodes.get(localNodeId);
}
/**
* Get the master node
*
* @return master node
*/
public DiscoveryNode masterNode() {
return nodes.get(masterNodeId);
} }
/** /**
@ -275,7 +195,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
* @return master node * @return master node
*/ */
public DiscoveryNode getMasterNode() { public DiscoveryNode getMasterNode() {
return masterNode(); return nodes.get(masterNodeId);
} }
/** /**
@ -287,7 +207,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
public DiscoveryNode findByAddress(TransportAddress address) { public DiscoveryNode findByAddress(TransportAddress address) {
for (ObjectCursor<DiscoveryNode> cursor : nodes.values()) { for (ObjectCursor<DiscoveryNode> cursor : nodes.values()) {
DiscoveryNode node = cursor.value; DiscoveryNode node = cursor.value;
if (node.address().equals(address)) { if (node.getAddress().equals(address)) {
return node; return node;
} }
} }
@ -304,7 +224,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
* *
* @return the oldest version in the cluster * @return the oldest version in the cluster
*/ */
public Version smallestVersion() { public Version getSmallestVersion() {
return minNodeVersion; return minNodeVersion;
} }
@ -313,7 +233,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
* *
* @return the oldest version in the cluster * @return the oldest version in the cluster
*/ */
public Version smallestNonClientNodeVersion() { public Version getSmallestNonClientNodeVersion() {
return minNonClientNodeVersion; return minNonClientNodeVersion;
} }
@ -340,19 +260,19 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
int index = 0; int index = 0;
nodesIds = new String[nodes.size()]; nodesIds = new String[nodes.size()];
for (DiscoveryNode node : this) { for (DiscoveryNode node : this) {
nodesIds[index++] = node.id(); nodesIds[index++] = node.getId();
} }
return nodesIds; return nodesIds;
} else { } else {
ObjectHashSet<String> resolvedNodesIds = new ObjectHashSet<>(nodesIds.length); ObjectHashSet<String> resolvedNodesIds = new ObjectHashSet<>(nodesIds.length);
for (String nodeId : nodesIds) { for (String nodeId : nodesIds) {
if (nodeId.equals("_local")) { if (nodeId.equals("_local")) {
String localNodeId = localNodeId(); String localNodeId = getLocalNodeId();
if (localNodeId != null) { if (localNodeId != null) {
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);
} }
@ -361,15 +281,15 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
} else { } else {
// not a node id, try and search by name // not a node id, try and search by name
for (DiscoveryNode node : this) { for (DiscoveryNode node : this) {
if (Regex.simpleMatch(nodeId, node.name())) { if (Regex.simpleMatch(nodeId, node.getName())) {
resolvedNodesIds.add(node.id()); resolvedNodesIds.add(node.getId());
} }
} }
for (DiscoveryNode node : this) { for (DiscoveryNode node : this) {
if (Regex.simpleMatch(nodeId, node.getHostAddress())) { if (Regex.simpleMatch(nodeId, node.getHostAddress())) {
resolvedNodesIds.add(node.id()); resolvedNodesIds.add(node.getId());
} else if (Regex.simpleMatch(nodeId, node.getHostName())) { } else if (Regex.simpleMatch(nodeId, node.getHostName())) {
resolvedNodesIds.add(node.id()); resolvedNodesIds.add(node.getId());
} }
} }
int index = nodeId.indexOf(':'); int index = nodeId.indexOf(':');
@ -400,7 +320,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
String attrName = entry.getKey(); String attrName = entry.getKey();
String attrValue = entry.getValue(); String attrValue = entry.getValue();
if (Regex.simpleMatch(matchAttrName, attrName) && Regex.simpleMatch(matchAttrValue, attrValue)) { if (Regex.simpleMatch(matchAttrName, attrName) && Regex.simpleMatch(matchAttrValue, attrValue)) {
resolvedNodesIds.add(node.id()); resolvedNodesIds.add(node.getId());
} }
} }
} }
@ -415,7 +335,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
public DiscoveryNodes removeDeadMembers(Set<String> newNodes, String masterNodeId) { public DiscoveryNodes removeDeadMembers(Set<String> newNodes, String masterNodeId) {
Builder builder = new Builder().masterNodeId(masterNodeId).localNodeId(localNodeId); Builder builder = new Builder().masterNodeId(masterNodeId).localNodeId(localNodeId);
for (DiscoveryNode node : this) { for (DiscoveryNode node : this) {
if (newNodes.contains(node.id())) { if (newNodes.contains(node.getId())) {
builder.put(node); builder.put(node);
} }
} }
@ -433,12 +353,12 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
List<DiscoveryNode> removed = new ArrayList<>(); List<DiscoveryNode> removed = new ArrayList<>();
List<DiscoveryNode> added = new ArrayList<>(); List<DiscoveryNode> added = new ArrayList<>();
for (DiscoveryNode node : other) { for (DiscoveryNode node : other) {
if (!this.nodeExists(node.id())) { if (!this.nodeExists(node.getId())) {
removed.add(node); removed.add(node);
} }
} }
for (DiscoveryNode node : this) { for (DiscoveryNode node : this) {
if (!other.nodeExists(node.id())) { if (!other.nodeExists(node.getId())) {
added.add(node); added.add(node);
} }
} }
@ -446,8 +366,8 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
DiscoveryNode newMasterNode = null; DiscoveryNode newMasterNode = null;
if (masterNodeId != null) { if (masterNodeId != null) {
if (other.masterNodeId == null || !other.masterNodeId.equals(masterNodeId)) { if (other.masterNodeId == null || !other.masterNodeId.equals(masterNodeId)) {
previousMasterNode = other.masterNode(); previousMasterNode = other.getMasterNode();
newMasterNode = masterNode(); newMasterNode = getMasterNode();
} }
} }
return new Delta(previousMasterNode, newMasterNode, localNodeId, Collections.unmodifiableList(removed), Collections.unmodifiableList(added)); return new Delta(previousMasterNode, newMasterNode, localNodeId, Collections.unmodifiableList(removed), Collections.unmodifiableList(added));
@ -469,10 +389,10 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
sb.append("nodes: \n"); sb.append("nodes: \n");
for (DiscoveryNode node : this) { for (DiscoveryNode node : this) {
sb.append(" ").append(node); sb.append(" ").append(node);
if (node == localNode()) { if (node == getLocalNode()) {
sb.append(", local"); sb.append(", local");
} }
if (node == masterNode()) { if (node == getMasterNode()) {
sb.append(", master"); sb.append(", master");
} }
sb.append("\n"); sb.append("\n");
@ -480,10 +400,6 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
return sb.toString(); return sb.toString();
} }
public Delta emptyDelta() {
return new Delta(null, null, localNodeId, DiscoveryNode.EMPTY_LIST, DiscoveryNode.EMPTY_LIST);
}
public static class Delta { public static class Delta {
private final String localNodeId; private final String localNodeId;
@ -539,7 +455,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
public String shortSummary() { public String shortSummary() {
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
if (!removed() && masterNodeChanged()) { if (!removed() && masterNodeChanged()) {
if (newMasterNode.id().equals(localNodeId)) { if (newMasterNode.getId().equals(localNodeId)) {
// we are the master, no nodes we removed, we are actually the first master // we are the master, no nodes we removed, we are actually the first master
sb.append("new_master ").append(newMasterNode()); sb.append("new_master ").append(newMasterNode());
} else { } else {
@ -567,13 +483,13 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
} }
if (added()) { if (added()) {
// don't print if there is one added, and it is us // don't print if there is one added, and it is us
if (!(addedNodes().size() == 1 && addedNodes().get(0).id().equals(localNodeId))) { if (!(addedNodes().size() == 1 && addedNodes().get(0).getId().equals(localNodeId))) {
if (removed() || masterNodeChanged()) { if (removed() || masterNodeChanged()) {
sb.append(", "); sb.append(", ");
} }
sb.append("added {"); sb.append("added {");
for (DiscoveryNode node : addedNodes()) { for (DiscoveryNode node : addedNodes()) {
if (!node.id().equals(localNodeId)) { if (!node.getId().equals(localNodeId)) {
// don't print ourself // don't print ourself
sb.append(node).append(','); sb.append(node).append(',');
} }
@ -605,12 +521,12 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
builder.masterNodeId(in.readString()); builder.masterNodeId(in.readString());
} }
if (localNode != null) { if (localNode != null) {
builder.localNodeId(localNode.id()); builder.localNodeId(localNode.getId());
} }
int size = in.readVInt(); int size = in.readVInt();
for (int i = 0; i < size; i++) { for (int i = 0; i < size; i++) {
DiscoveryNode node = new DiscoveryNode(in); DiscoveryNode node = new DiscoveryNode(in);
if (localNode != null && node.id().equals(localNode.id())) { if (localNode != null && node.getId().equals(localNode.getId())) {
// reuse the same instance of our address and local node id for faster equality // reuse the same instance of our address and local node id for faster equality
node = localNode; node = localNode;
} }
@ -621,7 +537,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
@Override @Override
public DiscoveryNodes readFrom(StreamInput in) throws IOException { public DiscoveryNodes readFrom(StreamInput in) throws IOException {
return readFrom(in, localNode()); return readFrom(in, getLocalNode());
} }
public static Builder builder() { public static Builder builder() {
@ -643,13 +559,13 @@ 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.getLocalNodeId();
this.nodes = ImmutableOpenMap.builder(nodes.nodes()); this.nodes = ImmutableOpenMap.builder(nodes.getNodes());
} }
public Builder put(DiscoveryNode node) { public Builder put(DiscoveryNode node) {
nodes.put(node.id(), node); nodes.put(node.getId(), node);
return this; return this;
} }
@ -675,18 +591,18 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
Version minNodeVersion = Version.CURRENT; Version minNodeVersion = Version.CURRENT;
Version minNonClientNodeVersion = Version.CURRENT; Version minNonClientNodeVersion = Version.CURRENT;
for (ObjectObjectCursor<String, DiscoveryNode> nodeEntry : nodes) { for (ObjectObjectCursor<String, DiscoveryNode> nodeEntry : nodes) {
if (nodeEntry.value.dataNode()) { if (nodeEntry.value.isDataNode()) {
dataNodesBuilder.put(nodeEntry.key, nodeEntry.value); dataNodesBuilder.put(nodeEntry.key, nodeEntry.value);
minNonClientNodeVersion = Version.smallest(minNonClientNodeVersion, nodeEntry.value.version()); minNonClientNodeVersion = Version.smallest(minNonClientNodeVersion, nodeEntry.value.getVersion());
} }
if (nodeEntry.value.masterNode()) { if (nodeEntry.value.isMasterNode()) {
masterNodesBuilder.put(nodeEntry.key, nodeEntry.value); masterNodesBuilder.put(nodeEntry.key, nodeEntry.value);
minNonClientNodeVersion = Version.smallest(minNonClientNodeVersion, nodeEntry.value.version()); minNonClientNodeVersion = Version.smallest(minNonClientNodeVersion, nodeEntry.value.getVersion());
} }
if (nodeEntry.value.isIngestNode()) { if (nodeEntry.value.isIngestNode()) {
ingestNodesBuilder.put(nodeEntry.key, nodeEntry.value); ingestNodesBuilder.put(nodeEntry.key, nodeEntry.value);
} }
minNodeVersion = Version.smallest(minNodeVersion, nodeEntry.value.version()); minNodeVersion = Version.smallest(minNodeVersion, nodeEntry.value.getVersion());
} }
return new DiscoveryNodes( return new DiscoveryNodes(

View File

@ -477,7 +477,7 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
private static List<ShardRouting> collectAttributeShards(AttributesKey key, DiscoveryNodes nodes, ArrayList<ShardRouting> from) { private static List<ShardRouting> collectAttributeShards(AttributesKey key, DiscoveryNodes nodes, ArrayList<ShardRouting> from) {
final ArrayList<ShardRouting> to = new ArrayList<>(); final ArrayList<ShardRouting> to = new ArrayList<>();
for (final String attribute : key.attributes) { for (final String attribute : key.attributes) {
final String localAttributeValue = nodes.localNode().getAttributes().get(attribute); final String localAttributeValue = nodes.getLocalNode().getAttributes().get(attribute);
if (localAttributeValue != null) { if (localAttributeValue != null) {
for (Iterator<ShardRouting> iterator = from.iterator(); iterator.hasNext(); ) { for (Iterator<ShardRouting> iterator = from.iterator(); iterator.hasNext(); ) {
ShardRouting fromShard = iterator.next(); ShardRouting fromShard = iterator.next();

View File

@ -58,12 +58,12 @@ public class OperationRouting extends AbstractComponent {
} }
public ShardIterator getShards(ClusterState clusterState, String index, String type, String id, @Nullable String routing, @Nullable String preference) { public ShardIterator getShards(ClusterState clusterState, String index, String type, String id, @Nullable String routing, @Nullable String preference) {
return preferenceActiveShardIterator(shards(clusterState, index, id, routing), clusterState.nodes().localNodeId(), clusterState.nodes(), preference); return preferenceActiveShardIterator(shards(clusterState, index, id, routing), clusterState.nodes().getLocalNodeId(), clusterState.nodes(), preference);
} }
public ShardIterator getShards(ClusterState clusterState, String index, int shardId, @Nullable String preference) { public ShardIterator getShards(ClusterState clusterState, String index, int shardId, @Nullable String preference) {
final IndexShardRoutingTable indexShard = clusterState.getRoutingTable().shardRoutingTable(index, shardId); final IndexShardRoutingTable indexShard = clusterState.getRoutingTable().shardRoutingTable(index, shardId);
return preferenceActiveShardIterator(indexShard, clusterState.nodes().localNodeId(), clusterState.nodes(), preference); return preferenceActiveShardIterator(indexShard, clusterState.nodes().getLocalNodeId(), clusterState.nodes(), preference);
} }
public int searchShardsCount(ClusterState clusterState, String[] concreteIndices, @Nullable Map<String, Set<String>> routing) { public int searchShardsCount(ClusterState clusterState, String[] concreteIndices, @Nullable Map<String, Set<String>> routing) {
@ -75,7 +75,7 @@ public class OperationRouting extends AbstractComponent {
final Set<IndexShardRoutingTable> shards = computeTargetedShards(clusterState, concreteIndices, routing); final Set<IndexShardRoutingTable> shards = computeTargetedShards(clusterState, concreteIndices, routing);
final Set<ShardIterator> set = new HashSet<>(shards.size()); final Set<ShardIterator> set = new HashSet<>(shards.size());
for (IndexShardRoutingTable shard : shards) { for (IndexShardRoutingTable shard : shards) {
ShardIterator iterator = preferenceActiveShardIterator(shard, clusterState.nodes().localNodeId(), clusterState.nodes(), preference); ShardIterator iterator = preferenceActiveShardIterator(shard, clusterState.nodes().getLocalNodeId(), clusterState.nodes(), preference);
if (iterator != null) { if (iterator != null) {
set.add(iterator); set.add(iterator);
} }
@ -230,7 +230,7 @@ public class OperationRouting extends AbstractComponent {
} }
private void ensureNodeIdExists(DiscoveryNodes nodes, String nodeId) { private void ensureNodeIdExists(DiscoveryNodes nodes, String nodeId) {
if (!nodes.dataNodes().keys().contains(nodeId)) { if (!nodes.getDataNodes().keys().contains(nodeId)) {
throw new IllegalArgumentException("No data node with id[" + nodeId + "] found"); throw new IllegalArgumentException("No data node with id[" + nodeId + "] found");
} }
} }

View File

@ -86,8 +86,8 @@ public class RoutingNodes implements Iterable<RoutingNode> {
Map<String, List<ShardRouting>> nodesToShards = new HashMap<>(); Map<String, List<ShardRouting>> nodesToShards = new HashMap<>();
// fill in the nodeToShards with the "live" nodes // fill in the nodeToShards with the "live" nodes
for (ObjectCursor<DiscoveryNode> cursor : clusterState.nodes().dataNodes().values()) { for (ObjectCursor<DiscoveryNode> cursor : clusterState.nodes().getDataNodes().values()) {
nodesToShards.put(cursor.value.id(), new ArrayList<>()); nodesToShards.put(cursor.value.getId(), new ArrayList<>());
} }
// fill in the inverse of node -> shards allocated // fill in the inverse of node -> shards allocated
@ -570,7 +570,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
public void addNode(DiscoveryNode node) { public void addNode(DiscoveryNode node) {
ensureMutable(); ensureMutable();
RoutingNode routingNode = new RoutingNode(node.id(), node); RoutingNode routingNode = new RoutingNode(node.getId(), node);
nodesToShards.put(routingNode.nodeId(), routingNode); nodesToShards.put(routingNode.nodeId(), routingNode);
} }

View File

@ -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.

View File

@ -406,7 +406,7 @@ public class AllocationService extends AbstractComponent {
*/ */
private void applyNewNodes(RoutingAllocation allocation) { private void applyNewNodes(RoutingAllocation allocation) {
final RoutingNodes routingNodes = allocation.routingNodes(); final RoutingNodes routingNodes = allocation.routingNodes();
for (ObjectCursor<DiscoveryNode> cursor : allocation.nodes().dataNodes().values()) { for (ObjectCursor<DiscoveryNode> cursor : allocation.nodes().getDataNodes().values()) {
DiscoveryNode node = cursor.value; DiscoveryNode node = cursor.value;
if (!routingNodes.isKnown(node)) { if (!routingNodes.isKnown(node)) {
routingNodes.addNode(node); routingNodes.addNode(node);
@ -418,7 +418,7 @@ public class AllocationService extends AbstractComponent {
boolean changed = false; boolean changed = false;
for (RoutingNodes.RoutingNodesIterator it = allocation.routingNodes().nodes(); it.hasNext(); ) { for (RoutingNodes.RoutingNodesIterator it = allocation.routingNodes().nodes(); it.hasNext(); ) {
RoutingNode node = it.next(); RoutingNode node = it.next();
if (allocation.nodes().dataNodes().containsKey(node.nodeId())) { if (allocation.nodes().getDataNodes().containsKey(node.nodeId())) {
// its a live node, continue // its a live node, continue
continue; continue;
} }

View File

@ -184,7 +184,7 @@ public abstract class AbstractAllocateAllocationCommand implements AllocationCom
* Handle case where a disco node cannot be found in the routing table. Usually means that it's not a data node. * Handle case where a disco node cannot be found in the routing table. Usually means that it's not a data node.
*/ */
protected RerouteExplanation explainOrThrowMissingRoutingNode(RoutingAllocation allocation, boolean explain, DiscoveryNode discoNode) { protected RerouteExplanation explainOrThrowMissingRoutingNode(RoutingAllocation allocation, boolean explain, DiscoveryNode discoNode) {
if (!discoNode.dataNode()) { if (!discoNode.isDataNode()) {
return explainOrThrowRejectedCommand(explain, allocation, "allocation can only be done on data nodes, not [" + node + "]"); return explainOrThrowRejectedCommand(explain, allocation, "allocation can only be done on data nodes, not [" + node + "]");
} else { } else {
return explainOrThrowRejectedCommand(explain, allocation, "could not find [" + node + "] among the routing nodes"); return explainOrThrowRejectedCommand(explain, allocation, "could not find [" + node + "] among the routing nodes");

View File

@ -91,7 +91,7 @@ public class AllocateEmptyPrimaryAllocationCommand extends BasePrimaryAllocation
return explainOrThrowRejectedCommand(explain, allocation, e); return explainOrThrowRejectedCommand(explain, allocation, e);
} }
final RoutingNodes routingNodes = allocation.routingNodes(); final RoutingNodes routingNodes = allocation.routingNodes();
RoutingNode routingNode = routingNodes.node(discoNode.id()); RoutingNode routingNode = routingNodes.node(discoNode.getId());
if (routingNode == null) { if (routingNode == null) {
return explainOrThrowMissingRoutingNode(allocation, explain, discoNode); return explainOrThrowMissingRoutingNode(allocation, explain, discoNode);
} }

View File

@ -30,7 +30,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException; import org.elasticsearch.index.shard.ShardNotFoundException;
import java.io.IOException; import java.io.IOException;
@ -90,7 +89,7 @@ public class AllocateReplicaAllocationCommand extends AbstractAllocateAllocation
return explainOrThrowRejectedCommand(explain, allocation, e); return explainOrThrowRejectedCommand(explain, allocation, e);
} }
final RoutingNodes routingNodes = allocation.routingNodes(); final RoutingNodes routingNodes = allocation.routingNodes();
RoutingNode routingNode = routingNodes.node(discoNode.id()); RoutingNode routingNode = routingNodes.node(discoNode.getId());
if (routingNode == null) { if (routingNode == null) {
return explainOrThrowMissingRoutingNode(allocation, explain, discoNode); return explainOrThrowMissingRoutingNode(allocation, explain, discoNode);
} }

View File

@ -30,7 +30,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException; import org.elasticsearch.index.shard.ShardNotFoundException;
import java.io.IOException; import java.io.IOException;
@ -92,7 +91,7 @@ public class AllocateStalePrimaryAllocationCommand extends BasePrimaryAllocation
return explainOrThrowRejectedCommand(explain, allocation, e); return explainOrThrowRejectedCommand(explain, allocation, e);
} }
final RoutingNodes routingNodes = allocation.routingNodes(); final RoutingNodes routingNodes = allocation.routingNodes();
RoutingNode routingNode = routingNodes.node(discoNode.id()); RoutingNode routingNode = routingNodes.node(discoNode.getId());
if (routingNode == null) { if (routingNode == null) {
return explainOrThrowMissingRoutingNode(allocation, explain, discoNode); return explainOrThrowMissingRoutingNode(allocation, explain, discoNode);
} }

View File

@ -175,7 +175,7 @@ public class CancelAllocationCommand implements AllocationCommand {
public RerouteExplanation execute(RoutingAllocation allocation, boolean explain) { public RerouteExplanation execute(RoutingAllocation allocation, boolean explain) {
DiscoveryNode discoNode = allocation.nodes().resolveNode(node); DiscoveryNode discoNode = allocation.nodes().resolveNode(node);
boolean found = false; boolean found = false;
for (RoutingNodes.RoutingNodeIterator it = allocation.routingNodes().routingNodeIter(discoNode.id()); it.hasNext(); ) { for (RoutingNodes.RoutingNodeIterator it = allocation.routingNodes().routingNodeIter(discoNode.getId()); it.hasNext(); ) {
ShardRouting shardRouting = it.next(); ShardRouting shardRouting = it.next();
if (!shardRouting.shardId().getIndex().getName().equals(index)) { if (!shardRouting.shardId().getIndex().getName().equals(index)) {
continue; continue;

View File

@ -155,7 +155,7 @@ public class MoveAllocationCommand implements AllocationCommand {
Decision decision = null; Decision decision = null;
boolean found = false; boolean found = false;
for (ShardRouting shardRouting : allocation.routingNodes().node(fromDiscoNode.id())) { for (ShardRouting shardRouting : allocation.routingNodes().node(fromDiscoNode.getId())) {
if (!shardRouting.shardId().getIndexName().equals(index)) { if (!shardRouting.shardId().getIndexName().equals(index)) {
continue; continue;
} }
@ -174,7 +174,7 @@ public class MoveAllocationCommand implements AllocationCommand {
", shard is not started (state = " + shardRouting.state() + "]"); ", shard is not started (state = " + shardRouting.state() + "]");
} }
RoutingNode toRoutingNode = allocation.routingNodes().node(toDiscoNode.id()); RoutingNode toRoutingNode = allocation.routingNodes().node(toDiscoNode.getId());
decision = allocation.deciders().canAllocate(shardRouting, toRoutingNode, allocation); decision = allocation.deciders().canAllocate(shardRouting, toRoutingNode, allocation);
if (decision.type() == Decision.Type.NO) { if (decision.type() == Decision.Type.NO) {
if (explain) { if (explain) {

View File

@ -519,7 +519,7 @@ public class DiskThresholdDecider extends AllocationDecider {
if (includeRelocations) { if (includeRelocations) {
long relocatingShardsSize = sizeOfRelocatingShards(node, clusterInfo, true, usage.getPath()); long relocatingShardsSize = sizeOfRelocatingShards(node, clusterInfo, true, usage.getPath());
DiskUsage usageIncludingRelocations = new DiskUsage(node.nodeId(), node.node().name(), usage.getPath(), DiskUsage usageIncludingRelocations = new DiskUsage(node.nodeId(), node.node().getName(), usage.getPath(),
usage.getTotalBytes(), usage.getFreeBytes() - relocatingShardsSize); usage.getTotalBytes(), usage.getFreeBytes() - relocatingShardsSize);
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("usage without relocations: {}", usage); logger.trace("usage without relocations: {}", usage);
@ -539,7 +539,7 @@ public class DiskThresholdDecider extends AllocationDecider {
*/ */
public DiskUsage averageUsage(RoutingNode node, ImmutableOpenMap<String, DiskUsage> usages) { public DiskUsage averageUsage(RoutingNode node, ImmutableOpenMap<String, DiskUsage> usages) {
if (usages.size() == 0) { if (usages.size() == 0) {
return new DiskUsage(node.nodeId(), node.node().name(), "_na_", 0, 0); return new DiskUsage(node.nodeId(), node.node().getName(), "_na_", 0, 0);
} }
long totalBytes = 0; long totalBytes = 0;
long freeBytes = 0; long freeBytes = 0;
@ -547,7 +547,7 @@ public class DiskThresholdDecider extends AllocationDecider {
totalBytes += du.value.getTotalBytes(); totalBytes += du.value.getTotalBytes();
freeBytes += du.value.getFreeBytes(); freeBytes += du.value.getFreeBytes();
} }
return new DiskUsage(node.nodeId(), node.node().name(), "_na_", totalBytes / usages.size(), freeBytes / usages.size()); return new DiskUsage(node.nodeId(), node.node().getName(), "_na_", totalBytes / usages.size(), freeBytes / usages.size());
} }
/** /**
@ -618,7 +618,7 @@ public class DiskThresholdDecider extends AllocationDecider {
} }
// Allow allocation regardless if only a single data node is available // Allow allocation regardless if only a single data node is available
if (allocation.nodes().dataNodes().size() <= 1) { if (allocation.nodes().getDataNodes().size() <= 1) {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("only a single data node is present, allowing allocation"); logger.trace("only a single data node is present, allowing allocation");
} }

View File

@ -73,26 +73,26 @@ public class NodeVersionAllocationDecider extends AllocationDecider {
private Decision isVersionCompatible(final RoutingNodes routingNodes, final String sourceNodeId, final RoutingNode target, private Decision isVersionCompatible(final RoutingNodes routingNodes, final String sourceNodeId, final RoutingNode target,
RoutingAllocation allocation) { RoutingAllocation allocation) {
final RoutingNode source = routingNodes.node(sourceNodeId); final RoutingNode source = routingNodes.node(sourceNodeId);
if (target.node().version().onOrAfter(source.node().version())) { if (target.node().getVersion().onOrAfter(source.node().getVersion())) {
/* we can allocate if we can recover from a node that is younger or on the same version /* we can allocate if we can recover from a node that is younger or on the same version
* if the primary is already running on a newer version that won't work due to possible * if the primary is already running on a newer version that won't work due to possible
* differences in the lucene index format etc.*/ * differences in the lucene index format etc.*/
return allocation.decision(Decision.YES, NAME, "target node version [%s] is the same or newer than source node version [%s]", return allocation.decision(Decision.YES, NAME, "target node version [%s] is the same or newer than source node version [%s]",
target.node().version(), source.node().version()); target.node().getVersion(), source.node().getVersion());
} else { } else {
return allocation.decision(Decision.NO, NAME, "target node version [%s] is older than the source node version [%s]", return allocation.decision(Decision.NO, NAME, "target node version [%s] is older than the source node version [%s]",
target.node().version(), source.node().version()); target.node().getVersion(), source.node().getVersion());
} }
} }
private Decision isVersionCompatible(RestoreSource restoreSource, final RoutingNode target, RoutingAllocation allocation) { private Decision isVersionCompatible(RestoreSource restoreSource, final RoutingNode target, RoutingAllocation allocation) {
if (target.node().version().onOrAfter(restoreSource.version())) { if (target.node().getVersion().onOrAfter(restoreSource.version())) {
/* we can allocate if we can restore from a snapshot that is older or on the same version */ /* we can allocate if we can restore from a snapshot that is older or on the same version */
return allocation.decision(Decision.YES, NAME, "target node version [%s] is the same or newer than snapshot version [%s]", return allocation.decision(Decision.YES, NAME, "target node version [%s] is the same or newer than snapshot version [%s]",
target.node().version(), restoreSource.version()); target.node().getVersion(), restoreSource.version());
} else { } else {
return allocation.decision(Decision.NO, NAME, "target node version [%s] is older than the snapshot version [%s]", return allocation.decision(Decision.NO, NAME, "target node version [%s] is older than the snapshot version [%s]",
target.node().version(), restoreSource.version()); target.node().getVersion(), restoreSource.version());
} }
} }
} }

View File

@ -158,8 +158,8 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
} }
synchronized public void setLocalNode(DiscoveryNode localNode) { synchronized public void setLocalNode(DiscoveryNode localNode) {
assert clusterState.nodes().localNodeId() == null : "local node is already set"; assert clusterState.nodes().getLocalNodeId() == null : "local node is already set";
DiscoveryNodes.Builder nodeBuilder = DiscoveryNodes.builder(clusterState.nodes()).put(localNode).localNodeId(localNode.id()); DiscoveryNodes.Builder nodeBuilder = DiscoveryNodes.builder(clusterState.nodes()).put(localNode).localNodeId(localNode.getId());
this.clusterState = ClusterState.builder(clusterState).nodes(nodeBuilder).build(); this.clusterState = ClusterState.builder(clusterState).nodes(nodeBuilder).build();
} }
@ -198,7 +198,7 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
@Override @Override
synchronized protected void doStart() { synchronized protected void doStart() {
Objects.requireNonNull(clusterStatePublisher, "please set a cluster state publisher before starting"); Objects.requireNonNull(clusterStatePublisher, "please set a cluster state publisher before starting");
Objects.requireNonNull(clusterState.nodes().localNode(), "please set the local node before starting"); Objects.requireNonNull(clusterState.nodes().getLocalNode(), "please set the local node before starting");
Objects.requireNonNull(nodeConnectionsService, "please set the node connection service before starting"); Objects.requireNonNull(nodeConnectionsService, "please set the node connection service before starting");
add(localNodeMasterListeners); add(localNodeMasterListeners);
this.clusterState = ClusterState.builder(clusterState).blocks(initialBlocks).build(); this.clusterState = ClusterState.builder(clusterState).blocks(initialBlocks).build();
@ -229,7 +229,7 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
* The local node. * The local node.
*/ */
public DiscoveryNode localNode() { public DiscoveryNode localNode() {
return clusterState.getNodes().localNode(); return clusterState.getNodes().getLocalNode();
} }
public OperationRouting operationRouting() { public OperationRouting operationRouting() {
@ -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,11 +661,11 @@ 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().localNode(), null); ackListener.onNodeAck(newClusterState.nodes().getLocalNode(), null);
} catch (Throwable t) { } catch (Throwable t) {
logger.debug("error while processing ack for master node [{}]", t, newClusterState.nodes().localNode()); logger.debug("error while processing ack for master node [{}]", t, newClusterState.nodes().getLocalNode());
} }
} }
@ -973,7 +973,7 @@ public class ClusterService extends AbstractLifecycleComponent<ClusterService> {
public void onNodeAck(DiscoveryNode node, @Nullable Throwable t) { public void onNodeAck(DiscoveryNode node, @Nullable Throwable t) {
if (!ackedTaskListener.mustAck(node)) { if (!ackedTaskListener.mustAck(node)) {
//we always wait for the master ack anyway //we always wait for the master ack anyway
if (!node.equals(nodes.masterNode())) { if (!node.equals(nodes.getMasterNode())) {
return; return;
} }
} }

View File

@ -387,7 +387,7 @@ public class NetworkModule extends AbstractModule {
bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry);
transportServiceTypes.bindType(binder(), settings, TRANSPORT_SERVICE_TYPE_KEY, NETTY_TRANSPORT); transportServiceTypes.bindType(binder(), settings, TRANSPORT_SERVICE_TYPE_KEY, NETTY_TRANSPORT);
String defaultTransport = DiscoveryNode.localNode(settings) ? LOCAL_TRANSPORT : NETTY_TRANSPORT; String defaultTransport = DiscoveryNode.isLocalNode(settings) ? LOCAL_TRANSPORT : NETTY_TRANSPORT;
transportTypes.bindType(binder(), settings, TRANSPORT_TYPE_KEY, defaultTransport); transportTypes.bindType(binder(), settings, TRANSPORT_TYPE_KEY, defaultTransport);
if (transportClient) { if (transportClient) {

View File

@ -47,7 +47,7 @@ import java.util.function.Function;
public class DiscoveryModule extends AbstractModule { public class DiscoveryModule extends AbstractModule {
public static final Setting<String> DISCOVERY_TYPE_SETTING = public static final Setting<String> DISCOVERY_TYPE_SETTING =
new Setting<>("discovery.type", settings -> DiscoveryNode.localNode(settings) ? "local" : "zen", Function.identity(), new Setting<>("discovery.type", settings -> DiscoveryNode.isLocalNode(settings) ? "local" : "zen", Function.identity(),
Property.NodeScope); Property.NodeScope);
public static final Setting<String> ZEN_MASTER_SERVICE_TYPE_SETTING = public static final Setting<String> ZEN_MASTER_SERVICE_TYPE_SETTING =
new Setting<>("discovery.zen.masterservice.type", "zen", Function.identity(), Property.NodeScope); new Setting<>("discovery.zen.masterservice.type", "zen", Function.identity(), Property.NodeScope);

View File

@ -106,7 +106,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
LocalDiscovery firstMaster = null; LocalDiscovery firstMaster = null;
for (LocalDiscovery localDiscovery : clusterGroup.members()) { for (LocalDiscovery localDiscovery : clusterGroup.members()) {
if (localDiscovery.localNode().masterNode()) { if (localDiscovery.localNode().isMasterNode()) {
firstMaster = localDiscovery; firstMaster = localDiscovery;
break; break;
} }
@ -129,7 +129,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
for (LocalDiscovery discovery : clusterGroups.get(clusterName).members()) { for (LocalDiscovery discovery : clusterGroups.get(clusterName).members()) {
nodesBuilder.put(discovery.localNode()); nodesBuilder.put(discovery.localNode());
} }
nodesBuilder.localNodeId(master.localNode().id()).masterNodeId(master.localNode().id()); nodesBuilder.localNodeId(master.localNode().getId()).masterNodeId(master.localNode().getId());
// remove the NO_MASTER block in this case // remove the NO_MASTER block in this case
ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()).removeGlobalBlock(discoverySettings.getNoMasterBlock()); ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()).removeGlobalBlock(discoverySettings.getNoMasterBlock());
return ClusterState.builder(currentState).nodes(nodesBuilder).blocks(blocks).build(); return ClusterState.builder(currentState).nodes(nodesBuilder).blocks(blocks).build();
@ -155,7 +155,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
for (LocalDiscovery discovery : clusterGroups.get(clusterName).members()) { for (LocalDiscovery discovery : clusterGroups.get(clusterName).members()) {
nodesBuilder.put(discovery.localNode()); nodesBuilder.put(discovery.localNode());
} }
nodesBuilder.localNodeId(master.localNode().id()).masterNodeId(master.localNode().id()); nodesBuilder.localNodeId(master.localNode().getId()).masterNodeId(master.localNode().getId());
return ClusterState.builder(currentState).nodes(nodesBuilder).build(); return ClusterState.builder(currentState).nodes(nodesBuilder).build();
} }
@ -193,7 +193,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
LocalDiscovery firstMaster = null; LocalDiscovery firstMaster = null;
for (LocalDiscovery localDiscovery : clusterGroup.members()) { for (LocalDiscovery localDiscovery : clusterGroup.members()) {
if (localDiscovery.localNode().masterNode()) { if (localDiscovery.localNode().isMasterNode()) {
firstMaster = localDiscovery; firstMaster = localDiscovery;
break; break;
} }
@ -207,7 +207,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
final Set<String> newMembers = new HashSet<>(); final Set<String> newMembers = new HashSet<>();
for (LocalDiscovery discovery : clusterGroup.members()) { for (LocalDiscovery discovery : clusterGroup.members()) {
newMembers.add(discovery.localNode().id()); newMembers.add(discovery.localNode().getId());
} }
final LocalDiscovery master = firstMaster; final LocalDiscovery master = firstMaster;
@ -219,7 +219,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
@Override @Override
public ClusterState execute(ClusterState currentState) { public ClusterState execute(ClusterState currentState) {
DiscoveryNodes newNodes = currentState.nodes().removeDeadMembers(newMembers, master.localNode().id()); DiscoveryNodes newNodes = currentState.nodes().removeDeadMembers(newMembers, master.localNode().getId());
DiscoveryNodes.Delta delta = newNodes.delta(currentState.nodes()); DiscoveryNodes.Delta delta = newNodes.delta(currentState.nodes());
if (delta.added()) { if (delta.added()) {
logger.warn("No new nodes should be created when a new discovery view is accepted"); logger.warn("No new nodes should be created when a new discovery view is accepted");
@ -251,7 +251,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
@Override @Override
public String nodeDescription() { public String nodeDescription() {
return clusterName.value() + "/" + localNode().id(); return clusterName.value() + "/" + localNode().getId();
} }
@Override @Override
@ -312,7 +312,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
synchronized (this) { synchronized (this) {
// we do the marshaling intentionally, to check it works well... // we do the marshaling intentionally, to check it works well...
// check if we published cluster state at least once and node was in the cluster when we published cluster state the last time // check if we published cluster state at least once and node was in the cluster when we published cluster state the last time
if (discovery.lastProcessedClusterState != null && clusterChangedEvent.previousState().nodes().nodeExists(discovery.localNode().id())) { if (discovery.lastProcessedClusterState != null && clusterChangedEvent.previousState().nodes().nodeExists(discovery.localNode().getId())) {
// both conditions are true - which means we can try sending cluster state as diffs // both conditions are true - which means we can try sending cluster state as diffs
if (clusterStateDiffBytes == null) { if (clusterStateDiffBytes == null) {
Diff diff = clusterState.diff(clusterChangedEvent.previousState()); Diff diff = clusterState.diff(clusterChangedEvent.previousState());
@ -339,8 +339,8 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
nodeSpecificClusterState.status(ClusterState.ClusterStateStatus.RECEIVED); nodeSpecificClusterState.status(ClusterState.ClusterStateStatus.RECEIVED);
// ignore cluster state messages that do not include "me", not in the game yet... // ignore cluster state messages that do not include "me", not in the game yet...
if (nodeSpecificClusterState.nodes().localNode() != null) { if (nodeSpecificClusterState.nodes().getLocalNode() != null) {
assert nodeSpecificClusterState.nodes().masterNode() != null : "received a cluster state without a master"; assert nodeSpecificClusterState.nodes().getMasterNode() != null : "received a cluster state without a master";
assert !nodeSpecificClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) : "received a cluster state with a master block"; assert !nodeSpecificClusterState.blocks().hasGlobalBlock(discoverySettings.getNoMasterBlock()) : "received a cluster state with a master block";
discovery.clusterService.submitStateUpdateTask("local-disco-receive(from master)", new ClusterStateUpdateTask() { discovery.clusterService.submitStateUpdateTask("local-disco-receive(from master)", new ClusterStateUpdateTask() {
@ -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

@ -240,13 +240,13 @@ public class NodeJoinController extends AbstractComponent {
// Take into account the previous known nodes, if they happen not to be available // Take into account the previous known nodes, if they happen not to be available
// then fault detection will remove these nodes. // then fault detection will remove these nodes.
if (currentState.nodes().masterNode() != null) { if (currentState.nodes().getMasterNode() != null) {
// TODO can we tie break here? we don't have a remote master cluster state version to decide on // TODO can we tie break here? we don't have a remote master cluster state version to decide on
logger.trace("join thread elected local node as master, but there is already a master in place: {}", currentState.nodes().masterNode()); logger.trace("join thread elected local node as master, but there is already a master in place: {}", currentState.nodes().getMasterNode());
throw new NotMasterException("Node [" + clusterService.localNode() + "] not master for join request"); throw new NotMasterException("Node [" + clusterService.localNode() + "] not master for join request");
} }
DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder(currentState.nodes()).masterNodeId(currentState.nodes().localNode().id()); DiscoveryNodes.Builder builder = new DiscoveryNodes.Builder(currentState.nodes()).masterNodeId(currentState.nodes().getLocalNode().getId());
// update the fact that we are the master... // update the fact that we are the master...
ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(currentState.blocks()).removeGlobalBlock(discoverySettings.getNoMasterBlock()).build(); ClusterBlocks clusterBlocks = ClusterBlocks.builder().blocks(currentState.blocks()).removeGlobalBlock(discoverySettings.getNoMasterBlock()).build();
currentState = ClusterState.builder(currentState).nodes(builder).blocks(clusterBlocks).build(); currentState = ClusterState.builder(currentState).nodes(builder).blocks(clusterBlocks).build();
@ -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();
@ -378,14 +378,14 @@ public class NodeJoinController extends AbstractComponent {
final DiscoveryNode node = entry.getKey(); final DiscoveryNode node = entry.getKey();
joinCallbacksToRespondTo.addAll(entry.getValue()); joinCallbacksToRespondTo.addAll(entry.getValue());
iterator.remove(); iterator.remove();
if (currentState.nodes().nodeExists(node.id())) { if (currentState.nodes().nodeExists(node.getId())) {
logger.debug("received a join request for an existing node [{}]", node); logger.debug("received a join request for an existing node [{}]", node);
} else { } else {
nodeAdded = true; nodeAdded = true;
nodesBuilder.put(node); nodesBuilder.put(node);
for (DiscoveryNode existingNode : currentState.nodes()) { for (DiscoveryNode existingNode : currentState.nodes()) {
if (node.address().equals(existingNode.address())) { if (node.getAddress().equals(existingNode.getAddress())) {
nodesBuilder.remove(existingNode.id()); nodesBuilder.remove(existingNode.getId());
logger.warn("received join request from node [{}], but found existing node {} with same address, removing existing node", node, existingNode); logger.warn("received join request from node [{}], but found existing node {} with same address, removing existing node", node, existingNode);
} }
} }

View File

@ -176,7 +176,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
clusterSettings.addSettingsUpdateConsumer(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, this::handleMinimumMasterNodesChanged, (value) -> { clusterSettings.addSettingsUpdateConsumer(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, this::handleMinimumMasterNodesChanged, (value) -> {
final ClusterState clusterState = clusterService.state(); final ClusterState clusterState = clusterService.state();
int masterNodes = clusterState.nodes().masterNodes().size(); int masterNodes = clusterState.nodes().getMasterNodes().size();
if (value > masterNodes) { if (value > masterNodes) {
throw new IllegalArgumentException("cannot set " + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + " to more than the current master nodes count [" + masterNodes + "]"); throw new IllegalArgumentException("cannot set " + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + " to more than the current master nodes count [" + masterNodes + "]");
} }
@ -242,25 +242,25 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
nodesFD.stop(); nodesFD.stop();
DiscoveryNodes nodes = nodes(); DiscoveryNodes nodes = nodes();
if (sendLeaveRequest) { if (sendLeaveRequest) {
if (nodes.masterNode() == 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.masterNode(), nodes.localNode(), TimeValue.timeValueSeconds(1)); membership.sendLeaveRequestBlocking(nodes.getMasterNode(), nodes.getLocalNode(), TimeValue.timeValueSeconds(1));
} catch (Exception e) { } catch (Exception e) {
logger.debug("failed to send leave request to master [{}]", e, nodes.masterNode()); logger.debug("failed to send leave request to master [{}]", e, nodes.getMasterNode());
} }
} else { } else {
// we're master -> let other potential master we left and start a master election now rather then wait for masterFD // we're master -> let other potential master we left and start a master election now rather then wait for masterFD
DiscoveryNode[] possibleMasters = electMaster.nextPossibleMasters(nodes.nodes().values(), 5); DiscoveryNode[] possibleMasters = electMaster.nextPossibleMasters(nodes.getNodes().values(), 5);
for (DiscoveryNode possibleMaster : possibleMasters) { for (DiscoveryNode possibleMaster : possibleMasters) {
if (nodes.localNode().equals(possibleMaster)) { if (nodes.getLocalNode().equals(possibleMaster)) {
continue; continue;
} }
try { try {
membership.sendLeaveRequest(nodes.localNode(), possibleMaster); membership.sendLeaveRequest(nodes.getLocalNode(), possibleMaster);
} catch (Exception e) { } catch (Exception e) {
logger.debug("failed to send leave request from master [{}] to possible master [{}]", e, nodes.masterNode(), possibleMaster); logger.debug("failed to send leave request from master [{}] to possible master [{}]", e, nodes.getMasterNode(), possibleMaster);
} }
} }
} }
@ -283,7 +283,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
@Override @Override
public String nodeDescription() { public String nodeDescription() {
return clusterName.value() + "/" + clusterService.localNode().id(); return clusterName.value() + "/" + clusterService.localNode().getId();
} }
/** start of {@link org.elasticsearch.discovery.zen.ping.PingContextProvider } implementation */ /** start of {@link org.elasticsearch.discovery.zen.ping.PingContextProvider } implementation */
@ -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());
@ -419,7 +419,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
return currentState; return currentState;
} }
if (currentState.getNodes().masterNode() == null) { if (currentState.getNodes().getMasterNode() == null) {
// Post 1.3.0, the master should publish a new cluster state before acking our join request. we now should have // Post 1.3.0, the master should publish a new cluster state before acking our join request. we now should have
// a valid master. // a valid master.
logger.debug("no master node is set, despite of join request completing. retrying pings."); logger.debug("no master node is set, despite of join request completing. retrying pings.");
@ -427,7 +427,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
return currentState; return currentState;
} }
if (!currentState.getNodes().masterNode().equals(finalMasterNode)) { if (!currentState.getNodes().getMasterNode().equals(finalMasterNode)) {
return joinThreadControl.stopRunningThreadAndRejoin(currentState, "master_switched_while_finalizing_join"); return joinThreadControl.stopRunningThreadAndRejoin(currentState, "master_switched_while_finalizing_join");
} }
@ -501,7 +501,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
clusterService.submitStateUpdateTask("zen-disco-node_left(" + node + ")", new ClusterStateUpdateTask(Priority.IMMEDIATE) { clusterService.submitStateUpdateTask("zen-disco-node_left(" + node + ")", new ClusterStateUpdateTask(Priority.IMMEDIATE) {
@Override @Override
public ClusterState execute(ClusterState currentState) { public ClusterState execute(ClusterState currentState) {
DiscoveryNodes.Builder builder = DiscoveryNodes.builder(currentState.nodes()).remove(node.id()); DiscoveryNodes.Builder builder = DiscoveryNodes.builder(currentState.nodes()).remove(node.getId());
currentState = ClusterState.builder(currentState).nodes(builder).build(); currentState = ClusterState.builder(currentState).nodes(builder).build();
// check if we have enough master nodes, if not, we need to move into joining the cluster again // check if we have enough master nodes, if not, we need to move into joining the cluster again
if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) { if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) {
@ -524,7 +524,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
logger.error("unexpected failure during [{}]", t, source); logger.error("unexpected failure during [{}]", t, source);
} }
}); });
} else if (node.equals(nodes().masterNode())) { } else if (node.equals(nodes().getMasterNode())) {
handleMasterGone(node, null, "shut_down"); handleMasterGone(node, null, "shut_down");
} }
} }
@ -541,12 +541,12 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
clusterService.submitStateUpdateTask("zen-disco-node_failed(" + node + "), reason " + reason, new ClusterStateUpdateTask(Priority.IMMEDIATE) { clusterService.submitStateUpdateTask("zen-disco-node_failed(" + node + "), reason " + reason, new ClusterStateUpdateTask(Priority.IMMEDIATE) {
@Override @Override
public ClusterState execute(ClusterState currentState) { public ClusterState execute(ClusterState currentState) {
if (currentState.nodes().get(node.id()) == null) { if (currentState.nodes().get(node.getId()) == null) {
logger.debug("node [{}] already removed from cluster state. ignoring.", node); logger.debug("node [{}] already removed from cluster state. ignoring.", node);
return currentState; return currentState;
} }
DiscoveryNodes.Builder builder = DiscoveryNodes.builder(currentState.nodes()) DiscoveryNodes.Builder builder = DiscoveryNodes.builder(currentState.nodes())
.remove(node.id()); .remove(node.getId());
currentState = ClusterState.builder(currentState).nodes(builder).build(); currentState = ClusterState.builder(currentState).nodes(builder).build();
// check if we have enough master nodes, if not, we need to move into joining the cluster again // check if we have enough master nodes, if not, we need to move into joining the cluster again
if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) { if (!electMaster.hasEnoughMasterNodes(currentState.nodes())) {
@ -634,14 +634,14 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
@Override @Override
public ClusterState execute(ClusterState currentState) { public ClusterState execute(ClusterState currentState) {
if (!masterNode.id().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;
} }
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(currentState.nodes()) DiscoveryNodes discoveryNodes = DiscoveryNodes.builder(currentState.nodes())
// make sure the old master node, which has failed, is not part of the nodes we publish // make sure the old master node, which has failed, is not part of the nodes we publish
.remove(masterNode.id()) .remove(masterNode.getId())
.masterNodeId(null).build(); .masterNodeId(null).build();
// flush any pending cluster states from old master, so it will not be set as master again // flush any pending cluster states from old master, so it will not be set as master again
@ -680,11 +680,11 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
return currentState; return currentState;
} }
assert newClusterState.nodes().masterNode() != 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().masterNode(), newClusterState.version(), "via a new cluster state"); return handleAnotherMaster(currentState, newClusterState.nodes().getMasterNode(), newClusterState.version(), "via a new cluster state");
} }
if (shouldIgnoreOrRejectNewClusterState(logger, currentState, newClusterState)) { if (shouldIgnoreOrRejectNewClusterState(logger, currentState, newClusterState)) {
@ -692,13 +692,13 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
} }
// check to see that we monitor the correct master of the cluster // check to see that we monitor the correct master of the cluster
if (masterFD.masterNode() == null || !masterFD.masterNode().equals(newClusterState.nodes().masterNode())) { if (masterFD.masterNode() == null || !masterFD.masterNode().equals(newClusterState.nodes().getMasterNode())) {
masterFD.restart(newClusterState.nodes().masterNode(), "new cluster state received and we are monitoring the wrong master [" + masterFD.masterNode() + "]"); masterFD.restart(newClusterState.nodes().getMasterNode(), "new cluster state received and we are monitoring the wrong master [" + masterFD.masterNode() + "]");
} }
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,17 +783,17 @@ 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().getMasterNode(), currentNodes.getMasterNode());
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().getMasterNode() + ", current " + currentNodes.getMasterNode() + ")");
} }
} }
void handleJoinRequest(final DiscoveryNode node, final ClusterState state, final MembershipAction.JoinCallback callback) { void handleJoinRequest(final DiscoveryNode node, final ClusterState state, final MembershipAction.JoinCallback callback) {
if (!transportService.addressSupported(node.address().getClass())) { if (!transportService.addressSupported(node.getAddress().getClass())) {
// TODO, what should we do now? Maybe inform that node that its crap? // TODO, what should we do now? Maybe inform that node that its crap?
logger.warn("received a wrong address type from [{}], ignoring...", node); logger.warn("received a wrong address type from [{}], ignoring...", node);
} else if (nodeJoinController == null) { } else if (nodeJoinController == null) {
@ -863,7 +863,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
Set<DiscoveryNode> activeNodes = new HashSet<>(); Set<DiscoveryNode> activeNodes = new HashSet<>();
// nodes discovered who has previously been part of the cluster and do not ping for the very first time // nodes discovered who has previously been part of the cluster and do not ping for the very first time
Set<DiscoveryNode> joinedOnceActiveNodes = new HashSet<>(); Set<DiscoveryNode> joinedOnceActiveNodes = new HashSet<>();
if (localNode.masterNode()) { if (localNode.isMasterNode()) {
activeNodes.add(localNode); activeNodes.add(localNode);
long joinsCounter = clusterJoinsCounter.get(); long joinsCounter = clusterJoinsCounter.get();
if (joinsCounter > 0) { if (joinsCounter > 0) {
@ -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()) {
@ -967,7 +967,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
// since the network connections are asymmetric, it may be that we received a state but have disconnected from the node // since the network connections are asymmetric, it may be that we received a state but have disconnected from the node
// in the past (after a master failure, for example) // in the past (after a master failure, for example)
transportService.connectToNode(otherMaster); transportService.connectToNode(otherMaster);
transportService.sendRequest(otherMaster, DISCOVERY_REJOIN_ACTION_NAME, new RejoinClusterRequest(localClusterState.nodes().localNodeId()), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { transportService.sendRequest(otherMaster, DISCOVERY_REJOIN_ACTION_NAME, new RejoinClusterRequest(localClusterState.nodes().getLocalNodeId()), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
@Override @Override
public void handleException(TransportException exp) { public void handleException(TransportException exp) {

View File

@ -73,7 +73,7 @@ public class ElectMasterService extends AbstractComponent {
} }
int count = 0; int count = 0;
for (DiscoveryNode node : nodes) { for (DiscoveryNode node : nodes) {
if (node.masterNode()) { if (node.isMasterNode()) {
count++; count++;
} }
} }
@ -136,7 +136,7 @@ public class ElectMasterService extends AbstractComponent {
// clean non master nodes // clean non master nodes
for (Iterator<DiscoveryNode> it = possibleNodes.iterator(); it.hasNext(); ) { for (Iterator<DiscoveryNode> it = possibleNodes.iterator(); it.hasNext(); ) {
DiscoveryNode node = it.next(); DiscoveryNode node = it.next();
if (!node.masterNode()) { if (!node.isMasterNode()) {
it.remove(); it.remove();
} }
} }
@ -148,13 +148,13 @@ public class ElectMasterService extends AbstractComponent {
@Override @Override
public int compare(DiscoveryNode o1, DiscoveryNode o2) { public int compare(DiscoveryNode o1, DiscoveryNode o2) {
if (o1.masterNode() && !o2.masterNode()) { if (o1.isMasterNode() && !o2.isMasterNode()) {
return -1; return -1;
} }
if (!o1.masterNode() && o2.masterNode()) { if (!o1.isMasterNode() && o2.isMasterNode()) {
return 1; return 1;
} }
return o1.id().compareTo(o2.id()); return o1.getId().compareTo(o2.getId());
} }
} }
} }

View File

@ -227,7 +227,7 @@ public class MasterFaultDetection extends FaultDetection {
threadPool.schedule(pingInterval, ThreadPool.Names.SAME, MasterPinger.this); threadPool.schedule(pingInterval, ThreadPool.Names.SAME, MasterPinger.this);
return; return;
} }
final MasterPingRequest request = new MasterPingRequest(clusterService.localNode().id(), masterToPing.id(), clusterName); final MasterPingRequest request = new MasterPingRequest(clusterService.localNode().getId(), masterToPing.getId(), clusterName);
final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING).withTimeout(pingRetryTimeout).build(); final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING).withTimeout(pingRetryTimeout).build();
transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, new BaseTransportResponseHandler<MasterPingResponseResponse>() { transportService.sendRequest(masterToPing, MASTER_PING_ACTION_NAME, request, options, new BaseTransportResponseHandler<MasterPingResponseResponse>() {
@ -328,7 +328,7 @@ public class MasterFaultDetection extends FaultDetection {
final DiscoveryNodes nodes = clusterService.state().nodes(); final DiscoveryNodes nodes = clusterService.state().nodes();
// check if we are really the same master as the one we seemed to be think we are // check if we are really the same master as the one we seemed to be think we are
// this can happen if the master got "kill -9" and then another node started using the same port // this can happen if the master got "kill -9" and then another node started using the same port
if (!request.masterNodeId.equals(nodes.localNodeId())) { if (!request.masterNodeId.equals(nodes.getLocalNodeId())) {
throw new ThisIsNotTheMasterYouAreLookingForException(); throw new ThisIsNotTheMasterYouAreLookingForException();
} }
@ -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() {

View File

@ -94,7 +94,7 @@ public class NodesFaultDetection extends FaultDetection {
public void updateNodesAndPing(ClusterState clusterState) { public void updateNodesAndPing(ClusterState clusterState) {
// remove any nodes we don't need, this will cause their FD to stop // remove any nodes we don't need, this will cause their FD to stop
for (DiscoveryNode monitoredNode : nodesFD.keySet()) { for (DiscoveryNode monitoredNode : nodesFD.keySet()) {
if (!clusterState.nodes().nodeExists(monitoredNode.id())) { if (!clusterState.nodes().nodeExists(monitoredNode.getId())) {
nodesFD.remove(monitoredNode); nodesFD.remove(monitoredNode);
} }
} }
@ -200,7 +200,7 @@ public class NodesFaultDetection extends FaultDetection {
if (!running()) { if (!running()) {
return; return;
} }
final PingRequest pingRequest = new PingRequest(node.id(), clusterName, localNode, clusterStateVersion); final PingRequest pingRequest = new PingRequest(node.getId(), clusterName, localNode, clusterStateVersion);
final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING).withTimeout(pingRetryTimeout).build(); final TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.PING).withTimeout(pingRetryTimeout).build();
transportService.sendRequest(node, PING_ACTION_NAME, pingRequest, options, new BaseTransportResponseHandler<PingResponse>() { transportService.sendRequest(node, PING_ACTION_NAME, pingRequest, options, new BaseTransportResponseHandler<PingResponse>() {
@Override @Override
@ -255,8 +255,8 @@ public class NodesFaultDetection extends FaultDetection {
public void messageReceived(PingRequest request, TransportChannel channel) throws Exception { public void messageReceived(PingRequest request, TransportChannel channel) throws Exception {
// if we are not the node we are supposed to be pinged, send an exception // if we are not the node we are supposed to be pinged, send an exception
// this can happen when a kill -9 is sent, and another node is started using the same port // this can happen when a kill -9 is sent, and another node is started using the same port
if (!localNode.id().equals(request.nodeId)) { if (!localNode.getId().equals(request.nodeId)) {
throw new IllegalStateException("Got pinged as node [" + request.nodeId + "], but I am node [" + localNode.id() + "]"); throw new IllegalStateException("Got pinged as node [" + request.nodeId + "], but I am node [" + localNode.getId() + "]");
} }
// PingRequest will have clusterName set to null if it came from a node of version <1.4.0 // PingRequest will have clusterName set to null if it came from a node of version <1.4.0

View File

@ -171,7 +171,7 @@ public class MembershipAction extends AbstractComponent {
@Override @Override
public void readFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException {
super.readFrom(in); super.readFrom(in);
this.state = ClusterState.Builder.readFrom(in, nodesProvider.nodes().localNode()); this.state = ClusterState.Builder.readFrom(in, nodesProvider.nodes().getLocalNode());
} }
@Override @Override

View File

@ -360,7 +360,7 @@ public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implemen
for (final DiscoveryNode node : nodesToPing) { for (final DiscoveryNode node : nodesToPing) {
// make sure we are connected // make sure we are connected
final boolean nodeFoundByAddress; final boolean nodeFoundByAddress;
DiscoveryNode nodeToSend = discoNodes.findByAddress(node.address()); DiscoveryNode nodeToSend = discoNodes.findByAddress(node.getAddress());
if (nodeToSend != null) { if (nodeToSend != null) {
nodeFoundByAddress = true; nodeFoundByAddress = true;
} else { } else {
@ -378,9 +378,9 @@ public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implemen
// to make sure we don't disconnect a true node which was temporarily removed from the DiscoveryNodes // to make sure we don't disconnect a true node which was temporarily removed from the DiscoveryNodes
// but will be added again during the pinging. We therefore create a new temporary node // but will be added again during the pinging. We therefore create a new temporary node
if (!nodeFoundByAddress) { if (!nodeFoundByAddress) {
if (!nodeToSend.id().startsWith(UNICAST_NODE_PREFIX)) { if (!nodeToSend.getId().startsWith(UNICAST_NODE_PREFIX)) {
DiscoveryNode tempNode = new DiscoveryNode("", DiscoveryNode tempNode = new DiscoveryNode("",
UNICAST_NODE_PREFIX + unicastNodeIdGenerator.incrementAndGet() + "_" + nodeToSend.id() + "#", UNICAST_NODE_PREFIX + unicastNodeIdGenerator.incrementAndGet() + "_" + nodeToSend.getId() + "#",
nodeToSend.getHostName(), nodeToSend.getHostAddress(), nodeToSend.getAddress(), nodeToSend.getAttributes(), nodeToSend.getHostName(), nodeToSend.getHostAddress(), nodeToSend.getAddress(), nodeToSend.getAttributes(),
nodeToSend.getRoles(), nodeToSend.getVersion()); nodeToSend.getRoles(), nodeToSend.getVersion());
@ -469,7 +469,7 @@ public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implemen
try { try {
DiscoveryNodes discoveryNodes = contextProvider.nodes(); DiscoveryNodes discoveryNodes = contextProvider.nodes();
for (PingResponse pingResponse : response.pingResponses) { for (PingResponse pingResponse : response.pingResponses) {
if (pingResponse.node().id().equals(discoveryNodes.localNodeId())) { if (pingResponse.node().getId().equals(discoveryNodes.getLocalNodeId())) {
// that's us, ignore // that's us, ignore
continue; continue;
} }
@ -565,7 +565,8 @@ public class UnicastZenPing extends AbstractLifecycleComponent<ZenPing> implemen
} }
private PingResponse createPingResponse(DiscoveryNodes discoNodes) { private PingResponse createPingResponse(DiscoveryNodes discoNodes) {
return new PingResponse(discoNodes.localNode(), discoNodes.masterNode(), clusterName, contextProvider.nodeHasJoinedClusterOnce()); return new PingResponse(discoNodes.getLocalNode(), discoNodes.getMasterNode(), clusterName,
contextProvider.nodeHasJoinedClusterOnce());
} }
static class UnicastPingResponse extends TransportResponse { static class UnicastPingResponse extends TransportResponse {

View File

@ -138,7 +138,7 @@ public class PendingClusterStatesQueue {
if (findState(state.stateUUID()) == null) { if (findState(state.stateUUID()) == null) {
throw new IllegalStateException("can't resolve processed cluster state with uuid [" + state.stateUUID() + "], version [" + state.version() + "]"); throw new IllegalStateException("can't resolve processed cluster state with uuid [" + state.stateUUID() + "], version [" + state.version() + "]");
} }
final DiscoveryNode currentMaster = state.nodes().masterNode(); final DiscoveryNode currentMaster = state.nodes().getMasterNode();
assert currentMaster != null : "processed cluster state mast have a master. " + state; assert currentMaster != null : "processed cluster state mast have a master. " + state;
// fail or remove any incoming state from a different master // fail or remove any incoming state from a different master
@ -147,7 +147,7 @@ public class PendingClusterStatesQueue {
for (int index = 0; index < pendingStates.size(); index++) { for (int index = 0; index < pendingStates.size(); index++) {
final ClusterStateContext pendingContext = pendingStates.get(index); final ClusterStateContext pendingContext = pendingStates.get(index);
final ClusterState pendingState = pendingContext.state; final ClusterState pendingState = pendingContext.state;
final DiscoveryNode pendingMasterNode = pendingState.nodes().masterNode(); final DiscoveryNode pendingMasterNode = pendingState.nodes().getMasterNode();
if (Objects.equals(currentMaster, pendingMasterNode) == false) { if (Objects.equals(currentMaster, pendingMasterNode) == false) {
contextsToRemove.add(pendingContext); contextsToRemove.add(pendingContext);
if (pendingContext.committed()) { if (pendingContext.committed()) {
@ -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

@ -125,9 +125,9 @@ public class PublishClusterStateAction extends AbstractComponent {
final boolean sendFullVersion; final boolean sendFullVersion;
try { try {
nodes = clusterChangedEvent.state().nodes(); nodes = clusterChangedEvent.state().nodes();
nodesToPublishTo = new HashSet<>(nodes.size()); nodesToPublishTo = new HashSet<>(nodes.getSize());
DiscoveryNode localNode = nodes.localNode(); DiscoveryNode localNode = nodes.getLocalNode();
final int totalMasterNodes = nodes.masterNodes().size(); final int totalMasterNodes = nodes.getMasterNodes().size();
for (final DiscoveryNode node : nodes) { for (final DiscoveryNode node : nodes) {
if (node.equals(localNode) == false) { if (node.equals(localNode) == false) {
nodesToPublishTo.add(node); nodesToPublishTo.add(node);
@ -179,7 +179,7 @@ public class PublishClusterStateAction extends AbstractComponent {
// try and serialize the cluster state once (or per version), so we don't serialize it // try and serialize the cluster state once (or per version), so we don't serialize it
// per node when we send it over the wire, compress it while we are at it... // per node when we send it over the wire, compress it while we are at it...
// we don't send full version if node didn't exist in the previous version of cluster state // we don't send full version if node didn't exist in the previous version of cluster state
if (sendFullVersion || !previousState.nodes().nodeExists(node.id())) { if (sendFullVersion || !previousState.nodes().nodeExists(node.getId())) {
sendFullClusterState(clusterState, serializedStates, node, publishTimeout, sendingController); sendFullClusterState(clusterState, serializedStates, node, publishTimeout, sendingController);
} else { } else {
sendClusterStateDiff(clusterState, serializedDiffs, serializedStates, node, publishTimeout, sendingController); sendClusterStateDiff(clusterState, serializedDiffs, serializedStates, node, publishTimeout, sendingController);
@ -210,18 +210,18 @@ public class PublishClusterStateAction extends AbstractComponent {
Diff<ClusterState> diff = null; Diff<ClusterState> diff = null;
for (final DiscoveryNode node : nodesToPublishTo) { for (final DiscoveryNode node : nodesToPublishTo) {
try { try {
if (sendFullVersion || !previousState.nodes().nodeExists(node.id())) { if (sendFullVersion || !previousState.nodes().nodeExists(node.getId())) {
// will send a full reference // will send a full reference
if (serializedStates.containsKey(node.version()) == false) { if (serializedStates.containsKey(node.getVersion()) == false) {
serializedStates.put(node.version(), serializeFullClusterState(clusterState, node.version())); serializedStates.put(node.getVersion(), serializeFullClusterState(clusterState, node.getVersion()));
} }
} else { } else {
// will send a diff // will send a diff
if (diff == null) { if (diff == null) {
diff = clusterState.diff(previousState); diff = clusterState.diff(previousState);
} }
if (serializedDiffs.containsKey(node.version()) == false) { if (serializedDiffs.containsKey(node.getVersion()) == false) {
serializedDiffs.put(node.version(), serializeDiffClusterState(diff, node.version())); serializedDiffs.put(node.getVersion(), serializeDiffClusterState(diff, node.getVersion()));
} }
} }
} catch (IOException e) { } catch (IOException e) {
@ -232,11 +232,11 @@ public class PublishClusterStateAction extends AbstractComponent {
private void sendFullClusterState(ClusterState clusterState, Map<Version, BytesReference> serializedStates, private void sendFullClusterState(ClusterState clusterState, Map<Version, BytesReference> serializedStates,
DiscoveryNode node, TimeValue publishTimeout, SendingController sendingController) { DiscoveryNode node, TimeValue publishTimeout, SendingController sendingController) {
BytesReference bytes = serializedStates.get(node.version()); BytesReference bytes = serializedStates.get(node.getVersion());
if (bytes == null) { if (bytes == null) {
try { try {
bytes = serializeFullClusterState(clusterState, node.version()); bytes = serializeFullClusterState(clusterState, node.getVersion());
serializedStates.put(node.version(), bytes); serializedStates.put(node.getVersion(), bytes);
} catch (Throwable e) { } catch (Throwable e) {
logger.warn("failed to serialize cluster_state before publishing it to node {}", e, node); logger.warn("failed to serialize cluster_state before publishing it to node {}", e, node);
sendingController.onNodeSendFailed(node, e); sendingController.onNodeSendFailed(node, e);
@ -249,8 +249,8 @@ public class PublishClusterStateAction extends AbstractComponent {
private void sendClusterStateDiff(ClusterState clusterState, private void sendClusterStateDiff(ClusterState clusterState,
Map<Version, BytesReference> serializedDiffs, Map<Version, BytesReference> serializedStates, Map<Version, BytesReference> serializedDiffs, Map<Version, BytesReference> serializedStates,
DiscoveryNode node, TimeValue publishTimeout, SendingController sendingController) { DiscoveryNode node, TimeValue publishTimeout, SendingController sendingController) {
BytesReference bytes = serializedDiffs.get(node.version()); BytesReference bytes = serializedDiffs.get(node.getVersion());
assert bytes != null : "failed to find serialized diff for node " + node + " of version [" + node.version() + "]"; assert bytes != null : "failed to find serialized diff for node " + node + " of version [" + node.getVersion() + "]";
sendClusterStateToNode(clusterState, bytes, node, publishTimeout, sendingController, true, serializedStates); sendClusterStateToNode(clusterState, bytes, node, publishTimeout, sendingController, true, serializedStates);
} }
@ -266,7 +266,7 @@ public class PublishClusterStateAction extends AbstractComponent {
// -> no need to compress, we already compressed the bytes // -> no need to compress, we already compressed the bytes
TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.STATE).withCompress(false).build(); TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.STATE).withCompress(false).build();
transportService.sendRequest(node, SEND_ACTION_NAME, transportService.sendRequest(node, SEND_ACTION_NAME,
new BytesTransportRequest(bytes, node.version()), new BytesTransportRequest(bytes, node.getVersion()),
options, options,
new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
@ -363,7 +363,7 @@ public class PublishClusterStateAction extends AbstractComponent {
final ClusterState incomingState; final ClusterState incomingState;
// If true we received full cluster state - otherwise diffs // If true we received full cluster state - otherwise diffs
if (in.readBoolean()) { if (in.readBoolean()) {
incomingState = ClusterState.Builder.readFrom(in, nodesProvider.nodes().localNode()); incomingState = ClusterState.Builder.readFrom(in, nodesProvider.nodes().getLocalNode());
logger.debug("received full cluster state version [{}] with size [{}]", incomingState.version(), request.bytes().length()); logger.debug("received full cluster state version [{}] with size [{}]", incomingState.version(), request.bytes().length());
} else if (lastSeenClusterState != null) { } else if (lastSeenClusterState != null) {
Diff<ClusterState> diff = lastSeenClusterState.readDiffFrom(in); Diff<ClusterState> diff = lastSeenClusterState.readDiffFrom(in);
@ -391,13 +391,13 @@ public class PublishClusterStateAction extends AbstractComponent {
void validateIncomingState(ClusterState incomingState, ClusterState lastSeenClusterState) { void validateIncomingState(ClusterState incomingState, ClusterState lastSeenClusterState) {
final ClusterName incomingClusterName = incomingState.getClusterName(); final ClusterName incomingClusterName = incomingState.getClusterName();
if (!incomingClusterName.equals(this.clusterName)) { if (!incomingClusterName.equals(this.clusterName)) {
logger.warn("received cluster state from [{}] which is also master but with a different cluster name [{}]", incomingState.nodes().masterNode(), incomingClusterName); logger.warn("received cluster state from [{}] which is also master but with a different cluster name [{}]", incomingState.nodes().getMasterNode(), incomingClusterName);
throw new IllegalStateException("received state from a node that is not part of the cluster"); throw new IllegalStateException("received state from a node that is not part of the cluster");
} }
final DiscoveryNodes currentNodes = nodesProvider.nodes(); final DiscoveryNodes currentNodes = nodesProvider.nodes();
if (currentNodes.localNode().equals(incomingState.nodes().localNode()) == false) { if (currentNodes.getLocalNode().equals(incomingState.nodes().getLocalNode()) == false) {
logger.warn("received a cluster state from [{}] and not part of the cluster, should not happen", incomingState.nodes().masterNode()); logger.warn("received a cluster state from [{}] and not part of the cluster, should not happen", incomingState.nodes().getMasterNode());
throw new IllegalStateException("received state from a node that is not part of the cluster"); throw new IllegalStateException("received state from a node that is not part of the cluster");
} }
@ -427,7 +427,7 @@ public class PublishClusterStateAction extends AbstractComponent {
} }
}); });
if (state != null) { if (state != null) {
newPendingClusterStatelistener.onNewClusterState("master " + state.nodes().masterNode() + " committed version [" + state.version() + "]"); newPendingClusterStatelistener.onNewClusterState("master " + state.nodes().getMasterNode() + " committed version [" + state.version() + "]");
} }
} }

View File

@ -223,7 +223,7 @@ public abstract class AsyncShardFetch<T extends BaseNodeResponse> implements Rel
*/ */
private void fillShardCacheWithDataNodes(Map<String, NodeEntry<T>> shardCache, DiscoveryNodes nodes) { private void fillShardCacheWithDataNodes(Map<String, NodeEntry<T>> shardCache, DiscoveryNodes nodes) {
// verify that all current data nodes are there // verify that all current data nodes are there
for (ObjectObjectCursor<String, DiscoveryNode> cursor : nodes.dataNodes()) { for (ObjectObjectCursor<String, DiscoveryNode> cursor : nodes.getDataNodes()) {
DiscoveryNode node = cursor.value; DiscoveryNode node = cursor.value;
if (shardCache.containsKey(node.getId()) == false) { if (shardCache.containsKey(node.getId()) == false) {
shardCache.put(node.getId(), new NodeEntry<T>(node.getId())); shardCache.put(node.getId(), new NodeEntry<T>(node.getId()));

View File

@ -74,7 +74,7 @@ public class Gateway extends AbstractComponent implements ClusterStateListener {
} }
public void performStateRecovery(final GatewayStateRecoveredListener listener) throws GatewayException { public void performStateRecovery(final GatewayStateRecoveredListener listener) throws GatewayException {
String[] nodesIds = clusterService.state().nodes().masterNodes().keys().toArray(String.class); String[] nodesIds = clusterService.state().nodes().getMasterNodes().keys().toArray(String.class);
logger.trace("performing state recovery from {}", Arrays.toString(nodesIds)); logger.trace("performing state recovery from {}", Arrays.toString(nodesIds));
TransportNodesListGatewayMetaState.NodesGatewayMetaState nodesState = listGatewayMetaState.list(nodesIds, null).actionGet(); TransportNodesListGatewayMetaState.NodesGatewayMetaState nodesState = listGatewayMetaState.list(nodesIds, null).actionGet();

View File

@ -68,9 +68,9 @@ public class GatewayAllocator extends AbstractComponent {
@Override @Override
public void clusterChanged(ClusterChangedEvent event) { public void clusterChanged(ClusterChangedEvent event) {
boolean cleanCache = false; boolean cleanCache = false;
DiscoveryNode localNode = event.state().nodes().localNode(); DiscoveryNode localNode = event.state().nodes().getLocalNode();
if (localNode != null) { if (localNode != null) {
if (localNode.masterNode() == true && event.localNodeMaster() == false) { if (localNode.isMasterNode() == true && event.localNodeMaster() == false) {
cleanCache = true; cleanCache = true;
} }
} else { } else {

View File

@ -76,14 +76,14 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
this.metaDataIndexUpgradeService = metaDataIndexUpgradeService; this.metaDataIndexUpgradeService = metaDataIndexUpgradeService;
nodesListGatewayMetaState.init(this); nodesListGatewayMetaState.init(this);
if (DiscoveryNode.dataNode(settings)) { if (DiscoveryNode.isDataNode(settings)) {
ensureNoPre019ShardState(nodeEnv); ensureNoPre019ShardState(nodeEnv);
} }
if (DiscoveryNode.masterNode(settings) || DiscoveryNode.dataNode(settings)) { if (DiscoveryNode.isMasterNode(settings) || DiscoveryNode.isDataNode(settings)) {
nodeEnv.ensureAtomicMoveSupported(); nodeEnv.ensureAtomicMoveSupported();
} }
if (DiscoveryNode.masterNode(settings) || DiscoveryNode.dataNode(settings)) { if (DiscoveryNode.isMasterNode(settings) || DiscoveryNode.isDataNode(settings)) {
try { try {
ensureNoPre019State(); ensureNoPre019State();
IndexFolderUpgrader.upgradeIndicesIfNeeded(settings, nodeEnv); IndexFolderUpgrader.upgradeIndicesIfNeeded(settings, nodeEnv);
@ -118,7 +118,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
Set<Index> relevantIndices = Collections.emptySet(); Set<Index> relevantIndices = Collections.emptySet();
boolean success = true; boolean success = true;
// write the state if this node is a master eligible node or if it is a data node and has shards allocated on it // write the state if this node is a master eligible node or if it is a data node and has shards allocated on it
if (state.nodes().localNode().masterNode() || state.nodes().localNode().dataNode()) { if (state.nodes().getLocalNode().isMasterNode() || state.nodes().getLocalNode().isDataNode()) {
if (previousMetaData == null) { if (previousMetaData == null) {
try { try {
// we determine if or if not we write meta data on data only nodes by looking at the shard routing // we determine if or if not we write meta data on data only nodes by looking at the shard routing
@ -178,7 +178,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
Set<Index> relevantIndices; Set<Index> relevantIndices;
if (isDataOnlyNode(state)) { if (isDataOnlyNode(state)) {
relevantIndices = getRelevantIndicesOnDataOnlyNode(state, previousState, previouslyWrittenIndices); relevantIndices = getRelevantIndicesOnDataOnlyNode(state, previousState, previouslyWrittenIndices);
} else if (state.nodes().localNode().masterNode() == true) { } else if (state.nodes().getLocalNode().isMasterNode() == true) {
relevantIndices = getRelevantIndicesForMasterEligibleNode(state); relevantIndices = getRelevantIndicesForMasterEligibleNode(state);
} else { } else {
relevantIndices = Collections.emptySet(); relevantIndices = Collections.emptySet();
@ -188,7 +188,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
protected static boolean isDataOnlyNode(ClusterState state) { protected static boolean isDataOnlyNode(ClusterState state) {
return ((state.nodes().localNode().masterNode() == false) && state.nodes().localNode().dataNode()); return ((state.nodes().getLocalNode().isMasterNode() == false) && state.nodes().getLocalNode().isDataNode());
} }
/** /**
@ -285,7 +285,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
} }
public static Set<Index> getRelevantIndicesOnDataOnlyNode(ClusterState state, ClusterState previousState, Set<Index> previouslyWrittenIndices) { public static Set<Index> getRelevantIndicesOnDataOnlyNode(ClusterState state, ClusterState previousState, Set<Index> previouslyWrittenIndices) {
RoutingNode newRoutingNode = state.getRoutingNodes().node(state.nodes().localNodeId()); RoutingNode newRoutingNode = state.getRoutingNodes().node(state.nodes().getLocalNodeId());
if (newRoutingNode == null) { if (newRoutingNode == null) {
throw new IllegalStateException("cluster state does not contain this node - cannot write index meta state"); throw new IllegalStateException("cluster state does not contain this node - cannot write index meta state");
} }

View File

@ -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;
} }
@ -163,17 +163,17 @@ 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.masterAndDataNodes().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 [{}]",
nodes.masterAndDataNodes().size(), recoverAfterNodes); nodes.getMasterAndDataNodes().size(), recoverAfterNodes);
} else if (recoverAfterDataNodes != -1 && nodes.dataNodes().size() < recoverAfterDataNodes) { } else if (recoverAfterDataNodes != -1 && nodes.getDataNodes().size() < recoverAfterDataNodes) {
logger.debug("not recovering from gateway, nodes_size (data) [{}] < recover_after_data_nodes [{}]", logger.debug("not recovering from gateway, nodes_size (data) [{}] < recover_after_data_nodes [{}]",
nodes.dataNodes().size(), recoverAfterDataNodes); nodes.getDataNodes().size(), recoverAfterDataNodes);
} else if (recoverAfterMasterNodes != -1 && nodes.masterNodes().size() < recoverAfterMasterNodes) { } else if (recoverAfterMasterNodes != -1 && nodes.getMasterNodes().size() < recoverAfterMasterNodes) {
logger.debug("not recovering from gateway, nodes_size (master) [{}] < recover_after_master_nodes [{}]", logger.debug("not recovering from gateway, nodes_size (master) [{}] < recover_after_master_nodes [{}]",
nodes.masterNodes().size(), recoverAfterMasterNodes); nodes.getMasterNodes().size(), recoverAfterMasterNodes);
} else { } else {
boolean enforceRecoverAfterTime; boolean enforceRecoverAfterTime;
String reason; String reason;
@ -185,15 +185,15 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
// one of the expected is set, see if all of them meet the need, and ignore the timeout in this case // one of the expected is set, see if all of them meet the need, and ignore the timeout in this case
enforceRecoverAfterTime = false; enforceRecoverAfterTime = false;
reason = ""; reason = "";
if (expectedNodes != -1 && (nodes.masterAndDataNodes().size() < expectedNodes)) { // does not meet the expected... if (expectedNodes != -1 && (nodes.getMasterAndDataNodes().size() < expectedNodes)) { // does not meet the expected...
enforceRecoverAfterTime = true; enforceRecoverAfterTime = true;
reason = "expecting [" + expectedNodes + "] nodes, but only have [" + nodes.masterAndDataNodes().size() + "]"; reason = "expecting [" + expectedNodes + "] nodes, but only have [" + nodes.getMasterAndDataNodes().size() + "]";
} else if (expectedDataNodes != -1 && (nodes.dataNodes().size() < expectedDataNodes)) { // does not meet the expected... } else if (expectedDataNodes != -1 && (nodes.getDataNodes().size() < expectedDataNodes)) { // does not meet the expected...
enforceRecoverAfterTime = true; enforceRecoverAfterTime = true;
reason = "expecting [" + expectedDataNodes + "] data nodes, but only have [" + nodes.dataNodes().size() + "]"; reason = "expecting [" + expectedDataNodes + "] data nodes, but only have [" + nodes.getDataNodes().size() + "]";
} else if (expectedMasterNodes != -1 && (nodes.masterNodes().size() < expectedMasterNodes)) { // does not meet the expected... } else if (expectedMasterNodes != -1 && (nodes.getMasterNodes().size() < expectedMasterNodes)) { // does not meet the expected...
enforceRecoverAfterTime = true; enforceRecoverAfterTime = true;
reason = "expecting [" + expectedMasterNodes + "] master nodes, but only have [" + nodes.masterNodes().size() + "]"; reason = "expecting [" + expectedMasterNodes + "] master nodes, but only have [" + nodes.getMasterNodes().size() + "]";
} }
} }
performStateRecovery(enforceRecoverAfterTime, reason); performStateRecovery(enforceRecoverAfterTime, reason);

View File

@ -76,7 +76,7 @@ public class LocalAllocateDangledIndices extends AbstractComponent {
public void allocateDangled(Collection<IndexMetaData> indices, final Listener listener) { public void allocateDangled(Collection<IndexMetaData> indices, final Listener listener) {
ClusterState clusterState = clusterService.state(); ClusterState clusterState = clusterService.state();
DiscoveryNode masterNode = clusterState.nodes().masterNode(); DiscoveryNode masterNode = clusterState.nodes().getMasterNode();
if (masterNode == null) { if (masterNode == null) {
listener.onFailure(new MasterNotDiscoveredException("no master to send allocate dangled request")); listener.onFailure(new MasterNotDiscoveredException("no master to send allocate dangled request"));
return; return;

View File

@ -158,12 +158,12 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
NodeGatewayStartedShards nodeShardState = nodesToAllocate.yesNodeShards.get(0); NodeGatewayStartedShards nodeShardState = nodesToAllocate.yesNodeShards.get(0);
logger.debug("[{}][{}]: allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodeShardState.getNode()); logger.debug("[{}][{}]: allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodeShardState.getNode());
changed = true; changed = true;
unassignedIterator.initialize(nodeShardState.getNode().id(), nodeShardState.allocationId(), ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE); unassignedIterator.initialize(nodeShardState.getNode().getId(), nodeShardState.allocationId(), ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE);
} else if (nodesToAllocate.throttleNodeShards.isEmpty() == true && nodesToAllocate.noNodeShards.isEmpty() == false) { } else if (nodesToAllocate.throttleNodeShards.isEmpty() == true && nodesToAllocate.noNodeShards.isEmpty() == false) {
NodeGatewayStartedShards nodeShardState = nodesToAllocate.noNodeShards.get(0); NodeGatewayStartedShards nodeShardState = nodesToAllocate.noNodeShards.get(0);
logger.debug("[{}][{}]: forcing allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodeShardState.getNode()); logger.debug("[{}][{}]: forcing allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodeShardState.getNode());
changed = true; changed = true;
unassignedIterator.initialize(nodeShardState.getNode().id(), nodeShardState.allocationId(), ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE); unassignedIterator.initialize(nodeShardState.getNode().getId(), nodeShardState.allocationId(), ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE);
} else { } else {
// we are throttling this, but we have enough to allocate to this node, ignore it for now // we are throttling this, but we have enough to allocate to this node, ignore it for now
logger.debug("[{}][{}]: throttling allocation [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodesToAllocate.throttleNodeShards); logger.debug("[{}][{}]: throttling allocation [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodesToAllocate.throttleNodeShards);
@ -187,7 +187,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
DiscoveryNode node = nodeShardState.getNode(); DiscoveryNode node = nodeShardState.getNode();
String allocationId = nodeShardState.allocationId(); String allocationId = nodeShardState.allocationId();
if (ignoreNodes.contains(node.id())) { if (ignoreNodes.contains(node.getId())) {
continue; continue;
} }
@ -272,7 +272,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
List<NodeGatewayStartedShards> throttledNodeShards = new ArrayList<>(); List<NodeGatewayStartedShards> throttledNodeShards = new ArrayList<>();
List<NodeGatewayStartedShards> noNodeShards = new ArrayList<>(); List<NodeGatewayStartedShards> noNodeShards = new ArrayList<>();
for (NodeGatewayStartedShards nodeShardState : nodeShardStates) { for (NodeGatewayStartedShards nodeShardState : nodeShardStates) {
RoutingNode node = allocation.routingNodes().node(nodeShardState.getNode().id()); RoutingNode node = allocation.routingNodes().node(nodeShardState.getNode().getId());
if (node == null) { if (node == null) {
continue; continue;
} }
@ -302,7 +302,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent {
long version = nodeShardState.legacyVersion(); long version = nodeShardState.legacyVersion();
DiscoveryNode node = nodeShardState.getNode(); DiscoveryNode node = nodeShardState.getNode();
if (ignoreNodes.contains(node.id())) { if (ignoreNodes.contains(node.getId())) {
continue; continue;
} }

View File

@ -164,7 +164,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
MatchingNodes matchingNodes = findMatchingNodes(shard, allocation, primaryStore, shardStores); MatchingNodes matchingNodes = findMatchingNodes(shard, allocation, primaryStore, shardStores);
if (matchingNodes.getNodeWithHighestMatch() != null) { if (matchingNodes.getNodeWithHighestMatch() != null) {
RoutingNode nodeWithHighestMatch = allocation.routingNodes().node(matchingNodes.getNodeWithHighestMatch().id()); RoutingNode nodeWithHighestMatch = allocation.routingNodes().node(matchingNodes.getNodeWithHighestMatch().getId());
// we only check on THROTTLE since we checked before before on NO // we only check on THROTTLE since we checked before before on NO
Decision decision = allocation.deciders().canAllocate(shard, nodeWithHighestMatch, allocation); Decision decision = allocation.deciders().canAllocate(shard, nodeWithHighestMatch, allocation);
if (decision.type() == Decision.Type.THROTTLE) { if (decision.type() == Decision.Type.THROTTLE) {
@ -216,8 +216,8 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
* Can the shard be allocated on at least one node based on the allocation deciders. * Can the shard be allocated on at least one node based on the allocation deciders.
*/ */
private boolean canBeAllocatedToAtLeastOneNode(ShardRouting shard, RoutingAllocation allocation) { private boolean canBeAllocatedToAtLeastOneNode(ShardRouting shard, RoutingAllocation allocation) {
for (ObjectCursor<DiscoveryNode> cursor : allocation.nodes().dataNodes().values()) { for (ObjectCursor<DiscoveryNode> cursor : allocation.nodes().getDataNodes().values()) {
RoutingNode node = allocation.routingNodes().node(cursor.value.id()); RoutingNode node = allocation.routingNodes().node(cursor.value.getId());
if (node == null) { if (node == null) {
continue; continue;
} }
@ -259,7 +259,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
continue; continue;
} }
RoutingNode node = allocation.routingNodes().node(discoNode.id()); RoutingNode node = allocation.routingNodes().node(discoNode.getId());
if (node == null) { if (node == null) {
continue; continue;
} }
@ -286,7 +286,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
String replicaSyncId = storeFilesMetaData.syncId(); String replicaSyncId = storeFilesMetaData.syncId();
// see if we have a sync id we can make use of // see if we have a sync id we can make use of
if (replicaSyncId != null && replicaSyncId.equals(primarySyncId)) { if (replicaSyncId != null && replicaSyncId.equals(primarySyncId)) {
logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.name(), replicaSyncId); logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.getName(), replicaSyncId);
nodesToSize.put(discoNode, Long.MAX_VALUE); nodesToSize.put(discoNode, Long.MAX_VALUE);
} else { } else {
long sizeMatched = 0; long sizeMatched = 0;
@ -297,7 +297,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
} }
} }
logger.trace("{}: node [{}] has [{}/{}] bytes of re-usable data", logger.trace("{}: node [{}] has [{}/{}] bytes of re-usable data",
shard, discoNode.name(), new ByteSizeValue(sizeMatched), sizeMatched); shard, discoNode.getName(), new ByteSizeValue(sizeMatched), sizeMatched);
nodesToSize.put(discoNode, sizeMatched); nodesToSize.put(discoNode, sizeMatched);
} }
} }

View File

@ -536,7 +536,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
throw new IllegalStateException("Can't delete index store for [" + index.getName() + "] - it's still part of the indices service [" + localUUid + "] [" + metaData.getIndexUUID() + "]"); throw new IllegalStateException("Can't delete index store for [" + index.getName() + "] - it's still part of the indices service [" + localUUid + "] [" + metaData.getIndexUUID() + "]");
} }
if (clusterState.metaData().hasIndex(index.getName()) && (clusterState.nodes().localNode().masterNode() == true)) { if (clusterState.metaData().hasIndex(index.getName()) && (clusterState.nodes().getLocalNode().isMasterNode() == true)) {
// we do not delete the store if it is a master eligible node and the index is still in the cluster state // we do not delete the store if it is a master eligible node and the index is still in the cluster state
// because we want to keep the meta data for indices around even if no shards are left here // because we want to keep the meta data for indices around even if no shards are left here
final IndexMetaData idxMeta = clusterState.metaData().index(index.getName()); final IndexMetaData idxMeta = clusterState.metaData().index(index.getName());
@ -609,7 +609,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
nodeEnv.deleteShardDirectorySafe(shardId, indexSettings); nodeEnv.deleteShardDirectorySafe(shardId, indexSettings);
logger.debug("{} deleted shard reason [{}]", shardId, reason); logger.debug("{} deleted shard reason [{}]", shardId, reason);
if (clusterState.nodes().localNode().isMasterNode() == false && // master nodes keep the index meta data, even if having no shards.. if (clusterState.nodes().getLocalNode().isMasterNode() == false && // master nodes keep the index meta data, even if having no shards..
canDeleteIndexContents(shardId.getIndex(), indexSettings, false)) { canDeleteIndexContents(shardId.getIndex(), indexSettings, false)) {
if (nodeEnv.findAllShardIds(shardId.getIndex()).isEmpty()) { if (nodeEnv.findAllShardIds(shardId.getIndex()).isEmpty()) {
try { try {

View File

@ -193,7 +193,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
} }
private void cleanFailedShards(final ClusterChangedEvent event) { private void cleanFailedShards(final ClusterChangedEvent event) {
RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId()); RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().getLocalNodeId());
if (routingNode == null) { if (routingNode == null) {
failedShards.clear(); failedShards.clear();
return; return;
@ -221,7 +221,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
private void applyDeletedIndices(final ClusterChangedEvent event) { private void applyDeletedIndices(final ClusterChangedEvent event) {
final ClusterState previousState = event.previousState(); final ClusterState previousState = event.previousState();
final String localNodeId = event.state().nodes().localNodeId(); final String localNodeId = event.state().nodes().getLocalNodeId();
assert localNodeId != null; assert localNodeId != null;
for (Index index : event.indicesDeleted()) { for (Index index : event.indicesDeleted()) {
@ -259,7 +259,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
} }
private void applyDeletedShards(final ClusterChangedEvent event) { private void applyDeletedShards(final ClusterChangedEvent event) {
RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId()); RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().getLocalNodeId());
if (routingNode == null) { if (routingNode == null) {
return; return;
} }
@ -315,7 +315,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
} }
final Set<Index> hasAllocations = new HashSet<>(); final Set<Index> hasAllocations = new HashSet<>();
final RoutingNode node = event.state().getRoutingNodes().node(event.state().nodes().localNodeId()); final RoutingNode node = event.state().getRoutingNodes().node(event.state().nodes().getLocalNodeId());
// if no shards are allocated ie. if this node is a master-only node it can return nul // if no shards are allocated ie. if this node is a master-only node it can return nul
if (node != null) { if (node != null) {
for (ShardRouting routing : node) { for (ShardRouting routing : node) {
@ -362,7 +362,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
private void applyNewIndices(final ClusterChangedEvent event) { private void applyNewIndices(final ClusterChangedEvent event) {
// we only create indices for shards that are allocated // we only create indices for shards that are allocated
RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId()); RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().getLocalNodeId());
if (routingNode == null) { if (routingNode == null) {
return; return;
} }
@ -407,7 +407,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
if (requireRefresh && sendRefreshMapping) { if (requireRefresh && sendRefreshMapping) {
nodeMappingRefreshAction.nodeMappingRefresh(event.state(), nodeMappingRefreshAction.nodeMappingRefresh(event.state(),
new NodeMappingRefreshAction.NodeMappingRefreshRequest(index.getName(), indexMetaData.getIndexUUID(), new NodeMappingRefreshAction.NodeMappingRefreshRequest(index.getName(), indexMetaData.getIndexUUID(),
event.state().nodes().localNodeId()) event.state().nodes().getLocalNodeId())
); );
} }
} catch (Throwable t) { } catch (Throwable t) {
@ -459,7 +459,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
} }
RoutingTable routingTable = event.state().routingTable(); RoutingTable routingTable = event.state().routingTable();
RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId()); RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().getLocalNodeId());
if (routingNode == null) { if (routingNode == null) {
failedShards.clear(); failedShards.clear();
@ -483,14 +483,14 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
if (!indexService.hasShard(shardId) && shardRouting.started()) { if (!indexService.hasShard(shardId) && shardRouting.started()) {
if (failedShards.containsKey(shardRouting.shardId())) { if (failedShards.containsKey(shardRouting.shardId())) {
if (nodes.masterNode() != null) { if (nodes.getMasterNode() != null) {
String message = "master " + nodes.masterNode() + " marked shard as started, but shard has previous failed. resending shard failure"; String message = "master " + nodes.getMasterNode() + " marked shard as started, but shard has previous failed. resending shard failure";
logger.trace("[{}] re-sending failed shard [{}], reason [{}]", shardRouting.shardId(), shardRouting, message); logger.trace("[{}] re-sending failed shard [{}], reason [{}]", shardRouting.shardId(), shardRouting, message);
shardStateAction.shardFailed(shardRouting, shardRouting, message, null, SHARD_STATE_ACTION_LISTENER); shardStateAction.shardFailed(shardRouting, shardRouting, message, null, SHARD_STATE_ACTION_LISTENER);
} }
} else { } else {
// the master thinks we are started, but we don't have this shard at all, mark it as failed // the master thinks we are started, but we don't have this shard at all, mark it as failed
sendFailShard(shardRouting, "master [" + nodes.masterNode() + "] marked shard as started, but shard has not been created, mark shard as failed", null); sendFailShard(shardRouting, "master [" + nodes.getMasterNode() + "] marked shard as started, but shard has not been created, mark shard as failed", null);
} }
continue; continue;
} }
@ -547,11 +547,11 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
// we managed to tell the master we started), mark us as started // we managed to tell the master we started), mark us as started
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("{} master marked shard as initializing, but shard has state [{}], resending shard started to {}", logger.trace("{} master marked shard as initializing, but shard has state [{}], resending shard started to {}",
indexShard.shardId(), indexShard.state(), nodes.masterNode()); indexShard.shardId(), indexShard.state(), nodes.getMasterNode());
} }
if (nodes.masterNode() != null) { if (nodes.getMasterNode() != null) {
shardStateAction.shardStarted(shardRouting, shardStateAction.shardStarted(shardRouting,
"master " + nodes.masterNode() + " marked shard as initializing, but shard state is [" + indexShard.state() + "], mark shard as started", "master " + nodes.getMasterNode() + " marked shard as initializing, but shard state is [" + indexShard.state() + "], mark shard as started",
SHARD_STATE_ACTION_LISTENER); SHARD_STATE_ACTION_LISTENER);
} }
return; return;
@ -576,8 +576,8 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
// if there is no shard, create it // if there is no shard, create it
if (!indexService.hasShard(shardId)) { if (!indexService.hasShard(shardId)) {
if (failedShards.containsKey(shardRouting.shardId())) { if (failedShards.containsKey(shardRouting.shardId())) {
if (nodes.masterNode() != null) { if (nodes.getMasterNode() != null) {
String message = "master " + nodes.masterNode() + " marked shard as initializing, but shard is marked as failed, resend shard failure"; String message = "master " + nodes.getMasterNode() + " marked shard as initializing, but shard is marked as failed, resend shard failure";
logger.trace("[{}] re-sending failed shard [{}], reason [{}]", shardRouting.shardId(), shardRouting, message); logger.trace("[{}] re-sending failed shard [{}], reason [{}]", shardRouting.shardId(), shardRouting, message);
shardStateAction.shardFailed(shardRouting, shardRouting, message, null, SHARD_STATE_ACTION_LISTENER); shardStateAction.shardFailed(shardRouting, shardRouting, message, null, SHARD_STATE_ACTION_LISTENER);
} }
@ -617,7 +617,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
// the edge case where its mark as relocated, and we might need to roll it back... // the edge case where its mark as relocated, and we might need to roll it back...
// For replicas: we are recovering a backup from a primary // For replicas: we are recovering a backup from a primary
RecoveryState.Type type = shardRouting.primary() ? RecoveryState.Type.PRIMARY_RELOCATION : RecoveryState.Type.REPLICA; RecoveryState.Type type = shardRouting.primary() ? RecoveryState.Type.PRIMARY_RELOCATION : RecoveryState.Type.REPLICA;
RecoveryState recoveryState = new RecoveryState(indexShard.shardId(), shardRouting.primary(), type, sourceNode, nodes.localNode()); RecoveryState recoveryState = new RecoveryState(indexShard.shardId(), shardRouting.primary(), type, sourceNode, nodes.getLocalNode());
indexShard.markAsRecovering("from " + sourceNode, recoveryState); indexShard.markAsRecovering("from " + sourceNode, recoveryState);
recoveryTargetService.startRecovery(indexShard, type, sourceNode, new PeerRecoveryListener(shardRouting, indexService, indexMetaData)); recoveryTargetService.startRecovery(indexShard, type, sourceNode, new PeerRecoveryListener(shardRouting, indexService, indexMetaData));
} catch (Throwable e) { } catch (Throwable e) {
@ -629,11 +629,11 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
// recover from filesystem store // recover from filesystem store
final RecoveryState recoveryState = new RecoveryState(indexShard.shardId(), shardRouting.primary(), final RecoveryState recoveryState = new RecoveryState(indexShard.shardId(), shardRouting.primary(),
RecoveryState.Type.STORE, RecoveryState.Type.STORE,
nodes.localNode(), nodes.localNode()); nodes.getLocalNode(), nodes.getLocalNode());
indexShard.markAsRecovering("from store", recoveryState); // mark the shard as recovering on the cluster state thread indexShard.markAsRecovering("from store", recoveryState); // mark the shard as recovering on the cluster state thread
threadPool.generic().execute(() -> { threadPool.generic().execute(() -> {
try { try {
if (indexShard.recoverFromStore(nodes.localNode())) { if (indexShard.recoverFromStore(nodes.getLocalNode())) {
shardStateAction.shardStarted(shardRouting, "after recovery from store", SHARD_STATE_ACTION_LISTENER); shardStateAction.shardStarted(shardRouting, "after recovery from store", SHARD_STATE_ACTION_LISTENER);
} }
} catch (Throwable t) { } catch (Throwable t) {
@ -644,13 +644,13 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
} else { } else {
// recover from a restore // recover from a restore
final RecoveryState recoveryState = new RecoveryState(indexShard.shardId(), shardRouting.primary(), final RecoveryState recoveryState = new RecoveryState(indexShard.shardId(), shardRouting.primary(),
RecoveryState.Type.SNAPSHOT, shardRouting.restoreSource(), nodes.localNode()); RecoveryState.Type.SNAPSHOT, shardRouting.restoreSource(), nodes.getLocalNode());
indexShard.markAsRecovering("from snapshot", recoveryState); // mark the shard as recovering on the cluster state thread indexShard.markAsRecovering("from snapshot", recoveryState); // mark the shard as recovering on the cluster state thread
threadPool.generic().execute(() -> { threadPool.generic().execute(() -> {
final ShardId sId = indexShard.shardId(); final ShardId sId = indexShard.shardId();
try { try {
final IndexShardRepository indexShardRepository = repositoriesService.indexShardRepository(restoreSource.snapshotId().getRepository()); final IndexShardRepository indexShardRepository = repositoriesService.indexShardRepository(restoreSource.snapshotId().getRepository());
if (indexShard.restoreFromRepository(indexShardRepository, nodes.localNode())) { if (indexShard.restoreFromRepository(indexShardRepository, nodes.getLocalNode())) {
restoreService.indexShardRestoreCompleted(restoreSource.snapshotId(), sId); restoreService.indexShardRestoreCompleted(restoreSource.snapshotId(), sId);
shardStateAction.shardStarted(shardRouting, "after recovery from repository", SHARD_STATE_ACTION_LISTENER); shardStateAction.shardStarted(shardRouting, "after recovery from repository", SHARD_STATE_ACTION_LISTENER);
} }

View File

@ -380,7 +380,7 @@ public class SyncedFlushService extends AbstractComponent implements IndexEventL
@Override @Override
public void handleResponse(PreSyncedFlushResponse response) { public void handleResponse(PreSyncedFlushResponse response) {
Engine.CommitId existing = commitIds.putIfAbsent(node.id(), response.commitId()); Engine.CommitId existing = commitIds.putIfAbsent(node.getId(), response.commitId());
assert existing == null : "got two answers for node [" + node + "]"; assert existing == null : "got two answers for node [" + node + "]";
// count after the assert so we won't decrement twice in handleException // count after the assert so we won't decrement twice in handleException
if (countDown.countDown()) { if (countDown.countDown()) {

View File

@ -88,7 +88,7 @@ public class RecoverySource extends AbstractComponent implements IndexEventListe
// starting recovery from that our (the source) shard state is marking the shard to be in recovery mode as well, otherwise // starting recovery from that our (the source) shard state is marking the shard to be in recovery mode as well, otherwise
// the index operations will not be routed to it properly // the index operations will not be routed to it properly
RoutingNode node = clusterService.state().getRoutingNodes().node(request.targetNode().id()); RoutingNode node = clusterService.state().getRoutingNodes().node(request.targetNode().getId());
if (node == null) { if (node == null) {
logger.debug("delaying recovery of {} as source node {} is unknown", request.shardId(), request.targetNode()); logger.debug("delaying recovery of {} as source node {} is unknown", request.shardId(), request.targetNode());
throw new DelayRecoveryException("source node does not have the node [" + request.targetNode() + "] in its state yet.."); throw new DelayRecoveryException("source node does not have the node [" + request.targetNode() + "] in its state yet..");

View File

@ -316,20 +316,20 @@ public class RecoveryState implements ToXContent, Streamable {
restoreSource.toXContent(builder, params); restoreSource.toXContent(builder, params);
} else { } else {
builder.startObject(Fields.SOURCE); builder.startObject(Fields.SOURCE);
builder.field(Fields.ID, sourceNode.id()); builder.field(Fields.ID, sourceNode.getId());
builder.field(Fields.HOST, sourceNode.getHostName()); builder.field(Fields.HOST, sourceNode.getHostName());
builder.field(Fields.TRANSPORT_ADDRESS, sourceNode.address().toString()); builder.field(Fields.TRANSPORT_ADDRESS, sourceNode.getAddress().toString());
builder.field(Fields.IP, sourceNode.getHostAddress()); builder.field(Fields.IP, sourceNode.getHostAddress());
builder.field(Fields.NAME, sourceNode.name()); builder.field(Fields.NAME, sourceNode.getName());
builder.endObject(); builder.endObject();
} }
builder.startObject(Fields.TARGET); builder.startObject(Fields.TARGET);
builder.field(Fields.ID, targetNode.id()); builder.field(Fields.ID, targetNode.getId());
builder.field(Fields.HOST, targetNode.getHostName()); builder.field(Fields.HOST, targetNode.getHostName());
builder.field(Fields.TRANSPORT_ADDRESS, targetNode.address().toString()); builder.field(Fields.TRANSPORT_ADDRESS, targetNode.getAddress().toString());
builder.field(Fields.IP, targetNode.getHostAddress()); builder.field(Fields.IP, targetNode.getHostAddress());
builder.field(Fields.NAME, targetNode.name()); builder.field(Fields.NAME, targetNode.getName());
builder.endObject(); builder.endObject();
builder.startObject(Fields.INDEX); builder.startObject(Fields.INDEX);

View File

@ -160,7 +160,7 @@ public class IndicesStore extends AbstractComponent implements ClusterStateListe
} }
// check if shard is active on the current node or is getting relocated to the our node // check if shard is active on the current node or is getting relocated to the our node
String localNodeId = state.getNodes().localNode().id(); String localNodeId = state.getNodes().getLocalNode().getId();
if (localNodeId.equals(shardRouting.currentNodeId()) || localNodeId.equals(shardRouting.relocatingNodeId())) { if (localNodeId.equals(shardRouting.currentNodeId()) || localNodeId.equals(shardRouting.relocatingNodeId())) {
return false; return false;
} }

View File

@ -216,7 +216,7 @@ public class Node implements Closeable {
modules.add(new ClusterModule(this.settings)); modules.add(new ClusterModule(this.settings));
modules.add(new IndicesModule()); modules.add(new IndicesModule());
modules.add(new SearchModule(settings, namedWriteableRegistry)); modules.add(new SearchModule(settings, namedWriteableRegistry));
modules.add(new ActionModule(DiscoveryNode.ingestNode(settings), false)); modules.add(new ActionModule(DiscoveryNode.isIngestNode(settings), false));
modules.add(new GatewayModule(settings)); modules.add(new GatewayModule(settings));
modules.add(new NodeClientModule()); modules.add(new NodeClientModule());
modules.add(new ResourceWatcherModule()); modules.add(new ResourceWatcherModule());
@ -333,7 +333,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

@ -77,7 +77,7 @@ public class RepositoriesService extends AbstractComponent implements ClusterSta
this.clusterService = clusterService; this.clusterService = clusterService;
// Doesn't make sense to maintain repositories on non-master and non-data nodes // Doesn't make sense to maintain repositories on non-master and non-data nodes
// Nothing happens there anyway // Nothing happens there anyway
if (DiscoveryNode.dataNode(settings) || DiscoveryNode.masterNode(settings)) { if (DiscoveryNode.isDataNode(settings) || DiscoveryNode.isMasterNode(settings)) {
clusterService.add(this); clusterService.add(this);
} }
this.verifyAction = new VerifyNodeRepositoryAction(settings, transportService, clusterService, this); this.verifyAction = new VerifyNodeRepositoryAction(settings, transportService, clusterService, this);
@ -154,7 +154,7 @@ public class RepositoriesService extends AbstractComponent implements ClusterSta
@Override @Override
public boolean mustAck(DiscoveryNode discoveryNode) { public boolean mustAck(DiscoveryNode discoveryNode) {
return discoveryNode.masterNode(); return discoveryNode.isMasterNode();
} }
}); });
} }
@ -205,7 +205,7 @@ public class RepositoriesService extends AbstractComponent implements ClusterSta
@Override @Override
public boolean mustAck(DiscoveryNode discoveryNode) { public boolean mustAck(DiscoveryNode discoveryNode) {
// Since operation occurs only on masters, it's enough that only master-eligible nodes acked // Since operation occurs only on masters, it's enough that only master-eligible nodes acked
return discoveryNode.masterNode(); return discoveryNode.isMasterNode();
} }
}); });
} }

View File

@ -69,9 +69,9 @@ public class VerifyNodeRepositoryAction extends AbstractComponent {
public void verify(String repository, String verificationToken, final ActionListener<VerifyResponse> listener) { public void verify(String repository, String verificationToken, final ActionListener<VerifyResponse> listener) {
final DiscoveryNodes discoNodes = clusterService.state().nodes(); final DiscoveryNodes discoNodes = clusterService.state().nodes();
final DiscoveryNode localNode = discoNodes.localNode(); final DiscoveryNode localNode = discoNodes.getLocalNode();
final ObjectContainer<DiscoveryNode> masterAndDataNodes = discoNodes.masterAndDataNodes().values(); final ObjectContainer<DiscoveryNode> masterAndDataNodes = discoNodes.getMasterAndDataNodes().values();
final List<DiscoveryNode> nodes = new ArrayList<>(); final List<DiscoveryNode> nodes = new ArrayList<>();
for (ObjectCursor<DiscoveryNode> cursor : masterAndDataNodes) { for (ObjectCursor<DiscoveryNode> cursor : masterAndDataNodes) {
DiscoveryNode node = cursor.value; DiscoveryNode node = cursor.value;
@ -85,7 +85,7 @@ public class VerifyNodeRepositoryAction extends AbstractComponent {
doVerify(repository, verificationToken); doVerify(repository, verificationToken);
} catch (Throwable t) { } catch (Throwable t) {
logger.warn("[{}] failed to verify repository", t, repository); logger.warn("[{}] failed to verify repository", t, repository);
errors.add(new VerificationFailure(node.id(), t)); errors.add(new VerificationFailure(node.getId(), t));
} }
if (counter.decrementAndGet() == 0) { if (counter.decrementAndGet() == 0) {
finishVerification(listener, nodes, errors); finishVerification(listener, nodes, errors);
@ -101,7 +101,7 @@ public class VerifyNodeRepositoryAction extends AbstractComponent {
@Override @Override
public void handleException(TransportException exp) { public void handleException(TransportException exp) {
errors.add(new VerificationFailure(node.id(), exp)); errors.add(new VerificationFailure(node.getId(), exp));
if (counter.decrementAndGet() == 0) { if (counter.decrementAndGet() == 0) {
finishVerification(listener, nodes, errors); finishVerification(listener, nodes, errors);
} }

View File

@ -120,7 +120,7 @@ public class RestAllocationAction extends AbstractCatAction {
for (NodeStats nodeStats : stats.getNodes()) { for (NodeStats nodeStats : stats.getNodes()) {
DiscoveryNode node = nodeStats.getNode(); DiscoveryNode node = nodeStats.getNode();
int shardCount = allocs.getOrDefault(node.id(), 0); int shardCount = allocs.getOrDefault(node.getId(), 0);
ByteSizeValue total = nodeStats.getFs().getTotal().getTotal(); ByteSizeValue total = nodeStats.getFs().getTotal().getTotal();
ByteSizeValue avail = nodeStats.getFs().getTotal().getAvailable(); ByteSizeValue avail = nodeStats.getFs().getTotal().getAvailable();
@ -143,7 +143,7 @@ public class RestAllocationAction extends AbstractCatAction {
table.addCell(diskPercent < 0 ? null : diskPercent); table.addCell(diskPercent < 0 ? null : diskPercent);
table.addCell(node.getHostName()); table.addCell(node.getHostName());
table.addCell(node.getHostAddress()); table.addCell(node.getHostAddress());
table.addCell(node.name()); table.addCell(node.getName());
table.endRow(); table.endRow();
} }

View File

@ -124,7 +124,7 @@ public class RestFielddataAction extends AbstractCatAction {
table.startRow(); table.startRow();
// add the node info and field data total before each individual field // add the node info and field data total before each individual field
NodeStats ns = statsEntry.getKey(); NodeStats ns = statsEntry.getKey();
table.addCell(ns.getNode().id()); table.addCell(ns.getNode().getId());
table.addCell(ns.getNode().getHostName()); table.addCell(ns.getNode().getHostName());
table.addCell(ns.getNode().getHostAddress()); table.addCell(ns.getNode().getHostAddress());
table.addCell(ns.getNode().getName()); table.addCell(ns.getNode().getName());

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

@ -102,16 +102,16 @@ public class RestNodeAttrsAction extends AbstractCatAction {
Table table = getTableWithHeader(req); Table table = getTableWithHeader(req);
for (DiscoveryNode node : nodes) { for (DiscoveryNode node : nodes) {
NodeInfo info = nodesInfo.getNodesMap().get(node.id()); NodeInfo info = nodesInfo.getNodesMap().get(node.getId());
for (Map.Entry<String, String> attrEntry : node.getAttributes().entrySet()) { for (Map.Entry<String, String> attrEntry : node.getAttributes().entrySet()) {
table.startRow(); table.startRow();
table.addCell(node.name()); table.addCell(node.getName());
table.addCell(fullId ? node.id() : Strings.substring(node.getId(), 0, 4)); table.addCell(fullId ? node.getId() : Strings.substring(node.getId(), 0, 4));
table.addCell(info == null ? null : info.getProcess().getId()); table.addCell(info == null ? null : info.getProcess().getId());
table.addCell(node.getHostName()); table.addCell(node.getHostName());
table.addCell(node.getHostAddress()); table.addCell(node.getHostAddress());
if (node.address() instanceof InetSocketTransportAddress) { if (node.getAddress() instanceof InetSocketTransportAddress) {
table.addCell(((InetSocketTransportAddress) node.address()).address().getPort()); table.addCell(((InetSocketTransportAddress) node.getAddress()).address().getPort());
} else { } else {
table.addCell("-"); table.addCell("-");
} }

View File

@ -223,12 +223,12 @@ 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) {
NodeInfo info = nodesInfo.getNodesMap().get(node.id()); NodeInfo info = nodesInfo.getNodesMap().get(node.getId());
NodeStats stats = nodesStats.getNodesMap().get(node.id()); NodeStats stats = nodesStats.getNodesMap().get(node.getId());
JvmInfo jvmInfo = info == null ? null : info.getJvm(); JvmInfo jvmInfo = info == null ? null : info.getJvm();
JvmStats jvmStats = stats == null ? null : stats.getJvm(); JvmStats jvmStats = stats == null ? null : stats.getJvm();
@ -239,11 +239,11 @@ public class RestNodesAction extends AbstractCatAction {
table.startRow(); table.startRow();
table.addCell(fullId ? node.id() : Strings.substring(node.getId(), 0, 4)); table.addCell(fullId ? node.getId() : Strings.substring(node.getId(), 0, 4));
table.addCell(info == null ? null : info.getProcess().getId()); table.addCell(info == null ? null : info.getProcess().getId());
table.addCell(node.getHostAddress()); table.addCell(node.getHostAddress());
if (node.address() instanceof InetSocketTransportAddress) { if (node.getAddress() instanceof InetSocketTransportAddress) {
table.addCell(((InetSocketTransportAddress) node.address()).address().getPort()); table.addCell(((InetSocketTransportAddress) node.getAddress()).address().getPort());
} else { } else {
table.addCell("-"); table.addCell("-");
} }
@ -287,8 +287,8 @@ public class RestNodesAction extends AbstractCatAction {
roles = node.getRoles().stream().map(DiscoveryNode.Role::getAbbreviation).collect(Collectors.joining()); roles = node.getRoles().stream().map(DiscoveryNode.Role::getAbbreviation).collect(Collectors.joining());
} }
table.addCell(roles); table.addCell(roles);
table.addCell(masterId == null ? "x" : masterId.equals(node.id()) ? "*" : "-"); table.addCell(masterId == null ? "x" : masterId.equals(node.getId()) ? "*" : "-");
table.addCell(node.name()); table.addCell(node.getName());
CompletionStats completionStats = indicesStats == null ? null : stats.getIndices().getCompletion(); CompletionStats completionStats = indicesStats == null ? null : stats.getIndices().getCompletion();
table.addCell(completionStats == null ? null : completionStats.getSize()); table.addCell(completionStats == null ? null : completionStats.getSize());

View File

@ -94,12 +94,12 @@ public class RestPluginsAction extends AbstractCatAction {
Table table = getTableWithHeader(req); Table table = getTableWithHeader(req);
for (DiscoveryNode node : nodes) { for (DiscoveryNode node : nodes) {
NodeInfo info = nodesInfo.getNodesMap().get(node.id()); NodeInfo info = nodesInfo.getNodesMap().get(node.getId());
for (PluginInfo pluginInfo : info.getPlugins().getPluginInfos()) { for (PluginInfo pluginInfo : info.getPlugins().getPluginInfos()) {
table.startRow(); table.startRow();
table.addCell(node.id()); table.addCell(node.getId());
table.addCell(node.name()); table.addCell(node.getName());
table.addCell(pluginInfo.getName()); table.addCell(pluginInfo.getName());
table.addCell(pluginInfo.getVersion()); table.addCell(pluginInfo.getVersion());
table.addCell(pluginInfo.getDescription()); table.addCell(pluginInfo.getDescription());

View File

@ -208,10 +208,10 @@ public class RestShardsAction extends AbstractCatAction {
String ip = state.getState().nodes().get(shard.currentNodeId()).getHostAddress(); String ip = state.getState().nodes().get(shard.currentNodeId()).getHostAddress();
String nodeId = shard.currentNodeId(); String nodeId = shard.currentNodeId();
StringBuilder name = new StringBuilder(); StringBuilder name = new StringBuilder();
name.append(state.getState().nodes().get(shard.currentNodeId()).name()); name.append(state.getState().nodes().get(shard.currentNodeId()).getName());
if (shard.relocating()) { if (shard.relocating()) {
String reloIp = state.getState().nodes().get(shard.relocatingNodeId()).getHostAddress(); String reloIp = state.getState().nodes().get(shard.relocatingNodeId()).getHostAddress();
String reloNme = state.getState().nodes().get(shard.relocatingNodeId()).name(); String reloNme = state.getState().nodes().get(shard.relocatingNodeId()).getName();
String reloNodeId = shard.relocatingNodeId(); String reloNodeId = shard.relocatingNodeId();
name.append(" -> "); name.append(" -> ");
name.append(reloIp); name.append(reloIp);

View File

@ -226,16 +226,16 @@ public class RestThreadPoolAction extends AbstractCatAction {
Table table = getTableWithHeader(req); Table table = getTableWithHeader(req);
for (DiscoveryNode node : nodes) { for (DiscoveryNode node : nodes) {
NodeInfo info = nodesInfo.getNodesMap().get(node.id()); NodeInfo info = nodesInfo.getNodesMap().get(node.getId());
NodeStats stats = nodesStats.getNodesMap().get(node.id()); NodeStats stats = nodesStats.getNodesMap().get(node.getId());
table.startRow(); table.startRow();
table.addCell(fullId ? node.id() : Strings.substring(node.getId(), 0, 4)); table.addCell(fullId ? node.getId() : Strings.substring(node.getId(), 0, 4));
table.addCell(info == null ? null : info.getProcess().getId()); table.addCell(info == null ? null : info.getProcess().getId());
table.addCell(node.getHostName()); table.addCell(node.getHostName());
table.addCell(node.getHostAddress()); table.addCell(node.getHostAddress());
if (node.address() instanceof InetSocketTransportAddress) { if (node.getAddress() instanceof InetSocketTransportAddress) {
table.addCell(((InetSocketTransportAddress) node.address()).address().getPort()); table.addCell(((InetSocketTransportAddress) node.getAddress()).address().getPort());
} else { } else {
table.addCell("-"); table.addCell("-");
} }

View File

@ -544,7 +544,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
final SearchContext createContext(ShardSearchRequest request, @Nullable Engine.Searcher searcher) throws IOException { final SearchContext createContext(ShardSearchRequest request, @Nullable Engine.Searcher searcher) throws IOException {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.getShard(request.shardId().getId()); IndexShard indexShard = indexService.getShard(request.shardId().getId());
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().id(), indexShard.shardId()); SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().getId(), indexShard.shardId());
Engine.Searcher engineSearcher = searcher == null ? indexShard.acquireSearcher("search") : searcher; Engine.Searcher engineSearcher = searcher == null ? indexShard.acquireSearcher("search") : searcher;

View File

@ -301,9 +301,9 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
for (int shard = 0; shard < snapshotIndexMetaData.getNumberOfShards(); shard++) { for (int shard = 0; shard < snapshotIndexMetaData.getNumberOfShards(); shard++) {
if (!ignoreShards.contains(shard)) { if (!ignoreShards.contains(shard)) {
shardsBuilder.put(new ShardId(renamedIndex, shard), new RestoreInProgress.ShardRestoreStatus(clusterService.state().nodes().localNodeId())); shardsBuilder.put(new ShardId(renamedIndex, shard), new RestoreInProgress.ShardRestoreStatus(clusterService.state().nodes().getLocalNodeId()));
} else { } else {
shardsBuilder.put(new ShardId(renamedIndex, shard), new RestoreInProgress.ShardRestoreStatus(clusterService.state().nodes().localNodeId(), RestoreInProgress.State.FAILURE)); shardsBuilder.put(new ShardId(renamedIndex, shard), new RestoreInProgress.ShardRestoreStatus(clusterService.state().nodes().getLocalNodeId(), RestoreInProgress.State.FAILURE));
} }
} }
} }
@ -486,8 +486,8 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
public void indexShardRestoreCompleted(SnapshotId snapshotId, ShardId shardId) { public void indexShardRestoreCompleted(SnapshotId snapshotId, ShardId shardId) {
logger.trace("[{}] successfully restored shard [{}]", snapshotId, shardId); logger.trace("[{}] successfully restored shard [{}]", snapshotId, shardId);
UpdateIndexShardRestoreStatusRequest request = new UpdateIndexShardRestoreStatusRequest(snapshotId, shardId, UpdateIndexShardRestoreStatusRequest request = new UpdateIndexShardRestoreStatusRequest(snapshotId, shardId,
new ShardRestoreStatus(clusterService.state().nodes().localNodeId(), RestoreInProgress.State.SUCCESS)); new ShardRestoreStatus(clusterService.state().nodes().getLocalNodeId(), RestoreInProgress.State.SUCCESS));
transportService.sendRequest(clusterService.state().nodes().masterNode(), transportService.sendRequest(clusterService.state().nodes().getMasterNode(),
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
} }
@ -760,8 +760,8 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
public void failRestore(SnapshotId snapshotId, ShardId shardId) { public void failRestore(SnapshotId snapshotId, ShardId shardId) {
logger.debug("[{}] failed to restore shard [{}]", snapshotId, shardId); logger.debug("[{}] failed to restore shard [{}]", snapshotId, shardId);
UpdateIndexShardRestoreStatusRequest request = new UpdateIndexShardRestoreStatusRequest(snapshotId, shardId, UpdateIndexShardRestoreStatusRequest request = new UpdateIndexShardRestoreStatusRequest(snapshotId, shardId,
new ShardRestoreStatus(clusterService.state().nodes().localNodeId(), RestoreInProgress.State.FAILURE)); new ShardRestoreStatus(clusterService.state().nodes().getLocalNodeId(), RestoreInProgress.State.FAILURE));
transportService.sendRequest(clusterService.state().nodes().masterNode(), transportService.sendRequest(clusterService.state().nodes().getMasterNode(),
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
} }

View File

@ -108,13 +108,13 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
this.transportService = transportService; this.transportService = transportService;
this.clusterService = clusterService; this.clusterService = clusterService;
this.threadPool = threadPool; this.threadPool = threadPool;
if (DiscoveryNode.dataNode(settings)) { if (DiscoveryNode.isDataNode(settings)) {
// this is only useful on the nodes that can hold data // this is only useful on the nodes that can hold data
// addLast to make sure that Repository will be created before snapshot // addLast to make sure that Repository will be created before snapshot
clusterService.addLast(this); clusterService.addLast(this);
} }
if (DiscoveryNode.masterNode(settings)) { if (DiscoveryNode.isMasterNode(settings)) {
// This needs to run only on nodes that can become masters // This needs to run only on nodes that can become masters
transportService.registerRequestHandler(UPDATE_SNAPSHOT_ACTION_NAME, UpdateIndexShardSnapshotStatusRequest::new, ThreadPool.Names.SAME, new UpdateSnapshotStateRequestHandler()); transportService.registerRequestHandler(UPDATE_SNAPSHOT_ACTION_NAME, UpdateIndexShardSnapshotStatusRequest::new, ThreadPool.Names.SAME, new UpdateSnapshotStateRequestHandler());
} }
@ -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);
} }
@ -207,7 +207,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
// snapshots in the future // snapshots in the future
Map<SnapshotId, Map<ShardId, IndexShardSnapshotStatus>> newSnapshots = new HashMap<>(); Map<SnapshotId, Map<ShardId, IndexShardSnapshotStatus>> newSnapshots = new HashMap<>();
// Now go through all snapshots and update existing or create missing // Now go through all snapshots and update existing or create missing
final String localNodeId = clusterService.localNode().id(); final String localNodeId = clusterService.localNode().getId();
if (snapshotsInProgress != null) { if (snapshotsInProgress != null) {
for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) {
if (entry.state() == SnapshotsInProgress.State.STARTED) { if (entry.state() == SnapshotsInProgress.State.STARTED) {
@ -255,12 +255,12 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
case DONE: case DONE:
logger.debug("[{}] trying to cancel snapshot on the shard [{}] that is already done, updating status on the master", entry.snapshotId(), shard.key); logger.debug("[{}] trying to cancel snapshot on the shard [{}] that is already done, updating status on the master", entry.snapshotId(), shard.key);
updateIndexShardSnapshotStatus(entry.snapshotId(), shard.key, updateIndexShardSnapshotStatus(entry.snapshotId(), shard.key,
new SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotsInProgress.State.SUCCESS)); new SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.SUCCESS));
break; break;
case FAILURE: case FAILURE:
logger.debug("[{}] trying to cancel snapshot on the shard [{}] that has already failed, updating status on the master", entry.snapshotId(), shard.key); logger.debug("[{}] trying to cancel snapshot on the shard [{}] that has already failed, updating status on the master", entry.snapshotId(), shard.key);
updateIndexShardSnapshotStatus(entry.snapshotId(), shard.key, updateIndexShardSnapshotStatus(entry.snapshotId(), shard.key,
new SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotsInProgress.State.FAILED, snapshotStatus.failure())); new SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.FAILED, snapshotStatus.failure()));
break; break;
default: default:
throw new IllegalStateException("Unknown snapshot shard stage " + snapshotStatus.stage()); throw new IllegalStateException("Unknown snapshot shard stage " + snapshotStatus.stage());
@ -382,12 +382,12 @@ public class SnapshotShardsService extends AbstractLifecycleComponent<SnapshotSh
// but we think the shard is done - we need to make new master know that the shard is done // but we think the shard is done - we need to make new master know that the shard is done
logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard is done locally, updating status on the master", snapshot.snapshotId(), shardId); logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard is done locally, updating status on the master", snapshot.snapshotId(), shardId);
updateIndexShardSnapshotStatus(snapshot.snapshotId(), shardId, updateIndexShardSnapshotStatus(snapshot.snapshotId(), shardId,
new SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotsInProgress.State.SUCCESS)); new SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.SUCCESS));
} else if (localShard.getValue().stage() == IndexShardSnapshotStatus.Stage.FAILURE) { } else if (localShard.getValue().stage() == IndexShardSnapshotStatus.Stage.FAILURE) {
// but we think the shard failed - we need to make new master know that the shard failed // but we think the shard failed - we need to make new master know that the shard failed
logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard failed locally, updating status on master", snapshot.snapshotId(), shardId); logger.debug("[{}] new master thinks the shard [{}] is not completed but the shard failed locally, updating status on master", snapshot.snapshotId(), shardId);
updateIndexShardSnapshotStatus(snapshot.snapshotId(), shardId, updateIndexShardSnapshotStatus(snapshot.snapshotId(), shardId,
new SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().localNodeId(), SnapshotsInProgress.State.FAILED, localShardStatus.failure())); new SnapshotsInProgress.ShardSnapshotStatus(event.state().nodes().getLocalNodeId(), SnapshotsInProgress.State.FAILED, localShardStatus.failure()));
} }
} }
@ -477,10 +477,10 @@ 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().masterNode(), transportService.sendRequest(clusterService.state().nodes().getMasterNode(),
UPDATE_SNAPSHOT_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME); UPDATE_SNAPSHOT_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
} }
} catch (Throwable t) { } catch (Throwable t) {

View File

@ -113,7 +113,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
this.repositoriesService = repositoriesService; this.repositoriesService = repositoriesService;
this.threadPool = threadPool; this.threadPool = threadPool;
if (DiscoveryNode.masterNode(settings)) { if (DiscoveryNode.isMasterNode(settings)) {
// addLast to make sure that Repository will be created before snapshot // addLast to make sure that Repository will be created before snapshot
clusterService.addLast(this); clusterService.addLast(this);
} }
@ -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;

View File

@ -231,7 +231,7 @@ public class TaskManager extends AbstractComponent implements ClusterStateListen
TaskId taskId = banIterator.next(); TaskId taskId = banIterator.next();
if (lastDiscoveryNodes.nodeExists(taskId.getNodeId()) == false) { if (lastDiscoveryNodes.nodeExists(taskId.getNodeId()) == false) {
logger.debug("Removing ban for the parent [{}] on the node [{}], reason: the parent node is gone", taskId, logger.debug("Removing ban for the parent [{}] on the node [{}], reason: the parent node is gone", taskId,
event.state().getNodes().localNode()); event.state().getNodes().getLocalNode());
banIterator.remove(); banIterator.remove();
} }
} }

View File

@ -45,7 +45,7 @@ public class ConnectTransportException extends ActionTransportException {
} }
public ConnectTransportException(DiscoveryNode node, String msg, String action, Throwable cause) { public ConnectTransportException(DiscoveryNode node, String msg, String action, Throwable cause) {
super(node == null ? null : node.name(), node == null ? null : node.address(), action, msg, cause); super(node == null ? null : node.getName(), node == null ? null : node.getAddress(), action, msg, cause);
this.node = node; this.node = node;
} }

View File

@ -30,7 +30,7 @@ import java.io.IOException;
public class ReceiveTimeoutTransportException extends ActionTransportException { public class ReceiveTimeoutTransportException extends ActionTransportException {
public ReceiveTimeoutTransportException(DiscoveryNode node, String action, String msg) { public ReceiveTimeoutTransportException(DiscoveryNode node, String action, String msg) {
super(node.name(), node.address(), action, msg, null); super(node.getName(), node.getAddress(), action, msg, null);
} }
public ReceiveTimeoutTransportException(StreamInput in) throws IOException { public ReceiveTimeoutTransportException(StreamInput in) throws IOException {

View File

@ -31,7 +31,7 @@ import java.io.IOException;
public class SendRequestTransportException extends ActionTransportException implements ElasticsearchWrapperException { public class SendRequestTransportException extends ActionTransportException implements ElasticsearchWrapperException {
public SendRequestTransportException(DiscoveryNode node, String action, Throwable cause) { public SendRequestTransportException(DiscoveryNode node, String action, Throwable cause) {
super(node == null ? null : node.name(), node == null ? null : node.address(), action, cause); super(node == null ? null : node.getName(), node == null ? null : node.getAddress(), action, cause);
} }
public SendRequestTransportException(StreamInput in) throws IOException { public SendRequestTransportException(StreamInput in) throws IOException {

View File

@ -873,7 +873,7 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
if (t instanceof RemoteTransportException) { if (t instanceof RemoteTransportException) {
return (RemoteTransportException) t; return (RemoteTransportException) t;
} }
return new RemoteTransportException(localNode.name(), localNode.getAddress(), action, t); return new RemoteTransportException(localNode.getName(), localNode.getAddress(), action, t);
} }
protected void processException(final TransportResponseHandler handler, final RemoteTransportException rtx) { protected void processException(final TransportResponseHandler handler, final RemoteTransportException rtx) {

View File

@ -175,7 +175,7 @@ public class LocalTransport extends AbstractLifecycleComponent<Transport> implem
if (connectedNodes.containsKey(node)) { if (connectedNodes.containsKey(node)) {
return; return;
} }
final LocalTransport targetTransport = transports.get(node.address()); final LocalTransport targetTransport = transports.get(node.getAddress());
if (targetTransport == null) { if (targetTransport == null) {
throw new ConnectTransportException(node, "Failed to connect"); throw new ConnectTransportException(node, "Failed to connect");
} }
@ -202,7 +202,7 @@ public class LocalTransport extends AbstractLifecycleComponent<Transport> implem
@Override @Override
public void sendRequest(final DiscoveryNode node, final long requestId, final String action, final TransportRequest request, public void sendRequest(final DiscoveryNode node, final long requestId, final String action, final TransportRequest request,
TransportRequestOptions options) throws IOException, TransportException { TransportRequestOptions options) throws IOException, TransportException {
final Version version = Version.smallest(node.version(), this.version); final Version version = Version.smallest(node.getVersion(), this.version);
try (BytesStreamOutput stream = new BytesStreamOutput()) { try (BytesStreamOutput stream = new BytesStreamOutput()) {
stream.setVersion(version); stream.setVersion(version);

View File

@ -887,7 +887,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
// we pick the smallest of the 2, to support both backward and forward compatibility // we pick the smallest of the 2, to support both backward and forward compatibility
// note, this is the only place we need to do this, since from here on, we use the serialized version // note, this is the only place we need to do this, since from here on, we use the serialized version
// as the version to use also when the node receiving this request will send the response with // as the version to use also when the node receiving this request will send the response with
Version version = Version.smallest(this.version, node.version()); Version version = Version.smallest(this.version, node.getVersion());
stream.setVersion(version); stream.setVersion(version);
threadPool.getThreadContext().writeTo(stream); threadPool.getThreadContext().writeTo(stream);
@ -900,7 +900,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
// more explicit). // more explicit).
if (request instanceof BytesTransportRequest) { if (request instanceof BytesTransportRequest) {
BytesTransportRequest bRequest = (BytesTransportRequest) request; BytesTransportRequest bRequest = (BytesTransportRequest) request;
assert node.version().equals(bRequest.version()); assert node.getVersion().equals(bRequest.version());
bRequest.writeThin(stream); bRequest.writeThin(stream);
stream.close(); stream.close();
bytes = bStream.bytes(); bytes = bStream.bytes();
@ -951,7 +951,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
globalLock.readLock().lock(); globalLock.readLock().lock();
try { try {
try (Releasable ignored = connectionLock.acquire(node.id())) { try (Releasable ignored = connectionLock.acquire(node.getId())) {
if (!lifecycle.started()) { if (!lifecycle.started()) {
throw new IllegalStateException("can't add nodes to a stopped transport"); throw new IllegalStateException("can't add nodes to a stopped transport");
} }
@ -993,7 +993,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
} }
protected NodeChannels connectToChannelsLight(DiscoveryNode node) { protected NodeChannels connectToChannelsLight(DiscoveryNode node) {
InetSocketAddress address = ((InetSocketTransportAddress) node.address()).address(); InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address();
ChannelFuture connect = clientBootstrap.connect(address); ChannelFuture connect = clientBootstrap.connect(address);
connect.awaitUninterruptibly((long) (connectTimeout.millis() * 1.5)); connect.awaitUninterruptibly((long) (connectTimeout.millis() * 1.5));
if (!connect.isSuccess()) { if (!connect.isSuccess()) {
@ -1011,7 +1011,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
ChannelFuture[] connectReg = new ChannelFuture[nodeChannels.reg.length]; ChannelFuture[] connectReg = new ChannelFuture[nodeChannels.reg.length];
ChannelFuture[] connectState = new ChannelFuture[nodeChannels.state.length]; ChannelFuture[] connectState = new ChannelFuture[nodeChannels.state.length];
ChannelFuture[] connectPing = new ChannelFuture[nodeChannels.ping.length]; ChannelFuture[] connectPing = new ChannelFuture[nodeChannels.ping.length];
InetSocketAddress address = ((InetSocketTransportAddress) node.address()).address(); InetSocketAddress address = ((InetSocketTransportAddress) node.getAddress()).address();
for (int i = 0; i < connectRecovery.length; i++) { for (int i = 0; i < connectRecovery.length; i++) {
connectRecovery[i] = clientBootstrap.connect(address); connectRecovery[i] = clientBootstrap.connect(address);
} }
@ -1109,7 +1109,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
@Override @Override
public void disconnectFromNode(DiscoveryNode node) { public void disconnectFromNode(DiscoveryNode node) {
try (Releasable ignored = connectionLock.acquire(node.id())) { try (Releasable ignored = connectionLock.acquire(node.getId())) {
NodeChannels nodeChannels = connectedNodes.remove(node); NodeChannels nodeChannels = connectedNodes.remove(node);
if (nodeChannels != null) { if (nodeChannels != null) {
try { try {
@ -1131,7 +1131,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
// check outside of the lock // check outside of the lock
NodeChannels nodeChannels = connectedNodes.get(node); NodeChannels nodeChannels = connectedNodes.get(node);
if (nodeChannels != null && nodeChannels.hasChannel(channel)) { if (nodeChannels != null && nodeChannels.hasChannel(channel)) {
try (Releasable ignored = connectionLock.acquire(node.id())) { try (Releasable ignored = connectionLock.acquire(node.getId())) {
nodeChannels = connectedNodes.get(node); nodeChannels = connectedNodes.get(node);
// check again within the connection lock, if its still applicable to remove it // check again within the connection lock, if its still applicable to remove it
if (nodeChannels != null && nodeChannels.hasChannel(channel)) { if (nodeChannels != null && nodeChannels.hasChannel(channel)) {

View File

@ -353,21 +353,21 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
for (DiscoveryNode discoNode : currentState.nodes()) { for (DiscoveryNode discoNode : currentState.nodes()) {
String markedTribeName = discoNode.getAttributes().get(TRIBE_NAME_SETTING.getKey()); String markedTribeName = discoNode.getAttributes().get(TRIBE_NAME_SETTING.getKey());
if (markedTribeName != null && markedTribeName.equals(tribeName)) { if (markedTribeName != null && markedTribeName.equals(tribeName)) {
if (tribeState.nodes().get(discoNode.id()) == null) { if (tribeState.nodes().get(discoNode.getId()) == null) {
clusterStateChanged = true; clusterStateChanged = true;
logger.info("[{}] removing node [{}]", tribeName, discoNode); logger.info("[{}] removing node [{}]", tribeName, discoNode);
nodes.remove(discoNode.id()); nodes.remove(discoNode.getId());
} }
} }
} }
// go over tribe nodes, and see if they need to be added // go over tribe nodes, and see if they need to be added
for (DiscoveryNode tribe : tribeState.nodes()) { for (DiscoveryNode tribe : tribeState.nodes()) {
if (currentState.nodes().get(tribe.id()) == null) { if (currentState.nodes().get(tribe.getId()) == null) {
// a new node, add it, but also add the tribe name to the attributes // a new node, add it, but also add the tribe name to the attributes
Map<String, String> tribeAttr = new HashMap<>(tribe.getAttributes()); Map<String, String> tribeAttr = new HashMap<>(tribe.getAttributes());
tribeAttr.put(TRIBE_NAME_SETTING.getKey(), tribeName); tribeAttr.put(TRIBE_NAME_SETTING.getKey(), tribeName);
DiscoveryNode discoNode = new DiscoveryNode(tribe.name(), tribe.id(), tribe.getHostName(), tribe.getHostAddress(), DiscoveryNode discoNode = new DiscoveryNode(tribe.getName(), tribe.getId(), tribe.getHostName(), tribe.getHostAddress(),
tribe.address(), unmodifiableMap(tribeAttr), tribe.getRoles(), tribe.version()); tribe.getAddress(), unmodifiableMap(tribeAttr), tribe.getRoles(), tribe.getVersion());
clusterStateChanged = true; clusterStateChanged = true;
logger.info("[{}] adding node [{}]", tribeName, discoNode); logger.info("[{}] adding node [{}]", tribeName, discoNode);
nodes.put(discoNode); nodes.put(discoNode);

View File

@ -188,7 +188,7 @@ public class TasksIT extends ESIntegTestCase {
logger.debug("number of shards, total: [{}], primaries: [{}] ", numberOfShards.totalNumShards, numberOfShards.numPrimaries); logger.debug("number of shards, total: [{}], primaries: [{}] ", numberOfShards.totalNumShards, numberOfShards.numPrimaries);
logger.debug("main events {}", numberOfEvents(RefreshAction.NAME, Tuple::v1)); logger.debug("main events {}", numberOfEvents(RefreshAction.NAME, Tuple::v1));
logger.debug("main event node {}", findEvents(RefreshAction.NAME, Tuple::v1).get(0).getNode().name()); logger.debug("main event node {}", findEvents(RefreshAction.NAME, Tuple::v1).get(0).getNode().getName());
logger.debug("[s] events {}", numberOfEvents(RefreshAction.NAME + "[s]", Tuple::v1)); logger.debug("[s] events {}", numberOfEvents(RefreshAction.NAME + "[s]", Tuple::v1));
logger.debug("[s][*] events {}", numberOfEvents(RefreshAction.NAME + "[s][*]", Tuple::v1)); logger.debug("[s][*] events {}", numberOfEvents(RefreshAction.NAME + "[s][*]", Tuple::v1));
logger.debug("nodes with the index {}", internalCluster().nodesInclude("test")); logger.debug("nodes with the index {}", internalCluster().nodesInclude("test"));
@ -436,7 +436,7 @@ public class TasksIT extends ESIntegTestCase {
DiscoveryNode node = internalCluster().getInstance(ClusterService.class, nodeName).localNode(); DiscoveryNode node = internalCluster().getInstance(ClusterService.class, nodeName).localNode();
RecordingTaskManagerListener listener = new RecordingTaskManagerListener(node, Strings.splitStringToArray(actionMasks, ',')); RecordingTaskManagerListener listener = new RecordingTaskManagerListener(node, Strings.splitStringToArray(actionMasks, ','));
((MockTaskManager) internalCluster().getInstance(TransportService.class, nodeName).getTaskManager()).addListener(listener); ((MockTaskManager) internalCluster().getInstance(TransportService.class, nodeName).getTaskManager()).addListener(listener);
RecordingTaskManagerListener oldListener = listeners.put(new Tuple<>(node.name(), actionMasks), listener); RecordingTaskManagerListener oldListener = listeners.put(new Tuple<>(node.getName(), actionMasks), listener);
assertNull(oldListener); assertNull(oldListener);
} }
} }

View File

@ -192,7 +192,7 @@ public class IndicesShardStoreRequestIT extends ESIntegTestCase {
for (IntObjectCursor<List<IndicesShardStoresResponse.StoreStatus>> shardStatus : shardStatuses) { for (IntObjectCursor<List<IndicesShardStoresResponse.StoreStatus>> shardStatus : shardStatuses) {
for (IndicesShardStoresResponse.StoreStatus status : shardStatus.value) { for (IndicesShardStoresResponse.StoreStatus status : shardStatus.value) {
if (corruptedShardIDMap.containsKey(shardStatus.key) if (corruptedShardIDMap.containsKey(shardStatus.key)
&& corruptedShardIDMap.get(shardStatus.key).contains(status.getNode().name())) { && corruptedShardIDMap.get(shardStatus.key).contains(status.getNode().getName())) {
assertThat(status.getLegacyVersion(), greaterThanOrEqualTo(0L)); assertThat(status.getLegacyVersion(), greaterThanOrEqualTo(0L));
assertThat(status.getStoreException(), notNullValue()); assertThat(status.getStoreException(), notNullValue());
} else { } else {

View File

@ -111,7 +111,7 @@ public class IndicesShardStoreResponseTests extends ESTestCase {
assertThat(eitherLegacyVersionOrAllocationIdSet, equalTo(true)); assertThat(eitherLegacyVersionOrAllocationIdSet, equalTo(true));
assertThat(storeInfo.containsKey("allocation"), equalTo(true)); assertThat(storeInfo.containsKey("allocation"), equalTo(true));
assertThat(((String) storeInfo.get("allocation")), equalTo(storeStatus.getAllocationStatus().value())); assertThat(((String) storeInfo.get("allocation")), equalTo(storeStatus.getAllocationStatus().value()));
assertThat(storeInfo.containsKey(storeStatus.getNode().id()), equalTo(true)); assertThat(storeInfo.containsKey(storeStatus.getNode().getId()), equalTo(true));
if (storeStatus.getStoreException() != null) { if (storeStatus.getStoreException() != null) {
assertThat(storeInfo.containsKey("store_exception"), equalTo(true)); assertThat(storeInfo.containsKey("store_exception"), equalTo(true));
} }

View File

@ -224,14 +224,14 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
totalIndexShards += numberOfShards; totalIndexShards += numberOfShards;
for (int j = 0; j < numberOfShards; j++) { for (int j = 0; j < numberOfShards; j++) {
final ShardId shardId = new ShardId(index, "_na_", ++shardIndex); final ShardId shardId = new ShardId(index, "_na_", ++shardIndex);
ShardRouting shard = TestShardRouting.newShardRouting(index, shardId.getId(), node.id(), true, ShardRoutingState.STARTED); ShardRouting shard = TestShardRouting.newShardRouting(index, shardId.getId(), node.getId(), true, ShardRoutingState.STARTED);
IndexShardRoutingTable.Builder indexShard = new IndexShardRoutingTable.Builder(shardId); IndexShardRoutingTable.Builder indexShard = new IndexShardRoutingTable.Builder(shardId);
indexShard.addShard(shard); indexShard.addShard(shard);
indexRoutingTable.addIndexShard(indexShard.build()); indexRoutingTable.addIndexShard(indexShard.build());
} }
} }
discoBuilder.localNodeId(newNode(0).id()); discoBuilder.localNodeId(newNode(0).getId());
discoBuilder.masterNodeId(newNode(numberOfNodes - 1).id()); discoBuilder.masterNodeId(newNode(numberOfNodes - 1).getId());
ClusterState.Builder stateBuilder = ClusterState.builder(new ClusterName(TEST_CLUSTER)); ClusterState.Builder stateBuilder = ClusterState.builder(new ClusterName(TEST_CLUSTER));
stateBuilder.nodes(discoBuilder); stateBuilder.nodes(discoBuilder);
final IndexMetaData.Builder indexMetaData = IndexMetaData.builder(index) final IndexMetaData.Builder indexMetaData = IndexMetaData.builder(index)
@ -318,9 +318,9 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
Request request = new Request(new String[]{TEST_INDEX}); Request request = new Request(new String[]{TEST_INDEX});
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<Response> listener = new PlainActionFuture<>();
DiscoveryNode masterNode = clusterService.state().nodes().masterNode(); DiscoveryNode masterNode = clusterService.state().nodes().getMasterNode();
DiscoveryNodes.Builder builder = DiscoveryNodes.builder(clusterService.state().getNodes()); DiscoveryNodes.Builder builder = DiscoveryNodes.builder(clusterService.state().getNodes());
builder.remove(masterNode.id()); builder.remove(masterNode.getId());
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(builder)); setState(clusterService, ClusterState.builder(clusterService.state()).nodes(builder));
@ -332,7 +332,7 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
ShardsIterator shardIt = clusterService.state().routingTable().allShards(new String[]{TEST_INDEX}); ShardsIterator shardIt = clusterService.state().routingTable().allShards(new String[]{TEST_INDEX});
Set<String> set = new HashSet<>(); Set<String> set = new HashSet<>();
for (ShardRouting shard : shardIt.asUnordered()) { for (ShardRouting shard : shardIt.asUnordered()) {
if (!shard.currentNodeId().equals(masterNode.id())) { if (!shard.currentNodeId().equals(masterNode.getId())) {
set.add(shard.currentNodeId()); set.add(shard.currentNodeId());
} }
} }
@ -403,9 +403,9 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
final boolean simulateFailedMasterNode = rarely(); final boolean simulateFailedMasterNode = rarely();
DiscoveryNode failedMasterNode = null; DiscoveryNode failedMasterNode = null;
if (simulateFailedMasterNode) { if (simulateFailedMasterNode) {
failedMasterNode = clusterService.state().nodes().masterNode(); failedMasterNode = clusterService.state().nodes().getMasterNode();
DiscoveryNodes.Builder builder = DiscoveryNodes.builder(clusterService.state().getNodes()); DiscoveryNodes.Builder builder = DiscoveryNodes.builder(clusterService.state().getNodes());
builder.remove(failedMasterNode.id()); builder.remove(failedMasterNode.getId());
builder.masterNodeId(null); builder.masterNodeId(null);
setState(clusterService, ClusterState.builder(clusterService.state()).nodes(builder)); setState(clusterService, ClusterState.builder(clusterService.state()).nodes(builder));
@ -453,7 +453,7 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
} }
} }
if (simulateFailedMasterNode) { if (simulateFailedMasterNode) {
totalShards += map.get(failedMasterNode.id()).size(); totalShards += map.get(failedMasterNode.getId()).size();
} }
Response response = listener.get(); Response response = listener.get();

Some files were not shown because too many files have changed in this diff Show More