Browse Source

HDFS-12497. Re-enable TestDFSStripedOutputStreamWithFailure tests. Contributed by Huafeng Wang.

(cherry picked from commit 0477eff8be4505ad2730ec16621105b6df9099ae)
Andrew Wang 7 years ago
parent
commit
ea149c10cf

+ 22 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
 import org.apache.hadoop.security.token.Token;
@@ -77,30 +79,29 @@ public class TestDFSStripedOutputStreamWithFailure {
         .getLogger().setLevel(Level.ALL);
   }
 
+  private final int cellSize = 64 * 1024; //64k
+  private final int stripesPerBlock = 4;
   private ErasureCodingPolicy ecPolicy;
   private int dataBlocks;
   private int parityBlocks;
-  private int cellSize;
-  private final int stripesPerBlock = 4;
   private int blockSize;
   private int blockGroupSize;
 
   private static final int FLUSH_POS =
       9 * DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
 
-  public ErasureCodingPolicy getEcPolicy() {
-    return StripedFileTestUtil.getDefaultECPolicy();
+  public ECSchema getEcSchema() {
+    return StripedFileTestUtil.getDefaultECPolicy().getSchema();
   }
 
   /*
    * Initialize erasure coding policy.
    */
   @Before
-  public void init(){
-    ecPolicy = getEcPolicy();
+  public void init() {
+    ecPolicy = new ErasureCodingPolicy(getEcSchema(), cellSize);
     dataBlocks = ecPolicy.getNumDataUnits();
     parityBlocks = ecPolicy.getNumParityUnits();
-    cellSize = ecPolicy.getCellSize();
     blockSize = cellSize * stripesPerBlock;
     blockGroupSize = blockSize * dataBlocks;
     dnIndexSuite = getDnIndexSuite();
@@ -189,7 +190,7 @@ public class TestDFSStripedOutputStreamWithFailure {
   private List<Integer> lengths;
 
   Integer getLength(int i) {
-    return i >= 0 && i < lengths.size()? lengths.get(i): null;
+    return i >= 0 && i < lengths.size() ? lengths.get(i): null;
   }
 
   private static final Random RANDOM = new Random();
@@ -220,6 +221,10 @@ public class TestDFSStripedOutputStreamWithFailure {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
+    AddErasureCodingPolicyResponse[] res =
+        dfs.addErasureCodingPolicies(new ErasureCodingPolicy[]{ecPolicy});
+    ecPolicy = res[0].getPolicy();
+    dfs.enableErasureCodingPolicy(ecPolicy.getName());
     DFSTestUtil.enableAllECPolicies(dfs);
     dfs.mkdirs(dir);
     dfs.setErasureCodingPolicy(dir, ecPolicy.getName());
@@ -241,7 +246,7 @@ public class TestDFSStripedOutputStreamWithFailure {
     return conf;
   }
 
-  @Test(timeout=240000)
+  @Test(timeout=300000)
   public void testMultipleDatanodeFailure56() throws Exception {
     runTestWithMultipleFailure(getLength(56));
   }
@@ -260,7 +265,8 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   @Test(timeout=240000)
   public void testBlockTokenExpired() throws Exception {
-    final int length = dataBlocks * (blockSize - cellSize);
+    // Make sure killPos is greater than the length of one stripe
+    final int length = dataBlocks * cellSize * 3;
     final HdfsConfiguration conf = newHdfsConfiguration();
 
     conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
@@ -300,13 +306,13 @@ public class TestDFSStripedOutputStreamWithFailure {
       cluster.triggerHeartbeats();
       DatanodeInfo[] info = dfs.getClient().datanodeReport(
           DatanodeReportType.LIVE);
-      assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
+      assertEquals("Mismatches number of live Dns", numDatanodes, info.length);
       final Path dirFile = new Path(dir, "ecfile");
       LambdaTestUtils.intercept(
           IOException.class,
           "File " + dirFile + " could only be written to " +
               numDatanodes + " of the " + dataBlocks + " required nodes for " +
-              getEcPolicy().getName(),
+              ecPolicy.getName(),
           () -> {
             try (FSDataOutputStream out = dfs.create(dirFile, true)) {
               out.write("something".getBytes());
@@ -413,7 +419,7 @@ public class TestDFSStripedOutputStreamWithFailure {
       cluster.triggerHeartbeats();
       DatanodeInfo[] info = dfs.getClient().datanodeReport(
           DatanodeReportType.LIVE);
-      assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
+      assertEquals("Mismatches number of live Dns", numDatanodes, info.length);
       Path srcPath = new Path(dir, "testAddBlockWhenNoSufficientParityNodes");
       int fileLength = cellSize - 1000;
       final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
@@ -432,7 +438,7 @@ public class TestDFSStripedOutputStreamWithFailure {
       try {
         LOG.info("runTest: dn=" + dn + ", length=" + length);
         setup(conf);
-        runTest(length, new int[]{length/2}, new int[]{dn}, false);
+        runTest(length, new int[]{length / 2}, new int[]{dn}, false);
       } catch (Throwable e) {
         final String err = "failed, dn=" + dn + ", length=" + length
             + StringUtils.stringifyException(e);
@@ -582,10 +588,10 @@ public class TestDFSStripedOutputStreamWithFailure {
     long oldGS = -1; // the old GS before bumping
     List<Long> gsList = new ArrayList<>();
     final List<DatanodeInfo> killedDN = new ArrayList<>();
-    int numKilled=0;
+    int numKilled = 0;
     for(; pos.get() < length;) {
       final int i = pos.getAndIncrement();
-      if (numKilled < killPos.length &&  i == killPos[numKilled]) {
+      if (numKilled < killPos.length && i == killPos[numKilled]) {
         assertTrue(firstGS != -1);
         final long gs = getGenerationStamp(stripedOut);
         if (numKilled == 0) {
@@ -706,8 +712,6 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   private void run(int offset) {
     int base = getBase();
-    // TODO: Fix and re-enable these flaky tests. See HDFS-12417.
-    assumeTrue("Test has been temporarily disabled. See HDFS-12417.", false);
     assumeTrue(base >= 0);
     final int i = offset + base;
     final Integer length = getLength(i);

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailureWithRandomECPolicy.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 /**
  * This tests write operation of DFS striped file with a random erasure code
@@ -28,18 +28,18 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 public class TestDFSStripedOutputStreamWithFailureWithRandomECPolicy extends
     TestDFSStripedOutputStreamWithFailure {
 
+  private final ECSchema schema;
+
   private static final Log LOG = LogFactory.getLog(
       TestDFSStripedOutputStreamWithRandomECPolicy.class.getName());
 
-  private ErasureCodingPolicy ecPolicy;
-
   public TestDFSStripedOutputStreamWithFailureWithRandomECPolicy() {
-    ecPolicy = StripedFileTestUtil.getRandomNonDefaultECPolicy();
-    LOG.info(ecPolicy);
+    schema = StripedFileTestUtil.getRandomNonDefaultECPolicy().getSchema();
+    LOG.info(schema);
   }
 
   @Override
-  public ErasureCodingPolicy getEcPolicy() {
-    return ecPolicy;
+  public ECSchema getEcSchema() {
+    return schema;
   }
 }