Browse Source

Merge r1578676 from branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.4@1578678 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 11 years ago
parent
commit
fd7c451063
37 changed files with 79 additions and 62 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java
  3. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
  4. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java
  5. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java
  6. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java
  7. 5 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java
  8. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java
  9. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
  10. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRClasspath.java
  11. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java
  12. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRWithDFSWithDistinctUsers.java
  13. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java
  14. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java
  15. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
  16. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java
  17. 4 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java
  18. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java
  19. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java
  20. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java
  21. 6 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
  22. 4 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestDelegatingInputFormat.java
  23. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java
  24. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java
  25. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestMRCredentials.java
  26. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java
  27. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java
  28. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java
  29. 1 1
      hadoop-tools/hadoop-datajoin/src/test/java/org/apache/hadoop/contrib/utils/join/TestDataJoin.java
  30. 1 1
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestGlobbedCopyListing.java
  31. 13 13
      hadoop-tools/hadoop-extras/src/test/java/org/apache/hadoop/tools/TestCopyFiles.java
  32. 2 1
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestDumpTypedBytes.java
  33. 1 1
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestFileArgs.java
  34. 2 1
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestLoadTypedBytes.java
  35. 1 1
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java
  36. 1 1
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestMultipleCachefiles.java
  37. 1 1
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestSymLink.java

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

@@ -151,6 +151,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-6106. Reduce default for
     dfs.namenode.path.based.cache.refresh.interval.ms (cmccabe)
 
+    HDFS-6090. Use MiniDFSCluster.Builder instead of deprecated constructors.
+    (Akira AJISAKA via jing9)
+
   OPTIMIZATIONS
 
     HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestFileSystem.java

