@Override public void onFailure(Exception t) { maybeFork(thread, () -> onShardFailure(shardIndex, shard, shard.currentNodeId(), shardIt, t)); } });
private static Set<String> getAllNodeIds(final List<ShardRouting> shards) { final Set<String> nodeIds = new HashSet<>(); for (ShardRouting shard : shards) { nodeIds.add(shard.currentNodeId()); } return nodeIds; }
private int numDocsOnPrimary(List<ShardRouting> shards, Map<String, PreSyncedFlushResponse> preSyncResponses) { for (ShardRouting shard : shards) { if (shard.primary()) { final PreSyncedFlushResponse resp = preSyncResponses.get(shard.currentNodeId()); if (resp != null) { return resp.numDocs; } } } return PreSyncedFlushResponse.UNKNOWN_NUM_DOCS; }
public ShardIterator preferNodeActiveInitializingShardsIt(Set<String> nodeIds) { ArrayList<ShardRouting> preferred = new ArrayList<>(activeShards.size() + allInitializingShards.size()); ArrayList<ShardRouting> notPreferred = new ArrayList<>(activeShards.size() + allInitializingShards.size()); // fill it in a randomized fashion for (ShardRouting shardRouting : shuffler.shuffle(activeShards)) { if (nodeIds.contains(shardRouting.currentNodeId())) { preferred.add(shardRouting); } else { notPreferred.add(shardRouting); } } preferred.addAll(notPreferred); if (!allInitializingShards.isEmpty()) { preferred.addAll(allInitializingShards); } return new PlainShardIterator(shardId, preferred); }
public ShardIterator onlyNodeActiveInitializingShardsIt(String nodeId) { ArrayList<ShardRouting> ordered = new ArrayList<>(activeShards.size() + allInitializingShards.size()); int seed = shuffler.nextSeed(); for (ShardRouting shardRouting : shuffler.shuffle(activeShards, seed)) { if (nodeId.equals(shardRouting.currentNodeId())) { ordered.add(shardRouting); } } for (ShardRouting shardRouting : shuffler.shuffle(allInitializingShards, seed)) { if (nodeId.equals(shardRouting.currentNodeId())) { ordered.add(shardRouting); } } return new PlainShardIterator(shardId, ordered); }
private void reportSuccessWithExistingSyncId(ShardId shardId, String existingSyncId, List<ShardRouting> shards, int totalShards, Map<String, PreSyncedFlushResponse> preSyncResponses, ActionListener<ShardsSyncedFlushResult> listener) { final Map<ShardRouting, ShardSyncedFlushResponse> results = new HashMap<>(); for (final ShardRouting shard : shards) { if (preSyncResponses.containsKey(shard.currentNodeId())) { results.put(shard, new ShardSyncedFlushResponse()); } } listener.onResponse(new ShardsSyncedFlushResult(shardId, existingSyncId, totalShards, results)); }
private void updateAssigned(ShardRouting oldShard, ShardRouting newShard) { assert oldShard.shardId().equals(newShard.shardId()) : "can only update " + oldShard + " by shard with same shard id but was " + newShard; assert oldShard.unassigned() == false && newShard.unassigned() == false : "only assigned shards can be updated in list of assigned shards (prev: " + oldShard + ", new: " + newShard + ")"; assert oldShard.currentNodeId().equals(newShard.currentNodeId()) : "shard to update " + oldShard + " can only update " + oldShard + " by shard assigned to same node but was " + newShard; node(oldShard.currentNodeId()).update(oldShard, newShard); List<ShardRouting> shardsWithMatchingShardId = assignedShards.computeIfAbsent(oldShard.shardId(), k -> new ArrayList<>()); int previousShardIndex = shardsWithMatchingShardId.indexOf(oldShard); assert previousShardIndex >= 0 : "shard to update " + oldShard + " does not exist in list of assigned shards"; shardsWithMatchingShardId.set(previousShardIndex, newShard); }
/** * Finds the store for the assigned shard in the fetched data, returns null if none is found. */ private TransportNodesListShardStoreMetaData.StoreFilesMetaData findStore(ShardRouting shard, RoutingAllocation allocation, AsyncShardFetch.FetchResult<NodeStoreFilesMetaData> data) { assert shard.currentNodeId() != null; DiscoveryNode primaryNode = allocation.nodes().get(shard.currentNodeId()); if (primaryNode == null) { return null; } NodeStoreFilesMetaData primaryNodeFilesStore = data.getData().get(primaryNode); if (primaryNodeFilesStore == null) { return null; } return primaryNodeFilesStore.storeFilesMetaData(); }
static boolean shardCanBeDeleted(String localNodeId, IndexShardRoutingTable indexShardRoutingTable) { // a shard can be deleted if all its copies are active, and its not allocated on this node if (indexShardRoutingTable.size() == 0) { // should not really happen, there should always be at least 1 (primary) shard in a // shard replication group, in any case, protected from deleting something by mistake return false; } for (ShardRouting shardRouting : indexShardRoutingTable) { // be conservative here, check on started, not even active if (shardRouting.started() == false) { return false; } // check if shard is active on the current node if (localNodeId.equals(shardRouting.currentNodeId())) { return false; } } return true; }
static boolean distinctNodes(List<ShardRouting> shards) { Set<String> nodes = new HashSet<>(); for (ShardRouting shard : shards) { if (shard.assignedToNode()) { if (nodes.add(shard.currentNodeId()) == false) { return false; } if (shard.relocating()) { if (nodes.add(shard.relocatingNodeId()) == false) { return false; } } } } return true; }
public Map<DiscoveryNode, Float> weighShard(ShardRouting shard) { final ModelNode[] modelNodes = sorter.modelNodes; final float[] weights = sorter.weights; buildWeightOrderedIndices(); Map<DiscoveryNode, Float> nodes = new HashMap<>(modelNodes.length); float currentNodeWeight = 0.0f; for (int i = 0; i < modelNodes.length; i++) { if (modelNodes[i].getNodeId().equals(shard.currentNodeId())) { // If a node was found with the shard, use that weight instead of 0.0 currentNodeWeight = weights[i]; break; } } for (int i = 0; i < modelNodes.length; i++) { final float delta = currentNodeWeight - weights[i]; nodes.put(modelNodes[i].getRoutingNode().node(), delta); } return nodes; }
@Override public void onFailure(Exception replicaException) { logger.trace(() -> new ParameterizedMessage( "[{}] failure while performing [{}] on replica {}, request [{}]", shard.shardId(), opType, shard, replicaRequest), replicaException); // Only report "critical" exceptions - TODO: Reach out to the master node to get the latest shard state then report. if (TransportActions.isShardNotAvailableException(replicaException) == false) { RestStatus restStatus = ExceptionsHelper.status(replicaException); shardReplicaFailures.add(new ReplicationResponse.ShardInfo.Failure( shard.shardId(), shard.currentNodeId(), replicaException, restStatus, false)); } String message = String.format(Locale.ROOT, "failed to perform %s on replica %s", opType, shard); replicasProxy.failShardIfNeeded(shard, message, replicaException, ReplicationOperation.this::decPendingAndFinishIfNeeded, ReplicationOperation.this::onPrimaryDemoted, throwable -> decPendingAndFinishIfNeeded()); } });
@Override public void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo newUnassignedInfo) { RecoverySource recoverySource = unassignedShard.recoverySource(); if (recoverySource.getType() == RecoverySource.Type.SNAPSHOT) { if (newUnassignedInfo.getLastAllocationStatus() == UnassignedInfo.AllocationStatus.DECIDERS_NO) { String reason = "shard could not be allocated to any of the nodes"; changes(recoverySource).shards.put( unassignedShard.shardId(), new ShardRestoreStatus(unassignedShard.currentNodeId(), RestoreInProgress.State.FAILURE, reason)); } } }
@Override public void shardStarted(ShardRouting initializingShard, ShardRouting startedShard) { // mark snapshot as completed if (initializingShard.primary()) { RecoverySource recoverySource = initializingShard.recoverySource(); if (recoverySource.getType() == RecoverySource.Type.SNAPSHOT) { changes(recoverySource).shards.put( initializingShard.shardId(), new ShardRestoreStatus(initializingShard.currentNodeId(), RestoreInProgress.State.SUCCESS)); } } }
@Override public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo) { if (failedShard.primary() && failedShard.initializing()) { RecoverySource recoverySource = failedShard.recoverySource(); if (recoverySource.getType() == RecoverySource.Type.SNAPSHOT) { // mark restore entry for this shard as failed when it's due to a file corruption. There is no need wait on retries // to restore this shard on another node if the snapshot files are corrupt. In case where a node just left or crashed, // however, we only want to acknowledge the restore operation once it has been successfully restored on another node. if (unassignedInfo.getFailure() != null && Lucene.isCorruptionException(unassignedInfo.getFailure().getCause())) { changes(recoverySource).shards.put( failedShard.shardId(), new ShardRestoreStatus(failedShard.currentNodeId(), RestoreInProgress.State.FAILURE, unassignedInfo.getFailure().getCause().getMessage())); } } } }
protected void executePhaseOnShard(final SearchShardIterator shardIt, final ShardRouting shard, final SearchActionListener<SearchPhaseResult> listener) { getSearchTransport().sendExecuteQuery(getConnection(shardIt.getClusterAlias(), shard.currentNodeId()), buildShardSearchRequest(shardIt), getTask(), listener); }
@Override protected void executePhaseOnShard(SearchShardIterator shardIt, ShardRouting shard, SearchActionListener<SearchService.CanMatchResponse> listener) { getSearchTransport().sendCanMatch(getConnection(shardIt.getClusterAlias(), shard.currentNodeId()), buildShardSearchRequest(shardIt), getTask(), listener); }
@Override protected void executePhaseOnShard(final SearchShardIterator shardIt, final ShardRouting shard, final SearchActionListener<DfsSearchResult> listener) { getSearchTransport().sendExecuteDfs(getConnection(shardIt.getClusterAlias(), shard.currentNodeId()), buildShardSearchRequest(shardIt) , getTask(), listener); }
private void deleteShardIfExistElseWhere(ClusterState state, IndexShardRoutingTable indexShardRoutingTable) { List<Tuple<DiscoveryNode, ShardActiveRequest>> requests = new ArrayList<>(indexShardRoutingTable.size()); String indexUUID = indexShardRoutingTable.shardId().getIndex().getUUID(); ClusterName clusterName = state.getClusterName(); for (ShardRouting shardRouting : indexShardRoutingTable) { assert shardRouting.started() : "expected started shard but was " + shardRouting; DiscoveryNode currentNode = state.nodes().get(shardRouting.currentNodeId()); requests.add(new Tuple<>(currentNode, new ShardActiveRequest(clusterName, indexUUID, shardRouting.shardId(), deleteShardTimeout))); } ShardActiveResponseHandler responseHandler = new ShardActiveResponseHandler(indexShardRoutingTable.shardId(), state.getVersion(), requests.size()); for (Tuple<DiscoveryNode, ShardActiveRequest> request : requests) { logger.trace("{} sending shard active check to {}", request.v2().shardId, request.v1()); transportService.sendRequest(request.v1(), ACTION_SHARD_EXISTS, request.v2(), responseHandler); } }
/** * Relocate a shard to another node, adding the target initializing * shard as well as assigning it. * * @return pair of source relocating and target initializing shards. */ public Tuple<ShardRouting,ShardRouting> relocateShard(ShardRouting startedShard, String nodeId, long expectedShardSize, RoutingChangesObserver changes) { ensureMutable(); relocatingShards++; ShardRouting source = startedShard.relocate(nodeId, expectedShardSize); ShardRouting target = source.getTargetRelocatingShard(); updateAssigned(startedShard, source); node(target.currentNodeId()).add(target); assignedShardsAdd(target); addRecovery(target); changes.relocationStarted(startedShard, target); return Tuple.tuple(source, target); }