浏览代码

HDFS-9016. Display upgrade domain information in fsck. (mingma)

Ming Ma 9 年之前
父节点
当前提交
b7436f4468

+ 18 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -118,6 +118,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   public static final String DECOMMISSIONED_STATUS = "is DECOMMISSIONED";
   public static final String NONEXISTENT_STATUS = "does not exist";
   public static final String FAILURE_STATUS = "FAILED";
+  public static final String UNDEFINED = "undefined";
 
   private final NameNode namenode;
   private final NetworkTopology networktopology;
@@ -136,6 +137,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   private boolean showCorruptFileBlocks = false;
 
   private boolean showReplicaDetails = false;
+  private boolean showUpgradeDomains = false;
   private long staleInterval;
   private Tracer tracer;
 
@@ -216,10 +218,13 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       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")) {
+      } else if (key.equals("upgradedomains")) {
+        this.showUpgradeDomains = true;
+      } else if (key.equals("storagepolicies")) {
+        this.showStoragePolcies = true;
+      } else if (key.equals("openforwrite")) {
+        this.showOpenFiles = true;
+      } else if (key.equals("listcorruptfileblocks")) {
         this.showCorruptFileBlocks = true;
       } else if (key.equals("startblockafter")) {
         this.currentCookie[0] = pmap.get("startblockafter")[0];
@@ -524,8 +529,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     if (res.totalFiles % 100 == 0) { out.println(); out.flush(); }
   }
 
