浏览代码

HDFS-9816. Erasure Coding: allow to use multiple EC policies in striping related tests [Part 3]. Contributed by Rui Li.

Change-Id: I64b57bab4722cdc6e1e3148c3a3a401370249afe
Zhe Zhang 9 年之前
父节点
当前提交
def754ec06
共有 13 个文件被更改,包括 161 次插入95 次删除
  1. 24 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
  2. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  3. 2 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockRecoveryWorker.java
  4. 38 20
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java
  5. 27 28
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
  6. 25 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
  7. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
  9. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  10. 11 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
  11. 14 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  12. 4 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReconstructStripedBlocks.java
  13. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java

@@ -171,6 +171,30 @@ public class StripedBlockUtil {
         + lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i);
   }
 
+  /**
+   * Compute the safe length given the internal block lengths.
+   *
+   * @param ecPolicy The EC policy used for the block group
+   * @param blockLens The lengths of internal blocks
+   * @return The safe length
+   */
+  public static long getSafeLength(ErasureCodingPolicy ecPolicy,
+      long[] blockLens) {
+    final int cellSize = ecPolicy.getCellSize();
+    final int dataBlkNum = ecPolicy.getNumDataUnits();
+    Preconditions.checkArgument(blockLens.length >= dataBlkNum);
+    final int stripeSize = dataBlkNum * cellSize;
+    long[] cpy = Arrays.copyOf(blockLens, blockLens.length);
+    Arrays.sort(cpy);
+    // full stripe is a stripe has at least dataBlkNum full cells.
+    // lastFullStripeIdx is the index of the last full stripe.
+    int lastFullStripeIdx =
+        (int) (cpy[cpy.length - dataBlkNum] / cellSize);
+    return lastFullStripeIdx * stripeSize; // return the safeLength
+    // TODO: Include lastFullStripeIdx+1 stripe in safeLength, if there exists
+    // such a stripe (and it must be partial).
+  }
+
   private static int lastCellSize(int size, int cellSize, int numDataBlocks,
       int i) {
     if (i < numDataBlocks) {

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -947,6 +947,9 @@ Trunk (Unreleased)
       HDFS-9830. Remove references to hftp in ViewFs documentation in trunk.
       (Wei-Chiu Chuang via aajisaka)
 
+      HDFS-9816. Erasure Coding: allow to use multiple EC policies in striping
+      related tests [Part 3]. (Rui Li via zhz)
+
 Release 2.9.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockRecoveryWorker.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlo
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
@@ -513,24 +514,15 @@ public class BlockRecoveryWorker {
      */
     @VisibleForTesting
     long getSafeLength(Map<Long, BlockRecord> syncBlocks) {
-      final int cellSize = ecPolicy.getCellSize();
       final int dataBlkNum = ecPolicy.getNumDataUnits();
       Preconditions.checkArgument(syncBlocks.size() >= dataBlkNum);
-      final int stripeSize = dataBlkNum * cellSize;
       long[] blockLengths = new long[syncBlocks.size()];
       int i = 0;
       for (BlockRecord r : syncBlocks.values()) {
         ReplicaRecoveryInfo rInfo = r.getReplicaRecoveryInfo();
         blockLengths[i++] = rInfo.getNumBytes();
       }
-      Arrays.sort(blockLengths);
-      // full stripe is a stripe has at least dataBlkNum full cells.
-      // lastFullStripeIdx is the index of the last full stripe.
-      int lastFullStripeIdx =
-          (int) (blockLengths[blockLengths.length - dataBlkNum] / cellSize);
-      return lastFullStripeIdx * stripeSize; // return the safeLength
-      // TODO: Include lastFullStripeIdx+1 stripe in safeLength, if there exists
-      // such a stripe (and it must be partial).
+      return StripedBlockUtil.getSafeLength(ecPolicy, blockLengths);
     }
 
     private void checkLocations(int locationCount)

+ 38 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java

@@ -27,7 +27,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -45,12 +47,16 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeoutException;
 
 public class TestLeaseRecoveryStriped {
   public static final Log LOG = LogFactory
       .getLog(TestLeaseRecoveryStriped.class);
 
+  private static final ErasureCodingPolicy ecPolicy =
+      StripedFileTestUtil.TEST_EC_POLICY;
   private static final int NUM_DATA_BLOCKS = StripedFileTestUtil.NUM_DATA_BLOCKS;
   private static final int NUM_PARITY_BLOCKS = StripedFileTestUtil.NUM_PARITY_BLOCKS;
   private static final int CELL_SIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
@@ -90,7 +96,7 @@ public class TestLeaseRecoveryStriped {
     cluster.waitActive();
     dfs = cluster.getFileSystem();
     dfs.mkdirs(dir);
-    dfs.setErasureCodingPolicy(dir, null);
+    dfs.setErasureCodingPolicy(dir, ecPolicy);
   }
 
   @After
@@ -100,26 +106,38 @@ public class TestLeaseRecoveryStriped {
     }
   }
 
-  public static final int[][][] BLOCK_LENGTHS_SUITE = {
-      { { 11 * CELL_SIZE, 10 * CELL_SIZE, 9 * CELL_SIZE, 8 * CELL_SIZE,
-          7 * CELL_SIZE, 6 * CELL_SIZE, 5 * CELL_SIZE, 4 * CELL_SIZE,
-          3 * CELL_SIZE }, { 36 * CELL_SIZE } },
-
-      { { 3 * CELL_SIZE, 4 * CELL_SIZE, 5 * CELL_SIZE, 6 * CELL_SIZE,
-          7 * CELL_SIZE, 8 * CELL_SIZE, 9 * CELL_SIZE, 10 * CELL_SIZE,
-          11 * CELL_SIZE }, { 36 * CELL_SIZE } },
-
-      { { 11 * CELL_SIZE, 7 * CELL_SIZE, 6 * CELL_SIZE, 5 * CELL_SIZE,
-          4 * CELL_SIZE, 2 * CELL_SIZE, 9 * CELL_SIZE, 10 * CELL_SIZE,
-          11 * CELL_SIZE }, { 36 * CELL_SIZE } },
+  private static int[][][] getBlockLengthsSuite() {
+    final int groups = 4;
+    final int minNumCell = 3;
+    final int maxNumCell = 11;
+    final int minNumDelta = -4;
+    final int maxNumDelta = 2;
+    int delta = 0;
+    int[][][] blkLenSuite = new int[groups][][];
+    Random random = ThreadLocalRandom.current();
+    for (int i = 0; i < blkLenSuite.length; i++) {
+      if (i == blkLenSuite.length - 1) {
+        delta = bytesPerChecksum;
+      }
+      int[][] suite = new int[2][];
+      int[] lens = new int[NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS];
+      long[] lenInLong = new long[lens.length];
+      for (int j = 0; j < lens.length; j++) {
+        int numCell = random.nextInt(maxNumCell - minNumCell + 1) + minNumCell;
+        int numDelta = j < NUM_DATA_BLOCKS ?
+            random.nextInt(maxNumDelta - minNumDelta + 1) + minNumDelta : 0;
+        lens[j] = CELL_SIZE * numCell + delta * numDelta;
+        lenInLong[j] = lens[j];
+      }
+      suite[0] = lens;
+      suite[1] = new int[]{
+          (int) StripedBlockUtil.getSafeLength(ecPolicy, lenInLong)};
+      blkLenSuite[i] = suite;
+    }
+    return blkLenSuite;
+  }
 
-      { { 8 * CELL_SIZE + bytesPerChecksum,
-          7 * CELL_SIZE + bytesPerChecksum * 2,
-          6 * CELL_SIZE + bytesPerChecksum * 2,
-          5 * CELL_SIZE - bytesPerChecksum * 3,
-          4 * CELL_SIZE - bytesPerChecksum * 4,
-          3 * CELL_SIZE - bytesPerChecksum * 4, 9 * CELL_SIZE, 10 * CELL_SIZE,
-          11 * CELL_SIZE }, { 36 * CELL_SIZE } }, };
+  private static final int[][][] BLOCK_LENGTHS_SUITE = getBlockLengthsSuite();
 
   @Test
   public void testLeaseRecovery() throws Exception {

+ 27 - 28
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java

@@ -111,37 +111,36 @@ public class TestUnderReplicatedBlockQueues {
     int groupSize = dataBlkNum + parityBlkNum;
     long numBytes = ecPolicy.getCellSize() * dataBlkNum;
     UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
-
-    // add a striped block which been left NUM_DATA_BLOCKS internal blocks
-    BlockInfo block1 = genStripedBlockInfo(-100, numBytes);
-    assertAdded(queues, block1, dataBlkNum, 0, groupSize);
-    assertEquals(1, queues.getUnderReplicatedBlockCount());
-    assertEquals(1, queues.size());
-    assertInLevel(queues, block1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY);
-
-    // add a striped block which been left NUM_DATA_BLOCKS+1 internal blocks
-    BlockInfo block2 = genStripedBlockInfo(-200, numBytes);
-    assertAdded(queues, block2, dataBlkNum + 1, 0, groupSize);
-    assertEquals(2, queues.getUnderReplicatedBlockCount());
-    assertEquals(2, queues.size());
-    assertInLevel(queues, block2,
-        UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED);
-
-    // add a striped block which been left NUM_DATA_BLOCKS+2 internal blocks
-    BlockInfo block3 = genStripedBlockInfo(-300, numBytes);
-    assertAdded(queues, block3, dataBlkNum + 2, 0, groupSize);
-    assertEquals(3, queues.getUnderReplicatedBlockCount());
-    assertEquals(3, queues.size());
-    assertInLevel(queues, block3,
-        UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED);
+    int numUR = 0;
+    int numCorrupt = 0;
+
+    // add under replicated blocks
+    for (int i = 0; dataBlkNum + i < groupSize; i++) {
+      BlockInfo block = genStripedBlockInfo(-100 - 100 * i, numBytes);
+      assertAdded(queues, block, dataBlkNum + i, 0, groupSize);
+      numUR++;
+      assertEquals(numUR, queues.getUnderReplicatedBlockCount());
+      assertEquals(numUR + numCorrupt, queues.size());
+      if (i == 0) {
+        assertInLevel(queues, block,
+            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY);
+      } else if (i * 3 < parityBlkNum + 1) {
+        assertInLevel(queues, block,
+            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED);
+      } else {
+        assertInLevel(queues, block,
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED);
+      }
+    }
 
     // add a corrupted block
-    BlockInfo block_corrupt = genStripedBlockInfo(-400, numBytes);
-    assertEquals(0, queues.getCorruptBlockSize());
+    BlockInfo block_corrupt = genStripedBlockInfo(-10, numBytes);
+    assertEquals(numCorrupt, queues.getCorruptBlockSize());
     assertAdded(queues, block_corrupt, dataBlkNum - 1, 0, groupSize);
-    assertEquals(4, queues.size());
-    assertEquals(3, queues.getUnderReplicatedBlockCount());
-    assertEquals(1, queues.getCorruptBlockSize());
+    numCorrupt++;
+    assertEquals(numUR + numCorrupt, queues.size());
+    assertEquals(numUR, queues.getUnderReplicatedBlockCount());
+    assertEquals(numCorrupt, queues.getCorruptBlockSize());
     assertInLevel(queues, block_corrupt,
         UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }

+ 25 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

@@ -61,6 +61,7 @@ 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.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@@ -98,8 +99,6 @@ import org.mockito.stubbing.Answer;
 
 import com.google.common.base.Supplier;
 
-import static org.apache.hadoop.hdfs.TestLeaseRecoveryStriped.BLOCK_LENGTHS_SUITE;
-
 /**
  * This tests if sync all replicas in block recovery works correctly
  */
@@ -124,6 +123,30 @@ public class TestBlockRecovery {
   private final static long REPLICA_LEN2 = 5000L;
   private final static ExtendedBlock block = new ExtendedBlock(POOL_ID,
       BLOCK_ID, BLOCK_LEN, GEN_STAMP);
+
+  private static final int CELL_SIZE =
+      StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
+  private static final int bytesPerChecksum = 512;
+  private static final int[][][] BLOCK_LENGTHS_SUITE = {
+      { { 11 * CELL_SIZE, 10 * CELL_SIZE, 9 * CELL_SIZE, 8 * CELL_SIZE,
+          7 * CELL_SIZE, 6 * CELL_SIZE, 5 * CELL_SIZE, 4 * CELL_SIZE,
+          3 * CELL_SIZE }, { 36 * CELL_SIZE } },
+
+      { { 3 * CELL_SIZE, 4 * CELL_SIZE, 5 * CELL_SIZE, 6 * CELL_SIZE,
+          7 * CELL_SIZE, 8 * CELL_SIZE, 9 * CELL_SIZE, 10 * CELL_SIZE,
+          11 * CELL_SIZE }, { 36 * CELL_SIZE } },
+
+      { { 11 * CELL_SIZE, 7 * CELL_SIZE, 6 * CELL_SIZE, 5 * CELL_SIZE,
+          4 * CELL_SIZE, 2 * CELL_SIZE, 9 * CELL_SIZE, 10 * CELL_SIZE,
+          11 * CELL_SIZE }, { 36 * CELL_SIZE } },
+
+      { { 8 * CELL_SIZE + bytesPerChecksum,
+          7 * CELL_SIZE + bytesPerChecksum * 2,
+          6 * CELL_SIZE + bytesPerChecksum * 2,
+          5 * CELL_SIZE - bytesPerChecksum * 3,
+          4 * CELL_SIZE - bytesPerChecksum * 4,
+          3 * CELL_SIZE - bytesPerChecksum * 4, 9 * CELL_SIZE, 10 * CELL_SIZE,
+          11 * CELL_SIZE }, { 36 * CELL_SIZE } }, };
   
   static {
     GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java

@@ -104,11 +104,11 @@ public class TestAddOverReplicatedStripedBlocks {
     cluster.triggerBlockReports();
 
     // let a internal block be over replicated with 2 redundant blocks.
-    blk.setBlockId(groupId + 2);
+    blk.setBlockId(groupId);
     cluster.injectBlocks(numDNs - 3, Arrays.asList(blk), bpid);
     cluster.injectBlocks(numDNs - 2, Arrays.asList(blk), bpid);
     // let a internal block be over replicated with 1 redundant block.
-    blk.setBlockId(groupId + 6);
+    blk.setBlockId(groupId + DATA_BLK_NUM);
     cluster.injectBlocks(numDNs - 1, Arrays.asList(blk), bpid);
 
     // update blocksMap
@@ -143,10 +143,9 @@ public class TestAddOverReplicatedStripedBlocks {
     cluster.triggerBlockReports();
     List<DatanodeInfo> infos = Arrays.asList(bg.getLocations());
 
-    // let a internal block be over replicated with 2 redundant blocks.
-    // Therefor number of internal blocks is over GROUP_SIZE. (5 data blocks +
-    // 3 parity blocks  + 2 redundant blocks > GROUP_SIZE)
-    blk.setBlockId(groupId + 2);
+    // let a internal block be over replicated with (numDNs - GROUP_SIZE + 1)
+    // redundant blocks. Therefor number of internal blocks is over GROUP_SIZE.
+    blk.setBlockId(groupId);
     List<DataNode> dataNodeList = cluster.getDataNodes();
     for (int i = 0; i < numDNs; i++) {
       if (!infos.contains(dataNodeList.get(i).getDatanodeId())) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java

@@ -454,7 +454,7 @@ public class TestAddStripedBlocks {
     reports = DFSTestUtil.makeReportForReceivedBlock(reported,
         ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
     ns.processIncrementalBlockReport(
-        cluster.getDataNodes().get(5).getDatanodeId(), reports[0]);
+        cluster.getDataNodes().get(0).getDatanodeId(), reports[0]);
     BlockManagerTestUtil.updateState(ns.getBlockManager());
     Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
     Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

@@ -75,7 +75,7 @@ public class TestFSEditLogLoader {
   private static final int NUM_DATA_NODES = 0;
 
   private static final ErasureCodingPolicy testECPolicy
-      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+      = StripedFileTestUtil.TEST_EC_POLICY;
   
   @Test
   public void testDisplayRecentEditLogOpCodes() throws IOException {
@@ -454,7 +454,8 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null);
+      fs.getClient().getNamenode().setErasureCodingPolicy(
+          testDir, testECPolicy);
 
       // Create a file with striped block
       Path p = new Path(testFilePath);
@@ -526,7 +527,8 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null);
+      fs.getClient().getNamenode().setErasureCodingPolicy(
+          testDir, testECPolicy);
 
       //create a file with striped blocks
       Path p = new Path(testFilePath);

+ 11 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java

@@ -143,7 +143,7 @@ public class TestFSImage {
   private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf,
                                                boolean isUC) throws IOException{
     // contruct a INode with StripedBlock for saving and loading
-    fsn.setErasureCodingPolicy("/", null, false);
+    fsn.setErasureCodingPolicy("/", testECPolicy, false);
     long id = 123456789;
     byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
     PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
@@ -439,8 +439,8 @@ public class TestFSImage {
    */
   @Test
   public void testSupportBlockGroup() throws IOException {
-    final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS
-        + StripedFileTestUtil.NUM_PARITY_BLOCKS);
+    final short GROUP_SIZE = (short) (testECPolicy.getNumDataUnits() +
+        testECPolicy.getNumParityUnits());
     final int BLOCK_SIZE = 8 * 1024 * 1024;
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -450,7 +450,7 @@ public class TestFSImage {
           .build();
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
-      fs.getClient().getNamenode().setErasureCodingPolicy("/", null);
+      fs.getClient().getNamenode().setErasureCodingPolicy("/", testECPolicy);
       Path file = new Path("/striped");
       FSDataOutputStream out = fs.create(file);
       byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE);
@@ -472,10 +472,14 @@ public class TestFSImage {
       BlockInfo[] blks = inode.getBlocks();
       assertEquals(1, blks.length);
       assertTrue(blks[0].isStriped());
-      assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum());
-      assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum());
+      assertEquals(testECPolicy.getNumDataUnits(),
+          ((BlockInfoStriped) blks[0]).getDataBlockNum());
+      assertEquals(testECPolicy.getNumParityUnits(),
+          ((BlockInfoStriped) blks[0]).getParityBlockNum());
     } finally {
-      cluster.shutdown();
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
   }
 

+ 14 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -76,6 +76,7 @@ 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.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -654,8 +655,11 @@ public class TestFsck {
         setNumFiles(4).build();
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(10)
-        .build();
+    ErasureCodingPolicy ecPolicy =
+        ErasureCodingPolicyManager.getSystemDefaultPolicy();
+    int numAllUnits = ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
+        numAllUnits + 1).build();
     FileSystem fs = null;
     try {
       String topDir = "/myDir";
@@ -666,7 +670,8 @@ public class TestFsck {
       fs = cluster.getFileSystem();
       util.createFiles(fs, topDir);
       // set topDir to EC when it has replicated files
-      cluster.getFileSystem().getClient().setErasureCodingPolicy(topDir, null);
+      cluster.getFileSystem().getClient().setErasureCodingPolicy(
+          topDir, ecPolicy);
 
       // create a new file under topDir
       DFSTestUtil.createFile(fs, new Path(topDir, "ecFile"), 1024, (short) 1, 0L);
@@ -687,16 +692,16 @@ public class TestFsck {
           "-blocks", "-openforwrite");
       assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
       assertTrue(outStr.contains("OPENFORWRITE"));
-      assertTrue(outStr.contains("Live_repl=9"));
-      assertTrue(outStr.contains("Expected_repl=9"));
+      assertTrue(outStr.contains("Live_repl=" + numAllUnits));
+      assertTrue(outStr.contains("Expected_repl=" + numAllUnits));
 
       // Use -openforwrite option to list open files
       outStr = runFsck(conf, 0, true, openFile.toString(), "-files", "-blocks",
           "-locations", "-openforwrite", "-replicaDetails");
       assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
       assertTrue(outStr.contains("OPENFORWRITE"));
-      assertTrue(outStr.contains("Live_repl=9"));
-      assertTrue(outStr.contains("Expected_repl=9"));
+      assertTrue(outStr.contains("Live_repl=" + numAllUnits));
+      assertTrue(outStr.contains("Expected_repl=" + numAllUnits));
       assertTrue(outStr.contains("Under Construction Block:"));
 
       // Close the file
@@ -708,8 +713,8 @@ public class TestFsck {
       assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
       assertFalse(outStr.contains("OPENFORWRITE"));
       assertFalse(outStr.contains("Under Construction Block:"));
-      assertFalse(outStr.contains("Expected_repl=9"));
-      assertTrue(outStr.contains("Live_repl=9"));
+      assertFalse(outStr.contains("Expected_repl=" + numAllUnits));
+      assertTrue(outStr.contains("Live_repl=" + numAllUnits));
       util.cleanup(fs, topDir);
     } finally {
       if (fs != null) {

+ 4 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReconstructStripedBlocks.java

@@ -77,13 +77,10 @@ public class TestReconstructStripedBlocks {
   }
 
   @Test
-  public void testMissingStripedBlockWithBusyNode1() throws Exception {
-    doTestMissingStripedBlock(2, 1);
-  }
-
-  @Test
-  public void testMissingStripedBlockWithBusyNode2() throws Exception {
-    doTestMissingStripedBlock(3, 1);
+  public void testMissingStripedBlockWithBusyNode() throws Exception {
+    for (int i = 1; i <= NUM_PARITY_BLOCKS; i++) {
+      doTestMissingStripedBlock(i, 1);
+    }
   }
 
   /**

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml

@@ -153,7 +153,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>ErasureCodingPolicy=[Name=RS-6-3-64k</expected-output>
+          <expected-output>ErasureCodingPolicy=[Name=</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -292,7 +292,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Policy 'invalidpolicy' does not match any of the supported policies. Please select any one of [RS-6-3-64k]</expected-output>
+          <expected-output>Policy 'invalidpolicy' does not match any of the supported policies. Please select any one of [</expected-output>
         </comparator>
       </comparators>
     </test>