@@ -523,7 +523,8 @@ public class TestFileSystem extends TestCase {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
-      cluster = new MiniDFSCluster(port, conf, 2, true, true, null, null);
+      cluster = new MiniDFSCluster.Builder(conf).nameNodePort(port)
+          .numDataNodes(2).build();
       URI uri = cluster.getFileSystem().getUri();
       LOG.info("uri=" + uri);
 

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java

@@ -146,7 +146,8 @@ public abstract class HadoopTestCase extends TestCase {
       fileSystem = FileSystem.getLocal(new JobConf());
     }
     else {
-      dfsCluster = new MiniDFSCluster(new JobConf(), dataNodes, true, null);
+      dfsCluster = new MiniDFSCluster.Builder(new JobConf())
+          .numDataNodes(dataNodes).build();
       fileSystem = dfsCluster.getFileSystem();
     }
     if (localMR) {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCommandLineJobSubmission.java

@@ -51,7 +51,7 @@ public class TestCommandLineJobSubmission extends TestCase {
     try {
       Configuration conf = new Configuration();
       //start the mini mr and dfs cluster.
-      dfs = new MiniDFSCluster(conf, 2 , true, null);
+      dfs = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       fs = dfs.getFileSystem();
       FSDataOutputStream stream = fs.create(testFile);
       stream.write("teststring".getBytes());

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobSysDirWithDFS.java

@@ -125,7 +125,7 @@ public class TestJobSysDirWithDFS extends TestCase {
 
       JobConf conf = new JobConf();
       conf.set(JTConfig.JT_SYSTEM_DIR, "/tmp/custom/mapred/system");
-      dfs = new MiniDFSCluster(conf, 4, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
       fileSys = dfs.getFileSystem();
       mr = new MiniMRCluster(taskTrackers, fileSys.getUri().toString(), 1, null, null, conf);
 

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java

@@ -141,7 +141,8 @@ public class TestLazyOutput extends TestCase {
       Configuration conf = new Configuration();
 
       // Start the mini-MR and mini-DFS clusters
-      dfs = new MiniDFSCluster(conf, NUM_HADOOP_SLAVES, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_HADOOP_SLAVES)
+          .build();
       fileSys = dfs.getFileSystem();
       mr = new MiniMRCluster(NUM_HADOOP_SLAVES, fileSys.getUri().toString(), 1);
 

+ 5 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRCJCFileInputFormat.java

@@ -44,11 +44,10 @@ public class TestMRCJCFileInputFormat extends TestCase {
   MiniDFSCluster dfs = null;
 
   private MiniDFSCluster newDFSCluster(JobConf conf) throws Exception {
-    return new MiniDFSCluster(conf, 4, true,
-                         new String[]{"/rack0", "/rack0",
-                                      "/rack1", "/rack1"},
-                         new String[]{"host0", "host1",
-                                      "host2", "host3"});
+    return new MiniDFSCluster.Builder(conf).numDataNodes(4)
+        .racks(new String[]{"/rack0", "/rack0", "/rack1", "/rack1"})
+        .hosts(new String[]{"host0", "host1", "host2", "host3"})
+        .build();
   }
 
   public void testLocality() throws Exception {
@@ -162,7 +161,7 @@ public class TestMRCJCFileInputFormat extends TestCase {
     JobConf job = new JobConf(conf);
 
     job.setBoolean("dfs.replication.considerLoad", false);
-    dfs = new MiniDFSCluster(job, 1, true, rack1, hosts1);
+    dfs = new MiniDFSCluster.Builder(job).racks(rack1).hosts(hosts1).build();
     dfs.waitActive();
 
     String namenode = (dfs.getFileSystem()).getUri().getHost() + ":" +

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMerge.java

@@ -76,7 +76,8 @@ public class TestMerge extends TestCase {
     try {
       Configuration conf = new Configuration();
       // Start the mini-MR and mini-DFS clusters
-      dfsCluster = new MiniDFSCluster(conf, NUM_HADOOP_DATA_NODES, true, null);
+      dfsCluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(NUM_HADOOP_DATA_NODES).build();
       fileSystem = dfsCluster.getFileSystem();
       mrCluster = MiniMRClientClusterFactory.create(this.getClass(),
                                                  NUM_HADOOP_DATA_NODES, conf);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java

@@ -324,7 +324,7 @@ public class TestMiniMRChildTask {
   @BeforeClass
   public static void setup() throws IOException {
     // create configuration, dfs, file system and mapred cluster 
-    dfs = new MiniDFSCluster(conf, 1, true, null);
+    dfs = new MiniDFSCluster.Builder(conf).build();
     fileSys = dfs.getFileSystem();
 
     if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRClasspath.java

@@ -169,7 +169,7 @@ public class TestMiniMRClasspath {
       final int jobTrackerPort = 60050;
 
       Configuration conf = new Configuration();
-      dfs = new MiniDFSCluster(conf, 1, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).build();
       fileSys = dfs.getFileSystem();
       namenode = fileSys.getUri().toString();
       mr = new MiniMRCluster(taskTrackers, namenode, 3);
@@ -201,7 +201,7 @@ public class TestMiniMRClasspath {
       final int taskTrackers = 4;
 
       Configuration conf = new Configuration();
-      dfs = new MiniDFSCluster(conf, 1, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).build();
       fileSys = dfs.getFileSystem();
       namenode = fileSys.getUri().toString();
       mr = new MiniMRCluster(taskTrackers, namenode, 3);      

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java

@@ -38,7 +38,7 @@ public class TestMiniMRDFSCaching extends TestCase {
     FileSystem fileSys = null;
     try {
       JobConf conf = new JobConf();
-      dfs = new MiniDFSCluster(conf, 1, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).build();
       fileSys = dfs.getFileSystem();
       mr = new MiniMRCluster(2, fileSys.getUri().toString(), 4);
       MRCaching.setupCache("/cachedir", fileSys);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRWithDFSWithDistinctUsers.java

@@ -75,7 +75,7 @@ public class TestMiniMRWithDFSWithDistinctUsers {
 
   @Before
   public void setUp() throws Exception {
-    dfs = new MiniDFSCluster(conf, 4, true, null);
+    dfs = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
 
     fs = DFS_UGI.doAs(new PrivilegedExceptionAction<FileSystem>() {
         public FileSystem run() throws IOException {

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultipleLevelCaching.java

@@ -92,8 +92,8 @@ public class TestMultipleLevelCaching extends TestCase {
       String rack2 = getRack(1, level);
       Configuration conf = new Configuration();
       // Run a datanode on host1 under /a/b/c/..../d1/e1/f1
-      dfs = new MiniDFSCluster(conf, 1, true, new String[] {rack1}, 
-                               new String[] {"host1.com"});
+      dfs = new MiniDFSCluster.Builder(conf).racks(new String[] {rack1})
+          .hosts(new String[] {"host1.com"}).build();
       dfs.waitActive();
       fileSys = dfs.getFileSystem();
       if (!fileSys.mkdirs(inDir)) {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestReduceFetchFromPartialMem.java

@@ -57,7 +57,7 @@ public class TestReduceFetchFromPartialMem extends TestCase {
     TestSetup setup = new TestSetup(mySuite) {
       protected void setUp() throws Exception {
         Configuration conf = new Configuration();
-        dfsCluster = new MiniDFSCluster(conf, 2, true, null);
+        dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
         mrCluster = new MiniMRCluster(2,
             dfsCluster.getFileSystem().getUri().toString(), 1);
       }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java

@@ -106,7 +106,7 @@ public class TestSpecialCharactersInOutputPath extends TestCase {
       final int taskTrackers = 4;
       final int jobTrackerPort = 60050;
       Configuration conf = new Configuration();
-      dfs = new MiniDFSCluster(conf, 1, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).build();
       fileSys = dfs.getFileSystem();
       namenode = fileSys.getUri().toString();
       mr = new MiniMRCluster(taskTrackers, namenode, 2);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestDatamerge.java

@@ -62,7 +62,7 @@ public class TestDatamerge extends TestCase {
     TestSetup setup = new TestSetup(new TestSuite(TestDatamerge.class)) {
       protected void setUp() throws Exception {
         Configuration conf = new Configuration();
-        cluster = new MiniDFSCluster(conf, 2, true, null);
+        cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       }
       protected void tearDown() throws Exception {
         if (cluster != null) {

+ 4 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestDelegatingInputFormat.java

@@ -39,9 +39,10 @@ public class TestDelegatingInputFormat extends TestCase {
     JobConf conf = new JobConf();
     MiniDFSCluster dfs = null;
     try {
-      dfs = new MiniDFSCluster(conf, 4, true, new String[] { "/rack0",
-         "/rack0", "/rack1", "/rack1" }, new String[] { "host0", "host1",
-         "host2", "host3" });
+      dfs = new MiniDFSCluster.Builder(conf).numDataNodes(4)
+          .racks(new String[] { "/rack0", "/rack0", "/rack1", "/rack1" })
+          .hosts(new String[] { "host0", "host1", "host2", "host3" })
+          .build();
       FileSystem fs = dfs.getFileSystem();
 
       Path path = getPath("/foo/bar", fs);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipes.java

@@ -79,7 +79,7 @@ public class TestPipes extends TestCase {
     try {
       final int numSlaves = 2;
       Configuration conf = new Configuration();
-      dfs = new MiniDFSCluster(conf, numSlaves, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).numDataNodes(numSlaves).build();
       mr = new MiniMRCluster(numSlaves, dfs.getFileSystem().getUri().toString(), 1);
       writeInputFile(dfs.getFileSystem(), inputPath);
       runProgram(mr, dfs, wordCountSimple, 

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/MiniHadoopClusterManager.java

@@ -151,8 +151,8 @@ public class MiniHadoopClusterManager {
   public void start() throws IOException, FileNotFoundException,
       URISyntaxException {
     if (!noDFS) {
-      dfs = new MiniDFSCluster(nnPort, conf, numDataNodes, true, true,
-          dfsOpts, null, null);
+      dfs = new MiniDFSCluster.Builder(conf).nameNodePort(nnPort)
+          .numDataNodes(numDataNodes).startupOption(dfsOpts).build();
       LOG.info("Started MiniDFSCluster -- namenode on port "
           + dfs.getNameNodePort());
     }

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java

@@ -131,7 +131,8 @@ public class TestMapReduceLazyOutput extends TestCase {
       Configuration conf = new Configuration();
 
       // Start the mini-MR and mini-DFS clusters
-      dfs = new MiniDFSCluster(conf, NUM_HADOOP_SLAVES, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_HADOOP_SLAVES)
+          .build();
       fileSys = dfs.getFileSystem();
       mr = new MiniMRCluster(NUM_HADOOP_SLAVES, fileSys.getUri().toString(), 1);
 

+ 6 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java

@@ -313,7 +313,8 @@ public class TestCombineFileInputFormat extends TestCase {
        */
       Configuration conf = new Configuration();
       conf.setBoolean("dfs.replication.considerLoad", false);
-      dfs = new MiniDFSCluster(conf, 1, true, rack1, hosts1);
+      dfs = new MiniDFSCluster.Builder(conf).racks(rack1).hosts(hosts1)
+          .build();
       dfs.waitActive();
 
       fileSys = dfs.getFileSystem();
@@ -855,7 +856,8 @@ public class TestCombineFileInputFormat extends TestCase {
        */
       Configuration conf = new Configuration();
       conf.setBoolean("dfs.replication.considerLoad", false);
-      dfs = new MiniDFSCluster(conf, 1, true, rack1, hosts1);
+      dfs = new MiniDFSCluster.Builder(conf).racks(rack1).hosts(hosts1)
+          .build();
       dfs.waitActive();
 
       fileSys = dfs.getFileSystem();
@@ -1197,7 +1199,8 @@ public class TestCombineFileInputFormat extends TestCase {
       Configuration conf = new Configuration();
       conf.set("fs.hdfs.impl", MissingBlockFileSystem.class.getName());
       conf.setBoolean("dfs.replication.considerLoad", false);
-      dfs = new MiniDFSCluster(conf, 1, true, rack1, hosts1);
+      dfs = new MiniDFSCluster.Builder(conf).racks(rack1).hosts(hosts1)
+          .build();
       dfs.waitActive();
 
       namenode = (dfs.getFileSystem()).getUri().getHost() + ":" +

+ 4 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestDelegatingInputFormat.java

@@ -37,9 +37,10 @@ public class TestDelegatingInputFormat extends TestCase {
     Job job = Job.getInstance();
     MiniDFSCluster dfs = null;
     try {
-      dfs = new MiniDFSCluster(job.getConfiguration(), 4, true, new String[] { "/rack0",
-         "/rack0", "/rack1", "/rack1" }, new String[] { "host0", "host1",
-         "host2", "host3" });
+      dfs = new MiniDFSCluster.Builder(job.getConfiguration()).numDataNodes(4)
+          .racks(new String[] { "/rack0", "/rack0", "/rack1", "/rack1" })
+          .hosts(new String[] { "host0", "host1", "host2", "host3" })
+          .build();
       FileSystem fs = dfs.getFileSystem();
 
       Path path = getPath("/foo/bar", fs);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java

@@ -45,7 +45,7 @@ public class TestJoinDatamerge extends TestCase {
     TestSetup setup = new TestSetup(new TestSuite(TestJoinDatamerge.class)) {
       protected void setUp() throws Exception {
         Configuration conf = new Configuration();
-        cluster = new MiniDFSCluster(conf, 2, true, null);
+        cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       }
       protected void tearDown() throws Exception {
         if (cluster != null) {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinProperties.java

@@ -50,7 +50,7 @@ public class TestJoinProperties extends TestCase {
     TestSetup setup = new TestSetup(new TestSuite(TestJoinProperties.class)) {
       protected void setUp() throws Exception {
         Configuration conf = new Configuration();
-        cluster = new MiniDFSCluster(conf, 2, true, null);
+        cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
         base = cluster.getFileSystem().makeQualified(new Path("/nested"));
         src = generateSources(conf);
       }

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestMRCredentials.java

@@ -58,7 +58,8 @@ public class TestMRCredentials {
   public static void setUp() throws Exception {
     System.setProperty("hadoop.log.dir", "logs");
     Configuration conf = new Configuration();
-    dfsCluster = new MiniDFSCluster(conf, numSlaves, true, null);  
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(numSlaves)
+        .build();
     jConf = new JobConf(conf);
     FileSystem.setDefaultUri(conf, dfsCluster.getFileSystem().getUri().toString());
     mrCluster = MiniMRClientClusterFactory.create(TestMRCredentials.class, 1, jConf);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java

@@ -92,7 +92,7 @@ public class TestEncryptedShuffle {
             YarnConfiguration.DEFAULT_YARN_CROSS_PLATFORM_APPLICATION_CLASSPATH))
         + File.pathSeparator + classpathDir;
     conf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, cp);
-    dfsCluster = new MiniDFSCluster(conf, 1, true, null);
+    dfsCluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fileSystem = dfsCluster.getFileSystem();
     fileSystem.mkdirs(new Path("/tmp"));
     fileSystem.mkdirs(new Path("/user"));

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMiniMRProxyUser.java

@@ -70,7 +70,8 @@ public class TestMiniMRProxyUser extends TestCase {
     UserGroupInformation.createUserForTesting("u1", userGroups);
     UserGroupInformation.createUserForTesting("u2", new String[]{"gg"});
 
-    dfsCluster = new MiniDFSCluster(conf, dataNodes, true, null);
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(dataNodes)
+        .build();
     FileSystem fileSystem = dfsCluster.getFileSystem();
     fileSystem.mkdirs(new Path("/tmp"));
     fileSystem.mkdirs(new Path("/user"));

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java

@@ -57,7 +57,8 @@ public class TestNonExistentJob extends TestCase {
     conf.set("dfs.permissions", "true");
     conf.set("hadoop.security.authentication", "simple");
 
-    dfsCluster = new MiniDFSCluster(conf, dataNodes, true, null);
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(dataNodes)
+        .build();
     FileSystem fileSystem = dfsCluster.getFileSystem();
     fileSystem.mkdirs(new Path("/tmp"));
     fileSystem.mkdirs(new Path("/user"));

+ 1 - 1
hadoop-tools/hadoop-datajoin/src/test/java/org/apache/hadoop/contrib/utils/join/TestDataJoin.java

@@ -43,7 +43,7 @@ public class TestDataJoin extends TestCase {
     TestSetup setup = new TestSetup(new TestSuite(TestDataJoin.class)) {
       protected void setUp() throws Exception {
         Configuration conf = new Configuration();
-        cluster = new MiniDFSCluster(conf, 2, true, null);
+        cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       }
       protected void tearDown() throws Exception {
         if (cluster != null) {

+ 1 - 1
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestGlobbedCopyListing.java

@@ -49,7 +49,7 @@ public class TestGlobbedCopyListing {
 
   @BeforeClass
   public static void setup() throws Exception {
-    cluster = new MiniDFSCluster(new Configuration(), 1, true, null);
+    cluster = new MiniDFSCluster.Builder(new Configuration()).build();
     createSourceData();
   }
 

+ 13 - 13
hadoop-tools/hadoop-extras/src/test/java/org/apache/hadoop/tools/TestCopyFiles.java

@@ -280,7 +280,7 @@ public class TestCopyFiles extends TestCase {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final FileSystem hdfs = cluster.getFileSystem();
       namenode = FileSystem.getDefaultUri(conf).toString();
       if (namenode.startsWith("hdfs://")) {
@@ -310,7 +310,7 @@ public class TestCopyFiles extends TestCase {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final FileSystem hdfs = cluster.getFileSystem();
       namenode = FileSystem.getDefaultUri(conf).toString();
       if (namenode.startsWith("hdfs://")) {
@@ -340,7 +340,7 @@ public class TestCopyFiles extends TestCase {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
-      cluster = new MiniDFSCluster(conf, 1, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).build();
       final FileSystem hdfs = cluster.getFileSystem();
       final String namenode = hdfs.getUri().toString();
       if (namenode.startsWith("hdfs://")) {
@@ -369,7 +369,7 @@ public class TestCopyFiles extends TestCase {
     try {
       Configuration conf = new Configuration();
       final FileSystem localfs = FileSystem.get(LOCAL_FS, conf);
-      cluster = new MiniDFSCluster(conf, 1, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).build();
       final FileSystem hdfs = cluster.getFileSystem();
       final String namenode = FileSystem.getDefaultUri(conf).toString();
       if (namenode.startsWith("hdfs://")) {
@@ -396,7 +396,7 @@ public class TestCopyFiles extends TestCase {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final FileSystem hdfs = cluster.getFileSystem();
       final String namenode = hdfs.getUri().toString();
       if (namenode.startsWith("hdfs://")) {
@@ -456,7 +456,7 @@ public class TestCopyFiles extends TestCase {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final FileSystem hdfs = cluster.getFileSystem();
       final String namenode = hdfs.getUri().toString();
       
@@ -614,7 +614,7 @@ public class TestCopyFiles extends TestCase {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final FileSystem hdfs = cluster.getFileSystem();
       namenode = FileSystem.getDefaultUri(conf).toString();
       if (namenode.startsWith("hdfs://")) {
@@ -639,7 +639,7 @@ public class TestCopyFiles extends TestCase {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       String nnUri = FileSystem.getDefaultUri(conf).toString();
       FileSystem fs = FileSystem.get(URI.create(nnUri), conf);
 
@@ -791,7 +791,7 @@ public class TestCopyFiles extends TestCase {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final String nnUri = FileSystem.getDefaultUri(conf).toString();
       final FileSystem fs = FileSystem.get(URI.create(nnUri), conf);
       final DistCpV1 distcp = new DistCpV1(conf);
@@ -899,7 +899,7 @@ public class TestCopyFiles extends TestCase {
 
       //start cluster by DFS_UGI
       final Configuration dfsConf = new Configuration();
-      cluster = new MiniDFSCluster(dfsConf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(dfsConf).numDataNodes(2).build();
       cluster.waitActive();
 
       final String httpAdd = dfsConf.get("dfs.http.address");
@@ -955,7 +955,7 @@ public class TestCopyFiles extends TestCase {
     conf.setInt("fs.trash.interval", 60);
     MiniDFSCluster cluster = null;
     try {
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final URI nnURI = FileSystem.getDefaultUri(conf);
       final String nnUri = nnURI.toString();
       final FileSystem fs = FileSystem.get(URI.create(nnUri), conf);
@@ -1027,7 +1027,7 @@ public class TestCopyFiles extends TestCase {
     try {
       Configuration conf = new Configuration();
       final FileSystem localfs = FileSystem.get(LOCAL_FS, conf);
-      cluster = new MiniDFSCluster(conf, 1, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).build();
       final FileSystem hdfs = cluster.getFileSystem();
       final String namenode = FileSystem.getDefaultUri(conf).toString();
       if (namenode.startsWith("hdfs://")) {
@@ -1060,7 +1060,7 @@ public class TestCopyFiles extends TestCase {
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new Configuration();
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final FileSystem hdfs = cluster.getFileSystem();
       namenode = FileSystem.getDefaultUri(conf).toString();
       if (namenode.startsWith("hdfs://")) {

+ 2 - 1
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestDumpTypedBytes.java

@@ -39,7 +39,8 @@ public class TestDumpTypedBytes {
   @Test
   public void testDumping() throws Exception {
     Configuration conf = new Configuration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
+        .build();
     FileSystem fs = cluster.getFileSystem();
     PrintStream psBackup = System.out;
     ByteArrayOutputStream out = new ByteArrayOutputStream();

+ 1 - 1
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestFileArgs.java

@@ -54,7 +54,7 @@ public class TestFileArgs extends TestStreaming
   {
     // Set up mini cluster
     conf = new Configuration();
-    dfs = new MiniDFSCluster(conf, 1, true, null);
+    dfs = new MiniDFSCluster.Builder(conf).build();
     fileSys = dfs.getFileSystem();
     namenode = fileSys.getUri().getAuthority();
     mr  = new MiniMRCluster(1, namenode, 1);

+ 2 - 1
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestLoadTypedBytes.java

@@ -39,7 +39,8 @@ public class TestLoadTypedBytes {
   @Test
   public void testLoading() throws Exception {
     Configuration conf = new Configuration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2)
+        .build();
     FileSystem fs = cluster.getFileSystem();
     
     ByteArrayOutputStream out = new ByteArrayOutputStream();

+ 1 - 1
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestMultipleArchiveFiles.java

@@ -65,7 +65,7 @@ public class TestMultipleArchiveFiles extends TestStreaming
     input = "HADOOP";
     expectedOutput = "HADOOP\t\nHADOOP\t\n";
     conf = new Configuration();
-    dfs = new MiniDFSCluster(conf, 1, true, null);
+    dfs = new MiniDFSCluster.Builder(conf).build();
     fileSys = dfs.getFileSystem();
     namenode = fileSys.getUri().getAuthority();
     mr  = new MiniMRCluster(1, namenode, 1);

+ 1 - 1
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestMultipleCachefiles.java

@@ -69,7 +69,7 @@ public class TestMultipleCachefiles
     MiniDFSCluster dfs = null; 
     try{
       Configuration conf = new Configuration();
-      dfs = new MiniDFSCluster(conf, 1, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).build();
       FileSystem fileSys = dfs.getFileSystem();
       String namenode = fileSys.getUri().toString();
 

+ 1 - 1
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestSymLink.java

@@ -61,7 +61,7 @@ public class TestSymLink
     MiniDFSCluster dfs = null; 
     try {
       Configuration conf = new Configuration();
-      dfs = new MiniDFSCluster(conf, 1, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).build();
       FileSystem fileSys = dfs.getFileSystem();
       String namenode = fileSys.getUri().toString();
       mr  = new MiniMRCluster(1, namenode, 3);