Browse Source

HDFS-3559. DFSTestUtil: use Builder class to construct DFSTestUtil instances. Contributed by Colin Patrick McCabe.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1355141 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 13 năm trước cách đây
mục cha
commit
9255cb83c8

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

@@ -370,6 +370,9 @@ Branch-2 ( Unreleased changes )
     HDFS-3524. Update TestFileLengthOnClusterRestart for HDFS-3522.  (Brandon
     Li via szetszwo)
 
+    HDFS-3559. DFSTestUtil: use Builder class to construct DFSTestUtil
+    instances. (Colin Patrick McCabe via atm)
+
   BREAKDOWN OF HDFS-3042 SUBTASKS
 
     HDFS-2185. HDFS portion of ZK-based FailoverController (todd)

+ 45 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil.Builder;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -95,10 +96,10 @@ public class DFSTestUtil {
     "zero", "one", "two", "three", "four", "five", "six", "seven", "eight", "nine"
   };
   
-  private int maxLevels;// = 3;
-  private int maxSize;// = 8*1024;
-  private int minSize = 1;
-  private int nFiles;
+  private final int maxLevels;
+  private final int maxSize;
+  private final int minSize;
+  private final int nFiles;
   private MyFile[] files;
   
   /** Creates a new instance of DFSTestUtil
@@ -108,10 +109,11 @@ public class DFSTestUtil {
    * @param maxLevels Maximum number of directory levels
    * @param maxSize Maximum size for file
    */
