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 [1/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...

Author: jghoman
Date: Wed Oct 20 22:58:52 2010
New Revision: 1025788

URL: http://svn.apache.org/viewvc?rev=1025788&view=rev
Log:
HDFS-1456. Provide builder for constructing instances of MiniDFSCluster.


Modified:
    hadoop/hdfs/branches/HDFS-1052/CHANGES.txt
    hadoop/hdfs/branches/HDFS-1052/src/contrib/fuse-dfs/src/test/TestFuseDFS.java
    hadoop/hdfs/branches/HDFS-1052/src/contrib/hdfsproxy/src/test/org/apache/hadoop/hdfsproxy/TestHdfsProxy.java
    hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/fs/TestFiListPath.java
    hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/TestFiHftp.java
    hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java
    hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/cli/TestHDFSCLI.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsPermission.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsSymlink.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestGlobPaths.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestUrlStreamHandler.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/permission/TestStickyBit.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/BenchmarkThroughput.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/FileAppendTest4.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlockMissingException.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestCrcCorruption.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSFinalize.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSMkdirs.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRemove.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRename.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRollback.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShell.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeConfig.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeDeath.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeReport.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDecommission.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSInputChecker.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFSOutputSummer.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend2.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend3.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend4.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCorruption.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationClient.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationDelete.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreationEmpty.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestFileStatus.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestGetBlocks.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSTrash.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHFlush.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestHftpFileSystem.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLargeBlock.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLease.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInDFS.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListFilesInFileContext.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestListPathServlet.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestLocalDFS.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestModTime.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestPread.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReadWhileWriting.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestRestartDFS.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSafeMode.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSeekBug.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetTimes.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSetrepIncreasing.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestSmallBlock.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationToken.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestDistributedUpgrade.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMXBean.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestWriteToReplica.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockTokenWithDFS.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlocksWithNotEnoughRacks.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestComputeInvalidateWork.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCorruptFilesJsp.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestLargeDirectoryDelete.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/security/TestPermission.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/security/TestRefreshUserMappings.java
    hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/tools/TestJMXGet.java

Modified: hadoop/hdfs/branches/HDFS-1052/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/CHANGES.txt?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/CHANGES.txt (original)
+++ hadoop/hdfs/branches/HDFS-1052/CHANGES.txt Wed Oct 20 22:58:52 2010
@@ -157,6 +157,9 @@ Trunk (unreleased changes)
 
     HDFS-1426. Remove unused method BlockInfo#listCount. (hairong)
 
+    HDFS-1456. Provide builder for constructing instances of MiniDFSCluster.
+    (jghoman)
+
   OPTIMIZATIONS
 
     HDFS-1140. Speedup INode.getPathComponents. (Dmytro Molkov via shv)

Modified: hadoop/hdfs/branches/HDFS-1052/src/contrib/fuse-dfs/src/test/TestFuseDFS.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/contrib/fuse-dfs/src/test/TestFuseDFS.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/contrib/fuse-dfs/src/test/TestFuseDFS.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/contrib/fuse-dfs/src/test/TestFuseDFS.java Wed Oct 20 22:58:52 2010
@@ -115,7 +115,7 @@ public class TestFuseDFS extends TestCas
     try {
       Configuration conf = new HdfsConfiguration();
       conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY,false);
-      cluster = new MiniDFSCluster(conf, 1, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).build();
       fileSys = (DistributedFileSystem)cluster.getFileSystem();
       assertTrue(fileSys.getFileStatus(new Path("/")).isDir());
       mount(mpoint, fileSys.getUri());

Modified: hadoop/hdfs/branches/HDFS-1052/src/contrib/hdfsproxy/src/test/org/apache/hadoop/hdfsproxy/TestHdfsProxy.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/contrib/hdfsproxy/src/test/org/apache/hadoop/hdfsproxy/TestHdfsProxy.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/contrib/hdfsproxy/src/test/org/apache/hadoop/hdfsproxy/TestHdfsProxy.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/contrib/hdfsproxy/src/test/org/apache/hadoop/hdfsproxy/TestHdfsProxy.java Wed Oct 20 22:58:52 2010
@@ -210,7 +210,7 @@ public class TestHdfsProxy extends TestC
       dfsConf.set("hadoop.proxyuser.users.ip-addresses", "localhost");
       dfsConf.set("hadoop.proxyuser." + System.getProperty("user.name") +
           ".ip-addresses", "localhost");
-      cluster = new MiniDFSCluster(dfsConf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(dfsConf).numDataNodes(2).build();
       cluster.waitActive();
 
       final FileSystem localfs = FileSystem.get(LOCAL_FS, dfsConf);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/fs/TestFiListPath.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/fs/TestFiListPath.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/fs/TestFiListPath.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/fs/TestFiListPath.java Wed Oct 20 22:58:52 2010
@@ -56,7 +56,7 @@ public class TestFiListPath {
   public static void setup() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, LIST_LIMIT);
-    cluster = new MiniDFSCluster(conf, 1, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).build();
     cluster.waitClusterUp();
     fs = cluster.getFileSystem();
   }

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

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/TestFiPipelines.java Wed Oct 20 22:58:52 2010
@@ -57,7 +57,7 @@ public class TestFiPipelines {
 
   @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/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java Wed Oct 20 22:58:52 2010
@@ -88,7 +88,7 @@ public class TestFiDataTransferProtocol2
     FiTestUtil.LOG.info("size=" + size + ", nPackets=" + nPackets
         + ", lastPacketSize=" + lastPacketSize);
 
-    final MiniDFSCluster cluster = new MiniDFSCluster(conf, REPLICATION, true, null);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION).build();
     final FileSystem dfs = cluster.getFileSystem();
     try {
       final Path p = new Path("/" + methodName + "/foo");

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/cli/TestHDFSCLI.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/cli/TestHDFSCLI.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/cli/TestHDFSCLI.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/cli/TestHDFSCLI.java Wed Oct 20 22:58:52 2010
@@ -52,7 +52,10 @@ public class TestHDFSCLI extends CLITest
                         "/rack2", "/rack3", "/rack4", "/rack4" };
     String [] hosts = {"host1", "host2", "host3", "host4",
                        "host5", "host6", "host7", "host8" };
-    dfsCluster = new MiniDFSCluster(conf, 8, true, racks, hosts);
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(8)
+                                                 .racks(racks)
+                                                 .hosts(hosts)
+                                                 .build();
     
     namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");
     

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsCreateMkdir.java Wed Oct 20 22:58:52 2010
@@ -42,7 +42,7 @@ public class TestFcHdfsCreateMkdir exten
   public static void clusterSetupAtBegining()
                                     throws IOException, LoginException, URISyntaxException  {
     Configuration conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster(conf, 2, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     fc = FileContext.getFileContext(cluster.getURI(), conf);
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
         UserGroupInformation.getCurrentUser().getShortUserName()));

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsPermission.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsPermission.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsPermission.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsPermission.java Wed Oct 20 22:58:52 2010
@@ -42,7 +42,7 @@ public class TestFcHdfsPermission extend
   public static void clusterSetupAtBegining()
                                     throws IOException, LoginException, URISyntaxException  {
     Configuration conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster(conf, 2, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     fc = FileContext.getFileContext(cluster.getURI(), conf);
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
         UserGroupInformation.getCurrentUser().getShortUserName()));

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsSymlink.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsSymlink.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsSymlink.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestFcHdfsSymlink.java Wed Oct 20 22:58:52 2010
@@ -71,7 +71,7 @@ public class TestFcHdfsSymlink extends F
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
     conf.set(FsPermission.UMASK_LABEL, "000");
-    cluster = new MiniDFSCluster(conf, 1, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).build();
     fc = FileContext.getFileContext(cluster.getURI());
   }
   

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestGlobPaths.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestGlobPaths.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestGlobPaths.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestGlobPaths.java Wed Oct 20 22:58:52 2010
@@ -50,7 +50,7 @@ public class TestGlobPaths extends TestC
   protected void setUp() throws Exception {
     try {
       Configuration conf = new HdfsConfiguration();
-      dfsCluster = new MiniDFSCluster(conf, 1, true, null);
+      dfsCluster = new MiniDFSCluster.Builder(conf).build();
       fs = FileSystem.get(conf);
     } catch (IOException e) {
       e.printStackTrace();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java Wed Oct 20 22:58:52 2010
@@ -48,7 +48,7 @@ public class TestHDFSFileContextMainOper
   @BeforeClass
   public static void clusterSetupAtBegining() throws IOException,
       LoginException, URISyntaxException {
-    cluster = new MiniDFSCluster(CONF, 2, true, null);
+    cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
     cluster.waitClusterUp();
     fc = FileContext.getFileContext(cluster.getURI(), CONF);
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
@@ -61,7 +61,8 @@ public class TestHDFSFileContextMainOper
       cluster.shutdown();
       cluster = null;
     }
-    cluster = new MiniDFSCluster(CONF, 1, false, null);
+    cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(1)
+                                              .format(false).build();
     cluster.waitClusterUp();
     fc = FileContext.getFileContext(cluster.getURI(), CONF);
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestUrlStreamHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestUrlStreamHandler.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestUrlStreamHandler.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/TestUrlStreamHandler.java Wed Oct 20 22:58:52 2010
@@ -50,7 +50,7 @@ public class TestUrlStreamHandler extend
   public void testDfsUrls() throws IOException {
 
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     FileSystem fs = cluster.getFileSystem();
 
     // Setup our own factory

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java Wed Oct 20 22:58:52 2010
@@ -150,7 +150,7 @@ public class TestLoadGenerator extends C
     writer.write(FILE_STRUCTURE_SECOND_LINE+"\n");
     writer.close();
     
-    MiniDFSCluster cluster = new MiniDFSCluster(CONF, 3, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(3).build();
     cluster.waitActive();
     
     try {

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/permission/TestStickyBit.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/permission/TestStickyBit.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/permission/TestStickyBit.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/fs/permission/TestStickyBit.java Wed Oct 20 22:58:52 2010
@@ -164,7 +164,7 @@ public class TestStickyBit extends TestC
       Configuration conf = new HdfsConfiguration();
       conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
       conf.setBoolean("dfs.support.append", true);
-      cluster = new MiniDFSCluster(conf, 4, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
 
       FileSystem hdfs = cluster.getFileSystem();
 
@@ -203,7 +203,7 @@ public class TestStickyBit extends TestC
       // Set up cluster for testing
       Configuration conf = new HdfsConfiguration();
       conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-      cluster = new MiniDFSCluster(conf, 4, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
       FileSystem hdfs = cluster.getFileSystem();
 
       assertTrue(hdfs instanceof DistributedFileSystem);
@@ -249,7 +249,7 @@ public class TestStickyBit extends TestC
     try {
       Configuration conf = new HdfsConfiguration();
       conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
-      cluster = new MiniDFSCluster(conf, 4, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
       FileSystem hdfs = cluster.getFileSystem();
 
       assertTrue(hdfs instanceof DistributedFileSystem);
@@ -269,7 +269,7 @@ public class TestStickyBit extends TestC
       cluster.shutdown();
 
       // Start file system up again
-      cluster = new MiniDFSCluster(conf, 4, false, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).format(false).build();
       hdfs = cluster.getFileSystem();
 
       assertTrue(hdfs.exists(sbSet));

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/BenchmarkThroughput.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/BenchmarkThroughput.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/BenchmarkThroughput.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/BenchmarkThroughput.java Wed Oct 20 22:58:52 2010
@@ -206,7 +206,8 @@ public class BenchmarkThroughput extends
     }
     MiniDFSCluster cluster = null;
     try {
-      cluster = new MiniDFSCluster(conf, 1, true, new String[]{"/foo"});
+      cluster = new MiniDFSCluster.Builder(conf)
+                                  .racks(new String[]{"/foo"}).build();
       cluster.waitActive();
       FileSystem dfs = cluster.getFileSystem();
       for(int i=0; i < reps; ++i) {

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/FileAppendTest4.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/FileAppendTest4.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/FileAppendTest4.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/FileAppendTest4.java Wed Oct 20 22:58:52 2010
@@ -61,7 +61,7 @@ public class FileAppendTest4 {
   public static void startUp () throws IOException {
     conf = new HdfsConfiguration();
     init(conf);
-    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/MiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java Wed Oct 20 22:58:52 2010
@@ -72,6 +72,121 @@ import org.apache.hadoop.util.ToolRunner
  */
 public class MiniDFSCluster {
 
+  /**
+   * Class to construct instances of MiniDFSClusters with specific options.
+   */
+  public static class Builder {
+    private int nameNodePort = 0;
+    private final Configuration conf;
+    private int numDataNodes = 1;
+    private boolean format = true;
+    private boolean manageNameDfsDirs = true;
+    private boolean manageDataDfsDirs = true;
+    private StartupOption option = null;
+    private String[] racks = null; 
+    private String [] hosts = null;
+    private long [] simulatedCapacities = null;
+    
+    public Builder(Configuration conf) {
+      this.conf = conf;
+    }
+    
+    /**
+     * Default: 0
+     */
+    public Builder nameNodePort(int val) {
+      this.nameNodePort = val;
+      return this;
+    }
+
+    /**
+     * Default: 1
+     */
+    public Builder numDataNodes(int val) {
+      this.numDataNodes = val;
+      return this;
+    }
+
+    /**
+     * Default: true
+     */
+    public Builder format(boolean val) {
+      this.format = val;
+      return this;
+    }
+
+    /**
+     * Default: true
+     */
+    public Builder manageNameDfsDirs(boolean val) {
+      this.manageNameDfsDirs = val;
+      return this;
+    }
+
+    /**
+     * Default: true
+     */
+    public Builder manageDataDfsDirs(boolean val) {
+      this.manageDataDfsDirs = val;
+      return this;
+    }
+
+    /**
+     * Default: null
+     */
+    public Builder startupOption(StartupOption val) {
+      this.option = val;
+      return this;
+    }
+
+    /**
+     * Default: null
+     */
+    public Builder racks(String[] val) {
+      this.racks = val;
+      return this;
+    }
+
+    /**
+     * Default: null
+     */
+    public Builder hosts(String[] val) {
+      this.hosts = val;
+      return this;
+    }
+
+    /**
+     * Default: null
+     */
+    public Builder simulatedCapacities(long[] val) {
+      this.simulatedCapacities = val;
+      return this;
+    }
+    
+    /**
+     * Construct the actual MiniDFSCluster
+     */
+    public MiniDFSCluster build() throws IOException {
+      return new MiniDFSCluster(this);
+    }
+  }
+  
+  /**
+   * Used by builder to create and return an instance of MiniDFSCluster
+   */
+  private MiniDFSCluster(Builder builder) throws IOException {
+    initMiniDFSCluster(builder.nameNodePort,
+                       builder.conf,
+                       builder.numDataNodes,
+                       builder.format,
+                       builder.manageNameDfsDirs,
+                       builder.manageDataDfsDirs,
+                       builder.option,
+                       builder.racks,
+                       builder.hosts,
+                       builder.simulatedCapacities);
+  }
+  
   public class DataNodeProperties {
     DataNode datanode;
     Configuration conf;
@@ -117,6 +232,7 @@ public class MiniDFSCluster {
    * @param nameNodeOperation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    */
+  @Deprecated // in 22 to be removed in 24. Use MiniDFSCluster.Builder instead
   public MiniDFSCluster(Configuration conf,
                         int numDataNodes,
                         StartupOption nameNodeOperation) throws IOException {
@@ -137,6 +253,7 @@ public class MiniDFSCluster {
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param racks array of strings indicating the rack that each DataNode is on
    */
+  @Deprecated // in 22 to be removed in 24. Use MiniDFSCluster.Builder instead
   public MiniDFSCluster(Configuration conf,
                         int numDataNodes,
                         boolean format,
@@ -158,6 +275,7 @@ public class MiniDFSCluster {
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param hosts array of strings indicating the hostname for each DataNode
    */
+  @Deprecated // in 22 to be removed in 24. Use MiniDFSCluster.Builder instead
   public MiniDFSCluster(Configuration conf,
                         int numDataNodes,
                         boolean format,
@@ -183,6 +301,7 @@ public class MiniDFSCluster {
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
    */
+  @Deprecated // in 22 to be removed in 24. Use MiniDFSCluster.Builder instead
   public MiniDFSCluster(int nameNodePort, 
                         Configuration conf,
                         int numDataNodes,
@@ -213,6 +332,7 @@ public class MiniDFSCluster {
    * @param racks array of strings indicating the rack that each DataNode is on
    * @param simulatedCapacities array of capacities of the simulated data nodes
    */
+  @Deprecated // in 22 to be removed in 24. Use MiniDFSCluster.Builder instead
   public MiniDFSCluster(int nameNodePort, 
                         Configuration conf,
                         int numDataNodes,
@@ -247,6 +367,7 @@ public class MiniDFSCluster {
    * @param hosts array of strings indicating the hostnames of each DataNode
    * @param simulatedCapacities array of capacities of the simulated data nodes
    */
+  @Deprecated // in 22 to be removed in 24. Use MiniDFSCluster.Builder instead
   public MiniDFSCluster(int nameNodePort, 
                         Configuration conf,
                         int numDataNodes,
@@ -256,6 +377,15 @@ public class MiniDFSCluster {
                         StartupOption operation,
                         String[] racks, String hosts[],
                         long[] simulatedCapacities) throws IOException {
+    initMiniDFSCluster(nameNodePort, conf, numDataNodes, format,
+        manageNameDfsDirs, manageDataDfsDirs, operation, racks, hosts,
+        simulatedCapacities);
+  }
+
+  private void initMiniDFSCluster(int nameNodePort, Configuration conf,
+      int numDataNodes, boolean format, boolean manageNameDfsDirs,
+      boolean manageDataDfsDirs, StartupOption operation, String[] racks,
+      String[] hosts, long[] simulatedCapacities) throws IOException {
     this.conf = conf;
     base_dir = new File(getBaseDirectory());
     data_dir = new File(base_dir, "data");

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestAbandonBlock.java Wed Oct 20 22:58:52 2010
@@ -36,7 +36,7 @@ public class TestAbandonBlock extends ju
       = "/" + TestAbandonBlock.class.getSimpleName() + "_"; 
 
   public void testAbandonBlock() throws IOException {
-    MiniDFSCluster cluster = new MiniDFSCluster(CONF, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
     FileSystem fs = cluster.getFileSystem();
 
     String src = FILE_NAME_PREFIX + "foo";

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlockMissingException.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlockMissingException.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlockMissingException.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlockMissingException.java Wed Oct 20 22:58:52 2010
@@ -52,7 +52,7 @@ public class TestBlockMissingException e
     int numBlocks = 4;
     conf = new HdfsConfiguration();
     try {
-      dfs = new MiniDFSCluster(conf, NUM_DATANODES, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
       dfs.waitActive();
       fileSys = (DistributedFileSystem)dfs.getFileSystem();
       Path file1 = new Path("/user/dhruba/raidtest/file1");

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java Wed Oct 20 22:58:52 2010
@@ -20,13 +20,12 @@ package org.apache.hadoop.hdfs;
 import java.io.IOException;
 import java.util.ArrayList;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
+import junit.framework.TestCase;
+
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-
-import junit.framework.TestCase;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 
 /**
  * This class tests DatanodeDescriptor.getBlocksScheduled() at the
@@ -36,9 +35,9 @@ import junit.framework.TestCase;
 public class TestBlocksScheduledCounter extends TestCase {
 
   public void testBlocksScheduledCounter() throws IOException {
-    
-    MiniDFSCluster cluster = new MiniDFSCluster(new HdfsConfiguration(), 1, 
-                                                true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
+                                               .build();
+
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
     

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientBlockVerification.java Wed Oct 20 22:58:52 2010
@@ -56,7 +56,7 @@ public class TestClientBlockVerification
     conf = new HdfsConfiguration();
     int numDataNodes = 1;
     conf.setInt("dfs.replication", numDataNodes);
-    cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java Wed Oct 20 22:58:52 2010
@@ -41,7 +41,7 @@ public class TestClientProtocolForPipeli
     int numDataNodes = 1;
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean("dfs.support.append", true);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     try {
       cluster.waitActive();
       FileSystem fileSys = cluster.getFileSystem();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestCrcCorruption.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestCrcCorruption.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestCrcCorruption.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestCrcCorruption.java Wed Oct 20 22:58:52 2010
@@ -72,7 +72,7 @@ public class TestCrcCorruption {
     Random random = new Random();
 
     try {
-      cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
       cluster.waitActive();
       FileSystem fs = cluster.getFileSystem();
       util.createFiles(fs, "/srcdat", replFactor);
@@ -248,7 +248,7 @@ public class TestCrcCorruption {
 
     Configuration conf = new Configuration();
     conf.setInt("dfs.replication", numDataNodes);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
 
     try {
       cluster.waitActive();

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java Wed Oct 20 22:58:52 2010
@@ -37,7 +37,7 @@ public class TestDFSClientExcludedNodes 
   @Test
   public void testExcludedNodes() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     FileSystem fs = cluster.getFileSystem();
     Path filePath = new Path("/testExcludedNodes");
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java Wed Oct 20 22:58:52 2010
@@ -92,7 +92,7 @@ public class TestDFSClientRetries extend
     final int bufferSize = 4096;
     conf.setInt("io.file.buffer.size", bufferSize);
 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     
     try {
       cluster.waitActive();
@@ -185,7 +185,7 @@ public class TestDFSClientRetries extend
     Configuration conf = new Configuration();
     // Set short retry timeout so this test runs faster
     conf.setInt(DFSConfigKeys.DFS_CLIENT_RETRY_WINDOW_BASE, 10);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
 
     try {
       cluster.waitActive();
@@ -384,7 +384,7 @@ public class TestDFSClientRetries extend
     conf.setInt("dfs.client.max.block.acquire.failures", retries);
     conf.setInt("dfs.client.retry.window.base", timeWin);
 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, replicationFactor, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(replicationFactor).build();
     cluster.waitActive();
     
     FileSystem fs = cluster.getFileSystem();
@@ -538,7 +538,7 @@ public class TestDFSClientRetries extend
     final Path p = new Path(f);
 
     final Configuration conf = new Configuration();
-    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     try {
       cluster.waitActive();
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSFinalize.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSFinalize.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSFinalize.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSFinalize.java Wed Oct 20 22:58:52 2010
@@ -99,7 +99,12 @@ public class TestDFSFinalize extends Tes
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
-      cluster = new MiniDFSCluster(conf, 1, StartupOption.REGULAR);
+      cluster = new MiniDFSCluster.Builder(conf)
+                                  .format(false)
+                                  .manageDataDfsDirs(false)
+                                  .manageNameDfsDirs(false)
+                                  .startupOption(StartupOption.REGULAR)
+                                  .build();
       cluster.finalizeCluster(conf);
       checkResult(nameNodeDirs, dataNodeDirs);
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSMkdirs.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSMkdirs.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSMkdirs.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSMkdirs.java Wed Oct 20 22:58:52 2010
@@ -44,7 +44,7 @@ public class TestDFSMkdirs extends TestC
    */
   public void testDFSMkdirs() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     FileSystem fileSys = cluster.getFileSystem();
     try {
       // First create a new directory with mkdirs
@@ -81,7 +81,7 @@ public class TestDFSMkdirs extends TestC
    */
   public void testMkdir() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
     try {
       // Create a dir in root dir, should succeed

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java Wed Oct 20 22:58:52 2010
@@ -112,7 +112,7 @@ public class TestDFSPermission extends T
    * setPermission works correctly
    */
   public void testPermissionSetting() throws Exception {
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     try {
       cluster.waitActive();
       fs = FileSystem.get(conf);
@@ -228,7 +228,7 @@ public class TestDFSPermission extends T
 
   /* check if the ownership of a file/directory is set correctly */
   public void testOwnership() throws Exception {
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     try {
       cluster.waitActive();
       testOwnership(OpType.CREATE); // test file creation
@@ -323,7 +323,7 @@ public class TestDFSPermission extends T
   /* Check if namenode performs permission checking correctly for
    * superuser, file owner, group owner, and other users */
   public void testPermissionChecking() throws Exception {
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     try {
       cluster.waitActive();
       fs = FileSystem.get(conf);

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRemove.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRemove.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRemove.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRemove.java Wed Oct 20 22:58:52 2010
@@ -56,7 +56,7 @@ public class TestDFSRemove extends junit
   
   public void testRemove() 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/TestDFSRename.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRename.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRename.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRename.java Wed Oct 20 22:58:52 2010
@@ -46,7 +46,7 @@ public class TestDFSRename extends junit
   
   public void testRename() 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/TestDFSRollback.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRollback.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRollback.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSRollback.java Wed Oct 20 22:58:52 2010
@@ -91,7 +91,12 @@ public class TestDFSRollback extends Tes
    */
   void startNameNodeShouldFail(StartupOption operation) {
     try {
-      cluster = new MiniDFSCluster(conf, 0, operation); // should fail
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                                .startupOption(operation)
+                                                .format(false)
+                                                .manageDataDfsDirs(false)
+                                                .manageNameDfsDirs(false)
+                                                .build(); // should fail
       throw new AssertionError("NameNode should have failed to start");
     } catch (Exception expected) {
       // expected
@@ -130,7 +135,12 @@ public class TestDFSRollback extends Tes
       log("Normal NameNode rollback", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.ROLLBACK);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                                .format(false)
+                                                .manageDataDfsDirs(false)
+                                                .manageNameDfsDirs(false)
+                                                .startupOption(StartupOption.ROLLBACK)
+                                                .build();
       checkResult(NAME_NODE, nameNodeDirs);
       cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
@@ -138,7 +148,12 @@ public class TestDFSRollback extends Tes
       log("Normal DataNode rollback", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.ROLLBACK);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                                .format(false)
+                                                .manageDataDfsDirs(false)
+                                                .manageNameDfsDirs(false)
+                                                .startupOption(StartupOption.ROLLBACK)
+                                                .build();
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
       cluster.startDataNodes(conf, 1, false, StartupOption.ROLLBACK, null);
@@ -154,7 +169,12 @@ public class TestDFSRollback extends Tes
       
       log("DataNode rollback without existing previous dir", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.UPGRADE);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                                .format(false)
+                                                .manageDataDfsDirs(false)
+                                                .manageNameDfsDirs(false)
+                                                .startupOption(StartupOption.UPGRADE)
+                                                .build();
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
       cluster.startDataNodes(conf, 1, false, StartupOption.ROLLBACK, null);
       cluster.shutdown();
@@ -164,7 +184,12 @@ public class TestDFSRollback extends Tes
       log("DataNode rollback with future stored layout version in previous", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.ROLLBACK);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                                .format(false)
+                                                .manageDataDfsDirs(false)
+                                                .manageNameDfsDirs(false)
+                                                .startupOption(StartupOption.ROLLBACK)
+                                                .build();
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
       baseDirs = UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
       UpgradeUtilities.createVersionFile(DATA_NODE, baseDirs,
@@ -181,7 +206,12 @@ public class TestDFSRollback extends Tes
       log("DataNode rollback with newer fsscTime in previous", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.ROLLBACK);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                                .format(false)
+                                                .manageDataDfsDirs(false)
+                                                .manageNameDfsDirs(false)
+                                                .startupOption(StartupOption.ROLLBACK)
+                                                .build();
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
       baseDirs = UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
       UpgradeUtilities.createVersionFile(DATA_NODE, baseDirs,

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShell.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShell.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShell.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShell.java Wed Oct 20 22:58:52 2010
@@ -94,7 +94,7 @@ public class TestDFSShell extends TestCa
 
   public void testZeroSizeFile() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
                fs instanceof DistributedFileSystem);
@@ -136,7 +136,7 @@ public class TestDFSShell extends TestCa
   
   public void testRecrusiveRm() throws IOException {
 	  Configuration conf = new HdfsConfiguration();
-	  MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+	  MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
 	  FileSystem fs = cluster.getFileSystem();
 	  assertTrue("Not a HDFS: " + fs.getUri(), 
 			  fs instanceof DistributedFileSystem);
@@ -161,7 +161,7 @@ public class TestDFSShell extends TestCa
     
   public void testDu() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
                 fs instanceof DistributedFileSystem);
@@ -210,7 +210,7 @@ public class TestDFSShell extends TestCa
   }
   public void testPut() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
                fs instanceof DistributedFileSystem);
@@ -309,7 +309,7 @@ public class TestDFSShell extends TestCa
     MiniDFSCluster cluster = null;
     PrintStream bak = null;
     try {
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       FileSystem srcFs = cluster.getFileSystem();
       Path root = new Path("/nonexistentfile");
       bak = System.err;
@@ -453,11 +453,11 @@ public class TestDFSShell extends TestCa
     MiniDFSCluster dstCluster = null;
     String bak = System.getProperty("test.build.data");
     try{
-      srcCluster = new MiniDFSCluster(srcConf, 2, true, null);
+      srcCluster = new MiniDFSCluster.Builder(srcConf).numDataNodes(2).build();
       File nameDir = new File(new File(bak), "dfs_tmp_uri/");
       nameDir.mkdirs();
       System.setProperty("test.build.data", nameDir.toString());
-      dstCluster = new MiniDFSCluster(dstConf, 2, true, null);
+      dstCluster = new MiniDFSCluster.Builder(dstConf).numDataNodes(2).build();
       FileSystem srcFs = srcCluster.getFileSystem();
       FileSystem dstFs = dstCluster.getFileSystem();
       FsShell shell = new FsShell();
@@ -542,7 +542,7 @@ public class TestDFSShell extends TestCa
     Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
     try {
-      cluster = new MiniDFSCluster(conf, 2, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       final FileSystem dfs = cluster.getFileSystem();
       textTest(new Path("/texttest").makeQualified(dfs.getUri(),
             dfs.getWorkingDirectory()), conf);
@@ -596,7 +596,7 @@ public class TestDFSShell extends TestCa
 
   public void testCopyToLocal() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
                fs instanceof DistributedFileSystem);
@@ -693,7 +693,7 @@ public class TestDFSShell extends TestCa
 
   public void testCount() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
     FsShell shell = new FsShell();
     shell.setConf(conf);
@@ -859,7 +859,7 @@ public class TestDFSShell extends TestCa
     conf.set(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, "true");
     
     //test chmod on DFS
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     fs = cluster.getFileSystem();
     testChmod(conf, fs, "/tmp/chmodTest");
     
@@ -917,7 +917,7 @@ public class TestDFSShell extends TestCa
     Configuration conf = new HdfsConfiguration();
     /* This tests some properties of ChecksumFileSystem as well.
      * Make sure that we create ChecksumDFS */
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
             fs instanceof DistributedFileSystem);
@@ -1140,7 +1140,7 @@ public class TestDFSShell extends TestCa
     PrintStream bak = null;
     try {
       final Configuration conf = new HdfsConfiguration();
-      dfs = new MiniDFSCluster(conf, 2, true, null);
+      dfs = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       FileSystem fs = dfs.getFileSystem();
       Path p = new Path("/foo");
       fs.mkdirs(p);
@@ -1179,7 +1179,7 @@ public class TestDFSShell extends TestCa
   public void testGet() throws IOException {
     DFSTestUtil.setLogLevel2All(FSInputChecker.LOG);
     final Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
 
     try {
@@ -1237,7 +1237,7 @@ public class TestDFSShell extends TestCa
 
   public void testLsr() throws Exception {
     final Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
 
     try {

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java Wed Oct 20 22:58:52 2010
@@ -38,7 +38,7 @@ public class TestDFSShellGenericOptions 
     MiniDFSCluster cluster = null;
     try {
       Configuration conf = new HdfsConfiguration();
-      cluster = new MiniDFSCluster(conf, 1, true, null);
+      cluster = new MiniDFSCluster.Builder(conf).build();
       namenode = FileSystem.getDefaultUri(conf).toString();
       String [] args = new String[4];
       args[2] = "-mkdir";

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStartupVersions.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStartupVersions.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStartupVersions.java Wed Oct 20 22:58:52 2010
@@ -175,7 +175,12 @@ public class TestDFSStartupVersions exte
     StorageInfo[] versions = initializeVersions();
     UpgradeUtilities.createStorageDirs(
                                        NAME_NODE, conf.getStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY), "current");
-    cluster = new MiniDFSCluster(conf, 0, StartupOption.REGULAR);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                              .format(false)
+                                              .manageDataDfsDirs(false)
+                                              .manageNameDfsDirs(false)
+                                              .startupOption(StartupOption.REGULAR)
+                                              .build();
     StorageInfo nameNodeVersion = new StorageInfo(
                                                   UpgradeUtilities.getCurrentLayoutVersion(),
                                                   UpgradeUtilities.getCurrentNamespaceID(cluster),

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java Wed Oct 20 22:58:52 2010
@@ -184,6 +184,15 @@ public class TestDFSStorageStateRecovery
     }
   }
  
+  private MiniDFSCluster createCluster(Configuration c) throws IOException {
+    return new MiniDFSCluster.Builder(c)
+                             .numDataNodes(0)
+                             .startupOption(StartupOption.REGULAR)
+                             .format(false)
+                             .manageDataDfsDirs(false)
+                             .manageNameDfsDirs(false)
+                             .build();
+  }
   /**
    * This test iterates over the testCases table and attempts
    * to startup the NameNode normally.
@@ -204,12 +213,12 @@ public class TestDFSStorageStateRecovery
         log("NAME_NODE recovery", numDirs, i, testCase);
         baseDirs = createStorageState(NAME_NODE, testCase);
         if (shouldRecover) {
-          cluster = new MiniDFSCluster(conf, 0, StartupOption.REGULAR);
+          cluster = createCluster(conf);
           checkResult(NAME_NODE, baseDirs, curAfterRecover, prevAfterRecover);
           cluster.shutdown();
         } else {
           try {
-            cluster = new MiniDFSCluster(conf, 0, StartupOption.REGULAR);
+            cluster = createCluster(conf);
             throw new AssertionError("NameNode should have failed to start");
           } catch (IOException expected) {
             // the exception is expected
@@ -247,7 +256,7 @@ public class TestDFSStorageStateRecovery
         log("DATA_NODE recovery", numDirs, i, testCase);
         createStorageState(NAME_NODE,
                            new boolean[] {true, true, false, false, false});
-        cluster = new MiniDFSCluster(conf, 0, StartupOption.REGULAR);
+        cluster = createCluster(conf);
         baseDirs = createStorageState(DATA_NODE, testCase);
         if (!testCase[0] && !testCase[1] && !testCase[2] && !testCase[3]) {
           // DataNode will create and format current if no directories exist

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java Wed Oct 20 22:58:52 2010
@@ -98,7 +98,12 @@ public class TestDFSUpgrade extends Test
    */
   void startNameNodeShouldFail(StartupOption operation) {
     try {
-      cluster = new MiniDFSCluster(conf, 0, operation); // should fail
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                                .startupOption(operation)
+                                                .format(false)
+                                                .manageDataDfsDirs(false)
+                                                .manageNameDfsDirs(false)
+                                                .build(); // should fail
       throw new AssertionError("NameNode should have failed to start");
     } catch (Exception expected) {
       // expected
@@ -120,6 +125,19 @@ public class TestDFSUpgrade extends Test
   }
  
   /**
+   * Create an instance of a newly configured cluster for testing that does
+   * not manage its own directories or files
+   */
+  private MiniDFSCluster createCluster() throws IOException {
+    return new MiniDFSCluster.Builder(conf).numDataNodes(0)
+                                           .format(false)
+                                           .manageDataDfsDirs(false)
+                                           .manageNameDfsDirs(false)
+                                           .startupOption(StartupOption.UPGRADE)
+                                           .build();
+  }
+  
+  /**
    * This test attempts to upgrade the NameNode and DataNode under
    * a number of valid and invalid conditions.
    */
@@ -136,14 +154,14 @@ public class TestDFSUpgrade extends Test
       
       log("Normal NameNode upgrade", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.UPGRADE);
+      cluster = createCluster();
       checkResult(NAME_NODE, nameNodeDirs);
       cluster.shutdown();
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
       
       log("Normal DataNode upgrade", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.UPGRADE);
+      cluster = createCluster();
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
       cluster.startDataNodes(conf, 1, false, StartupOption.REGULAR, null);
       checkResult(DATA_NODE, dataNodeDirs);
@@ -159,7 +177,7 @@ public class TestDFSUpgrade extends Test
       
       log("DataNode upgrade with existing previous dir", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.UPGRADE);
+      cluster = createCluster();
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
       UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
       cluster.startDataNodes(conf, 1, false, StartupOption.REGULAR, null);
@@ -170,7 +188,7 @@ public class TestDFSUpgrade extends Test
 
       log("DataNode upgrade with future stored layout version in current", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.UPGRADE);
+      cluster = createCluster();
       baseDirs = UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
       UpgradeUtilities.createVersionFile(DATA_NODE, baseDirs,
                                          new StorageInfo(Integer.MIN_VALUE,
@@ -185,7 +203,7 @@ public class TestDFSUpgrade extends Test
       
       log("DataNode upgrade with newer fsscTime in current", numDirs);
       UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
-      cluster = new MiniDFSCluster(conf, 0, StartupOption.UPGRADE);
+      cluster = createCluster();
       baseDirs = UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
       UpgradeUtilities.createVersionFile(DATA_NODE, baseDirs,
                                          new StorageInfo(UpgradeUtilities.getCurrentLayoutVersion(),

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java Wed Oct 20 22:58:52 2010
@@ -182,8 +182,11 @@ public class TestDFSUpgradeFromImage ext
         System.setProperty("test.build.data", "build/test/data");
       }
       conf.setInt("dfs.datanode.scan.period.hours", -1); // block scanning off
-      cluster = new MiniDFSCluster(0, conf, numDataNodes, false, true,
-                                   StartupOption.UPGRADE, null);
+      cluster = new MiniDFSCluster.Builder(conf)
+                                  .numDataNodes(numDataNodes)
+                                  .format(false)
+                                  .startupOption(StartupOption.UPGRADE)
+                                  .build();
       cluster.waitActive();
       DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
       DFSClient dfsClient = dfs.dfs;

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java Wed Oct 20 22:58:52 2010
@@ -192,7 +192,7 @@ public class TestDataTransferProtocol ex
     int numDataNodes = 1;
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean("dfs.support.append", true);
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     try {
       cluster.waitActive();
       datanode = cluster.getDataNodes().get(0).dnRegistration;
@@ -323,7 +323,7 @@ public class TestDataTransferProtocol ex
     
     Configuration conf = new HdfsConfiguration();
     conf.setInt("dfs.replication", numDataNodes); 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     try {
     cluster.waitActive();
     DFSClient dfsClient = new DFSClient(

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java Wed Oct 20 22:58:52 2010
@@ -103,7 +103,7 @@ public class TestDatanodeBlockScanner ex
     long startTime = System.currentTimeMillis();
     
     Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     cluster.waitActive();
     
     FileSystem fs = cluster.getFileSystem();
@@ -115,7 +115,9 @@ public class TestDatanodeBlockScanner ex
      */
     DFSTestUtil.createFile(fs, file1, 10, (short)1, 0);
     cluster.shutdown();
-    cluster = new MiniDFSCluster(conf, 1, false, null);
+    cluster = new MiniDFSCluster.Builder(conf)
+                                .numDataNodes(1)
+                                .format(false).build();
     cluster.waitActive();
     
     DFSClient dfsClient =  new DFSClient(new InetSocketAddress("localhost", 
@@ -174,7 +176,7 @@ public class TestDatanodeBlockScanner ex
     int blockCount = 0;
     int rand = random.nextInt(3);
 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
     Path file1 = new Path("/tmp/testBlockVerification/file1");
@@ -285,7 +287,7 @@ public class TestDatanodeBlockScanner ex
     LocatedBlocks blocks = null;
     int replicaCount = 0;
 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
     Path file1 = new Path("/tmp/testBlockCorruptRecovery/file");
@@ -386,7 +388,7 @@ public class TestDatanodeBlockScanner ex
     final Configuration conf = new HdfsConfiguration();
     final short REPLICATION_FACTOR = (short)2;
 
-    MiniDFSCluster cluster = new MiniDFSCluster(conf, REPLICATION_FACTOR, true, null);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION_FACTOR).build();
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
     try {
@@ -402,8 +404,10 @@ public class TestDatanodeBlockScanner ex
       cluster.shutdown();
 
       // restart the cluster
-      cluster = new MiniDFSCluster(
-          0, conf, REPLICATION_FACTOR, false, true, null, null, null);
+      cluster = new MiniDFSCluster.Builder(conf)
+                                  .numDataNodes(REPLICATION_FACTOR)
+                                  .format(false)
+                                  .build();
       cluster.startDataNodes(conf, 1, true, null, null);
       cluster.waitActive();  // now we have 3 datanodes
 

Modified: hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeConfig.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeConfig.java?rev=1025788&r1=1025787&r2=1025788&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeConfig.java (original)
+++ hadoop/hdfs/branches/HDFS-1052/src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeConfig.java Wed Oct 20 22:58:52 2010
@@ -47,7 +47,7 @@ public class TestDatanodeConfig {
   public static void setUp() throws Exception {
     clearBaseDir();
     Configuration conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster(conf, 0, true, null);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
     cluster.waitActive();
   }