Pārlūkot izejas kodu

Revert "HDFS-11259. Update fsck to display maintenance state info. (Manoj Govindassamy via lei)"

This reverts commit c18590fce283378edb09acd4e764706a9a4a8b5f.
Lei Xu 8 gadi atpakaļ
vecāks
revīzija
2f8e9b7e4b

+ 9 - 52
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -117,9 +117,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   public static final String HEALTHY_STATUS = "is HEALTHY";
   public static final String DECOMMISSIONING_STATUS = "is DECOMMISSIONING";
   public static final String DECOMMISSIONED_STATUS = "is DECOMMISSIONED";
-  public static final String ENTERING_MAINTENANCE_STATUS =
-      "is ENTERING MAINTENANCE";
-  public static final String IN_MAINTENANCE_STATUS = "is IN MAINTENANCE";
   public static final String NONEXISTENT_STATUS = "does not exist";
   public static final String FAILURE_STATUS = "FAILED";
   public static final String UNDEFINED = "undefined";
@@ -283,10 +280,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           + numberReplicas.decommissioned());
       out.println("No. of decommissioning Replica: "
           + numberReplicas.decommissioning());
-      out.println("No. of entering maintenance Replica: "
-          + numberReplicas.liveEnteringMaintenanceReplicas());
-      out.println("No. of in maintenance Replica: "
-          + numberReplicas.maintenanceNotForReadReplicas());
       out.println("No. of corrupted Replica: " +
           numberReplicas.corruptReplicas());
       //record datanodes that have corrupted block replica
@@ -307,10 +300,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           out.print(DECOMMISSIONED_STATUS);
         } else if (dn.isDecommissionInProgress()) {
           out.print(DECOMMISSIONING_STATUS);
-        } else if (dn.isEnteringMaintenance()) {
-          out.print(ENTERING_MAINTENANCE_STATUS);
-        } else if (dn.isInMaintenance()) {
-          out.print(IN_MAINTENANCE_STATUS);
         } else {
           out.print(HEALTHY_STATUS);
         }
@@ -609,10 +598,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           sb.append("DECOMMISSIONED)");
         } else if (dnDesc.isDecommissionInProgress()) {
           sb.append("DECOMMISSIONING)");
-        } else if (dnDesc.isEnteringMaintenance()) {
-          sb.append("ENTERING MAINTENANCE)");
-        } else if (dnDesc.isInMaintenance()) {
-          sb.append("IN MAINTENANCE)");
         } else if (corruptReplicas != null
             && corruptReplicas.contains(dnDesc)) {
           sb.append("CORRUPT)");
@@ -635,7 +620,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   }
 
   private void collectBlocksSummary(String parent, HdfsFileStatus file,
-      Result res, LocatedBlocks blocks) throws IOException {
+                 Result res, LocatedBlocks blocks) throws IOException {
     String path = file.getFullName(parent);
     boolean isOpen = blocks.isUnderConstruction();
     if (isOpen && !showOpenFiles) {
@@ -666,21 +651,13 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
       int decommissionedReplicas = numberReplicas.decommissioned();
       int decommissioningReplicas = numberReplicas.decommissioning();
-      int enteringMaintenanceReplicas =
-          numberReplicas.liveEnteringMaintenanceReplicas();
-      int inMaintenanceReplicas =
-          numberReplicas.maintenanceNotForReadReplicas();
       res.decommissionedReplicas +=  decommissionedReplicas;
       res.decommissioningReplicas += decommissioningReplicas;
-      res.enteringMaintenanceReplicas += enteringMaintenanceReplicas;
-      res.inMaintenanceReplicas += inMaintenanceReplicas;
 
       // count total replicas
       int liveReplicas = numberReplicas.liveReplicas();
-      int totalReplicasPerBlock = liveReplicas + decommissionedReplicas
-          + decommissioningReplicas
-          + enteringMaintenanceReplicas
-          + inMaintenanceReplicas;
+      int totalReplicasPerBlock = liveReplicas + decommissionedReplicas +
+          decommissioningReplicas;
       res.totalReplicas += totalReplicasPerBlock;
 
       boolean isMissing;
@@ -734,14 +711,12 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         if (!showFiles) {
           out.print("\n" + path + ": ");
         }
-        out.println(" Under replicated " + block + ". Target Replicas is "
-            + targetFileReplication + " but found "
-            + liveReplicas+ " live replica(s), "
-            + decommissionedReplicas + " decommissioned replica(s), "
-            + decommissioningReplicas + " decommissioning replica(s), "
-            + enteringMaintenanceReplicas
-            + " entering maintenance replica(s) and "
-            + inMaintenanceReplicas + " in maintenance replica(s).");
+        out.println(" Under replicated " + block +
+                    ". Target Replicas is " +
+                    targetFileReplication + " but found " +
+                    liveReplicas + " live replica(s), " +
+                    decommissionedReplicas + " decommissioned replica(s) and " +
+                    decommissioningReplicas + " decommissioning replica(s).");
       }
 
       // count mis replicated blocks
@@ -1120,8 +1095,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     long missingReplicas = 0L;
     long decommissionedReplicas = 0L;
     long decommissioningReplicas = 0L;
-    long enteringMaintenanceReplicas = 0L;
-    long inMaintenanceReplicas = 0L;
     long numUnderMinReplicatedBlocks = 0L;
     long numOverReplicatedBlocks = 0L;
     long numUnderReplicatedBlocks = 0L;
@@ -1270,14 +1243,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         res.append("\n DecommissioningReplicas:\t").append(
             decommissioningReplicas);
       }
