You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2016/10/17 18:31:00 UTC

[04/50] [abbrv] hadoop git commit: HDFS-10827. When there are unrecoverable ec block groups, Namenode Web UI doesn't show the block names. Contributed by Takanobu Asanuma.

HDFS-10827. When there are unrecoverable ec block groups, Namenode Web UI doesn't show the block names. Contributed by Takanobu Asanuma.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/adb96e10
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/adb96e10
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/adb96e10

Branch: refs/heads/YARN-3368
Commit: adb96e109f1ab4a2c3d469e716c084d0a891b951
Parents: 0007360
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Oct 14 13:21:53 2016 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Fri Oct 14 13:21:53 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      |   2 +-
 .../server/namenode/TestNameNodeMXBean.java     | 105 +++++++++++++++++++
 2 files changed, 106 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb96e10/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 8c59186..563682f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4999,7 +4999,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         BlockInfo blk = blkIterator.next();
         final INodeFile inode = getBlockCollection(blk);
         skip++;
-        if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
+        if (inode != null) {
           String src = inode.getFullPathName();
           if (src.startsWith(path)){
             corruptFiles.add(new CorruptFileBlockInfo(src, blk));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb96e10/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index ac97a36..47f1c85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -17,35 +17,48 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import com.google.common.base.Supplier;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
 import org.apache.hadoop.hdfs.util.HostsFileWriter;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator;
 import org.apache.hadoop.net.ServerSocketUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Assert;
 import org.junit.Test;
 import org.mortbay.util.ajax.JSON;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 import java.io.File;
+import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.net.BindException;
 import java.net.URI;
@@ -495,4 +508,96 @@ public class TestNameNodeMXBean {
           FileUtils.sizeOfDirectory(dir));
     }
   }
+
+  @Test
+  public void testVerifyMissingBlockGroupsMetrics() throws Exception {
+    MiniDFSCluster cluster = null;
+    DistributedFileSystem fs = null;
+    try {
+      Configuration conf = new HdfsConfiguration();
+      int dataBlocks = ErasureCodingPolicyManager
+          .getSystemDefaultPolicy().getNumDataUnits();
+      int parityBlocks = ErasureCodingPolicyManager
+          .getSystemDefaultPolicy().getNumParityUnits();
+      int cellSize = ErasureCodingPolicyManager
+          .getSystemDefaultPolicy().getCellSize();
+      int totalSize = dataBlocks + parityBlocks;
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(totalSize).build();
+      fs = cluster.getFileSystem();
+
+      // create file
+      Path ecDirPath = new Path("/striped");
+      fs.mkdir(ecDirPath, FsPermission.getDirDefault());
+      fs.getClient().setErasureCodingPolicy(ecDirPath.toString(), null);
+      Path file = new Path(ecDirPath, "corrupted");
+      final int length = cellSize * dataBlocks;
+      final byte[] bytes = StripedFileTestUtil.generateBytes(length);
+      DFSTestUtil.writeFile(fs, file, bytes);
+
+      LocatedStripedBlock lsb = (LocatedStripedBlock)fs.getClient()
+          .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0);
+      final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb,
+          cellSize, dataBlocks, parityBlocks);
+
+      // make an unrecoverable ec file with corrupted blocks
+      for(int i = 0; i < parityBlocks + 1; i++) {
+        int ipcPort = blks[i].getLocations()[0].getIpcPort();
+        cluster.corruptReplica(cluster.getDataNode(ipcPort),
+            blks[i].getBlock());
+      }
+
+      // disable the heart beat from DN so that the corrupted block record is
+      // kept in NameNode
+      for (DataNode dn : cluster.getDataNodes()) {
+        DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
+      }
+
+      // Read the file to trigger reportBadBlocks
+      try {
+        IOUtils.copyBytes(fs.open(file), new IOUtils.NullOutputStream(), conf,
+            true);
+      } catch (IOException ie) {
+        assertTrue(ie.getMessage().contains(
+            "missingChunksNum=" + (parityBlocks + 1)));
+      }
+
+      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      ObjectName mxbeanName = new ObjectName(
+          "Hadoop:service=NameNode,name=NameNodeInfo");
+
+      // Wait for the metrics to discover the unrecoverable block group
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            Long numMissingBlocks =
+                (Long) mbs.getAttribute(mxbeanName, "NumberOfMissingBlocks");
+            if (numMissingBlocks == 1L) {
+              return true;
+            }
+          } catch (Exception e) {
+            Assert.fail("Caught unexpected exception.");
+          }
+          return false;
+        }
+      }, 1000, 60000);
+
+      String corruptFiles = (String) (mbs.getAttribute(mxbeanName,
+          "CorruptFiles"));
+      int numCorruptFiles = ((Object[]) JSON.parse(corruptFiles)).length;
+      assertEquals(1, numCorruptFiles);
+    } finally {
+      if (fs != null) {
+        try {
+          fs.close();
+        } catch (Exception e) {
+          throw e;
+        }
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org