浏览代码

HDFS-9658. Erasure Coding: allow to use multiple EC policies in striping related tests. Contributed by Rui Li.

Change-Id: I5b02f5bde4d343b7529c6a7fe5df73bd99c6cb24
zhezhang 9 年之前
父节点
当前提交
a0f5e83f28

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

@@ -918,6 +918,9 @@ Trunk (Unreleased)
       HDFS-9403. Erasure coding: some EC tests are missing timeout.
       (Gao Rui via zhz)
 
+      HDFS-9658. Erasure Coding: allow to use multiple EC policies in striping
+      related tests. (Rui Li via zhz)
+
 Release 2.9.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 10 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java

@@ -26,10 +26,12 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem.WebHdfsInputStream;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
@@ -56,9 +58,14 @@ public class StripedFileTestUtil {
    * These values correspond to the values used by the system default erasure
    * coding policy.
    */
-  public static final short NUM_DATA_BLOCKS = (short) 6;
-  public static final short NUM_PARITY_BLOCKS = (short) 3;
-  public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024;
+  public static final ErasureCodingPolicy TEST_EC_POLICY =
+      ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  public static final short NUM_DATA_BLOCKS =
+      (short) TEST_EC_POLICY.getNumDataUnits();
+  public static final short NUM_PARITY_BLOCKS =
+      (short) TEST_EC_POLICY.getNumParityUnits();
+  public static final int BLOCK_STRIPED_CELL_SIZE =
+      TEST_EC_POLICY.getCellSize();
 
   static int stripesPerBlock = 4;
   public static int blockSize = BLOCK_STRIPED_CELL_SIZE * stripesPerBlock;

+ 20 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java

@@ -60,7 +60,8 @@ public class TestDFSStripedInputStream {
   private DistributedFileSystem fs;
   private final Path dirPath = new Path("/striped");
   private Path filePath = new Path(dirPath, "file");
-  private final ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
+  private final ErasureCodingPolicy ecPolicy =
+      ErasureCodingPolicyManager.getSystemDefaultPolicy();
   private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS;
   private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS;
   private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
@@ -182,7 +183,7 @@ public class TestDFSStripedInputStream {
   @Test
   public void testPreadWithDNFailure() throws Exception {
     final int numBlocks = 4;
-    final int failedDNIdx = 2;
+    final int failedDNIdx = DATA_BLK_NUM - 1;
     DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks,
         NUM_STRIPE_PER_BLOCK, false);
     LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
@@ -200,11 +201,10 @@ public class TestDFSStripedInputStream {
     }
     DFSStripedInputStream in =
         new DFSStripedInputStream(fs.getClient(), filePath.toString(), false,
-            ErasureCodingPolicyManager.getSystemDefaultPolicy(), null);
+            ecPolicy, null);
     int readSize = BLOCK_GROUP_SIZE;
     byte[] readBuffer = new byte[readSize];
     byte[] expected = new byte[readSize];
-    cluster.stopDataNode(failedDNIdx);
     /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */
     for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
       for (int j = 0; j < DATA_BLK_NUM; j++) {
@@ -221,20 +221,30 @@ public class TestDFSStripedInputStream {
         DATA_BLK_NUM, PARITY_BLK_NUM);
 
     // Update the expected content for decoded data
+    int[] missingBlkIdx = new int[PARITY_BLK_NUM];
+    for (int i = 0; i < missingBlkIdx.length; i++) {
+      if (i == 0) {
+        missingBlkIdx[i] = failedDNIdx;
+      } else {
+        missingBlkIdx[i] = DATA_BLK_NUM + i;
+      }
+    }
+    cluster.stopDataNode(failedDNIdx);
     for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) {
       byte[][] decodeInputs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][CELLSIZE];
-      int[] missingBlkIdx = new int[]{failedDNIdx, 7, 8};
-      byte[][] decodeOutputs = new byte[PARITY_BLK_NUM][CELLSIZE];
+      byte[][] decodeOutputs = new byte[missingBlkIdx.length][CELLSIZE];
       for (int j = 0; j < DATA_BLK_NUM; j++) {
         int posInBuf = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE;
         if (j != failedDNIdx) {
           System.arraycopy(expected, posInBuf, decodeInputs[j], 0, CELLSIZE);
         }
       }
-      for (int k = 0; k < CELLSIZE; k++) {
-        int posInBlk = i * CELLSIZE + k;
-        decodeInputs[DATA_BLK_NUM][k] = SimulatedFSDataset.simulatedByte(
-            new Block(bg.getBlock().getBlockId() + DATA_BLK_NUM), posInBlk);
+      for (int j = DATA_BLK_NUM; j < DATA_BLK_NUM + PARITY_BLK_NUM; j++) {
+        for (int k = 0; k < CELLSIZE; k++) {
+          int posInBlk = i * CELLSIZE + k;
+          decodeInputs[j][k] = SimulatedFSDataset.simulatedByte(
+              new Block(bg.getBlock().getBlockId() + j), posInBlk);
+        }
       }
       for (int m : missingBlkIdx) {
         decodeInputs[m] = null;

+ 80 - 39
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java

@@ -17,16 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicInteger;
-
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -52,7 +43,18 @@ import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
 
-import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.Stack;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 
 public class TestDFSStripedOutputStreamWithFailure {
@@ -73,8 +75,8 @@ public class TestDFSStripedOutputStreamWithFailure {
   private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK;
   private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS;
 
-  private static final int FLUSH_POS
-      = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
+  private static final int FLUSH_POS =
+      9 * DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
 
   static {
     System.out.println("NUM_DATA_BLOCKS  = " + NUM_DATA_BLOCKS);
@@ -103,23 +105,60 @@ public class TestDFSStripedOutputStreamWithFailure {
     return lengths;
   }
 
-  private static final int[][] dnIndexSuite = {
-      {0, 1},
-      {0, 5},
-      {0, 6},
-      {0, 8},
-      {1, 5},
-      {1, 6},
-      {6, 8},
-      {0, 1, 2},
-      {3, 4, 5},
-      {0, 1, 6},
-      {0, 5, 6},
-      {0, 5, 8},
-      {0, 6, 7},
-      {5, 6, 7},
-      {6, 7, 8},
-  };
+  private static final int[][] dnIndexSuite = getDnIndexSuite();
+
+  private static int[][] getDnIndexSuite() {
+    final int maxNumLevel = 2;
+    final int maxPerLevel = 8;
+    List<List<Integer>> allLists = new ArrayList<>();
+    int numIndex = NUM_PARITY_BLOCKS;
+    for (int i = 0; i < maxNumLevel && numIndex > 1; i++) {
+      List<List<Integer>> lists =
+          combinations(NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS, numIndex);
+      if (lists.size() > maxPerLevel) {
+        Collections.shuffle(lists);
+        lists = lists.subList(0, maxPerLevel);
+      }
+      allLists.addAll(lists);
+      numIndex--;
+    }
+    int[][] dnIndexSuite = new int[allLists.size()][];
+    for (int i = 0; i < dnIndexSuite.length; i++) {
+      int[] list = new int[allLists.get(i).size()];
+      for (int j = 0; j < list.length; j++) {
+        list[j] = allLists.get(i).get(j);
+      }
+      dnIndexSuite[i] = list;
+    }
+    return dnIndexSuite;
+  }
+
+  // get all combinations of k integers from {0,...,n-1}
+  private static List<List<Integer>> combinations(int n, int k) {
+    List<List<Integer>> res = new LinkedList<List<Integer>>();
+    if (k >= 1 && n >= k) {
+      getComb(n, k, new Stack<Integer>(), res);
+    }
+    return res;
+  }
+
+  private static void getComb(int n, int k, Stack<Integer> stack,
+      List<List<Integer>> res) {
+    if (stack.size() == k) {
+      List<Integer> list = new ArrayList<Integer>(stack);
+      res.add(list);
+    } else {
+      int next = stack.empty() ? 0 : stack.peek() + 1;
+      while (next < n) {
+        stack.push(next);
+        getComb(n, k, stack, res);
+        next++;
+      }
+    }
+    if (!stack.empty()) {
+      stack.pop();
+    }
+  }
 
   private int[] getKillPositions(int fileLen, int num) {
     int[] positions = new int[num];
@@ -193,10 +232,10 @@ public class TestDFSStripedOutputStreamWithFailure {
     conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
     // Set short retry timeouts so this test runs faster
     conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
-    for (int dn = 0; dn < 9; dn += 2) {
+    for (int dn = 0; dn < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; dn += 2) {
       try {
         setup(conf);
-        runTest(length, new int[]{length/2}, new int[]{dn}, true);
+        runTest(length, new int[]{length / 2}, new int[]{dn}, true);
       } catch (Exception e) {
         LOG.error("failed, dn=" + dn + ", length=" + length);
         throw e;
@@ -216,10 +255,10 @@ public class TestDFSStripedOutputStreamWithFailure {
       ArrayList<DataNode> dataNodes = cluster.getDataNodes();
       // shutdown few datanodes to avoid getting sufficient data blocks number
       // of datanodes
-      int killDns = dataNodes.size() / 2;
-      int numDatanodes = dataNodes.size() - killDns;
-      for (int i = 0; i < killDns; i++) {
-        cluster.stopDataNode(i);
+      int numDatanodes = dataNodes.size();
+      while (numDatanodes >= NUM_DATA_BLOCKS) {
+        cluster.stopDataNode(0);
+        numDatanodes--;
       }
       cluster.restartNameNodes();
       cluster.triggerHeartbeats();
@@ -235,8 +274,10 @@ public class TestDFSStripedOutputStreamWithFailure {
         Assert.fail("Failed to validate available dns against blkGroupSize");
       } catch (IOException ioe) {
         // expected
-        GenericTestUtils.assertExceptionContains("Failed to get 6 nodes from" +
-            " namenode: blockGroupSize= 9, blocks.length= 5", ioe);
+        GenericTestUtils.assertExceptionContains("Failed to get " +
+            NUM_DATA_BLOCKS + " nodes from namenode: blockGroupSize= " +
+            (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length= " +
+            numDatanodes, ioe);
       }
     } finally {
       tearDown();
@@ -274,7 +315,7 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   void runTest(final int length) {
     final HdfsConfiguration conf = newHdfsConfiguration();
-    for (int dn = 0; dn < 9; dn++) {
+    for (int dn = 0; dn < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; dn++) {
       try {
         LOG.info("runTest: dn=" + dn + ", length=" + length);
         setup(conf);

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java

@@ -39,11 +39,11 @@ public class TestErasureCodingPolicyWithSnapshot {
   private DistributedFileSystem fs;
   private Configuration conf;
 
-  private final static short GROUP_SIZE = StripedFileTestUtil.NUM_DATA_BLOCKS
-      + StripedFileTestUtil.NUM_PARITY_BLOCKS;
+  private final static short GROUP_SIZE = (short) (StripedFileTestUtil.
+      NUM_DATA_BLOCKS + StripedFileTestUtil.NUM_PARITY_BLOCKS);
   private final static int SUCCESS = 0;
-  private final ErasureCodingPolicy sysDefaultPolicy = ErasureCodingPolicyManager
-      .getSystemDefaultPolicy();
+  private final ErasureCodingPolicy sysDefaultPolicy =
+      StripedFileTestUtil.TEST_EC_POLICY;
 
   @Before
   public void setupCluster() throws IOException {