-  private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res,
-      LocatedBlocks blocks) throws IOException {
+  private void collectBlocksSummary(String parent, HdfsFileStatus file,
+      Result res, LocatedBlocks blocks) throws IOException {
     String path = file.getFullName(parent);
     boolean isOpen = blocks.isUnderConstruction();
     if (isOpen && !showOpenFiles) {
@@ -638,7 +643,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         missize += block.getNumBytes();
       } else {
         report.append(" Live_repl=" + liveReplicas);
-        if (showLocations || showRacks || showReplicaDetails) {
+        if (showLocations || showRacks || showReplicaDetails ||
+            showUpgradeDomains) {
           StringBuilder sb = new StringBuilder("[");
           Iterable<DatanodeStorageInfo> storages = bm.getStorages(block.getLocalBlock());
           for (Iterator<DatanodeStorageInfo> iterator = storages.iterator(); iterator.hasNext();) {
@@ -650,6 +656,11 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
               sb.append(new DatanodeInfoWithStorage(dnDesc, storage.getStorageID(), storage
                   .getStorageType()));
             }
+            if (showUpgradeDomains) {
+              String upgradeDomain = (dnDesc.getUpgradeDomain() != null) ?
+                  dnDesc.getUpgradeDomain() : UNDEFINED;
+              sb.append("(ud=" + upgradeDomain +")");
+            }
             if (showReplicaDetails) {
               LightWeightHashSet<Block> blocksExcess =
                   bm.excessReplicateMap.get(dnDesc.getDatanodeUuid());

+ 9 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java

@@ -77,7 +77,8 @@ public class DFSck extends Configured implements Tool {
   private static final String USAGE = "Usage: hdfs fsck <path> "
       + "[-list-corruptfileblocks | "
       + "[-move | -delete | -openforwrite] "
-      + "[-files [-blocks [-locations | -racks | -replicaDetails]]]] "
+      + "[-files [-blocks [-locations | -racks | -replicaDetails | " +
+          "-upgradedomains]]]] "
       + "[-includeSnapshots] "
       + "[-storagepolicies] [-blockId <blk_Id>]\n"
       + "\t<path>\tstart checking from this path\n"
@@ -95,6 +96,8 @@ public class DFSck extends Configured implements Tool {
       + "\t-files -blocks -racks" 
       + "\tprint out network topology for data-node locations\n"
       + "\t-files -blocks -replicaDetails\tprint out each replica details \n"
+      + "\t-files -blocks -upgradedomains\tprint out upgrade domains for " +
+          "every block\n"
       + "\t-storagepolicies\tprint out storage policy summary for the blocks\n"
       + "\t-blockId\tprint out which file this blockId belongs to, locations"
       + " (nodes, racks) of this block, and other diagnostics info"
@@ -271,9 +274,11 @@ public class DFSck extends Configured implements Tool {
       else if (args[idx].equals("-racks")) { url.append("&racks=1"); }
       else if (args[idx].equals("-replicaDetails")) {
         url.append("&replicadetails=1");
-      }
-      else if (args[idx].equals("-storagepolicies")) { url.append("&storagepolicies=1"); }
-      else if (args[idx].equals("-list-corruptfileblocks")) {
+      } else if (args[idx].equals("-upgradedomains")) {
+        url.append("&upgradedomains=1");
+      } else if (args[idx].equals("-storagepolicies")) {
+        url.append("&storagepolicies=1");
+      } else if (args[idx].equals("-list-corruptfileblocks")) {
         url.append("&listcorruptfileblocks=1");
         doListCorruptFileBlocks = true;
       } else if (args[idx].equals("-includeSnapshots")) {

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

@@ -109,7 +109,7 @@ Usage:
        hdfs fsck <path>
               [-list-corruptfileblocks |
               [-move | -delete | -openforwrite]
-              [-files [-blocks [-locations | -racks | -replicaDetails]]]
+              [-files [-blocks [-locations | -racks | -replicaDetails | -upgradedomains]]]
               [-includeSnapshots]
               [-storagepolicies] [-blockId <blk_Id>]
 
@@ -122,6 +122,7 @@ Usage:
 | `-files` `-blocks` `-locations` | Print out locations for every block. |
 | `-files` `-blocks` `-racks` | Print out network topology for data-node locations. |
 | `-files` `-blocks` `-replicaDetails` | Print out each replica details. |
+| `-files` `-blocks` `-upgradedomains` | Print out upgrade domains for every block. |
 | `-includeSnapshots` | Include snapshot data if the given path indicates a snapshottable directory or there are snapshottable directories under it. |
 | `-list-corruptfileblocks` | Print out list of missing blocks and files they belong to. |
 | `-move` | Move corrupted files to /lost+found. |

+ 72 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -24,9 +24,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -81,6 +79,8 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -89,11 +89,14 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSck;
+import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.AccessControlException;
@@ -1852,4 +1855,71 @@ public class TestFsck {
     }
   }
 
+  @Test(timeout = 60000)
+  public void testFsckUpgradeDomain() throws Exception {
+    testUpgradeDomain(false, false);
+    testUpgradeDomain(false, true);
+    testUpgradeDomain(true, false);
+    testUpgradeDomain(true, true);
+  }
+
+  private void testUpgradeDomain(boolean defineUpgradeDomain,
+      boolean displayUpgradeDomain) throws Exception {
+    final short replFactor = 1;
+    final short numDN = 1;
+    final long blockSize = 512;
+    final long fileSize = 1024;
+    final String upgradeDomain = "ud1";
+    final String[] racks = {"/rack1"};
+    final String[] hosts = {"127.0.0.1"};
+    HostsFileWriter hostsFileWriter = new HostsFileWriter();
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, replFactor);
+    if (defineUpgradeDomain) {
+      conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY,
+          CombinedHostFileManager.class, HostConfigManager.class);
+      hostsFileWriter.initialize(conf, "temp/fsckupgradedomain");
+    }
+
+    MiniDFSCluster cluster;
+    DistributedFileSystem dfs;
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDN).
+        hosts(hosts).racks(racks).build();
+    cluster.waitClusterUp();
+    dfs = cluster.getFileSystem();
+
+    // Configure the upgrade domain on the datanode
+    if (defineUpgradeDomain) {
+      DatanodeAdminProperties dnProp = new DatanodeAdminProperties();
+      DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
+      dnProp.setHostName(datanodeID.getHostName());
+      dnProp.setPort(datanodeID.getXferPort());
+      dnProp.setUpgradeDomain(upgradeDomain);
+      hostsFileWriter.initIncludeHosts(new DatanodeAdminProperties[]{dnProp});
+      cluster.getFileSystem().refreshNodes();
+    }
+
+    // create files
+    final String testFile = new String("/testfile");
+    final Path path = new Path(testFile);
+    DFSTestUtil.createFile(dfs, path, fileSize, replFactor, 1000L);
+    DFSTestUtil.waitReplication(dfs, path, replFactor);
+    try {
+      String fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
+          displayUpgradeDomain ? "-upgradedomains" : "-locations");
+      assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
+      String udValue = defineUpgradeDomain ? upgradeDomain :
+          NamenodeFsck.UNDEFINED;
+      assertEquals(displayUpgradeDomain,
+          fsckOut.contains("(ud=" + udValue + ")"));
+    } finally {
+      if (defineUpgradeDomain) {
+        hostsFileWriter.cleanup();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }