|
@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
|
|
|
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
@@ -68,9 +69,11 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
|
+import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.net.NetworkTopology;
|
|
|
import org.apache.hadoop.net.NodeBase;
|
|
@@ -132,6 +135,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|
|
private boolean showStoragePolcies = false;
|
|
|
private boolean showCorruptFileBlocks = false;
|
|
|
|
|
|
+ private boolean showReplicaDetails = false;
|
|
|
+ private long staleInterval;
|
|
|
/**
|
|
|
* True if we encountered an internal error during FSCK, such as not being
|
|
|
* able to delete a corrupt file.
|
|
@@ -193,6 +198,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|
|
networktopology,
|
|
|
namenode.getNamesystem().getBlockManager().getDatanodeManager()
|
|
|
.getHost2DatanodeMap());
|
|
|
+ this.staleInterval =
|
|
|
+ conf.getLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);
|
|
|
|
|
|
for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) {
|
|
|
String key = it.next();
|
|
@@ -203,6 +211,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|
|
else if (key.equals("blocks")) { this.showBlocks = true; }
|
|
|
else if (key.equals("locations")) { this.showLocations = true; }
|
|
|
else if (key.equals("racks")) { this.showRacks = true; }
|
|
|
+ else if (key.equals("replicadetails")) {
|
|
|
+ this.showReplicaDetails = true;
|
|
|
+ }
|
|
|
else if (key.equals("storagepolicies")) { this.showStoragePolcies = true; }
|
|
|
else if (key.equals("openforwrite")) {this.showOpenFiles = true; }
|
|
|
else if (key.equals("listcorruptfileblocks")) {
|
|
@@ -502,9 +513,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|
|
ExtendedBlock block = lBlk.getBlock();
|
|
|
boolean isCorrupt = lBlk.isCorrupt();
|
|
|
String blkName = block.toString();
|
|
|
- DatanodeInfo[] locs = lBlk.getLocations();
|
|
|
- NumberReplicas numberReplicas =
|
|
|
- namenode.getNamesystem().getBlockManager().countNodes(block.getLocalBlock());
|
|
|
+ BlockManager bm = namenode.getNamesystem().getBlockManager();
|
|
|
+ NumberReplicas numberReplicas = bm.countNodes(block.getLocalBlock());
|
|
|
int liveReplicas = numberReplicas.liveReplicas();
|
|
|
int decommissionedReplicas = numberReplicas.decommissioned();;
|
|
|
int decommissioningReplicas = numberReplicas.decommissioning();
|
|
@@ -513,6 +523,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|
|
int totalReplicas = liveReplicas + decommissionedReplicas +
|
|
|
decommissioningReplicas;
|
|
|
res.totalReplicas += totalReplicas;
|
|
|
+ Collection<DatanodeDescriptor> corruptReplicas = null;
|
|
|
+ if (showReplicaDetails) {
|
|
|
+ corruptReplicas = bm.getCorruptReplicas(block.getLocalBlock());
|
|
|
+ }
|
|
|
short targetFileReplication = file.getReplication();
|
|
|
res.numExpectedReplicas += targetFileReplication;
|
|
|
if(totalReplicas < minReplication){
|
|
@@ -573,14 +587,41 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
|
|
|
missize += block.getNumBytes();
|
|
|
} else {
|
|
|
report.append(" repl=" + liveReplicas);
|
|
|
- if (showLocations || showRacks) {
|
|
|
+ if (showLocations || showRacks || showReplicaDetails) {
|
|
|
StringBuilder sb = new StringBuilder("[");
|
|
|
- for (int j = 0; j < locs.length; j++) {
|
|
|
- if (j > 0) { sb.append(", "); }
|
|
|
- if (showRacks)
|
|
|
- sb.append(NodeBase.getPath(locs[j]));
|
|
|
- else
|
|
|
- sb.append(locs[j]);
|
|
|
+ Iterable<DatanodeStorageInfo> storages = bm.getStorages(block.getLocalBlock());
|
|
|
+ for (Iterator<DatanodeStorageInfo> iterator = storages.iterator(); iterator.hasNext();) {
|
|
|
+ DatanodeStorageInfo storage = iterator.next();
|
|
|
+ DatanodeDescriptor dnDesc = storage.getDatanodeDescriptor();
|
|
|
+ if (showRacks) {
|
|
|
+ sb.append(NodeBase.getPath(dnDesc));
|
|
|
+ } else {
|
|
|
+ sb.append(new DatanodeInfoWithStorage(dnDesc, storage.getStorageID(), storage
|
|
|
+ .getStorageType()));
|
|
|
+ }
|
|
|
+ if (showReplicaDetails) {
|
|
|
+ LightWeightLinkedSet<Block> blocksExcess =
|
|
|
+ bm.excessReplicateMap.get(dnDesc.getDatanodeUuid());
|
|
|
+ sb.append("(");
|
|
|
+ if (dnDesc.isDecommissioned()) {
|
|
|
+ sb.append("DECOMMISSIONED)");
|
|
|
+ } else if (dnDesc.isDecommissionInProgress()) {
|
|
|
+ sb.append("DECOMMISSIONING)");
|
|
|
+ } else if (corruptReplicas != null && corruptReplicas.contains(dnDesc)) {
|
|
|
+ sb.append("CORRUPT)");
|
|
|
+ } else if (blocksExcess != null && blocksExcess.contains(block.getLocalBlock())) {
|
|
|
+ sb.append("EXCESS)");
|
|
|
+ } else if (dnDesc.isStale(this.staleInterval)) {
|
|
|
+ sb.append("STALE_NODE)");
|
|
|
+ } else if (storage.areBlockContentsStale()) {
|
|
|
+ sb.append("STALE_BLOCK_CONTENT)");
|
|
|
+ } else {
|
|
|
+ sb.append("LIVE)");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (iterator.hasNext()) {
|
|
|
+ sb.append(", ");
|
|
|
+ }
|
|
|
}
|
|
|
sb.append(']');
|
|
|
report.append(" " + sb.toString());
|