瀏覽代碼

HDFS-9403. Erasure coding: some EC tests are missing timeout. Contributed by Gao Rui.

Change-Id: I13c4bc0e566c4f8af5371d26f8e18a749ed41d8b
zhezhang 9 年之前
父節點
當前提交
6d1213860f
共有 21 個文件被更改,包括 104 次插入1 次删除
  1. 6 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java
  2. 4 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
  3. 5 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
  4. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  5. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java
  6. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java
  7. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
  8. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodeBenchmarkThroughput.java
  9. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java
  10. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java
  11. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java
  12. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java
  13. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
  14. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
  15. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java
  16. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeECN.java
  17. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddOverReplicatedStripedBlocks.java
  18. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
  19. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
  20. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
  21. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java

+ 6 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java

@@ -17,13 +17,19 @@
  */
 package org.apache.hadoop.io.erasurecode;
 
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
+
 import static org.junit.Assert.assertEquals;
 import java.util.HashMap;
 import java.util.Map;
 
 public class TestECSchema {
 
+   @Rule
+   public Timeout globalTimeout = new Timeout(300000);
+
   @Test
   public void testGoodSchema() {
     int numDataUnits = 6;

+ 4 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java

@@ -21,12 +21,16 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 /**
  * Test Reed-Solomon encoding and decoding.
  */
 public class TestRSErasureCoder extends TestErasureCoderBase {
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
 
   @Before
   public void setup() {

+ 5 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java

@@ -18,13 +18,18 @@
 package org.apache.hadoop.io.erasurecode.coder;
 
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 /**
  * Test XOR encoding and decoding.
  */
 public class TestXORCoder extends TestErasureCoderBase {
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() {
     this.encoderClass = XORErasureEncoder.class;

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

@@ -915,6 +915,9 @@ Trunk (Unreleased)
       HDFS-9731. Erasure Coding: Rename BlockECRecoveryCommand to
       BlockECReconstructionCommand. (Rakesh R via zhz)
 
+      HDFS-9403. Erasure coding: some EC tests are missing timeout.
+      (Gao Rui via zhz)
+
 Release 2.9.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java

@@ -1,4 +1,3 @@
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -27,7 +26,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 import org.xml.sax.SAXException;
 
 public class TestErasureCodingCLI extends CLITestHelper {
@@ -36,6 +37,9 @@ public class TestErasureCodingCLI extends CLITestHelper {
   private FileSystem fs = null;
   private String namenode = null;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   @Override
   public void setUp() throws Exception {

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java

@@ -43,6 +43,8 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -66,6 +68,9 @@ public class TestDFSStripedInputStream {
   private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE;
   private final int BLOCK_GROUP_SIZE =  DATA_BLK_NUM * INTERNAL_BLOCK_SIZE;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE);

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java

@@ -31,7 +31,9 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 public class TestDFSStripedOutputStream {
   public static final Log LOG = LogFactory.getLog(
@@ -52,6 +54,9 @@ public class TestDFSStripedOutputStream {
   private final int stripesPerBlock = 4;
   private final int blockSize = cellSize * stripesPerBlock;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     int numDNs = dataBlocks + parityBlocks + 2;

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodeBenchmarkThroughput.java

@@ -27,6 +27,8 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 
@@ -38,6 +40,9 @@ public class TestErasureCodeBenchmarkThroughput {
   private static Configuration conf;
   private static FileSystem fs;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @BeforeClass
   public static void setup() throws IOException {
     conf = new HdfsConfiguration();

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java

@@ -30,13 +30,18 @@ import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 public class TestFileStatusWithECPolicy {
   private MiniDFSCluster cluster;
   private DistributedFileSystem fs;
   private DFSClient client;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void before() throws IOException {
     cluster =

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java

@@ -44,6 +44,8 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -74,6 +76,9 @@ public class TestReadStripedFileWithDecoding {
   private final int[] fileLengths = {smallFileLength, largeFileLength};
   private final int[] dnFailureNums = {1, 2, 3};
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     Configuration conf = new HdfsConfiguration();

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java

@@ -28,6 +28,8 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 
@@ -48,6 +50,9 @@ public class TestReadStripedFileWithMissingBlocks {
   private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE;
   private final int fileLength = blockSize * dataBlocks + 123;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java

@@ -29,7 +29,9 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 import java.util.List;
@@ -49,6 +51,9 @@ public class TestSafeModeWithStripedFile {
   private MiniDFSCluster cluster;
   private Configuration conf;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     conf = new HdfsConfiguration();

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java

@@ -35,6 +35,8 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -61,6 +63,9 @@ public class TestWriteReadStripedFile {
         .getLogger().setLevel(Level.ALL);
   }
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java

@@ -23,7 +23,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBloc
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 import org.mockito.internal.util.reflection.Whitebox;
 
 import java.io.DataOutput;
@@ -57,6 +59,9 @@ public class TestBlockInfoStriped {
     return blocks;
   }
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   /**
    * Test adding storage and reported block
    */

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java

@@ -25,7 +25,9 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 
@@ -44,6 +46,9 @@ public class TestBlockTokenWithDFSStriped extends TestBlockTokenWithDFS {
     FILE_SIZE =  BLOCK_SIZE * dataBlocks * 3;
   }
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   private Configuration getConf() {
     Configuration conf = super.getConf(numDNs);
     conf.setInt("io.bytes.per.checksum", cellSize);

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeECN.java

@@ -22,11 +22,17 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 
 public class TestDataNodeECN {
+
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Test
   public void testECNFlag() throws IOException {
     Configuration conf = new Configuration();

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

@@ -34,7 +34,9 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -56,6 +58,9 @@ public class TestAddOverReplicatedStripedBlocks {
   private final int BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE;
   private final int numDNs = GROUP_SIZE + 3;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     Configuration conf = new Configuration();

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

@@ -52,6 +52,8 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -70,6 +72,9 @@ public class TestAddStripedBlocks {
   private MiniDFSCluster cluster;
   private DistributedFileSystem dfs;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java

@@ -35,6 +35,8 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
 
 import java.io.IOException;
 
@@ -55,6 +57,9 @@ public class TestQuotaWithStripedBlocks {
   private FSDirectory dir;
   private DistributedFileSystem dfs;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setUp() throws IOException {
     final Configuration conf = new Configuration();

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java

@@ -43,7 +43,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 /**
  * This class tests INodeFile with striped feature.
@@ -62,6 +64,9 @@ public class TestStripedINodeFile {
   private static final ErasureCodingPolicy testECPolicy
       = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   private static INodeFile createStripedINodeFile() {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID, true);

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java

@@ -32,7 +32,9 @@ import static org.apache.hadoop.hdfs.util.StripedBlockUtil.*;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 import java.util.Random;
 
@@ -92,6 +94,9 @@ public class TestStripedBlockUtil {
   private int[] byteRangeStartOffsets;
   private int[] byteRangeSizes;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup(){
     blockGroupSizes = new int[]{1, getDelta(CELLSIZE), CELLSIZE,