/** * Remove decommissioned datanode from the the list of live or dead nodes. * This is used to not to display a decommissioned datanode to the operators. * @param nodeList , array list of live or dead nodes. */ private static void removeDecomNodeFromList( final List<DatanodeDescriptor> nodeList) { for (Iterator<DatanodeDescriptor> it = nodeList.iterator(); it.hasNext();) { DatanodeDescriptor node = it.next(); if (node.isDecommissioned()) { it.remove(); } } }
@Override // FSNamesystemMBean @Metric({"NumDecomDeadDataNodes", "Number of datanodes which have been decommissioned and are now dead"}) public int getNumDecomDeadDataNodes() { final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); getBlockManager().getDatanodeManager().fetchDatanodes(null, dead, false); int deadDecommissioned = 0; for (DatanodeDescriptor node : dead) { deadDecommissioned += node.isDecommissioned() ? 1 : 0; } return deadDecommissioned; }
/** * Returned information is a JSON representation of map with host name as the * key and value is a map of dead node attribute keys to its values */ @Override // NameNodeMXBean public String getDeadNodes() { final Map<String, Map<String, Object>> info = new HashMap<String, Map<String, Object>>(); final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); blockManager.getDatanodeManager().fetchDatanodes(null, dead, false); for (DatanodeDescriptor node : dead) { Map<String, Object> innerinfo = ImmutableMap.<String, Object>builder() .put("lastContact", getLastContact(node)) .put("decommissioned", node.isDecommissioned()) .put("adminState", node.getAdminState().toString()) .put("xferaddr", node.getXferAddr()) .build(); info.put(node.getHostName() + ":" + node.getXferPort(), innerinfo); } return JSON.toString(info); }
@Override // FSNamesystemMBean @Metric({"NumDecomLiveDataNodes", "Number of datanodes which have been decommissioned and are now live"}) public int getNumDecomLiveDataNodes() { final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); getBlockManager().getDatanodeManager().fetchDatanodes(live, null, false); int liveDecommissioned = 0; for (DatanodeDescriptor node : live) { liveDecommissioned += node.isDecommissioned() ? 1 : 0; } return liveDecommissioned; }
/** * Stop decommissioning the specified datanode. * @param node */ @VisibleForTesting public void stopDecommission(DatanodeDescriptor node) { if (node.isDecommissionInProgress() || node.isDecommissioned()) { // Update DN stats maintained by HeartbeatManager hbManager.stopDecommission(node); // extra redundancy blocks will be detected and processed when // the dead node comes back and send in its full block report. if (node.isAlive()) { blockManager.processExtraRedundancyBlocksOnInService(node); } // Remove from tracking in DatanodeAdminManager pendingNodes.remove(node); outOfServiceNodeBlocks.remove(node); } else { LOG.trace("stopDecommission: Node {} in {}, nothing to do.", node, node.getAdminState()); } }
synchronized void startMaintenance(final DatanodeDescriptor node) { if (!node.isAlive()) { LOG.info("Dead node {} is put in maintenance state immediately.", node); node.setInMaintenance(); } else { stats.subtract(node); if (node.isDecommissioned()) { LOG.info("Decommissioned node " + node + " is put in maintenance state" + " immediately."); node.setInMaintenance(); } else if (blockManager.getMinReplicationToBeInMaintenance() == 0) { LOG.info("MinReplicationToBeInMaintenance is set to zero. " + node + " is put in maintenance state" + " immediately."); node.setInMaintenance(); } else { node.startMaintenance(); } stats.add(node); } }
/** * Start decommissioning the specified datanode. * @param node */ @VisibleForTesting public void startDecommission(DatanodeDescriptor node) { if (!node.isDecommissionInProgress() && !node.isDecommissioned()) { // Update DN stats maintained by HeartbeatManager hbManager.startDecommission(node); // hbManager.startDecommission will set dead node to decommissioned. if (node.isDecommissionInProgress()) { for (DatanodeStorageInfo storage : node.getStorageInfos()) { LOG.info("Starting decommission of {} {} with {} blocks", node, storage, storage.numBlocks()); } node.getLeavingServiceStatus().setStartTime(monotonicNow()); pendingNodes.add(node); } } else { LOG.trace("startDecommission: Node {} in {}, nothing to do.", node, node.getAdminState()); } }
blockManager.getCorruptReplicas(storedBlock); sb.append("("); if (dnDesc.isDecommissioned()) { sb.append("DECOMMISSIONED)"); } else if (dnDesc.isDecommissionInProgress()) {
boolean isPlacementPolicySatisfied(BlockInfo storedBlock) { List<DatanodeDescriptor> liveNodes = new ArrayList<>(); Collection<DatanodeDescriptor> corruptNodes = corruptReplicas .getNodes(storedBlock); for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) { if (storage.getStorageType() == StorageType.PROVIDED && storage.getState() == State.NORMAL) { // assume the policy is satisfied for blocks on PROVIDED storage // as long as the storage is in normal state. return true; } final DatanodeDescriptor cur = getDatanodeDescriptorFromStorage(storage); // Nodes under maintenance should be counted as valid replicas from // rack policy point of view. if (!cur.isDecommissionInProgress() && !cur.isDecommissioned() && ((corruptNodes == null) || !corruptNodes.contains(cur))) { liveNodes.add(cur); } } DatanodeInfo[] locs = liveNodes.toArray(new DatanodeInfo[liveNodes.size()]); BlockType blockType = storedBlock.getBlockType(); BlockPlacementPolicy placementPolicy = placementPolicies .getPolicy(blockType); int numReplicas = blockType == STRIPED ? ((BlockInfoStriped) storedBlock) .getRealTotalBlockNum() : storedBlock.getReplication(); return placementPolicy.verifyBlockPlacement(locs, numReplicas) .isPlacementPolicySatisfied(); }
out.print(CORRUPT_STATUS + "\t ReasonCode: " + blockManager.getCorruptReason(block, dn)); } else if (dn.isDecommissioned() ){ out.print(DECOMMISSIONED_STATUS); } else if (dn.isDecommissionInProgress()) {
Preconditions.checkState(dn.isDecommissioned() || dn.isInService(), "Removing a node that is not yet decommissioned or in service!"); outOfServiceNodeBlocks.remove(dn);
if (corruptNodes != null && corruptNodes.contains(node)) { state = "(corrupt)"; } else if (node.isDecommissioned() || node.isDecommissionInProgress()) { state = "(decommissioned)";
} else if (node.isDecommissionInProgress()) { s = StoredReplicaState.DECOMMISSIONING; } else if (node.isDecommissioned()) { s = StoredReplicaState.DECOMMISSIONED; } else if (node.isMaintenance()) {
curReplicaDelta = (node.isDecommissioned()) ? 0 : 1; if (logEveryBlock) { blockLog.debug("BLOCK* addStoredBlock: {} is added to {} (size={})",
@Override // FSNamesystemMBean public int getNumDecomLiveDataNodes() { final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); getBlockManager().getDatanodeManager().fetchDatanodes(live, null, false); int liveDecommissioned = 0; for (DatanodeDescriptor node : live) { liveDecommissioned += node.isDecommissioned() ? 1 : 0; } return liveDecommissioned; }
@Override // FSNamesystemMBean public int getNumDecomDeadDataNodes() { final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); getBlockManager().getDatanodeManager().fetchDatanodes(null, dead, false); int deadDecommissioned = 0; for (DatanodeDescriptor node : dead) { deadDecommissioned += node.isDecommissioned() ? 1 : 0; } return deadDecommissioned; }
boolean isPlacementPolicySatisfied(Block b) { List<DatanodeDescriptor> liveNodes = new ArrayList<>(); Collection<DatanodeDescriptor> corruptNodes = corruptReplicas .getNodes(b); for (DatanodeStorageInfo storage : blocksMap.getStorages(b)) { final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); if (!cur.isDecommissionInProgress() && !cur.isDecommissioned() && ((corruptNodes == null) || !corruptNodes.contains(cur))) { liveNodes.add(cur); } } DatanodeInfo[] locs = liveNodes.toArray(new DatanodeInfo[liveNodes.size()]); return blockplacement.verifyBlockPlacement(locs, getReplication(b)).isPlacementPolicySatisfied(); }
@Override // FSNamesystemMBean @Metric({"NumDecomLiveDataNodes", "Number of datanodes which have been decommissioned and are now live"}) public int getNumDecomLiveDataNodes() { final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); getBlockManager().getDatanodeManager().fetchDatanodes(live, null, false); int liveDecommissioned = 0; for (DatanodeDescriptor node : live) { liveDecommissioned += node.isDecommissioned() ? 1 : 0; } return liveDecommissioned; }
@Override // FSNamesystemMBean @Metric({"NumDecomDeadDataNodes", "Number of datanodes which have been decommissioned and are now dead"}) public int getNumDecomDeadDataNodes() { final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); getBlockManager().getDatanodeManager().fetchDatanodes(null, dead, false); int deadDecommissioned = 0; for (DatanodeDescriptor node : dead) { deadDecommissioned += node.isDecommissioned() ? 1 : 0; } return deadDecommissioned; }
private void add(final DatanodeDescriptor node) { capacityUsed += node.getDfsUsed(); capacityUsedNonDfs += node.getNonDfsUsed(); blockPoolUsed += node.getBlockPoolUsed(); xceiverCount += node.getXceiverCount(); if (!(node.isDecommissionInProgress() || node.isDecommissioned())) { nodesInService++; nodesInServiceXceiverCount += node.getXceiverCount(); capacityTotal += node.getCapacity(); capacityRemaining += node.getRemaining(); } else { capacityTotal += node.getDfsUsed(); } cacheCapacity += node.getCacheCapacity(); cacheUsed += node.getCacheUsed(); }