-      if (enteringMaintenanceReplicas > 0) {
-        res.append("\n EnteringMaintenanceReplicas:\t").append(
-            enteringMaintenanceReplicas);
-      }
-      if (inMaintenanceReplicas > 0) {
-        res.append("\n InMaintenanceReplicas:\t").append(
-            inMaintenanceReplicas);
-      }
       return res.toString();
     }
   }
@@ -1384,14 +1349,6 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         res.append("\n Decommissioning internal blocks:\t").append(
             decommissioningReplicas);
       }
-      if (enteringMaintenanceReplicas > 0) {
-        res.append("\n EnteringMaintenanceReplicas:\t").append(
-            enteringMaintenanceReplicas);
-      }
-      if (inMaintenanceReplicas > 0) {
-        res.append("\n InMaintenanceReplicas:\t").append(
-            inMaintenanceReplicas);
-      }
       return res.toString();
     }
   }

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

@@ -372,10 +372,6 @@ public class DFSck extends Configured implements Tool {
       errCode = 2;
     } else if (lastLine.endsWith(NamenodeFsck.DECOMMISSIONING_STATUS)) {
       errCode = 3;
-    } else if (lastLine.endsWith(NamenodeFsck.IN_MAINTENANCE_STATUS))  {
-      errCode = 4;
-    } else if (lastLine.endsWith(NamenodeFsck.ENTERING_MAINTENANCE_STATUS)) {
-      errCode = 5;
     }
     return errCode;
   }

+ 28 - 312
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -54,7 +54,6 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -169,11 +168,11 @@ public class TestFsck {
     PrintStream out = new PrintStream(bStream, true);
     GenericTestUtils.setLogLevel(FSPermissionChecker.LOG, Level.ALL);
     int errCode = ToolRunner.run(new DFSck(conf, out), path);
-    LOG.info("OUTPUT = " + bStream.toString());
     if (checkErrorCode) {
       assertEquals(expectedErrCode, errCode);
     }
     GenericTestUtils.setLogLevel(FSPermissionChecker.LOG, Level.INFO);
+    LOG.info("OUTPUT = " + bStream.toString());
     return bStream.toString();
   }
 
@@ -879,13 +878,14 @@ public class TestFsck {
     assertTrue(outStr.contains("dfs.namenode.replication.min:\t2"));
   }
 
