Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 0EFFE200C0E for ; Wed, 18 Jan 2017 00:29:15 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 0E061160B58; Tue, 17 Jan 2017 23:29:15 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id B9BCF160B55 for ; Wed, 18 Jan 2017 00:29:13 +0100 (CET) Received: (qmail 94518 invoked by uid 500); 17 Jan 2017 23:29:11 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 93425 invoked by uid 99); 17 Jan 2017 23:29:11 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 17 Jan 2017 23:29:11 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 329F3DFCDC; Tue, 17 Jan 2017 23:29:11 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: subru@apache.org To: common-commits@hadoop.apache.org Date: Tue, 17 Jan 2017 23:29:16 -0000 Message-Id: <0e1307cbc8b74c53ab79ffab86dde622@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [06/50] [abbrv] hadoop git commit: Revert "HDFS-11259. Update fsck to display maintenance state info. (Manoj Govindassamy via lei)" archived-at: Tue, 17 Jan 2017 23:29:15 -0000 Revert "HDFS-11259. Update fsck to display maintenance state info. (Manoj Govindassamy via lei)" This reverts commit c18590fce283378edb09acd4e764706a9a4a8b5f. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2f8e9b7e Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2f8e9b7e Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2f8e9b7e Branch: refs/heads/YARN-2915 Commit: 2f8e9b7e4b1721ed5c7db8882eff70f83164e320 Parents: ed09c14 Author: Lei Xu Authored: Mon Jan 16 15:11:53 2017 +0800 Committer: Lei Xu Committed: Mon Jan 16 15:11:53 2017 +0800 ---------------------------------------------------------------------- .../hdfs/server/namenode/NamenodeFsck.java | 61 +--- .../org/apache/hadoop/hdfs/tools/DFSck.java | 4 - .../hadoop/hdfs/server/namenode/TestFsck.java | 340 ++----------------- 3 files changed, 37 insertions(+), 368 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f8e9b7e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java index 3941378..756a2f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java +++ b/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(); } } http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f8e9b7e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java index 0fd6122..9cf234a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java +++ b/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; } http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f8e9b7e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java index 0956ed2..81a25fd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java +++ b/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() { - @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() { - @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. @@ -1521,118 +1460,6 @@ public class TestFsck { } /** - * 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() { - @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() { - @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. */ @Test @@ -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() { - @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() { - @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; --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-commits-help@hadoop.apache.org