Remove DiscoveryNodes#localNodeId in favour of existing DiscoveryNodes#getLocalNodeId
This commit is contained in:
parent
f8b5d1f5b0
commit
f26d05eac8
|
@ -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);
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -179,22 +179,13 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
|
||||||
return this.masterNodeId;
|
return this.masterNodeId;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the id of the local node
|
|
||||||
*
|
|
||||||
* @return id of the local node
|
|
||||||
*/
|
|
||||||
public String localNodeId() {
|
|
||||||
return this.localNodeId;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the id of the local node
|
* Get the id of the local node
|
||||||
*
|
*
|
||||||
* @return id of the local node
|
* @return id of the local node
|
||||||
*/
|
*/
|
||||||
public String getLocalNodeId() {
|
public String getLocalNodeId() {
|
||||||
return localNodeId();
|
return this.localNodeId;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -302,7 +293,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
|
||||||
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);
|
||||||
}
|
}
|
||||||
|
@ -595,7 +586,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
|
||||||
|
|
||||||
public Builder(DiscoveryNodes nodes) {
|
public Builder(DiscoveryNodes nodes) {
|
||||||
this.masterNodeId = nodes.getMasterNodeId();
|
this.masterNodeId = nodes.getMasterNodeId();
|
||||||
this.localNodeId = nodes.localNodeId();
|
this.localNodeId = nodes.getLocalNodeId();
|
||||||
this.nodes = ImmutableOpenMap.builder(nodes.getNodes());
|
this.nodes = ImmutableOpenMap.builder(nodes.getNodes());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
|
@ -158,7 +158,7 @@ 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.getId());
|
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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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().getId().equals(discoveryNodes.localNodeId())) {
|
if (pingResponse.node().getId().equals(discoveryNodes.getLocalNodeId())) {
|
||||||
// that's us, ignore
|
// that's us, ignore
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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");
|
||||||
}
|
}
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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,7 +486,7 @@ 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().masterNode(),
|
||||||
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
|
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
|
||||||
}
|
}
|
||||||
|
@ -760,7 +760,7 @@ 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().masterNode(),
|
||||||
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
|
UPDATE_RESTORE_ACTION_NAME, request, EmptyTransportResponseHandler.INSTANCE_SAME);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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()));
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -376,7 +376,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
final List<CapturingTransport.CapturedRequest> capturedRequests = transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
|
final List<CapturingTransport.CapturedRequest> capturedRequests = transport.getCapturedRequestsByTargetNodeAndClear().get(primaryNodeId);
|
||||||
assertThat(capturedRequests, notNullValue());
|
assertThat(capturedRequests, notNullValue());
|
||||||
assertThat(capturedRequests.size(), equalTo(1));
|
assertThat(capturedRequests.size(), equalTo(1));
|
||||||
if (clusterService.state().nodes().localNodeId().equals(primaryNodeId)) {
|
if (clusterService.state().nodes().getLocalNodeId().equals(primaryNodeId)) {
|
||||||
assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
|
assertThat(capturedRequests.get(0).action, equalTo("testAction[p]"));
|
||||||
assertPhase(task, "waiting_on_primary");
|
assertPhase(task, "waiting_on_primary");
|
||||||
} else {
|
} else {
|
||||||
|
@ -722,7 +722,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
||||||
assertEquals(request.shardId, replicationRequest.shardId);
|
assertEquals(request.shardId, replicationRequest.shardId);
|
||||||
}
|
}
|
||||||
|
|
||||||
String localNodeId = clusterService.state().getNodes().localNodeId();
|
String localNodeId = clusterService.state().getNodes().getLocalNodeId();
|
||||||
// no request was sent to the local node
|
// no request was sent to the local node
|
||||||
assertThat(nodesSentTo.keySet(), not(hasItem(localNodeId)));
|
assertThat(nodesSentTo.keySet(), not(hasItem(localNodeId)));
|
||||||
|
|
||||||
|
|
|
@ -134,7 +134,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
|
||||||
|
|
||||||
// Check nodes
|
// Check nodes
|
||||||
assertThat(clusterStateFromDiffs.nodes().getNodes(), equalTo(clusterState.nodes().getNodes()));
|
assertThat(clusterStateFromDiffs.nodes().getNodes(), equalTo(clusterState.nodes().getNodes()));
|
||||||
assertThat(clusterStateFromDiffs.nodes().localNodeId(), equalTo(previousClusterStateFromDiffs.nodes().localNodeId()));
|
assertThat(clusterStateFromDiffs.nodes().getLocalNodeId(), equalTo(previousClusterStateFromDiffs.nodes().getLocalNodeId()));
|
||||||
assertThat(clusterStateFromDiffs.nodes().getNodes(), equalTo(clusterState.nodes().getNodes()));
|
assertThat(clusterStateFromDiffs.nodes().getNodes(), equalTo(clusterState.nodes().getNodes()));
|
||||||
for (ObjectCursor<String> node : clusterStateFromDiffs.nodes().getNodes().keys()) {
|
for (ObjectCursor<String> node : clusterStateFromDiffs.nodes().getNodes().keys()) {
|
||||||
DiscoveryNode node1 = clusterState.nodes().get(node.value);
|
DiscoveryNode node1 = clusterState.nodes().get(node.value);
|
||||||
|
|
|
@ -124,7 +124,7 @@ public class DiscoveryNodesTests extends ESTestCase {
|
||||||
LOCAL("_local") {
|
LOCAL("_local") {
|
||||||
@Override
|
@Override
|
||||||
Set<String> matchingNodeIds(DiscoveryNodes nodes) {
|
Set<String> matchingNodeIds(DiscoveryNodes nodes) {
|
||||||
return Collections.singleton(nodes.localNodeId());
|
return Collections.singleton(nodes.getLocalNodeId());
|
||||||
}
|
}
|
||||||
}, ELECTED_MASTER("_master") {
|
}, ELECTED_MASTER("_master") {
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -123,7 +123,7 @@ public class ClusterServiceTests extends ESTestCase {
|
||||||
ClusterState state = timedClusterService.state();
|
ClusterState state = timedClusterService.state();
|
||||||
final DiscoveryNodes nodes = state.nodes();
|
final DiscoveryNodes nodes = state.nodes();
|
||||||
final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(nodes)
|
final DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(nodes)
|
||||||
.masterNodeId(makeMaster ? nodes.localNodeId() : null);
|
.masterNodeId(makeMaster ? nodes.getLocalNodeId() : null);
|
||||||
state = ClusterState.builder(state).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK)
|
state = ClusterState.builder(state).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK)
|
||||||
.nodes(nodesBuilder).build();
|
.nodes(nodesBuilder).build();
|
||||||
setState(timedClusterService, state);
|
setState(timedClusterService, state);
|
||||||
|
|
|
@ -49,8 +49,8 @@ public class SimpleNodesInfoIT extends ESIntegTestCase {
|
||||||
ClusterHealthResponse clusterHealth = client().admin().cluster().prepareHealth().setWaitForGreenStatus().setWaitForNodes("2").get();
|
ClusterHealthResponse clusterHealth = client().admin().cluster().prepareHealth().setWaitForGreenStatus().setWaitForNodes("2").get();
|
||||||
logger.info("--> done cluster_health, status {}", clusterHealth.getStatus());
|
logger.info("--> done cluster_health, status {}", clusterHealth.getStatus());
|
||||||
|
|
||||||
String server1NodeId = internalCluster().getInstance(ClusterService.class, node_1).state().nodes().localNodeId();
|
String server1NodeId = internalCluster().getInstance(ClusterService.class, node_1).state().nodes().getLocalNodeId();
|
||||||
String server2NodeId = internalCluster().getInstance(ClusterService.class, node_2).state().nodes().localNodeId();
|
String server2NodeId = internalCluster().getInstance(ClusterService.class, node_2).state().nodes().getLocalNodeId();
|
||||||
logger.info("--> started nodes: {} and {}", server1NodeId, server2NodeId);
|
logger.info("--> started nodes: {} and {}", server1NodeId, server2NodeId);
|
||||||
|
|
||||||
NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().execute().actionGet();
|
NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().execute().actionGet();
|
||||||
|
@ -93,8 +93,8 @@ public class SimpleNodesInfoIT extends ESIntegTestCase {
|
||||||
ClusterHealthResponse clusterHealth = client().admin().cluster().prepareHealth().setWaitForGreenStatus().setWaitForNodes("2").get();
|
ClusterHealthResponse clusterHealth = client().admin().cluster().prepareHealth().setWaitForGreenStatus().setWaitForNodes("2").get();
|
||||||
logger.info("--> done cluster_health, status {}", clusterHealth.getStatus());
|
logger.info("--> done cluster_health, status {}", clusterHealth.getStatus());
|
||||||
|
|
||||||
String server1NodeId = internalCluster().getInstance(ClusterService.class, node_1).state().nodes().localNodeId();
|
String server1NodeId = internalCluster().getInstance(ClusterService.class, node_1).state().nodes().getLocalNodeId();
|
||||||
String server2NodeId = internalCluster().getInstance(ClusterService.class, node_2).state().nodes().localNodeId();
|
String server2NodeId = internalCluster().getInstance(ClusterService.class, node_2).state().nodes().getLocalNodeId();
|
||||||
logger.info("--> started nodes: {} and {}", server1NodeId, server2NodeId);
|
logger.info("--> started nodes: {} and {}", server1NodeId, server2NodeId);
|
||||||
|
|
||||||
NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().execute().actionGet();
|
NodesInfoResponse response = client().admin().cluster().prepareNodesInfo().execute().actionGet();
|
||||||
|
|
Loading…
Reference in New Issue