You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by jg...@apache.org on 2010/10/21 00:58:56 UTC

svn commit: r1025788 [2/3] - in /hadoop/hdfs/branches/HDFS-1052: ./ src/contrib/fuse-dfs/src/test/ src/contrib/hdfsproxy/src/test/org/apache/hadoop/hdfsproxy/ src/test/aop/org/apache/hadoop/fs/ src/test/aop/org/apache/hadoop/hdfs/ src/test/aop/org/apac...

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeDeath.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeDeath.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeDeath.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeDeath.java Wed Oct 20 22:58:52 2010
@@ -288,7 +288,8 @@ public class TestDatanodeDeath extends T
     conf.setInt("dfs.heartbeat.interval", 2);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 2);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 5000);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                                               .numDataNodes(numDatanodes).build();
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
     Modify modThread = null;
@@ -346,7 +347,7 @@ public class TestDatanodeDeath extends T
     int myMaxNodes = 5;
     System.out.println("SimpleTest starting with DataNode to Kill " + 
                        datanodeToKill);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, myMaxNodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(myMaxNodes).build();
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
     short repl = 3;

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeRegistration.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeRegistration.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeRegistration.java Wed Oct 20 22:58:52 2010
@@ -43,7 +43,7 @@ public class TestDatanodeRegistration ex
     MiniDFSCluster cluster = null;
     FileSystem fs = null;
     try {
-      cluster = new MiniDFSCluster(conf, 1, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).build();
       fs = cluster.getFileSystem();
 
       InetSocketAddress addr = new InetSocketAddress(

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeReport.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeReport.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeReport.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeReport.java Wed Oct 20 22:58:52 2010
@@ -43,7 +43,7 @@ public class TestDatanodeReport extends 
         DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500); // 0.5s
     conf.setLong("dfs.heartbeat.interval", 1L);
     MiniDFSCluster cluster = 
-      new MiniDFSCluster(conf, NUM_OF_DATANODES, true, null);
+      new MiniDFSCluster.Builder(conf).numDataNodes(NUM_OF_DATANODES).build();
     try {
       //wait until the cluster is up
       cluster.waitActive();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDecommission.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDecommission.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDecommission.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDecommission.java Wed Oct 20 22:58:52 2010
@@ -256,7 +256,8 @@ public class TestDecommission extends Te
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 4);
     writeConfigFile(localFileSys, excludeFile, null);
 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                                               .numDataNodes(numDatanodes).build();
     cluster.waitActive();
     InetSocketAddress addr = new InetSocketAddress("localhost", 
                                                    cluster.getNameNodePort());

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDistributedFileSystem.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDistributedFileSystem.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDistributedFileSystem.java Wed Oct 20 22:58:52 2010
@@ -58,7 +58,7 @@ public class TestDistributedFileSystem {
   @Test
   public void testFileSystemCloseAll() throws Exception {
     Configuration conf = getTestConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 0, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
     URI address = FileSystem.getDefaultUri(conf);
 
     try {
@@ -82,7 +82,7 @@ public class TestDistributedFileSystem {
   @Test
   public void testDFSClose() throws Exception {
     Configuration conf = getTestConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     FileSystem fileSys = cluster.getFileSystem();
 
     try {
@@ -103,7 +103,7 @@ public class TestDistributedFileSystem {
     MiniDFSCluster cluster = null;
 
     try {
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final Path filepath = new Path("/test/LeaseChecker/foo");
       final long millis = System.currentTimeMillis();
 
@@ -186,7 +186,7 @@ public class TestDistributedFileSystem {
     int lsLimit = 2;
     final Configuration conf = getTestConfiguration();
     conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, lsLimit);
-    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     try {
       final FileSystem fs = cluster.getFileSystem();
       Path dir = new Path("/test");
@@ -290,7 +290,7 @@ public class TestDistributedFileSystem {
     final Configuration conf = getTestConfiguration();
     conf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "localhost");
 
-    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem hdfs = cluster.getFileSystem();
     final String hftpuri = "hftp://" + conf.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
     System.out.println("hftpuri=" + hftpuri);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSInputChecker.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSInputChecker.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSInputChecker.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSInputChecker.java Wed Oct 20 22:58:52 2010
@@ -299,7 +299,7 @@ public class TestFSInputChecker extends 
     rand.nextBytes(expected);
 
     // test DFS
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fileSys = cluster.getFileSystem();
     try {
       testChecker(fileSys, true);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSOutputSummer.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSOutputSummer.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSOutputSummer.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSOutputSummer.java Wed Oct 20 22:58:52 2010
@@ -113,8 +113,9 @@ public class TestFSOutputSummer extends 
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, BYTES_PER_CHECKSUM);
-    MiniDFSCluster cluster = new MiniDFSCluster(
-        conf, NUM_OF_DATANODES, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                                               .numDataNodes(NUM_OF_DATANODES)
+                                               .build();
     fileSys = cluster.getFileSystem();
     try {
       Path file = new Path("try.dat");

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend.java Wed Oct 20 22:58:52 2010
@@ -106,7 +106,7 @@ public class TestFileAppend extends Test
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     InetSocketAddress addr = new InetSocketAddress("localhost",
                                                    cluster.getNameNodePort());
@@ -177,7 +177,7 @@ public class TestFileAppend extends Test
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     try {
 
@@ -232,7 +232,7 @@ public class TestFileAppend extends Test
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     try {
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend2.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend2.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend2.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend2.java Wed Oct 20 22:58:52 2010
@@ -86,7 +86,7 @@ public class TestFileAppend2 extends Tes
     conf.setInt("dfs.datanode.handler.count", 50);
     conf.setBoolean("dfs.support.append", true);
     fileContents = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     try {
       { // test appending to a file.
@@ -339,8 +339,9 @@ public class TestFileAppend2 extends Tes
     conf.setInt("dfs.datanode.handler.count", 50);
     conf.setBoolean("dfs.support.append", true);
 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, 
-                                                true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                                               .numDataNodes(numDatanodes)
+                                               .build();
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend3.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend3.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend3.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend3.java Wed Oct 20 22:58:52 2010
@@ -69,7 +69,7 @@ public class TestFileAppend3 extends jun
         conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
         conf.setBoolean("dfs.support.append", true);
         buffersize = conf.getInt("io.file.buffer.size", 4096);
-        cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null);
+        cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
         fs = (DistributedFileSystem)cluster.getFileSystem();
       }
     

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend4.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend4.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend4.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend4.java Wed Oct 20 22:58:52 2010
@@ -149,7 +149,7 @@ public class TestFileAppend4 {
    */
   @Test(timeout=60000)
   public void testRecoverFinalizedBlock() throws Throwable {
-    cluster = new MiniDFSCluster(conf, 3, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
  
     try {
       cluster.waitActive();
@@ -220,7 +220,7 @@ public class TestFileAppend4 {
    */
   @Test(timeout=60000)
   public void testCompleteOtherLeaseHoldersFile() throws Throwable {
-    cluster = new MiniDFSCluster(conf, 3, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
  
     try {
       cluster.waitActive();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileConcurrentReader.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileConcurrentReader.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileConcurrentReader.java Wed Oct 20 22:58:52 2010
@@ -85,7 +85,7 @@ public class TestFileConcurrentReader ex
     if (cluster != null) {
       cluster.shutdown();
     }
-    cluster = new MiniDFSCluster(conf, 1, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).build();
     cluster.waitClusterUp();
     fileSystem = cluster.getFileSystem();
   }

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCorruption.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCorruption.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCorruption.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCorruption.java Wed Oct 20 22:58:52 2010
@@ -58,7 +58,7 @@ public class TestFileCorruption extends 
     DFSTestUtil util = new DFSTestUtil("TestFileCorruption", 20, 3, 8*1024);
     try {
       Configuration conf = new HdfsConfiguration();
-      cluster = new MiniDFSCluster(conf, 3, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
       FileSystem fs = cluster.getFileSystem();
       util.createFiles(fs, "/srcdat");
       // Now deliberately remove the blocks
@@ -113,7 +113,7 @@ public class TestFileCorruption extends 
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new HdfsConfiguration();
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       cluster.waitActive();
       
       FileSystem fs = cluster.getFileSystem();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java Wed Oct 20 22:58:52 2010
@@ -106,8 +106,9 @@ public class TestFileCreation extends ju
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, FSConstants.DEFAULT_WRITE_PACKET_SIZE);
     conf.setInt("dfs.replication", FSConstants.DEFAULT_REPLICATION_FACTOR + 1);
     conf.setInt("io.file.buffer.size", FSConstants.DEFAULT_FILE_BUFFER_SIZE);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf,
-        FSConstants.DEFAULT_REPLICATION_FACTOR + 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                     .numDataNodes(FSConstants.DEFAULT_REPLICATION_FACTOR + 1)
+                     .build();
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
     try {
@@ -131,7 +132,7 @@ public class TestFileCreation extends ju
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     try {
 
@@ -210,7 +211,7 @@ public class TestFileCreation extends ju
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     FileSystem localfs = FileSystem.getLocal(conf);
 
@@ -275,7 +276,7 @@ public class TestFileCreation extends ju
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     cluster.waitActive();
     InetSocketAddress addr = new InetSocketAddress("localhost",
@@ -349,7 +350,7 @@ public class TestFileCreation extends ju
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     DistributedFileSystem dfs = null;
     try {
       cluster.waitActive();
@@ -419,7 +420,7 @@ public class TestFileCreation extends ju
     }
 
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = null;
     try {
       cluster.waitActive();
@@ -489,8 +490,9 @@ public class TestFileCreation extends ju
         Thread.sleep(2*MAX_IDLE_TIME);
       } catch (InterruptedException e) {
       }
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                               .format(false)
+                                               .build();
       cluster.waitActive();
 
       // restart cluster yet again. This triggers the code to read in
@@ -500,8 +502,9 @@ public class TestFileCreation extends ju
         Thread.sleep(5000);
       } catch (InterruptedException e) {
       }
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
       fs = cluster.getFileSystem();
 
@@ -554,7 +557,7 @@ public class TestFileCreation extends ju
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     DistributedFileSystem dfs = (DistributedFileSystem) fs;
     DFSClient dfsclient = dfs.dfs;
@@ -589,7 +592,7 @@ public class TestFileCreation extends ju
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     final Path path = new Path("/" + System.currentTimeMillis()
         + "-testFileCreationNonRecursive");
@@ -689,7 +692,7 @@ public class TestFileCreation extends ju
    */
   public void testConcurrentFileCreation() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
 
     try {
       FileSystem fs = cluster.getFileSystem();
@@ -731,7 +734,7 @@ public class TestFileCreation extends ju
     conf.setInt("dfs.heartbeat.interval", 1);
 
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
     DistributedFileSystem dfs = null;
     try {
       cluster.waitActive();
@@ -790,7 +793,7 @@ public class TestFileCreation extends ju
     Configuration conf = new HdfsConfiguration();
 
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
     DistributedFileSystem dfs = null;
     try {
       cluster.waitActive();
@@ -821,7 +824,7 @@ public class TestFileCreation extends ju
     conf.setInt("ipc.ping.interval", 10000); // hdfs timeout is now 10 second
 
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
     DistributedFileSystem dfs = null;
     try {
       cluster.waitActive();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationClient.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationClient.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationClient.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationClient.java Wed Oct 20 22:58:52 2010
@@ -50,7 +50,7 @@ public class TestFileCreationClient exte
     Configuration conf = new HdfsConfiguration();
     conf.setInt("dfs.datanode.handler.count", 1);
     conf.setInt("dfs.replication", REPLICATION);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, REPLICATION, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION).build();
 
     try {
       final FileSystem fs = cluster.getFileSystem();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationDelete.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationDelete.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationDelete.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationDelete.java Wed Oct 20 22:58:52 2010
@@ -45,7 +45,7 @@ public class TestFileCreationDelete exte
     conf.setBoolean("dfs.support.append", true);
 
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = null;
     try {
       cluster.waitActive();
@@ -76,16 +76,18 @@ public class TestFileCreationDelete exte
       // This ensures that leases are persisted in fsimage.
       cluster.shutdown();
       try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
 
       // restart cluster yet again. This triggers the code to read in
       // persistent leases from fsimage.
       cluster.shutdown();
       try {Thread.sleep(5000);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
       fs = cluster.getFileSystem();
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationEmpty.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationEmpty.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationEmpty.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationEmpty.java Wed Oct 20 22:58:52 2010
@@ -55,7 +55,7 @@ public class TestFileCreationEmpty exten
     conf.setInt("dfs.heartbeat.interval", 1);
 
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
     try {
       cluster.waitActive();
       DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileStatus.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileStatus.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileStatus.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileStatus.java Wed Oct 20 22:58:52 2010
@@ -71,7 +71,7 @@ public class TestFileStatus {
   public static void testSetUp() throws Exception {
     conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, 2);
-    cluster = new MiniDFSCluster(conf, 1, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).build();
     fs = cluster.getFileSystem();
     fc = FileContext.getFileContext(cluster.getURI(), conf);
     hftpfs = cluster.getHftpFileSystem();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestGetBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestGetBlocks.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestGetBlocks.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestGetBlocks.java Wed Oct 20 22:58:52 2010
@@ -51,8 +51,9 @@ public class TestGetBlocks extends TestC
     final Random r = new Random();
     
     CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
-    MiniDFSCluster cluster = new MiniDFSCluster(
-          CONF, REPLICATION_FACTOR, true, null );
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF)
+                                               .numDataNodes(REPLICATION_FACTOR)
+                                               .build();
     try {
       cluster.waitActive();
       

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java Wed Oct 20 22:58:52 2010
@@ -30,7 +30,7 @@ public class TestHDFSFileSystemContract 
   @Override
   protected void setUp() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster(conf, 2, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     fs = cluster.getFileSystem();
     defaultWorkingDirectory = "/user/" + 
            UserGroupInformation.getCurrentUser().getShortUserName();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSTrash.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSTrash.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSTrash.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSTrash.java Wed Oct 20 22:58:52 2010
@@ -38,7 +38,7 @@ public class TestHDFSTrash extends TestT
     TestSetup setup = new TestSetup(new TestSuite(TestHDFSTrash.class)) {
       protected void setUp() throws Exception {
         Configuration conf = new HdfsConfiguration();
-        cluster = new MiniDFSCluster(conf, 2, true, null);
+        cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       }
       protected void tearDown() throws Exception {
         if (cluster != null) { cluster.shutdown(); }

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHFlush.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHFlush.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHFlush.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHFlush.java Wed Oct 20 22:58:52 2010
@@ -98,7 +98,8 @@ public class TestHFlush {
     final int SECTIONS = 10;
 
     fileContent = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, replicas, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                                               .numDataNodes(replicas).build();
     // Make sure we work with DFS in order to utilize all its functionality
     DistributedFileSystem fileSystem =
         (DistributedFileSystem)cluster.getFileSystem();
@@ -168,7 +169,7 @@ public class TestHFlush {
     final Path p = new Path("/pipelineHeartbeat/foo");
     System.out.println("p=" + p);
     
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
     DistributedFileSystem fs = (DistributedFileSystem)cluster.getFileSystem();
 
     byte[] fileContents = AppendTestUtil.initBuffer(fileLen);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHftpFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHftpFileSystem.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHftpFileSystem.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHftpFileSystem.java Wed Oct 20 22:58:52 2010
@@ -68,7 +68,7 @@ public class TestHftpFileSystem extends 
     config = new Configuration();
     config.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "localhost");
 
-    cluster = new MiniDFSCluster(config, 2, true, null);
+    cluster = new MiniDFSCluster.Builder(config).numDataNodes(2).build();
     hdfs = cluster.getFileSystem();
     final String hftpuri = 
       "hftp://" + config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java Wed Oct 20 22:58:52 2010
@@ -138,8 +138,7 @@ public class TestInjectionForSimulatedSt
       conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, checksumSize);
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
       //first time format
-      cluster = new MiniDFSCluster(0, conf, numDataNodes, true,
-                                   true, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
       cluster.waitActive();
       DFSClient dfsClient = new DFSClient(new InetSocketAddress("localhost",
                                             cluster.getNameNodePort()),
@@ -169,8 +168,10 @@ public class TestInjectionForSimulatedSt
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
       conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, "0.0f"); 
       
-      cluster = new MiniDFSCluster(0, conf, numDataNodes*2, false,
-                                   true, null, null);
+      cluster = new MiniDFSCluster.Builder(conf)
+                                  .numDataNodes(numDataNodes * 2)
+                                  .format(false)
+                                  .build();
       cluster.waitActive();
       Set<Block> uniqueBlocks = new HashSet<Block>();
       for (int i=0; i<blocksList.length; ++i) {

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLargeBlock.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLargeBlock.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLargeBlock.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLargeBlock.java Wed Oct 20 22:58:52 2010
@@ -183,7 +183,7 @@ public class TestLargeBlock extends juni
     if (simulatedStorage) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
     try {
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLease.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLease.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLease.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLease.java Wed Oct 20 22:58:52 2010
@@ -32,7 +32,7 @@ public class TestLease extends junit.fra
 
   public void testLease() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     try {
       FileSystem fs = cluster.getFileSystem();
       assertTrue(fs.mkdirs(dir));

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery.java Wed Oct 20 22:58:52 2010
@@ -72,7 +72,7 @@ public class TestLeaseRecovery extends j
     MiniDFSCluster cluster = null;
 
     try {
-      cluster = new MiniDFSCluster(conf, 5, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build();
       cluster.waitActive();
 
       //create a file

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery2.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery2.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery2.java Wed Oct 20 22:58:52 2010
@@ -72,7 +72,7 @@ public class TestLeaseRecovery2 {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setInt("dfs.heartbeat.interval", 1);
 
-    cluster = new MiniDFSCluster(conf, 5, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(5).build();
     cluster.waitActive();
     dfs = (DistributedFileSystem)cluster.getFileSystem();
   }

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInDFS.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInDFS.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInDFS.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInDFS.java Wed Oct 20 22:58:52 2010
@@ -39,7 +39,7 @@ public class TestListFilesInDFS extends 
 
   @BeforeClass
   public static void testSetUp() throws Exception {
-    cluster = new MiniDFSCluster(conf, 1, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).build();
     fs = cluster.getFileSystem();
     fs.delete(TEST_DIR, true);
   }

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInFileContext.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInFileContext.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInFileContext.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInFileContext.java Wed Oct 20 22:58:52 2010
@@ -63,7 +63,7 @@ public class TestListFilesInFileContext 
 
   @BeforeClass
   public static void testSetUp() throws Exception {
-    cluster = new MiniDFSCluster(conf, 1, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).build();
     fc = FileContext.getFileContext(cluster.getConfiguration());
     fc.delete(TEST_DIR, true);
   }

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListPathServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListPathServlet.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListPathServlet.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListPathServlet.java Wed Oct 20 22:58:52 2010
@@ -55,7 +55,7 @@ public class TestListPathServlet {
   @BeforeClass
   public static void setup() throws Exception {
     // start a cluster with single datanode
-    cluster = new MiniDFSCluster(CONF, 1, true, null);
+    cluster = new MiniDFSCluster.Builder(CONF).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLocalDFS.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLocalDFS.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLocalDFS.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLocalDFS.java Wed Oct 20 22:58:52 2010
@@ -61,7 +61,7 @@ public class TestLocalDFS extends TestCa
    */
   public void testWorkingDirectory() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fileSys = cluster.getFileSystem();
     try {
       Path orig_path = fileSys.getWorkingDirectory();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java Wed Oct 20 22:58:52 2010
@@ -52,7 +52,7 @@ public class TestMissingBlocksAlert exte
       int fileLen = 10*1024;
 
       //start a cluster with single datanode
-      cluster = new MiniDFSCluster(conf, 1, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
 
       DistributedFileSystem dfs = 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestModTime.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestModTime.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestModTime.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestModTime.java Wed Oct 20 22:58:52 2010
@@ -76,7 +76,8 @@ public class TestModTime extends TestCas
   public void testModTime() throws IOException {
     Configuration conf = new HdfsConfiguration();
 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                                               .numDataNodes(numDatanodes).build();
     cluster.waitActive();
     InetSocketAddress addr = new InetSocketAddress("localhost", 
                                                    cluster.getNameNodePort());

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java Wed Oct 20 22:58:52 2010
@@ -61,7 +61,7 @@ public class TestPipelines {
 
   @Before
   public void startUpCluster() throws IOException {
-    cluster = new MiniDFSCluster(conf, REPL_FACTOR, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL_FACTOR).build();
     fs = (DistributedFileSystem) cluster.getFileSystem();
   }
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPread.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPread.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPread.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPread.java Wed Oct 20 22:58:52 2010
@@ -210,7 +210,7 @@ public class TestPread extends TestCase 
     if (disableTransferTo) {
       conf.setBoolean("dfs.datanode.transferTo.allowed", false);
     }
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     FileSystem fileSys = cluster.getFileSystem();
     try {
       Path file1 = new Path("preadtest.dat");

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java Wed Oct 20 22:58:52 2010
@@ -62,7 +62,7 @@ public class TestQuota extends TestCase 
     // Space quotas
     conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "512");
     conf.setBoolean("dfs.support.append", true);
-    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
                 fs instanceof DistributedFileSystem);
@@ -283,7 +283,7 @@ public class TestQuota extends TestCase 
    *  mkdirs, rename, and delete */
   public void testNamespaceCommands() throws Exception {
     final Configuration conf = new HdfsConfiguration();
-    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
                 fs instanceof DistributedFileSystem);
@@ -455,7 +455,7 @@ public class TestQuota extends TestCase 
     // diskspace quotas
     conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "512");
     conf.setBoolean("dfs.support.append", true);
-    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
                 fs instanceof DistributedFileSystem);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReadWhileWriting.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReadWhileWriting.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReadWhileWriting.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReadWhileWriting.java Wed Oct 20 22:58:52 2010
@@ -59,7 +59,7 @@ public class TestReadWhileWriting {
     conf.setLong("dfs.heartbeat.interval", 1);
 
     // create cluster
-    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     try {
       //change the lease limits.
       cluster.setLeasePeriod(SOFT_LEASE_LIMIT, HARD_LEASE_LIMIT);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRenameWhileOpen.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRenameWhileOpen.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRenameWhileOpen.java Wed Oct 20 22:58:52 2010
@@ -57,7 +57,7 @@ public class TestRenameWhileOpen extends
 
     // create cluster
     System.out.println("Test 1*****************************");
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = null;
     try {
       cluster.waitActive();
@@ -102,16 +102,18 @@ public class TestRenameWhileOpen extends
       // This ensures that leases are persisted in fsimage.
       cluster.shutdown();
       try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
 
       // restart cluster yet again. This triggers the code to read in
       // persistent leases from fsimage.
       cluster.shutdown();
       try {Thread.sleep(5000);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
       fs = cluster.getFileSystem();
 
@@ -141,7 +143,7 @@ public class TestRenameWhileOpen extends
     System.out.println("Test 2************************************");
 
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = null;
     try {
       cluster.waitActive();
@@ -174,16 +176,18 @@ public class TestRenameWhileOpen extends
       // This ensures that leases are persisted in fsimage.
       cluster.shutdown();
       try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
 
       // restart cluster yet again. This triggers the code to read in
       // persistent leases from fsimage.
       cluster.shutdown();
       try {Thread.sleep(5000);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
       fs = cluster.getFileSystem();
 
@@ -214,7 +218,7 @@ public class TestRenameWhileOpen extends
     System.out.println("Test 3************************************");
 
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = null;
     try {
       cluster.waitActive();
@@ -239,16 +243,18 @@ public class TestRenameWhileOpen extends
       // This ensures that leases are persisted in fsimage.
       cluster.shutdown();
       try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
 
       // restart cluster yet again. This triggers the code to read in
       // persistent leases from fsimage.
       cluster.shutdown();
       try {Thread.sleep(5000);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
       fs = cluster.getFileSystem();
 
@@ -277,7 +283,7 @@ public class TestRenameWhileOpen extends
     System.out.println("Test 4************************************");
 
     // create cluster
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = null;
     try {
       cluster.waitActive();
@@ -301,16 +307,18 @@ public class TestRenameWhileOpen extends
       // This ensures that leases are persisted in fsimage.
       cluster.shutdown();
       try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
 
       // restart cluster yet again. This triggers the code to read in
       // persistent leases from fsimage.
       cluster.shutdown();
       try {Thread.sleep(5000);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
       fs = cluster.getFileSystem();
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java Wed Oct 20 22:58:52 2010
@@ -152,7 +152,7 @@ public class TestReplication extends Tes
     DFSClient dfsClient = null;
     LocatedBlocks blocks = null;
     int replicaCount = 0;
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
     dfsClient = new DFSClient(new InetSocketAddress("localhost",
@@ -197,7 +197,9 @@ public class TestReplication extends Tes
     if (simulated) {
       conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
     }
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, racks);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                                               .numDataNodes(numDatanodes)
+                                               .racks(racks).build();
     cluster.waitActive();
     
     InetSocketAddress addr = new InetSocketAddress("localhost",
@@ -311,8 +313,7 @@ public class TestReplication extends Tes
       Configuration conf = new HdfsConfiguration();
       conf.set("dfs.replication", Integer.toString(numDataNodes));
       //first time format
-      cluster = new MiniDFSCluster(0, conf, numDataNodes, true,
-                                   true, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
       cluster.waitActive();
       DFSClient dfsClient = new DFSClient(new InetSocketAddress("localhost",
                                             cluster.getNameNodePort()),
@@ -380,8 +381,10 @@ public class TestReplication extends Tes
       conf.set("dfs.datanode.block.write.timeout.sec", Integer.toString(5));
       conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, "0.75f"); // only 3 copies exist
       
-      cluster = new MiniDFSCluster(0, conf, numDataNodes*2, false,
-                                   true, null, null);
+      cluster = new MiniDFSCluster.Builder(conf)
+                                  .numDataNodes(numDataNodes * 2)
+                                  .format(false)
+                                  .build();
       cluster.waitActive();
       
       dfsClient = new DFSClient(new InetSocketAddress("localhost",
@@ -402,7 +405,7 @@ public class TestReplication extends Tes
    * @throws Exception
    */
   public void testReplicateLenMismatchedBlock() throws Exception {
-    MiniDFSCluster cluster = new MiniDFSCluster(new HdfsConfiguration(), 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()).numDataNodes(2).build();
     try {
       cluster.waitActive();
       // test truncated block

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRestartDFS.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRestartDFS.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRestartDFS.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRestartDFS.java Wed Oct 20 22:58:52 2010
@@ -46,7 +46,7 @@ public class TestRestartDFS extends Test
         conf.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
                  "localhost:0");
       }
-      cluster = new MiniDFSCluster(conf, 4, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
       FileSystem fs = cluster.getFileSystem();
       files.createFiles(fs, dir);
 
@@ -65,7 +65,7 @@ public class TestRestartDFS extends Test
                  "localhost:0");
       }
       // Here we restart the MiniDFScluster without formatting namenode
-      cluster = new MiniDFSCluster(conf, 4, false, null); 
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).format(false).build(); 
       FileSystem fs = cluster.getFileSystem();
       assertTrue("Filesystem corrupted after restart.",
                  files.checkFiles(fs, dir));
@@ -89,7 +89,7 @@ public class TestRestartDFS extends Test
       }
       // This is a second restart to check that after the first restart
       // the image written in parallel to both places did not get corrupted
-      cluster = new MiniDFSCluster(conf, 4, false, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).format(false).build();
       FileSystem fs = cluster.getFileSystem();
       assertTrue("Filesystem corrupted after restart.",
                  files.checkFiles(fs, dir));

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSafeMode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSafeMode.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSafeMode.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSafeMode.java Wed Oct 20 22:58:52 2010
@@ -58,7 +58,7 @@ public class TestSafeMode extends TestCa
       Configuration conf = new HdfsConfiguration();
       // disable safemode extension to make the test run faster.
       conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, "1");
-      cluster = new MiniDFSCluster(conf, 1, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
       
       fs = (DistributedFileSystem)cluster.getFileSystem();
@@ -74,7 +74,7 @@ public class TestSafeMode extends TestCa
       cluster.shutdown();
       
       // now bring up just the NameNode.
-      cluster = new MiniDFSCluster(conf, 0, false, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build();
       cluster.waitActive();
       fs = (DistributedFileSystem)cluster.getFileSystem();
       

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSeekBug.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSeekBug.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSeekBug.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSeekBug.java Wed Oct 20 22:58:52 2010
@@ -124,7 +124,7 @@ public class TestSeekBug extends TestCas
    */
   public void testSeekBugDFS() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fileSys = cluster.getFileSystem();
     try {
       Path file1 = new Path("seektest.dat");

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetTimes.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetTimes.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetTimes.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetTimes.java Wed Oct 20 22:58:52 2010
@@ -84,7 +84,9 @@ public class TestSetTimes extends TestCa
     conf.setInt("dfs.heartbeat.interval", 1);
 
 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                                               .numDataNodes(numDatanodes)
+                                               .build();
     cluster.waitActive();
     final int nnport = cluster.getNameNodePort();
     InetSocketAddress addr = new InetSocketAddress("localhost", 
@@ -159,8 +161,9 @@ public class TestSetTimes extends TestCa
       // shutdown cluster and restart
       cluster.shutdown();
       try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
-      cluster = new MiniDFSCluster(nnport, conf, 1, false, true,
-                                   null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
+                                                .format(false)
+                                                .build();
       cluster.waitActive();
       fileSys = cluster.getFileSystem();
 
@@ -196,7 +199,9 @@ public class TestSetTimes extends TestCa
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt("dfs.heartbeat.interval", 1);
     conf.setInt("dfs.datanode.handler.count", 50);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+                                               .numDataNodes(numDatanodes)
+                                               .build();
     cluster.waitActive();
     InetSocketAddress addr = new InetSocketAddress("localhost",
                                                      cluster.getNameNodePort());

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetrepIncreasing.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetrepIncreasing.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetrepIncreasing.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetrepIncreasing.java Wed Oct 20 22:58:52 2010
@@ -33,7 +33,7 @@ public class TestSetrepIncreasing extend
     conf.set("dfs.replication", "" + fromREP);
     conf.setLong("dfs.blockreport.intervalMsec", 1000L);
     conf.set(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2));
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 10, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(10).build();
     FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(), fs instanceof DistributedFileSystem);
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSmallBlock.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSmallBlock.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSmallBlock.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSmallBlock.java Wed Oct 20 22:58:52 2010
@@ -95,7 +95,7 @@ public class TestSmallBlock extends Test
       conf.setBoolean("dfs.datanode.simulateddatastorage", true);
     }
     conf.set(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, "1");
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fileSys = cluster.getFileSystem();
     try {
       Path file1 = new Path("smallblocktest.dat");

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java Wed Oct 20 22:58:52 2010
@@ -104,7 +104,13 @@ public class UpgradeUtilities {
       
       // format and start NameNode and start DataNode
       GenericTestUtils.formatNamenode(config);
-      cluster = new MiniDFSCluster(config, 1, StartupOption.REGULAR);
+      cluster =  new MiniDFSCluster.Builder(config)
+                                   .numDataNodes(1)
+                                   .startupOption(StartupOption.REGULAR)
+                                   .format(false)
+                                   .manageDataDfsDirs(false)
+                                   .manageNameDfsDirs(false)
+                                   .build();
         
       NameNode namenode = cluster.getNameNode();
       namenodeStorageNamespaceID = namenode.versionRequest().getNamespaceID();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationToken.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationToken.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationToken.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationToken.java Wed Oct 20 22:58:52 2010
@@ -59,7 +59,7 @@ public class TestDelegationToken {
     config.set("hadoop.security.auth_to_local",
         "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT");
     FileSystem.setDefaultUri(config, "hdfs://localhost:" + "0");
-    cluster = new MiniDFSCluster(0, config, 1, true, true, true,  null, null, null, null);
+    cluster = new MiniDFSCluster.Builder(config).build();
     cluster.waitActive();
     cluster.getNamesystem().getDelegationTokenSecretManager().startThreads();
   }

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java Wed Oct 20 22:58:52 2010
@@ -96,8 +96,7 @@ public class TestDelegationTokenForProxy
         "group1");
     configureSuperUserIPAddresses(config, REAL_USER);
     FileSystem.setDefaultUri(config, "hdfs://localhost:" + "0");
-    cluster = new MiniDFSCluster(0, config, 1, true, true, true, null, null,
-        null, null);
+    cluster = new MiniDFSCluster.Builder(config).build();
     cluster.waitActive();
     cluster.getNamesystem().getDelegationTokenSecretManager().startThreads();
     ProxyUsers.refreshSuperUserGroupsConfiguration(config);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java Wed Oct 20 22:58:52 2010
@@ -85,7 +85,7 @@ public class TestBalancer extends TestCa
    */
   private ExtendedBlock[] generateBlocks(Configuration conf, long size,
       short numNodes) throws IOException {
-    cluster = new MiniDFSCluster( conf, numNodes, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numNodes).build();
     try {
       cluster.waitActive();
       client = DFSClient.createNamenode(conf);
@@ -171,8 +171,11 @@ public class TestBalancer extends TestCa
 
     // restart the cluster: do NOT format the cluster
     conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, "0.0f"); 
-    cluster = new MiniDFSCluster(0, conf, numDatanodes,
-        false, true, null, racks, capacities);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
+                                              .format(false)
+                                              .racks(racks)
+                                              .simulatedCapacities(capacities)
+                                              .build();
     cluster.waitActive();
     client = DFSClient.createNamenode(conf);
 
@@ -209,8 +212,11 @@ public class TestBalancer extends TestCa
       long newCapacity, String newRack) throws Exception {
     int numOfDatanodes = capacities.length;
     assertEquals(numOfDatanodes, racks.length);
-    cluster = new MiniDFSCluster(0, conf, capacities.length, true, true, null, 
-        racks, capacities);
+    cluster = new MiniDFSCluster.Builder(conf)
+                                .numDataNodes(capacities.length)
+                                .racks(racks)
+                                .simulatedCapacities(capacities)
+                                .build();
     try {
       cluster.waitActive();
       client = DFSClient.createNamenode(conf);
@@ -348,8 +354,11 @@ public class TestBalancer extends TestCa
       throws Exception {
     int numOfDatanodes = capacities.length;
     assertEquals(numOfDatanodes, racks.length);
-    cluster = new MiniDFSCluster(0, conf, capacities.length, true, true, null,
-        racks, capacities);
+    cluster = new MiniDFSCluster.Builder(conf)
+                                .numDataNodes(capacities.length)
+                                .racks(racks)
+                                .simulatedCapacities(capacities)
+                                .build();
     try {
       cluster.waitActive();
       client = DFSClient.createNamenode(conf);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java Wed Oct 20 22:58:52 2010
@@ -58,12 +58,14 @@ public class TestDistributedUpgrade exte
    */
   void startNameNodeShouldFail(StartupOption operation) {
     try {
-      //cluster = new MiniDFSCluster(conf, 0, operation); // should fail
+      //cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).startupOption(operation).build(); // should fail
       // we set manage dirs to true as NN has to start from untar'ed image with 
       // nn dirs set to name1 and name2
-      cluster = new MiniDFSCluster(0, conf, 0, false, true,
-          operation, null); // Should fail
-      throw new AssertionError("NameNode should have failed to start");
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                              .format(false)
+                                              .startupOption(operation)
+                                              .build(); // should fail
+      throw new AssertionError("Jakob was here. NameNode should have failed to start");
     } catch (Exception expected) {
       expected = null;
       // expected
@@ -111,9 +113,12 @@ public class TestDistributedUpgrade exte
     startNameNodeShouldFail(StartupOption.REGULAR);
 
     log("Start NameNode only distributed upgrade", numDirs);
-    // cluster = new MiniDFSCluster(conf, 0, StartupOption.UPGRADE);
-    cluster = new MiniDFSCluster(0, conf, 0, false, true,
-                                  StartupOption.UPGRADE, null);
+    // cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false)
+    // .startupOption(StartupOption.UPGRADE).build();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                              .format(false)
+                                              .startupOption(StartupOption.UPGRADE)
+                                              .build();
     cluster.shutdown();
 
     log("NameNode start in regular mode when dustributed upgrade has been started", numDirs);
@@ -123,8 +128,11 @@ public class TestDistributedUpgrade exte
     startNameNodeShouldFail(StartupOption.ROLLBACK);
 
     log("Normal distributed upgrade for the cluster", numDirs);
-    cluster = new MiniDFSCluster(0, conf, numDNs, false, true,
-                                  StartupOption.UPGRADE, null);
+    cluster = new MiniDFSCluster.Builder(conf)
+                                .numDataNodes(numDNs)
+                                .format(false)
+                                .startupOption(StartupOption.UPGRADE)
+                                .build();
     DFSAdmin dfsAdmin = new DFSAdmin();
     dfsAdmin.setConf(conf);
     dfsAdmin.run(new String[] {"-safemode", "wait"});
@@ -132,8 +140,12 @@ public class TestDistributedUpgrade exte
 
     // it should be ok to start in regular mode
     log("NameCluster regular startup after the upgrade", numDirs);
-    cluster = new MiniDFSCluster(0, conf, numDNs, false, true,
-                                  StartupOption.REGULAR, null);
+    cluster = new MiniDFSCluster.Builder(conf)
+                                .numDataNodes(numDNs)
+                                .format(false)
+                                .startupOption(StartupOption.REGULAR)
+                                .build();
+
     cluster.waitActive();
     cluster.shutdown();
   }

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java Wed Oct 20 22:58:52 2010
@@ -93,8 +93,9 @@ public class TestBlockReplacement extend
     CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
     CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE/2);
     CONF.setLong("dfs.blockreport.intervalMsec",500);
-    cluster = new MiniDFSCluster(
-          CONF, REPLICATION_FACTOR, true, INITIAL_RACKS );
+    cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(REPLICATION_FACTOR)
+                                              .racks(INITIAL_RACKS).build();
+
     try {
       cluster.waitActive();
       

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java Wed Oct 20 22:58:52 2010
@@ -84,7 +84,7 @@ public class TestBlockReport {
   @Before
   public void startUpCluster() throws IOException {
     REPL_FACTOR = 1; //Reset if case a test has modified the value
-    cluster = new MiniDFSCluster(conf, REPL_FACTOR, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL_FACTOR).build();
     fs = (DistributedFileSystem) cluster.getFileSystem();
   }
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java Wed Oct 20 22:58:52 2010
@@ -35,7 +35,7 @@ public class TestDataNodeMXBean {
   @Test
   public void testDataNodeMXBean() throws Exception {
     Configuration conf = new Configuration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
 
     try {
       List<DataNode> datanodes = cluster.getDataNodes();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java Wed Oct 20 22:58:52 2010
@@ -33,7 +33,7 @@ public class TestDataNodeMetrics extends
   public void testDataNodeMetrics() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     try {
       FileSystem fs = cluster.getFileSystem();
       final long LONG_FILE_LEN = Integer.MAX_VALUE+1L; 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java Wed Oct 20 22:58:52 2010
@@ -73,7 +73,7 @@ public class TestDataNodeVolumeFailure e
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, block_size);
     // Allow a single volume failure (there are two volumes)
     conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
-    cluster = new MiniDFSCluster(conf, dn_num, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dn_num).build();
     cluster.waitActive();
   }
   

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java Wed Oct 20 22:58:52 2010
@@ -50,7 +50,7 @@ public class TestDatanodeRestart {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
     try {
@@ -74,7 +74,7 @@ public class TestDatanodeRestart {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
     conf.setBoolean("dfs.support.append", true);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     cluster.waitActive();
     try {
       testRbwReplicas(cluster, false);
@@ -137,7 +137,7 @@ public class TestDatanodeRestart {
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024L);
     conf.setInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 512);
     conf.setBoolean("dfs.support.append", true);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     cluster.waitActive();
     try {
       FileSystem fs = cluster.getFileSystem();