-  @Test(timeout = 90000)
+  @Test(timeout = 60000)
   public void testFsckReplicaDetails() throws Exception {
 
     final short replFactor = 1;
     short numDn = 1;
     final long blockSize = 512;
     final long fileSize = 1024;
+    boolean checkDecommissionInProgress = false;
     String[] racks = {"/rack1"};
     String[] hosts = {"host1"};
 
@@ -910,110 +910,49 @@ public class TestFsck {
         "-replicaDetails");
     assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
     assertTrue(fsckOut.contains("(LIVE)"));
-    assertTrue(!fsckOut.contains("(ENTERING MAINTENANCE)"));
-    assertTrue(!fsckOut.contains("(IN MAINTENANCE)"));
 
     // decommission datanode
+    ExtendedBlock eb = DFSTestUtil.getFirstBlock(dfs, path);
     FSNamesystem fsn = cluster.getNameNode().getNamesystem();
     BlockManager bm = fsn.getBlockManager();
-    final DatanodeManager dnm = bm.getDatanodeManager();
-    DatanodeDescriptor dnDesc0 = dnm.getDatanode(
-        cluster.getDataNodes().get(0).getDatanodeId());
-
-    bm.getDatanodeManager().getDecomManager().startDecommission(dnDesc0);
-    final String dn0Name = dnDesc0.getXferAddr();
+    BlockCollection bc = null;
+    try {
+      fsn.writeLock();
+      BlockInfo bi = bm.getStoredBlock(eb.getLocalBlock());
+      bc = fsn.getBlockCollection(bi);
+    } finally {
+      fsn.writeUnlock();
+    }
+    DatanodeDescriptor dn = bc.getBlocks()[0]
+        .getDatanode(0);
+    bm.getDatanodeManager().getDecomManager().startDecommission(dn);
+    String dnName = dn.getXferAddr();
 
     // check the replica status while decommissioning
     fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
         "-replicaDetails");
     assertTrue(fsckOut.contains("(DECOMMISSIONING)"));
-    assertTrue(!fsckOut.contains("(ENTERING MAINTENANCE)"));
-    assertTrue(!fsckOut.contains("(IN MAINTENANCE)"));
-
-    // Start 2nd DataNode
-    cluster.startDataNodes(conf, 1, true, null,
-        new String[] {"/rack2"}, new String[] {"host2"}, null, false);
 
-    // Wait for decommission to start
-    final AtomicBoolean checkDecommissionInProgress =
-        new AtomicBoolean(false);
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        DatanodeInfo datanodeInfo = null;
-        try {
-          for (DatanodeInfo info : dfs.getDataNodeStats()) {
-            if (dn0Name.equals(info.getXferAddr())) {
-              datanodeInfo = info;
-            }
-          }
-          if (!checkDecommissionInProgress.get() && datanodeInfo != null
-              && datanodeInfo.isDecommissionInProgress()) {
-            checkDecommissionInProgress.set(true);
-          }
-          if (datanodeInfo != null && datanodeInfo.isDecommissioned()) {
-            return true;
-          }
-        } catch (Exception e) {
-          LOG.warn("Unexpected exception: " + e);
-          return false;
+    // Start 2nd Datanode and wait for decommission to start
+    cluster.startDataNodes(conf, 1, true, null, null, null);
+    DatanodeInfo datanodeInfo = null;
+    do {
+      Thread.sleep(2000);
+      for (DatanodeInfo info : dfs.getDataNodeStats()) {
+        if (dnName.equals(info.getXferAddr())) {
+          datanodeInfo = info;
         }
-        return false;
       }
-    }, 500, 30000);
-
-    // check the replica status after decommission is done
-    fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
-        "-replicaDetails");
-    assertTrue(fsckOut.contains("(DECOMMISSIONED)"));
-    assertTrue(!fsckOut.contains("(ENTERING MAINTENANCE)"));
-    assertTrue(!fsckOut.contains("(IN MAINTENANCE)"));
-
-    DatanodeDescriptor dnDesc1 = dnm.getDatanode(
-        cluster.getDataNodes().get(1).getDatanodeId());
-    final String dn1Name = dnDesc1.getXferAddr();
-
-    bm.getDatanodeManager().getDecomManager().startMaintenance(dnDesc1,
-        Long.MAX_VALUE);
-    // check the replica status while entering maintenance
-    fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
-        "-replicaDetails");
-    assertTrue(fsckOut.contains("(DECOMMISSIONED)"));
-    assertTrue(fsckOut.contains("(ENTERING MAINTENANCE)"));
-    assertTrue(!fsckOut.contains("(IN MAINTENANCE)"));
-
-    // Start 3rd DataNode
-    cluster.startDataNodes(conf, 1, true, null,
-        new String[] {"/rack3"}, new String[] {"host3"}, null, false);
-
-    // Wait for the 2nd node to reach in maintenance state
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        DatanodeInfo dnInfo = null;
-        try {
-          for (DatanodeInfo info : dfs.getDataNodeStats()) {
-            if (dn1Name.equals(info.getXferAddr())) {
-              dnInfo = info;
-            }
-          }
-          if (dnInfo != null && dnInfo.isInMaintenance()) {
-            return true;
-          }
-        } catch (Exception e) {
-          LOG.warn("Unexpected exception: " + e);
-          return false;
-        }
-        return false;
+      if (!checkDecommissionInProgress && datanodeInfo != null
+          && datanodeInfo.isDecommissionInProgress()) {
+        checkDecommissionInProgress = true;
       }
