public Builder(DiscoveryNodes nodes) { this.masterNodeId = nodes.getMasterNodeId(); this.localNodeId = nodes.getLocalNodeId(); this.nodes = ImmutableOpenMap.builder(nodes.getNodes()); }
/** * Removes shard entries from the failed shards cache that are no longer allocated to this node by the master. * Sends shard failures for shards that are marked as actively allocated to this node but don't actually exist on the node. * Resends shard failures for shards that are still marked as allocated to this node but previously failed. * * @param state new cluster state */ private void updateFailedShardsCache(final ClusterState state) { RoutingNode localRoutingNode = state.getRoutingNodes().node(state.nodes().getLocalNodeId()); if (localRoutingNode == null) { failedShardsCache.clear(); return; } DiscoveryNode masterNode = state.nodes().getMasterNode(); // remove items from cache which are not in our routing table anymore and resend failures that have not executed on master yet for (Iterator<Map.Entry<ShardId, ShardRouting>> iterator = failedShardsCache.entrySet().iterator(); iterator.hasNext(); ) { ShardRouting failedShardRouting = iterator.next().getValue(); ShardRouting matchedRouting = localRoutingNode.getByShardId(failedShardRouting.shardId()); if (matchedRouting == null || matchedRouting.isSameAllocation(failedShardRouting) == false) { iterator.remove(); } else { if (masterNode != null) { // TODO: can we remove this? Is resending shard failures the responsibility of shardStateAction? String message = "master " + masterNode + " has not removed previously failed shard. resending shard failure"; logger.trace("[{}] re-sending failed shard [{}], reason [{}]", matchedRouting.shardId(), matchedRouting, message); shardStateAction.localShardFailed(matchedRouting, message, null, SHARD_STATE_ACTION_LISTENER, state); } } } }
public GroupShardsIterator<ShardIterator> searchShards(ClusterState clusterState, String[] concreteIndices, @Nullable Map<String, Set<String>> routing, @Nullable String preference, @Nullable ResponseCollectorService collectorService, @Nullable Map<String, Long> nodeCounts) { final Set<IndexShardRoutingTable> shards = computeTargetedShards(clusterState, concreteIndices, routing); final Set<ShardIterator> set = new HashSet<>(shards.size()); for (IndexShardRoutingTable shard : shards) { ShardIterator iterator = preferenceActiveShardIterator(shard, clusterState.nodes().getLocalNodeId(), clusterState.nodes(), preference, collectorService, nodeCounts); if (iterator != null) { set.add(iterator); } } return new GroupShardsIterator<>(new ArrayList<>(set)); }
public ShardIterator getShards(ClusterState clusterState, String index, int shardId, @Nullable String preference) { final IndexShardRoutingTable indexShard = clusterState.getRoutingTable().shardRoutingTable(index, shardId); return preferenceActiveShardIterator(indexShard, clusterState.nodes().getLocalNodeId(), clusterState.nodes(), preference, null, null); }
public ShardIterator getShards(ClusterState clusterState, String index, String id, @Nullable String routing, @Nullable String preference) { return preferenceActiveShardIterator(shards(clusterState, index, id, routing), clusterState.nodes().getLocalNodeId(), clusterState.nodes(), preference, null, null); }
private void createIndices(final ClusterState state) { RoutingNode localRoutingNode = state.getRoutingNodes().node(state.nodes().getLocalNodeId()); if (localRoutingNode == null) { return; nodeMappingRefreshAction.nodeMappingRefresh(state.nodes().getMasterNode(), new NodeMappingRefreshAction.NodeMappingRefreshRequest(indexMetaData.getIndex().getName(), indexMetaData.getIndexUUID(), state.nodes().getLocalNodeId()) );
final String localNodeId = event.state().nodes().getLocalNodeId(); final DiscoveryNode masterNode = event.state().nodes().getMasterNode(); final Map<Snapshot, Map<String, IndexId>> snapshotIndices = new HashMap<>();
/** * Notifies master about shards that don't exist but are supposed to be active on this node. * * @param state new cluster state */ private void failMissingShards(final ClusterState state) { RoutingNode localRoutingNode = state.getRoutingNodes().node(state.nodes().getLocalNodeId()); if (localRoutingNode == null) { return; } for (final ShardRouting shardRouting : localRoutingNode) { ShardId shardId = shardRouting.shardId(); if (shardRouting.initializing() == false && failedShardsCache.containsKey(shardId) == false && indicesService.getShardOrNull(shardId) == null) { // the master thinks we are active, but we don't have this shard at all, mark it as failed sendFailShard(shardRouting, "master marked shard as active, but shard has not been created, mark shard as failed", null, state); } } }
public static Set<Index> getRelevantIndicesOnDataOnlyNode(ClusterState state, ClusterState previousState, Set<Index> previouslyWrittenIndices) { RoutingNode newRoutingNode = state.getRoutingNodes().node(state.nodes().getLocalNodeId()); if (newRoutingNode == null) { throw new IllegalStateException("cluster state does not contain this node - cannot write index meta state"); } Set<Index> indices = new HashSet<>(); for (ShardRouting routing : newRoutingNode) { indices.add(routing.index()); } // we have to check the meta data also: closed indices will not appear in the routing table, but we must still write the state if // we have it written on disk previously for (IndexMetaData indexMetaData : state.metaData()) { boolean isOrWasClosed = indexMetaData.getState().equals(IndexMetaData.State.CLOSE); // if the index is open we might still have to write the state if it just transitioned from closed to open // so we have to check for that as well. IndexMetaData previousMetaData = previousState.metaData().index(indexMetaData.getIndex()); if (previousMetaData != null) { isOrWasClosed = isOrWasClosed || previousMetaData.getState().equals(IndexMetaData.State.CLOSE); } if (previouslyWrittenIndices.contains(indexMetaData.getIndex()) && isOrWasClosed) { indices.add(indexMetaData.getIndex()); } } return indices; }
final String localNodeId = state.nodes().getLocalNodeId(); assert localNodeId != null;
private void handleAnotherMaster(ClusterState localClusterState, final DiscoveryNode otherMaster, long otherClusterStateVersion, String reason) { assert localClusterState.nodes().isLocalNodeElectedMaster() : "handleAnotherMaster called but current node is not a master"; assert Thread.holdsLock(stateMutex); if (otherClusterStateVersion > localClusterState.version()) { rejoin("zen-disco-discovered another master with a new cluster_state [" + otherMaster + "][" + reason + "]"); } else { // TODO: do this outside mutex logger.warn("discovered [{}] which is also master but with an older cluster_state, telling [{}] to rejoin the cluster ([{}])", otherMaster, otherMaster, reason); try { // make sure we're connected to this node (connect to node does nothing if we're already connected) // 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) transportService.connectToNode(otherMaster); transportService.sendRequest(otherMaster, DISCOVERY_REJOIN_ACTION_NAME, new RejoinClusterRequest(localClusterState.nodes().getLocalNodeId()), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { @Override public void handleException(TransportException exp) { logger.warn(() -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), exp); } }); } catch (Exception e) { logger.warn(() -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), e); } } }
String localNodeId = event.state().getNodes().getLocalNodeId(); Set<Long> notVisitedTasks = new HashSet<>(runningTasks.keySet()); if (tasks != null) {
final String localNodeId = event.state().nodes().getLocalNodeId(); final DiscoveryNode masterNode = event.state().nodes().getMasterNode(); for (SnapshotsInProgress.Entry snapshot : snapshotsInProgress.entries()) {
for (String nodeId : nodes) { if (nodeId.equals("_local")) { String localNodeId = getLocalNodeId(); if (localNodeId != null) { resolvedNodesIds.add(localNodeId);
final String localNodeId = state.nodes().getLocalNodeId(); assert localNodeId != null;
private void createOrUpdateShards(final ClusterState state) { RoutingNode localRoutingNode = state.getRoutingNodes().node(state.nodes().getLocalNodeId()); if (localRoutingNode == null) { return; } DiscoveryNodes nodes = state.nodes(); RoutingTable routingTable = state.routingTable(); for (final ShardRouting shardRouting : localRoutingNode) { ShardId shardId = shardRouting.shardId(); if (failedShardsCache.containsKey(shardId) == false) { AllocatedIndex<? extends Shard> indexService = indicesService.indexService(shardId.getIndex()); assert indexService != null : "index " + shardId.getIndex() + " should have been created by createIndices"; Shard shard = indexService.getShardOrNull(shardId.id()); if (shard == null) { assert shardRouting.initializing() : shardRouting + " should have been removed by failMissingShards"; createShard(nodes, routingTable, shardRouting, state); } else { updateShard(nodes, shardRouting, shard, routingTable, state); } } } }
final ClusterState previousState = event.previousState(); final ClusterState state = event.state(); final String localNodeId = state.nodes().getLocalNodeId(); assert localNodeId != null;
final String localNodeId = event.state().nodes().getLocalNodeId(); RoutingNode localRoutingNode = event.state().getRoutingNodes().node(localNodeId); if (localRoutingNode != null) {
nodeMappingRefreshAction.nodeMappingRefresh(state.nodes().getMasterNode(), new NodeMappingRefreshAction.NodeMappingRefreshRequest(newIndexMetaData.getIndex().getName(), newIndexMetaData.getIndexUUID(), state.nodes().getLocalNodeId()) ); RoutingNode localRoutingNode = state.getRoutingNodes().node(state.nodes().getLocalNodeId()); if (localRoutingNode != null) { for (final ShardRouting shardRouting : localRoutingNode) {
private ClusterState.Builder becomeMasterAndTrimConflictingNodes(ClusterState currentState, List<DiscoveryNode> joiningNodes) { assert currentState.nodes().getMasterNodeId() == null : currentState; DiscoveryNodes currentNodes = currentState.nodes(); DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(currentNodes); nodesBuilder.masterNodeId(currentState.nodes().getLocalNodeId()); for (final DiscoveryNode joiningNode : joiningNodes) { final DiscoveryNode nodeWithSameId = nodesBuilder.get(joiningNode.getId()); if (nodeWithSameId != null && nodeWithSameId.equals(joiningNode) == false) { logger.debug("removing existing node [{}], which conflicts with incoming join from [{}]", nodeWithSameId, joiningNode); nodesBuilder.remove(nodeWithSameId.getId()); } final DiscoveryNode nodeWithSameAddress = currentNodes.findByAddress(joiningNode.getAddress()); if (nodeWithSameAddress != null && nodeWithSameAddress.equals(joiningNode) == false) { logger.debug("removing existing node [{}], which conflicts with incoming join from [{}]", nodeWithSameAddress, joiningNode); nodesBuilder.remove(nodeWithSameAddress.getId()); } } // now trim any left over dead nodes - either left there when the previous master stepped down // or removed by us above ClusterState tmpState = ClusterState.builder(currentState).nodes(nodesBuilder).blocks(ClusterBlocks.builder() .blocks(currentState.blocks()) .removeGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID)).build(); return ClusterState.builder(allocationService.deassociateDeadNodes(tmpState, false, "removed dead nodes on election")); }