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 su...@apache.org on 2012/10/05 08:22:27 UTC

svn commit: r1394376 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/test/java/org/apache/hadoop/hdfs/ src/test/java/org/apache/hadoop/hdfs/server/namenode/ src/test/java/org/apache/hadoop/tools/

Author: suresh
Date: Fri Oct  5 06:22:26 2012
New Revision: 1394376

URL: http://svn.apache.org/viewvc?rev=1394376&view=rev
Log:
HDFS-3995. Use DFSTestUtil.createFile() for file creation and writing in test cases. Contributed by Jing Zhao.

Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Fri Oct  5 06:22:26 2012
@@ -140,6 +140,9 @@ Trunk (Unreleased)
     HDFS-2127. Add a test that ensure AccessControlExceptions contain
     a full path. (Stephen Chu via eli)
 
+    HDFS-3995. Use DFSTestUtil.createFile() for file creation and 
+    writing in test cases. (Jing Zhao via suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java Fri Oct  5 06:22:26 2012
@@ -210,27 +210,40 @@ public class DFSTestUtil {
   
   public static void createFile(FileSystem fs, Path fileName, long fileLen, 
       short replFactor, long seed) throws IOException {
+    createFile(fs, fileName, 1024, fileLen, fs.getDefaultBlockSize(fileName),
+        replFactor, seed);
+  }
+  
+  public static void createFile(FileSystem fs, Path fileName, int bufferLen,
+      long fileLen, long blockSize, short replFactor, long seed)
+      throws IOException {
+    assert bufferLen > 0;
     if (!fs.mkdirs(fileName.getParent())) {
       throw new IOException("Mkdirs failed to create " + 
                             fileName.getParent().toString());
     }
     FSDataOutputStream out = null;
     try {
-      out = fs.create(fileName, replFactor);
-      byte[] toWrite = new byte[1024];
-      Random rb = new Random(seed);
-      long bytesToWrite = fileLen;
-      while (bytesToWrite>0) {
-        rb.nextBytes(toWrite);
-        int bytesToWriteNext = (1024<bytesToWrite)?1024:(int)bytesToWrite;
-
-        out.write(toWrite, 0, bytesToWriteNext);
-        bytesToWrite -= bytesToWriteNext;
+      out = fs.create(fileName, true, fs.getConf()
+        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
+        replFactor, blockSize);
+      if (fileLen > 0) {
+        byte[] toWrite = new byte[bufferLen];
+        Random rb = new Random(seed);
+        long bytesToWrite = fileLen;
+        while (bytesToWrite>0) {
+          rb.nextBytes(toWrite);
+          int bytesToWriteNext = (bufferLen < bytesToWrite) ? bufferLen
+              : (int) bytesToWrite;
+  
+          out.write(toWrite, 0, bytesToWriteNext);
+          bytesToWrite -= bytesToWriteNext;
+        }
       }
-      out.close();
-      out = null;
     } finally {
-      IOUtils.closeStream(out);
+      if (out != null) {
+        out.close();
+      }
     }
   }
   

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java Fri Oct  5 06:22:26 2012
@@ -37,7 +37,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -141,13 +140,6 @@ public class TestDataTransferProtocol {
     }
   }
   
-  void createFile(FileSystem fs, Path path, int fileLen) throws IOException {
-    byte [] arr = new byte[fileLen];
-    FSDataOutputStream out = fs.create(path);
-    out.write(arr);
-    out.close();
-  }
-  
   void readFile(FileSystem fs, Path path, int fileLen) throws IOException {
     byte [] arr = new byte[fileLen];
     FSDataInputStream in = fs.open(path);
@@ -357,7 +349,9 @@ public class TestDataTransferProtocol {
     
     int fileLen = Math.min(conf.getInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096), 4096);
     
-    createFile(fileSys, file, fileLen);
+      DFSTestUtil.createFile(fileSys, file, fileLen, fileLen,
+          fileSys.getDefaultBlockSize(file),
+          fileSys.getDefaultReplication(file), 0L);
 
     // get the first blockid for the file
     final ExtendedBlock firstBlock = DFSTestUtil.getFirstBlock(fileSys, file);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java Fri Oct  5 06:22:26 2012
@@ -79,7 +79,8 @@ public class TestFileStatus {
     hftpfs = cluster.getHftpFileSystem(0);
     dfsClient = new DFSClient(NameNode.getAddress(conf), conf);
     file1 = new Path("filestatus.dat");
-    writeFile(fs, file1, 1, fileSize, blockSize);
+    DFSTestUtil.createFile(fs, file1, fileSize, fileSize, blockSize, (short) 1,
+        seed);
   }
   
   @AfterClass
@@ -87,18 +88,6 @@ public class TestFileStatus {
     fs.close();
     cluster.shutdown();
   }
-
-  private static void writeFile(FileSystem fileSys, Path name, int repl,
-      int fileSize, int blockSize) throws IOException {
-    // Create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true,
-        HdfsConstants.IO_FILE_BUFFER_SIZE, (short)repl, (long)blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
   
   private void checkFile(FileSystem fileSys, Path name, int repl)
       throws IOException, InterruptedException, TimeoutException {
@@ -218,7 +207,8 @@ public class TestFileStatus {
 
     // create another file that is smaller than a block.
     Path file2 = new Path(dir, "filestatus2.dat");
-    writeFile(fs, file2, 1, blockSize/4, blockSize);
+    DFSTestUtil.createFile(fs, file2, blockSize/4, blockSize/4, blockSize,
+        (short) 1, seed);
     checkFile(fs, file2, 1);
     
     // verify file attributes
@@ -230,7 +220,8 @@ public class TestFileStatus {
 
     // Create another file in the same directory
     Path file3 = new Path(dir, "filestatus3.dat");
-    writeFile(fs, file3, 1, blockSize/4, blockSize);
+    DFSTestUtil.createFile(fs, file3, blockSize/4, blockSize/4, blockSize,
+        (short) 1, seed);
     checkFile(fs, file3, 1);
     file3 = fs.makeQualified(file3);
 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java Fri Oct  5 06:22:26 2012
@@ -110,9 +110,7 @@ public class TestGetBlocks {
       // do the writing but do not close the FSDataOutputStream
       // in order to mimic the ongoing writing
       final Path fileName = new Path("/file1");
-      stm = fileSys.create(
-          fileName,
-          true,
+      stm = fileSys.create(fileName, true,
           fileSys.getConf().getInt(
               CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
           (short) 3, blockSize);
@@ -180,29 +178,15 @@ public class TestGetBlocks {
 
     final short REPLICATION_FACTOR = (short) 2;
     final int DEFAULT_BLOCK_SIZE = 1024;
-    final Random r = new Random();
 
     CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(
         REPLICATION_FACTOR).build();
     try {
       cluster.waitActive();
-
-      // create a file with two blocks
-      FileSystem fs = cluster.getFileSystem();
-      FSDataOutputStream out = fs.create(new Path("/tmp.txt"),
-          REPLICATION_FACTOR);
-      byte[] data = new byte[1024];
       long fileLen = 2 * DEFAULT_BLOCK_SIZE;
-      long bytesToWrite = fileLen;
-      while (bytesToWrite > 0) {
-        r.nextBytes(data);
-        int bytesToWriteNext = (1024 < bytesToWrite) ? 1024
-            : (int) bytesToWrite;
-        out.write(data, 0, bytesToWriteNext);
-        bytesToWrite -= bytesToWriteNext;
-      }
-      out.close();
+      DFSTestUtil.createFile(cluster.getFileSystem(), new Path("/tmp.txt"),
+          fileLen, REPLICATION_FACTOR, 0L);
 
       // get blocks & data nodes
       List<LocatedBlock> locatedBlocks;

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java Fri Oct  5 06:22:26 2012
@@ -52,22 +52,6 @@ public class TestInjectionForSimulatedSt
   private static final Log LOG = LogFactory.getLog(
       "org.apache.hadoop.hdfs.TestInjectionForSimulatedStorage");
 
-  
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-                                                throws IOException {
-    // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, blockSize);
-    byte[] buffer = new byte[filesize];
-    for (int i=0; i<buffer.length; i++) {
-      buffer[i] = '1';
-    }
-    stm.write(buffer);
-    stm.close();
-  }
-  
-  // Waits for all of the blocks to have expected replication
 
   // Waits for all of the blocks to have expected replication
   private void waitForBlockReplication(String filename, 
@@ -149,7 +133,8 @@ public class TestInjectionForSimulatedSt
                                             cluster.getNameNodePort()),
                                             conf);
       
-      writeFile(cluster.getFileSystem(), testPath, numDataNodes);
+      DFSTestUtil.createFile(cluster.getFileSystem(), testPath, filesize,
+          filesize, blockSize, (short) numDataNodes, 0L);
       waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, 20);
       Iterable<Block>[] blocksList = cluster.getAllBlockReports(bpid);
       

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java Fri Oct  5 06:22:26 2012
@@ -50,19 +50,6 @@ public class TestModTime {
   Random myrand = new Random();
   Path hostsFile;
   Path excludeFile;
-
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-    throws IOException {
-    // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
   
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
@@ -105,7 +92,8 @@ public class TestModTime {
      System.out.println("Creating testdir1 and testdir1/test1.dat.");
      Path dir1 = new Path("testdir1");
      Path file1 = new Path(dir1, "test1.dat");
-     writeFile(fileSys, file1, replicas);
+     DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+         (short) replicas, seed);
      FileStatus stat = fileSys.getFileStatus(file1);
      long mtime1 = stat.getModificationTime();
      assertTrue(mtime1 != 0);
@@ -120,7 +108,8 @@ public class TestModTime {
      //
      System.out.println("Creating testdir1/test2.dat.");
      Path file2 = new Path(dir1, "test2.dat");
-     writeFile(fileSys, file2, replicas);
+     DFSTestUtil.createFile(fileSys, file2, fileSize, fileSize, blockSize,
+         (short) replicas, seed);
      stat = fileSys.getFileStatus(file2);
 
      //

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java Fri Oct  5 06:22:26 2012
@@ -41,11 +41,9 @@ public class TestPread {
   boolean simulatedStorage = false;
 
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
-    // create and write a file that contains three blocks of data
-    DataOutputStream stm = fileSys.create(name, true, 4096, (short)1,
-                                          blockSize);
     // test empty file open and read
-    stm.close();
+    DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 0,
+        blockSize, (short) 1, seed);
     FSDataInputStream in = fileSys.open(name);
     byte[] buffer = new byte[12 * blockSize];
     in.readFully(0, buffer, 0, 0);
@@ -62,11 +60,8 @@ public class TestPread {
       assertTrue("Cannot delete file", false);
     
     // now create the real file
-    stm = fileSys.create(name, true, 4096, (short)1, blockSize);
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
+    DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 12 * blockSize,
+        blockSize, (short) 1, seed);
   }
   
   private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java Fri Oct  5 06:22:26 2012
@@ -26,15 +26,12 @@ import java.io.OutputStream;
 import java.io.RandomAccessFile;
 import java.net.InetSocketAddress;
 import java.util.Iterator;
-import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -61,19 +58,6 @@ public class TestReplication {
   private static final int numDatanodes = racks.length;
   private static final Log LOG = LogFactory.getLog(
                                        "org.apache.hadoop.hdfs.TestReplication");
-
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-    throws IOException {
-    // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
   
   /* check if there are at least two nodes are on the same rack */
   private void checkFile(FileSystem fileSys, Path name, int repl)
@@ -222,19 +206,25 @@ public class TestReplication {
     FileSystem fileSys = cluster.getFileSystem();
     try {
       Path file1 = new Path("/smallblocktest.dat");
-      writeFile(fileSys, file1, 3);
+      //writeFile(fileSys, file1, 3);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          (short) 3, seed);
       checkFile(fileSys, file1, 3);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file1, 10);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          (short) 10, seed);
       checkFile(fileSys, file1, 10);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file1, 4);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          (short) 4, seed);
       checkFile(fileSys, file1, 4);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file1, 1);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          (short) 1, seed);
       checkFile(fileSys, file1, 1);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file1, 2);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          (short) 2, seed);
       checkFile(fileSys, file1, 2);
       cleanupFile(fileSys, file1);
     } finally {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java Fri Oct  5 06:22:26 2012
@@ -40,16 +40,6 @@ public class TestSeekBug {
   static final long seed = 0xDEADBEEFL;
   static final int ONEMB = 1 << 20;
   
-  private void writeFile(FileSystem fileSys, Path name) throws IOException {
-    // create and write a file that contains 1MB
-    DataOutputStream stm = fileSys.create(name);
-    byte[] buffer = new byte[ONEMB];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-  
   private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
     for (int idx = 0; idx < actual.length; idx++) {
       assertEquals(message+" byte "+(from+idx)+" differs. expected "+
@@ -132,7 +122,9 @@ public class TestSeekBug {
     FileSystem fileSys = cluster.getFileSystem();
     try {
       Path file1 = new Path("seektest.dat");
-      writeFile(fileSys, file1);
+      DFSTestUtil.createFile(fileSys, file1, ONEMB, ONEMB,
+          fileSys.getDefaultBlockSize(file1),
+          fileSys.getDefaultReplication(file1), seed);
       seekReadFile(fileSys, file1);
       smallReadSeek(fileSys, file1);
       cleanupFile(fileSys, file1);
@@ -151,7 +143,9 @@ public class TestSeekBug {
     FileSystem fileSys = FileSystem.getLocal(conf);
     try {
       Path file1 = new Path("build/test/data", "seektest.dat");
-      writeFile(fileSys, file1);
+      DFSTestUtil.createFile(fileSys, file1, ONEMB, ONEMB,
+          fileSys.getDefaultBlockSize(file1),
+          fileSys.getDefaultReplication(file1), seed);
       seekReadFile(fileSys, file1);
       cleanupFile(fileSys, file1);
     } finally {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSmallBlock.java Fri Oct  5 06:22:26 2012
@@ -42,18 +42,6 @@ public class TestSmallBlock {
   static final int blockSize = 1;
   static final int fileSize = 20;
   boolean simulatedStorage = false;
-
-  private void writeFile(FileSystem fileSys, Path name) throws IOException {
-    // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) 1, blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
   
   private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
     for (int idx = 0; idx < actual.length; idx++) {
@@ -105,7 +93,8 @@ public class TestSmallBlock {
     FileSystem fileSys = cluster.getFileSystem();
     try {
       Path file1 = new Path("smallblocktest.dat");
-      writeFile(fileSys, file1);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          (short) 1, seed);
       checkFile(fileSys, file1);
       cleanupFile(fileSys, file1);
     } finally {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java Fri Oct  5 06:22:26 2012
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -64,6 +65,10 @@ public class TestBackupNode {
   }
   
   static final String BASE_DIR = MiniDFSCluster.getBaseDirectory();
+  
+  static final long seed = 0xDEADBEEFL;
+  static final int blockSize = 4096;
+  static final int fileSize = 8192;
 
   @Before
   public void setUp() throws Exception {
@@ -350,14 +355,17 @@ public class TestBackupNode {
           + NetUtils.getHostPortString(add)).toUri(), conf);
       boolean canWrite = true;
       try {
-        TestCheckpoint.writeFile(bnFS, file3, replication);
+        DFSTestUtil.createFile(bnFS, file3, fileSize, fileSize, blockSize,
+            replication, seed);
       } catch (IOException eio) {
         LOG.info("Write to BN failed as expected: ", eio);
         canWrite = false;
       }
       assertFalse("Write to BackupNode must be prohibited.", canWrite);
 
-      TestCheckpoint.writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, fileSize, fileSize, blockSize,
+          replication, seed);
+      
       TestCheckpoint.checkFile(fileSys, file3, replication);
       // should also be on BN right away
       assertTrue("file3 does not exist on BackupNode",

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java Fri Oct  5 06:22:26 2012
@@ -28,7 +28,6 @@ import static org.junit.Assert.assertTru
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.FilenameFilter;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.net.InetSocketAddress;
@@ -36,14 +35,12 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.Random;
 
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
@@ -51,6 +48,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -116,19 +114,6 @@ public class TestCheckpoint {
     faultInjector = Mockito.mock(CheckpointFaultInjector.class);
     CheckpointFaultInjector.instance = faultInjector;
   }
-
-  static void writeFile(FileSystem fileSys, Path name, int repl)
-    throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, blockSize);
-    byte[] buffer = new byte[TestCheckpoint.fileSize];
-    Random rand = new Random(TestCheckpoint.seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-  
   
   static void checkFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
@@ -259,7 +244,8 @@ public class TestCheckpoint {
       //
       // Create a new file
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
     } finally {
       fileSys.close();
@@ -323,7 +309,8 @@ public class TestCheckpoint {
       //
       // Create a new file
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
     } finally {
       fileSys.close();
@@ -394,7 +381,8 @@ public class TestCheckpoint {
       //
       // Create a new file
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
     } finally {
       fileSys.close();
@@ -580,7 +568,8 @@ public class TestCheckpoint {
       //
       // Create a new file
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
     } finally {
       fileSys.close();
@@ -906,7 +895,8 @@ public class TestCheckpoint {
       //
       // Create file1
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
 
       //
@@ -933,7 +923,8 @@ public class TestCheckpoint {
       cleanupFile(fileSys, file1);
 
       // create new file file2
-      writeFile(fileSys, file2, replication);
+      DFSTestUtil.createFile(fileSys, file2, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file2, replication);
 
       //
@@ -999,7 +990,8 @@ public class TestCheckpoint {
       }
       // create new file
       Path file = new Path("namespace.dat");
-      writeFile(fs, file, replication);
+      DFSTestUtil.createFile(fs, file, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fs, file, replication);
 
       // create new link

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java Fri Oct  5 06:22:26 2012
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -116,19 +117,6 @@ public class TestDecommissioningStatus {
     stm.close();
   }
 
-  private void writeFile(FileSystem fileSys, Path name, short repl)
-      throws IOException {
-    // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
-        blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
- 
   private FSDataOutputStream writeIncompleteFile(FileSystem fileSys, Path name,
       short repl) throws IOException {
     // create and write a file that contains three blocks of data
@@ -198,7 +186,8 @@ public class TestDecommissioningStatus {
     // Decommission one node. Verify the decommission status
     // 
     Path file1 = new Path("decommission.dat");
-    writeFile(fileSys, file1, replicas);
+    DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+        replicas, seed);
 
     Path file2 = new Path("decommission1.dat");
     FSDataOutputStream st1 = writeIncompleteFile(fileSys, file2, replicas);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java Fri Oct  5 06:22:26 2012
@@ -20,14 +20,12 @@ package org.apache.hadoop.hdfs.server.na
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
@@ -43,19 +41,6 @@ public class TestFileLimit {
   static final int blockSize = 8192;
   boolean simulatedStorage = false;
 
-  // creates a zero file.
-  private void createFile(FileSystem fileSys, Path name)
-    throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) 1, blockSize);
-    byte[] buffer = new byte[1024];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
   private void waitForLimit(FSNamesystem namesys, long num)
   {
     // wait for number of blocks to decrease
@@ -106,7 +91,7 @@ public class TestFileLimit {
       //
       for (int i = 0; i < maxObjects/2; i++) {
         Path file = new Path("/filestatus" + i);
-        createFile(fs, file);
+        DFSTestUtil.createFile(fs, file, 1024, 1024, blockSize, (short) 1, seed);
         System.out.println("Created file " + file);
         currentNodes += 2;      // two more objects for this creation.
       }
@@ -115,7 +100,7 @@ public class TestFileLimit {
       boolean hitException = false;
       try {
         Path file = new Path("/filestatus");
-        createFile(fs, file);
+        DFSTestUtil.createFile(fs, file, 1024, 1024, blockSize, (short) 1, seed);
         System.out.println("Created file " + file);
       } catch (IOException e) {
         hitException = true;
@@ -132,7 +117,7 @@ public class TestFileLimit {
       waitForLimit(namesys, currentNodes);
 
       // now, we shud be able to create a new file
-      createFile(fs, file0);
+      DFSTestUtil.createFile(fs, file0, 1024, 1024, blockSize, (short) 1, seed);
       System.out.println("Created file " + file0 + " again.");
       currentNodes += 2;
 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java Fri Oct  5 06:22:26 2012
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.AfterClass;
@@ -48,17 +49,6 @@ public class TestMetaSave {
   private static MiniDFSCluster cluster = null;
   private static FileSystem fileSys = null;
 
-  private void createFile(FileSystem fileSys, Path name) throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) 2, blockSize);
-    byte[] buffer = new byte[1024];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
   @BeforeClass
   public static void setUp() throws IOException {
     // start a cluster
@@ -84,7 +74,8 @@ public class TestMetaSave {
 
     for (int i = 0; i < 2; i++) {
       Path file = new Path("/filestatus" + i);
-      createFile(fileSys, file);
+      DFSTestUtil.createFile(fileSys, file, 1024, 1024, blockSize, (short) 2,
+          seed);
     }
 
     cluster.stopDataNode(1);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java Fri Oct  5 06:22:26 2012
@@ -25,17 +25,15 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
-import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -71,18 +69,6 @@ public class TestNameEditsConfigs {
     }
   }
 
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-      throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, BLOCK_SIZE);
-    byte[] buffer = new byte[FILE_SIZE];
-    Random rand = new Random(SEED);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
   void checkImageAndEditsFilesExistence(File dir, 
                                         boolean shouldHaveImages,
                                         boolean shouldHaveEdits)
@@ -187,7 +173,8 @@ public class TestNameEditsConfigs {
 
     try {
       assertTrue(!fileSys.exists(file1));
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file1, replication);
       secondary.doCheckpoint();
     } finally {
@@ -224,7 +211,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file1));
       checkFile(fileSys, file1, replication);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file2, replication);
+      DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file2, replication);
       secondary.doCheckpoint();
     } finally {
@@ -260,7 +248,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file2));
       checkFile(fileSys, file2, replication);
       cleanupFile(fileSys, file2);
-      writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file3, replication);
       secondary.doCheckpoint();
     } finally {
@@ -364,7 +353,8 @@ public class TestNameEditsConfigs {
       fileSys = cluster.getFileSystem();
 
       assertTrue(!fileSys.exists(file1));
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file1, replication);
     } finally  {
       fileSys.close();
@@ -402,7 +392,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file1));
       checkFile(fileSys, file1, replication);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file2, replication);
+      DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file2, replication);
     } finally {
       fileSys.close();
@@ -429,7 +420,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file2));
       checkFile(fileSys, file2, replication);
       cleanupFile(fileSys, file2);
-      writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file3, replication);
     } finally {
       fileSys.close();
@@ -483,7 +475,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file3));
       checkFile(fileSys, file3, replication);
       cleanupFile(fileSys, file3);
-      writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file3, replication);
     } finally {
       fileSys.close();

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java Fri Oct  5 06:22:26 2012
@@ -77,20 +77,6 @@ public class TestStartup {
   static final int fileSize = 8192;
   private long editsLength=0, fsimageLength=0;
 
-
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-  throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
-
   @Before
   public void setUp() throws Exception {
     config = new HdfsConfiguration();
@@ -150,7 +136,8 @@ public class TestStartup {
       // create a file
       FileSystem fileSys = cluster.getFileSystem();
       Path file1 = new Path("t1");
-      this.writeFile(fileSys, file1, 1);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize, 
+          (short) 1, seed);
 
       LOG.info("--doing checkpoint");
       sn.doCheckpoint();  // this shouldn't fail

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java?rev=1394376&r1=1394375&r2=1394376&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java Fri Oct  5 06:22:26 2012
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.tools.JMXGet;
@@ -53,19 +54,6 @@ public class TestJMXGet {
   static final int blockSize = 4096;
   static final int fileSize = 8192;
 
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-  throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-        fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short)repl, blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
-
   @Before
   public void setUp() throws Exception {
     config = new HdfsConfiguration();
@@ -96,7 +84,8 @@ public class TestJMXGet {
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes).build();
     cluster.waitActive();
 
-    writeFile(cluster.getFileSystem(), new Path("/test1"), 2);
+    DFSTestUtil.createFile(cluster.getFileSystem(), new Path("/test1"),
+        fileSize, fileSize, blockSize, (short) 2, seed);
 
     JMXGet jmx = new JMXGet();
     //jmx.setService("*"); // list all hadoop services
@@ -125,7 +114,8 @@ public class TestJMXGet {
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes).build();
     cluster.waitActive();
 
-    writeFile(cluster.getFileSystem(), new Path("/test"), 2);
+    DFSTestUtil.createFile(cluster.getFileSystem(), new Path("/test"),
+        fileSize, fileSize, blockSize, (short) 2, seed);
 
     JMXGet jmx = new JMXGet();
     //jmx.setService("*"); // list all hadoop services