-    }, 500, 30000);
+    } while (datanodeInfo != null && !datanodeInfo.isDecommissioned());
 
     // check the replica status after decommission is done
     fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
         "-replicaDetails");
     assertTrue(fsckOut.contains("(DECOMMISSIONED)"));
-    assertTrue(!fsckOut.contains("(ENTERING MAINTENANCE)"));
-    assertTrue(fsckOut.contains("(IN MAINTENANCE)"));
   }
 
   /** Test if fsck can return -1 in case of failure.
@@ -1520,118 +1459,6 @@ public class TestFsck {
     assertTrue(fsckOut.contains(NamenodeFsck.DECOMMISSIONED_STATUS));
   }
 
-  /**
-   * Test for blockIdCK with datanode maintenance.
-   */
-  @Test (timeout = 90000)
-  public void testBlockIdCKMaintenance() throws Exception {
-    final short replFactor = 2;
-    short numDn = 2;
-    final long blockSize = 512;
-    String[] hosts = {"host1", "host2"};
-    String[] racks = {"/rack1", "/rack2"};
-
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, replFactor);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, replFactor);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY,
-        replFactor);
-
-    DistributedFileSystem dfs;
-    cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(numDn)
-        .hosts(hosts)
-        .racks(racks)
-        .build();
-
-    assertNotNull("Failed Cluster Creation", cluster);
-    cluster.waitClusterUp();
-    dfs = cluster.getFileSystem();
-    assertNotNull("Failed to get FileSystem", dfs);
-
-    DFSTestUtil util = new DFSTestUtil.Builder().
-        setName(getClass().getSimpleName()).setNumFiles(1).build();
-    //create files
-    final String pathString = new String("/testfile");
-    final Path path = new Path(pathString);
-    util.createFile(dfs, path, 1024, replFactor, 1000L);
-    util.waitReplication(dfs, path, replFactor);
-    StringBuilder sb = new StringBuilder();
-    for (LocatedBlock lb: util.getAllBlocks(dfs, path)){
-      sb.append(lb.getBlock().getLocalBlock().getBlockName()+" ");
-    }
-    String[] bIds = sb.toString().split(" ");
-
-    //make sure datanode that has replica is fine before maintenance
-    String outStr = runFsck(conf, 0, true, "/", "-blockId", bIds[0]);
-    System.out.println(outStr);
-    assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
-
-    FSNamesystem fsn = cluster.getNameNode().getNamesystem();
-    BlockManager bm = fsn.getBlockManager();
-    DatanodeManager dnm = bm.getDatanodeManager();
-    DatanodeDescriptor dn = dnm.getDatanode(cluster.getDataNodes().get(0)
-        .getDatanodeId());
-    bm.getDatanodeManager().getDecomManager().startMaintenance(dn,
-        Long.MAX_VALUE);
-    final String dnName = dn.getXferAddr();
-
-    //wait for the node to enter maintenance state
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        DatanodeInfo datanodeInfo = null;
-        try {
-          for (DatanodeInfo info : dfs.getDataNodeStats()) {
-            if (dnName.equals(info.getXferAddr())) {
-              datanodeInfo = info;
-            }
-          }
-          if (datanodeInfo != null && datanodeInfo.isEnteringMaintenance()) {
-            String fsckOut = runFsck(conf, 5, false, "/", "-blockId", bIds[0]);
-            assertTrue(fsckOut.contains(
-                NamenodeFsck.ENTERING_MAINTENANCE_STATUS));
-            return true;
-          }
-        } catch (Exception e) {
-          LOG.warn("Unexpected exception: " + e);
-          return false;
-        }
-        return false;
-      }
-    }, 500, 30000);
-
-    // Start 3rd DataNode
-    cluster.startDataNodes(conf, 1, true, null,
-        new String[] {"/rack3"}, new String[] {"host3"}, null, false);
-
-    // Wait for 1st node to reach in maintenance state
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        try {
-          DatanodeInfo datanodeInfo = null;
-          for (DatanodeInfo info : dfs.getDataNodeStats()) {
-            if (dnName.equals(info.getXferAddr())) {
-              datanodeInfo = info;
-            }
-          }
-          if (datanodeInfo != null && datanodeInfo.isInMaintenance()) {
-            return true;
-          }
-        } catch (Exception e) {
-          LOG.warn("Unexpected exception: " + e);
-          return false;
-        }
-        return false;
-      }
-    }, 500, 30000);
-
-    //check in maintenance node
-    String fsckOut = runFsck(conf, 4, false, "/", "-blockId", bIds[0]);
-    assertTrue(fsckOut.contains(NamenodeFsck.IN_MAINTENANCE_STATUS));
-  }
-
   /**
    * Test for blockIdCK with block corruption.
    */