-  public DFSTestUtil(String testName, int nFiles, int maxLevels, int maxSize) {
+  private DFSTestUtil(int nFiles, int maxLevels, int maxSize, int minSize) {
     this.nFiles = nFiles;
     this.maxLevels = maxLevels;
     this.maxSize = maxSize;
+    this.minSize = minSize;
   }
   
   /**
@@ -778,4 +780,42 @@ public class DFSTestUtil {
     return new DatanodeRegistration(getLocalDatanodeID(),
         new StorageInfo(), new ExportedBlockKeys(), VersionInfo.getVersion());
   }
+
+  public static class Builder {
+    private int maxLevels = 3;
+    private int maxSize = 8*1024;
+    private int minSize = 1;
+    private int nFiles = 1;
+    
+    public Builder() {
+    }
+    
+    public Builder setName(String string) {
+      return this;
+    }
+
+    public Builder setNumFiles(int nFiles) {
+      this.nFiles = nFiles;
+      return this;
+    }
+    
+    public Builder setMaxLevels(int maxLevels) {
+      this.maxLevels = maxLevels;
+      return this;
+    }
+
+    public Builder setMaxSize(int maxSize) {
+      this.maxSize = maxSize;
+      return this;
+    }
+
+    public Builder setMinSize(int minSize) {
+      this.minSize = minSize;
+      return this;
+    }
+    
+    public DFSTestUtil build() {
+      return new DFSTestUtil(nFiles, maxLevels, maxSize, minSize);
+    }
+  }
 }

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

@@ -206,7 +206,8 @@ public class TestCrcCorruption {
     System.out.println("TestCrcCorruption with default parameters");
     Configuration conf1 = new HdfsConfiguration();
     conf1.setInt(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 3 * 1000);
-    DFSTestUtil util1 = new DFSTestUtil("TestCrcCorruption", 40, 3, 8*1024);
+    DFSTestUtil util1 = new DFSTestUtil.Builder().setName("TestCrcCorruption").
+        setNumFiles(40).build();
     thistest(conf1, util1);
 
     //
@@ -216,7 +217,8 @@ public class TestCrcCorruption {
     Configuration conf2 = new HdfsConfiguration();
     conf2.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 17);
     conf2.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 34);
-    DFSTestUtil util2 = new DFSTestUtil("TestCrcCorruption", 40, 3, 400);
+    DFSTestUtil util2 = new DFSTestUtil.Builder().setName("TestCrcCorruption").
+        setNumFiles(40).setMaxSize(400).build();
     thistest(conf2, util2);
   }
 

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java

@@ -58,7 +58,8 @@ public class TestFileCorruption extends TestCase {
   /** check if DFS can handle corrupted blocks properly */
   public void testFileCorruption() throws Exception {
     MiniDFSCluster cluster = null;
-    DFSTestUtil util = new DFSTestUtil("TestFileCorruption", 20, 3, 8*1024);
+    DFSTestUtil util = new DFSTestUtil.Builder().setName("TestFileCorruption").
+        setNumFiles(20).build();
     try {
       Configuration conf = new HdfsConfiguration();
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRestartDFS.java

@@ -31,7 +31,8 @@ import org.apache.hadoop.fs.Path;
 public class TestRestartDFS extends TestCase {
   public void runTests(Configuration conf, boolean serviceTest) throws Exception {
     MiniDFSCluster cluster = null;
-    DFSTestUtil files = new DFSTestUtil("TestRestartDFS", 20, 3, 8*1024);
+    DFSTestUtil files = new DFSTestUtil.Builder().setName("TestRestartDFS").
+        setNumFiles(20).build();
 
     final String dir = "/srcdat";
     final Path rootpath = new Path("/");

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java

@@ -59,7 +59,8 @@ public class TestDatanodeRestart {
     try {
       // test finalized replicas
       final String TopDir = "/test";
-      DFSTestUtil util = new DFSTestUtil("TestCrcCorruption", 2, 3, 8*1024);
+      DFSTestUtil util = new DFSTestUtil.Builder().
+          setName("TestDatanodeRestart").setNumFiles(2).build();
       util.createFiles(fs, TopDir, (short)3);
       util.waitReplication(fs, TopDir, (short)3);
       util.checkFiles(fs, TopDir);

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

@@ -80,7 +80,8 @@ public class TestAuditLogs {
     final long precision = 1L;
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
-    util = new DFSTestUtil("TestAuditAllowed", 20, 3, 8*1024);
+    util = new DFSTestUtil.Builder().setName("TestAuditAllowed").
+        setNumFiles(20).build();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
     fs = cluster.getFileSystem();
     util.createFiles(fs, fileName);

+ 13 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -101,7 +101,8 @@ public class TestFsck {
   /** do fsck */
   @Test
   public void testFsck() throws Exception {
-    DFSTestUtil util = new DFSTestUtil("TestFsck", 20, 3, 8*1024);
+    DFSTestUtil util = new DFSTestUtil.Builder().setName("TestFsck").
+        setNumFiles(20).build();
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
     try {
@@ -174,7 +175,8 @@ public class TestFsck {
   
   @Test
   public void testFsckNonExistent() throws Exception {
-    DFSTestUtil util = new DFSTestUtil("TestFsck", 20, 3, 8*1024);
+    DFSTestUtil util = new DFSTestUtil.Builder().setName("TestFsck").
+        setNumFiles(20).build();
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
     try {
@@ -197,7 +199,8 @@ public class TestFsck {
   /** Test fsck with permission set on inodes */
   @Test
   public void testFsckPermission() throws Exception {
-    final DFSTestUtil util = new DFSTestUtil(getClass().getSimpleName(), 20, 3, 8*1024);
+    final DFSTestUtil util = new DFSTestUtil.Builder().
+        setName(getClass().getSimpleName()).setNumFiles(20).build();
     final Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
 
@@ -246,7 +249,8 @@ public class TestFsck {
   @Test
   public void testFsckMoveAndDelete() throws Exception {
     final int MAX_MOVE_TRIES = 5;
-    DFSTestUtil util = new DFSTestUtil("TestFsck", 5, 3, 8*1024);
+    DFSTestUtil util = new DFSTestUtil.Builder().
+        setName("TestFsckMoveAndDelete").setNumFiles(5).build();
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
     try {
@@ -319,7 +323,8 @@ public class TestFsck {
   
   @Test
   public void testFsckOpenFiles() throws Exception {
-    DFSTestUtil util = new DFSTestUtil("TestFsck", 4, 3, 8*1024); 
+    DFSTestUtil util = new DFSTestUtil.Builder().setName("TestFsck").
+        setNumFiles(4).build();
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
     try {
@@ -492,7 +497,9 @@ public class TestFsck {
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
       fs = cluster.getFileSystem();
-      DFSTestUtil util = new DFSTestUtil("testGetCorruptFiles", 3, 1, 1024);
+      DFSTestUtil util = new DFSTestUtil.Builder().
+          setName("testGetCorruptFiles").setNumFiles(3).setMaxLevels(1).
+          setMaxSize(1024).build();
       util.createFiles(fs, "/corruptData", (short) 1);
       util.waitReplication(fs, "/corruptData", (short) 1);
 

+ 15 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java

@@ -68,7 +68,9 @@ public class TestListCorruptFileBlocks {
       FileSystem fs = cluster.getFileSystem();
 
       // create two files with one block each
-      DFSTestUtil util = new DFSTestUtil("testCorruptFilesCorruptedBlock", 2, 1, 512);
+      DFSTestUtil util = new DFSTestUtil.Builder().
+          setName("testCorruptFilesCorruptedBlock").setNumFiles(2).
+          setMaxLevels(1).setMaxSize(512).build();
       util.createFiles(fs, "/srcdat10");
 
       // fetch bad file list from namenode. There should be none.
@@ -152,8 +154,9 @@ public class TestListCorruptFileBlocks {
       FileSystem fs = cluster.getFileSystem();
 
       // create two files with one block each
-      DFSTestUtil util = new DFSTestUtil("testListCorruptFileBlocksInSafeMode",
-                                         2, 1, 512);
+      DFSTestUtil util = new DFSTestUtil.Builder().
+          setName("testListCorruptFileBlocksInSafeMode").setNumFiles(2).
+          setMaxLevels(1).setMaxSize(512).build();
       util.createFiles(fs, "/srcdat10");
 
       // fetch bad file list from namenode. There should be none.
@@ -272,7 +275,9 @@ public class TestListCorruptFileBlocks {
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
       fs = cluster.getFileSystem();
-      DFSTestUtil util = new DFSTestUtil("testGetCorruptFiles", 3, 1, 1024);
+      DFSTestUtil util = new DFSTestUtil.Builder().
+          setName("testGetCorruptFiles").setNumFiles(3).setMaxLevels(1).
+          setMaxSize(1024).build();
       util.createFiles(fs, "/corruptData");
 
       final NameNode namenode = cluster.getNameNode();
@@ -381,7 +386,9 @@ public class TestListCorruptFileBlocks {
       cluster.waitActive();
       fs = cluster.getFileSystem();
       DistributedFileSystem dfs = (DistributedFileSystem) fs;
-      DFSTestUtil util = new DFSTestUtil("testGetCorruptFiles", 3, 1, 1024);
+      DFSTestUtil util = new DFSTestUtil.Builder().
+          setName("testGetCorruptFiles").setNumFiles(3).
+          setMaxLevels(1).setMaxSize(1024).build();
       util.createFiles(fs, "/corruptData");
 
       RemoteIterator<Path> corruptFileBlocks = 
@@ -451,8 +458,9 @@ public class TestListCorruptFileBlocks {
         FSNamesystem.DEFAULT_MAX_CORRUPT_FILEBLOCKS_RETURNED;
 
       // create 110 files with one block each
-      DFSTestUtil util = new DFSTestUtil("testMaxCorruptFiles", 
-          maxCorruptFileBlocks * 3, 1, 512);
+      DFSTestUtil util = new DFSTestUtil.Builder().setName("testMaxCorruptFiles").
+          setNumFiles(maxCorruptFileBlocks * 3).setMaxLevels(1).setMaxSize(512).
+          build();
       util.createFiles(fs, "/srcdat2", (short) 1);
       util.waitReplication(fs, "/srcdat2", (short) 1);
 

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

@@ -49,7 +49,8 @@ public class TestParallelImageWrite extends TestCase {
     MiniDFSCluster cluster = null;
     FSNamesystem fsn = null;
     int numNamenodeDirs;
-    DFSTestUtil files = new DFSTestUtil("TestRestartDFS", 200, 3, 8*1024);
+    DFSTestUtil files = new DFSTestUtil.Builder().setName("TestRestartDFS").
+        setNumFiles(200).build();
 
     final String dir = "/srcdat";
     final Path rootpath = new Path("/");