|
@@ -54,6 +54,7 @@ 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;
|
|
|
|
|
@@ -168,11 +169,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();
|
|
|
}
|
|
|
|
|
@@ -878,14 +879,13 @@ public class TestFsck {
|
|
|
assertTrue(outStr.contains("dfs.namenode.replication.min:\t2"));
|
|
|
}
|
|
|
|
|
|
- @Test(timeout = 60000)
|
|
|
+ @Test(timeout = 90000)
|
|
|
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"};
|
|
|
|
|
@@ -906,53 +906,132 @@ public class TestFsck {
|
|
|
DFSTestUtil.waitReplication(dfs, path, replFactor);
|
|
|
|
|
|
// make sure datanode that has replica is fine before decommission
|
|
|
- String fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
|
|
|
- "-replicaDetails");
|
|
|
+ String fsckOut = runFsck(conf, 0, true, testFile, "-files",
|
|
|
+ "-maintenance", "-blocks", "-replicaDetails");
|
|
|
assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
|
|
|
assertTrue(fsckOut.contains("(LIVE)"));
|
|
|
+ assertFalse(fsckOut.contains("(ENTERING MAINTENANCE)"));
|
|
|
+ assertFalse(fsckOut.contains("(IN MAINTENANCE)"));
|
|
|
|
|
|
// decommission datanode
|
|
|
- ExtendedBlock eb = DFSTestUtil.getFirstBlock(dfs, path);
|
|
|
FSNamesystem fsn = cluster.getNameNode().getNamesystem();
|
|
|
BlockManager bm = fsn.getBlockManager();
|
|
|
- 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();
|
|
|
+ final DatanodeManager dnm = bm.getDatanodeManager();
|
|
|
+ DatanodeDescriptor dnDesc0 = dnm.getDatanode(
|
|
|
+ cluster.getDataNodes().get(0).getDatanodeId());
|
|
|
+
|
|
|
+ bm.getDatanodeManager().getDecomManager().startDecommission(dnDesc0);
|
|
|
+ final String dn0Name = dnDesc0.getXferAddr();
|
|
|
|
|
|
// check the replica status while decommissioning
|
|
|
- fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
|
|
|
- "-replicaDetails");
|
|
|
+ fsckOut = runFsck(conf, 0, true, testFile, "-files",
|
|
|
+ "-maintenance", "-blocks", "-replicaDetails");
|
|
|
assertTrue(fsckOut.contains("(DECOMMISSIONING)"));
|
|
|
+ assertFalse(fsckOut.contains("(ENTERING MAINTENANCE)"));
|
|
|
+ assertFalse(fsckOut.contains("(IN MAINTENANCE)"));
|
|
|
|
|
|
- // 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;
|
|
|
+ // 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;
|
|
|
}
|
|
|
+ return false;
|
|
|
}
|
|
|
- if (!checkDecommissionInProgress && datanodeInfo != null
|
|
|
- && datanodeInfo.isDecommissionInProgress()) {
|
|
|
- checkDecommissionInProgress = true;
|
|
|
+ }, 500, 30000);
|
|
|
+
|
|
|
+ // check the replica status after decommission is done
|
|
|
+ fsckOut = runFsck(conf, 0, true, testFile, "-files",
|
|
|
+ "-maintenance", "-blocks", "-replicaDetails");
|
|
|
+ assertTrue(fsckOut.contains("(DECOMMISSIONED)"));
|
|
|
+ assertFalse(fsckOut.contains("(ENTERING MAINTENANCE)"));
|
|
|
+ assertFalse(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",
|
|
|
+ "-maintenance", "-blocks", "-replicaDetails");
|
|
|
+ assertTrue(fsckOut.contains("(DECOMMISSIONED)"));
|
|
|
+ assertTrue(fsckOut.contains("(ENTERING MAINTENANCE)"));
|
|
|
+ assertFalse(fsckOut.contains("(IN MAINTENANCE)"));
|
|
|
+
|
|
|
+ // check entering maintenance replicas are printed only when requested
|
|
|
+ fsckOut = runFsck(conf, 0, true, testFile, "-files",
|
|
|
+ "-blocks", "-replicaDetails");
|
|
|
+ assertTrue(fsckOut.contains("(DECOMMISSIONED)"));
|
|
|
+ assertFalse(fsckOut.contains("(ENTERING MAINTENANCE)"));
|
|
|
+ assertFalse(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;
|
|
|
}
|
|
|
- } while (datanodeInfo != null && !datanodeInfo.isDecommissioned());
|
|
|
+ }, 500, 30000);
|
|
|
|
|
|
// check the replica status after decommission is done
|
|
|
- fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
|
|
|
- "-replicaDetails");
|
|
|
+ fsckOut = runFsck(conf, 0, true, testFile, "-files",
|
|
|
+ "-maintenance", "-blocks", "-replicaDetails");
|
|
|
+ assertTrue(fsckOut.contains("(DECOMMISSIONED)"));
|
|
|
+ assertFalse(fsckOut.contains("(ENTERING MAINTENANCE)"));
|
|
|
+ assertTrue(fsckOut.contains("(IN MAINTENANCE)"));
|
|
|
+
|
|
|
+ // check in maintenance replicas are not printed when not requested
|
|
|
+ fsckOut = runFsck(conf, 0, true, testFile, "-files",
|
|
|
+ "-blocks", "-replicaDetails");
|
|
|
assertTrue(fsckOut.contains("(DECOMMISSIONED)"));
|
|
|
+ assertFalse(fsckOut.contains("(ENTERING MAINTENANCE)"));
|
|
|
+ assertFalse(fsckOut.contains("(IN MAINTENANCE)"));
|
|
|
+
|
|
|
+
|
|
|
}
|
|
|
|
|
|
/** Test if fsck can return -1 in case of failure.
|
|
@@ -1459,6 +1538,125 @@ 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, "/",
|
|
|
+ "-maintenance", "-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, "/",
|
|
|
+ "-maintenance", "-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, "/",
|
|
|
+ "-maintenance", "-blockId", bIds[0]);
|
|
|
+ assertTrue(fsckOut.contains(NamenodeFsck.IN_MAINTENANCE_STATUS));
|
|
|
+
|
|
|
+ //check in maintenance node are not printed when not requested
|
|
|
+ fsckOut = runFsck(conf, 4, false, "/", "-blockId", bIds[0]);
|
|
|
+ assertFalse(fsckOut.contains(NamenodeFsck.IN_MAINTENANCE_STATUS));
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Test for blockIdCK with block corruption.
|
|
|
*/
|
|
@@ -1655,6 +1853,121 @@ 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, "-maintenance");
|
|
|
+ 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, "-maintenance");
|
|
|
+ assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
|
|
|
+
|
|
|
+ // verify fsck returns Healthy status even without maintenance option
|
|
|
+ fsckOut = runFsck(conf, 0, true, testFile);
|
|
|
+ assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testECFsck() throws Exception {
|
|
|
FileSystem fs = null;
|