@@ -1828,117 +1655,6 @@ public class TestFsck {
     String fsckOut = runFsck(conf, 0, true, testFile);
   }
 
-  /**
-   * Test for blocks on maintenance hosts are not shown as missing.
-   */
-  @Test (timeout = 90000)
-  public void testFsckWithMaintenanceReplicas() throws Exception {
-    final short replFactor = 2;
-    short numDn = 2;
-    final long blockSize = 512;
-    String[] hosts = {"host1", "host2"};
-    String[] racks = {"/rack1", "/rack2"};
-
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, replFactor);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, replFactor);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY,
-        replFactor);
-
-    DistributedFileSystem dfs;
-    cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(numDn)
-        .hosts(hosts)
-        .racks(racks)
-        .build();
-
-    assertNotNull("Failed Cluster Creation", cluster);
-    cluster.waitClusterUp();
-    dfs = cluster.getFileSystem();
-    assertNotNull("Failed to get FileSystem", dfs);
-
-    DFSTestUtil util = new DFSTestUtil.Builder().
-        setName(getClass().getSimpleName()).setNumFiles(1).build();
-    //create files
-    final String testFile = new String("/testfile");
-    final Path path = new Path(testFile);
-    util.createFile(dfs, path, 1024, replFactor, 1000L);
-    util.waitReplication(dfs, path, replFactor);
-    StringBuilder sb = new StringBuilder();
-    for (LocatedBlock lb: util.getAllBlocks(dfs, path)){
-      sb.append(lb.getBlock().getLocalBlock().getBlockName()+" ");
-    }
-    String[] bIds = sb.toString().split(" ");
-
-    //make sure datanode that has replica is fine before maintenance
-    String outStr = runFsck(conf, 0, true, testFile);
-    System.out.println(outStr);
-    assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
-
-    FSNamesystem fsn = cluster.getNameNode().getNamesystem();
-    BlockManager bm = fsn.getBlockManager();
-    DatanodeManager dnm = bm.getDatanodeManager();
-    DatanodeDescriptor dn = dnm.getDatanode(cluster.getDataNodes().get(0)
-        .getDatanodeId());
-    bm.getDatanodeManager().getDecomManager().startMaintenance(dn,
-        Long.MAX_VALUE);
-    final String dnName = dn.getXferAddr();
-
-    //wait for the node to enter maintenance state
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        DatanodeInfo datanodeInfo = null;
-        try {
-          for (DatanodeInfo info : dfs.getDataNodeStats()) {
-            if (dnName.equals(info.getXferAddr())) {
-              datanodeInfo = info;
-            }
-          }
-          if (datanodeInfo != null && datanodeInfo.isEnteringMaintenance()) {
-            // verify fsck returns Healthy status
-            String fsckOut = runFsck(conf, 0, true, testFile);
-            assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
-            return true;
-          }
-        } catch (Exception e) {
-          LOG.warn("Unexpected exception: " + e);
-          return false;
-        }
-        return false;
-      }
-    }, 500, 30000);
-
-    // Start 3rd DataNode and wait for node to reach in maintenance state
-    cluster.startDataNodes(conf, 1, true, null,
-        new String[] {"/rack3"}, new String[] {"host3"}, null, false);
-
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        DatanodeInfo datanodeInfo = null;
-        try {
-          for (DatanodeInfo info : dfs.getDataNodeStats()) {
-            if (dnName.equals(info.getXferAddr())) {
-              datanodeInfo = info;
-            }
-          }
-          if (datanodeInfo != null && datanodeInfo.isInMaintenance()) {
-            return true;
-          }
-        } catch (Exception e) {
-          LOG.warn("Unexpected exception: " + e);
-          return false;
-        }
-        return false;
-      }
-    }, 500, 30000);
-
-    // verify fsck returns Healthy status
-    String fsckOut = runFsck(conf, 0, true, testFile);
-    assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
-  }
-
   @Test
   public void testECFsck() throws Exception {
     FileSystem fs = null;