You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by el...@apache.org on 2011/09/02 05:20:13 UTC

svn commit: r1164339 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoop/util/ src/test/java/org/apache/hadoop/fs/ src/test/java/org/apache/hadoop/io/nativeio/ src/test/j...

Author: eli
Date: Fri Sep  2 03:20:12 2011
New Revision: 1164339

URL: http://svn.apache.org/viewvc?rev=1164339&view=rev
Log:
HADOOP-7552. FileUtil#fullyDelete doesn't throw IOE but lists it in the throws clause. Contributed by Eli Collins

Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Fri Sep  2 03:20:12 2011
@@ -356,6 +356,9 @@ Release 0.23.0 - Unreleased
 
     HADOOP-7594. Support HTTP REST in HttpServer.  (szetszwo)
 
+    HADOOP-7552. FileUtil#fullyDelete doesn't throw IOE but lists it
+    in the throws clause. (eli)
+
   OPTIMIZATIONS
   
     HADOOP-7333. Performance improvement in PureJavaCrc32. (Eric Caspole

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java Fri Sep  2 03:20:12 2011
@@ -28,7 +28,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 
@@ -88,7 +87,7 @@ public class FileUtil {
    * (4) If dir is a normal directory, then dir and all its contents recursively
    *     are deleted.
    */
-  public static boolean fullyDelete(File dir) throws IOException {
+  public static boolean fullyDelete(File dir) {
     if (dir.delete()) {
       // dir is (a) normal file, (b) symlink to a file, (c) empty directory or
       // (d) symlink to a directory
@@ -108,7 +107,7 @@ public class FileUtil {
    * If dir is a symlink to a directory, all the contents of the actual
    * directory pointed to by dir will be deleted.
    */
-  public static boolean fullyDeleteContents(File dir) throws IOException {
+  public static boolean fullyDeleteContents(File dir) {
     boolean deletionSucceeded = true;
     File contents[] = dir.listFiles();
     if (contents != null) {

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java Fri Sep  2 03:20:12 2011
@@ -228,10 +228,10 @@ public class RawLocalFileSystem extends 
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
     if (!exists(f)) {
-      throw new FileNotFoundException("File " + f + " not found.");
+      throw new FileNotFoundException("File " + f + " not found");
     }
     if (getFileStatus(f).isDirectory()) {
-      throw new IOException("Cannot append to a diretory (=" + f + " ).");
+      throw new IOException("Cannot append to a diretory (=" + f + " )");
     }
     return new FSDataOutputStream(new BufferedOutputStream(
         new LocalFSFileOutputStream(f, true), bufferSize), statistics);
@@ -242,7 +242,7 @@ public class RawLocalFileSystem extends 
     short replication, long blockSize, Progressable progress)
     throws IOException {
     if (exists(f) && !overwrite) {
-      throw new IOException("File already exists:"+f);
+      throw new IOException("File already exists: "+f);
     }
     Path parent = f.getParent();
     if (parent != null && !mkdirs(parent)) {
@@ -271,11 +271,18 @@ public class RawLocalFileSystem extends 
     return FileUtil.copy(this, src, this, dst, true, getConf());
   }
   
+  /**
+   * Delete the given path to a file or directory.
+   * @param p the path to delete
+   * @param recursive to delete sub-directories
+   * @return true if the file or directory and all its contents were deleted
+   * @throws IOException if p is non-empty and recursive is false 
+   */
   public boolean delete(Path p, boolean recursive) throws IOException {
     File f = pathToFile(p);
     if (f.isFile()) {
       return f.delete();
-    } else if ((!recursive) && f.isDirectory() && 
+    } else if (!recursive && f.isDirectory() && 
         (FileUtil.listFiles(f).length != 0)) {
       throw new IOException("Directory " + f.toString() + " is not empty");
     }
@@ -287,7 +294,7 @@ public class RawLocalFileSystem extends 
     FileStatus[] results;
 
     if (!localf.exists()) {
-      throw new FileNotFoundException("File " + f + " does not exist.");
+      throw new FileNotFoundException("File " + f + " does not exist");
     }
     if (localf.isFile()) {
       return new FileStatus[] {
@@ -421,7 +428,7 @@ public class RawLocalFileSystem extends 
     if (path.exists()) {
       return new RawLocalFileStatus(pathToFile(f), getDefaultBlockSize(), this);
     } else {
-      throw new FileNotFoundException("File " + f + " does not exist.");
+      throw new FileNotFoundException("File " + f + " does not exist");
     }
   }
 

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java Fri Sep  2 03:20:12 2011
@@ -158,10 +158,7 @@ public class RunJar {
 
     Runtime.getRuntime().addShutdownHook(new Thread() {
         public void run() {
-          try {
-            FileUtil.fullyDelete(workDir);
-          } catch (IOException e) {
-          }
+          FileUtil.fullyDelete(workDir);
         }
       });
 

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestHelper.java Fri Sep  2 03:20:12 2011
@@ -17,16 +17,15 @@
  */
 package org.apache.hadoop.fs;
 
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.FileNotFoundException;
 import java.net.URI;
+import java.util.Random;
 
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
-
+import static org.junit.Assert.*;
 
 /**
  * Helper class for unit tests.
@@ -143,23 +142,33 @@ public final class FileSystemTestHelper 
     }
   }
   
-  
-  public static void writeFile(FileSystem fSys, Path path,byte b[])
-    throws Exception {
-    FSDataOutputStream out = 
-      fSys.create(path);
-    out.write(b);
-    out.close();
+  static String writeFile(FileSystem fileSys, Path name, int fileSize)
+    throws IOException {
+    final long seed = 0xDEADBEEFL;
+    // Create and write a file that contains three blocks of data
+    FSDataOutputStream stm = fileSys.create(name);
+    byte[] buffer = new byte[fileSize];
+    Random rand = new Random(seed);
+    rand.nextBytes(buffer);
+    stm.write(buffer);
+    stm.close();
+    return new String(buffer);
   }
   
-  public static byte[] readFile(FileSystem fSys, Path path, int len )
-    throws Exception {
-    DataInputStream dis = fSys.open(path);
-    byte[] buffer = new byte[len];
-    IOUtils.readFully(dis, buffer, 0, len);
-    dis.close();
-    return buffer;
+  static String readFile(FileSystem fs, Path name, int buflen) 
+    throws IOException {
+    byte[] b = new byte[buflen];
+    int offset = 0;
+    FSDataInputStream in = fs.open(name);
+    for (int remaining, n;
+        (remaining = b.length - offset) > 0 && (n = in.read(b, offset, remaining)) != -1;
+        offset += n); 
+    assertEquals(offset, Math.min(b.length, in.getPos()));
+    in.close();
+    String s = new String(b, 0, offset);
+    return s;
   }
+
   public static FileStatus containsPath(FileSystem fSys, Path path,
       FileStatus[] dirList)
     throws IOException {

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestChecksumFileSystem.java Fri Sep  2 03:20:12 2011
@@ -18,10 +18,9 @@
 
 package org.apache.hadoop.fs;
 
-import java.net.URI;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import static org.apache.hadoop.fs.FileSystemTestHelper.*;
 import org.apache.hadoop.conf.Configuration;
 import junit.framework.TestCase;
 
@@ -56,13 +55,13 @@ public class TestChecksumFileSystem exte
 
     // Exercise some boundary cases - a divisor of the chunk size
     // the chunk size, 2x chunk size, and +/-1 around these.
-    TestLocalFileSystem.readFile(localFs, testPath, 128);
-    TestLocalFileSystem.readFile(localFs, testPath, 511);
-    TestLocalFileSystem.readFile(localFs, testPath, 512);
-    TestLocalFileSystem.readFile(localFs, testPath, 513);
-    TestLocalFileSystem.readFile(localFs, testPath, 1023);
-    TestLocalFileSystem.readFile(localFs, testPath, 1024);
-    TestLocalFileSystem.readFile(localFs, testPath, 1025);
+    readFile(localFs, testPath, 128);
+    readFile(localFs, testPath, 511);
+    readFile(localFs, testPath, 512);
+    readFile(localFs, testPath, 513);
+    readFile(localFs, testPath, 1023);
+    readFile(localFs, testPath, 1024);
+    readFile(localFs, testPath, 1025);
 
     localFs.delete(localFs.getChecksumFile(testPath), true);
     assertTrue("checksum deleted", !localFs.exists(localFs.getChecksumFile(testPath)));
@@ -74,7 +73,7 @@ public class TestChecksumFileSystem exte
     
     boolean errorRead = false;
     try {
-      TestLocalFileSystem.readFile(localFs, testPath, 1024);
+      readFile(localFs, testPath, 1024);
     }catch(ChecksumException ie) {
       errorRead = true;
     }
@@ -83,7 +82,7 @@ public class TestChecksumFileSystem exte
     //now setting verify false, the read should succeed
     try {
       localFs.setVerifyChecksum(false);
-      String str = TestLocalFileSystem.readFile(localFs, testPath, 1024);
+      String str = readFile(localFs, testPath, 1024).toString();
       assertTrue("read", "testing".equals(str));
     } finally {
       // reset for other tests
@@ -104,13 +103,13 @@ public class TestChecksumFileSystem exte
 
     // Exercise some boundary cases - a divisor of the chunk size
     // the chunk size, 2x chunk size, and +/-1 around these.
-    TestLocalFileSystem.readFile(localFs, testPath, 128);
-    TestLocalFileSystem.readFile(localFs, testPath, 511);
-    TestLocalFileSystem.readFile(localFs, testPath, 512);
-    TestLocalFileSystem.readFile(localFs, testPath, 513);
-    TestLocalFileSystem.readFile(localFs, testPath, 1023);
-    TestLocalFileSystem.readFile(localFs, testPath, 1024);
-    TestLocalFileSystem.readFile(localFs, testPath, 1025);
+    readFile(localFs, testPath, 128);
+    readFile(localFs, testPath, 511);
+    readFile(localFs, testPath, 512);
+    readFile(localFs, testPath, 513);
+    readFile(localFs, testPath, 1023);
+    readFile(localFs, testPath, 1024);
+    readFile(localFs, testPath, 1025);
   }
 
   /**
@@ -140,7 +139,7 @@ public class TestChecksumFileSystem exte
 
     // Now reading the file should fail with a ChecksumException
     try {
-      TestLocalFileSystem.readFile(localFs, testPath, 1024);
+      readFile(localFs, testPath, 1024);
       fail("Did not throw a ChecksumException when reading truncated " +
            "crc file");
     } catch(ChecksumException ie) {
@@ -149,7 +148,7 @@ public class TestChecksumFileSystem exte
     // telling it not to verify checksums, should avoid issue.
     try {
       localFs.setVerifyChecksum(false);
-      String str = TestLocalFileSystem.readFile(localFs, testPath, 1024);
+      String str = readFile(localFs, testPath, 1024).toString();
       assertTrue("read", "testing truncation".equals(str));
     } finally {
       // reset for other tests

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDU.java Fri Sep  2 03:20:12 2011
@@ -29,7 +29,7 @@ public class TestDU extends TestCase {
   final static private File DU_DIR = new File(
       System.getProperty("test.build.data","/tmp"), "dutmp");
 
-  public void setUp() throws IOException {
+  public void setUp() {
       FileUtil.fullyDelete(DU_DIR);
       assertTrue(DU_DIR.mkdirs());
   }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java Fri Sep  2 03:20:12 2011
@@ -98,7 +98,7 @@ public class TestHardLink {
    * @throws IOException
    */
   @BeforeClass
-  public static void setupClean() throws IOException {
+  public static void setupClean() {
     //delete source and target directories if they exist
     FileUtil.fullyDelete(src);
     FileUtil.fullyDelete(tgt_one);

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java Fri Sep  2 03:20:12 2011
@@ -18,37 +18,23 @@
 package org.apache.hadoop.fs;
 
 import org.apache.hadoop.conf.Configuration;
+import static org.apache.hadoop.fs.FileSystemTestHelper.*;
+
 import java.io.*;
-import junit.framework.*;
+
+import static org.junit.Assert.*;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * This class tests the local file system via the FileSystem abstraction.
  */
-public class TestLocalFileSystem extends TestCase {
+public class TestLocalFileSystem {
   private static String TEST_ROOT_DIR
     = System.getProperty("test.build.data","build/test/data/work-dir/localfs");
 
-
-  static void writeFile(FileSystem fs, Path name) throws IOException {
-    FSDataOutputStream stm = fs.create(name);
-    stm.writeBytes("42\n");
-    stm.close();
-  }
-  
-  static String readFile(FileSystem fs, Path name, int buflen) throws IOException {
-    byte[] b = new byte[buflen];
-    int offset = 0;
-    FSDataInputStream in = fs.open(name);
-    for(int remaining, n;
-        (remaining = b.length - offset) > 0 && (n = in.read(b, offset, remaining)) != -1;
-        offset += n); 
-    assertEquals(offset, Math.min(b.length, in.getPos()));
-    in.close();
-
-    String s = new String(b, 0, offset);
-    System.out.println("s=" + s);
-    return s;
-  }
+  private Configuration conf;
+  private FileSystem fileSys;
 
   private void cleanupFile(FileSystem fs, Path name) throws IOException {
     assertTrue(fs.exists(name));
@@ -56,12 +42,18 @@ public class TestLocalFileSystem extends
     assertTrue(!fs.exists(name));
   }
   
+  @Before
+  public void setup() throws IOException {
+    conf = new Configuration();
+    fileSys = FileSystem.getLocal(conf);
+    fileSys.delete(new Path(TEST_ROOT_DIR), true);
+  }
+
   /**
    * Test the capability of setting the working directory.
    */
+  @Test
   public void testWorkingDirectory() throws IOException {
-    Configuration conf = new Configuration();
-    FileSystem fileSys = FileSystem.getLocal(conf);
     Path origDir = fileSys.getWorkingDirectory();
     Path subdir = new Path(TEST_ROOT_DIR, "new");
     try {
@@ -85,7 +77,7 @@ public class TestLocalFileSystem extends
       // create files and manipulate them.
       Path file1 = new Path("file1");
       Path file2 = new Path("sub/file2");
-      writeFile(fileSys, file1);
+      String contents = writeFile(fileSys, file1, 1);
       fileSys.copyFromLocalFile(file1, file2);
       assertTrue(fileSys.exists(file1));
       assertTrue(fileSys.isFile(file1));
@@ -103,11 +95,10 @@ public class TestLocalFileSystem extends
       InputStream stm = fileSys.open(file1);
       byte[] buffer = new byte[3];
       int bytesRead = stm.read(buffer, 0, 3);
-      assertEquals("42\n", new String(buffer, 0, bytesRead));
+      assertEquals(contents, new String(buffer, 0, bytesRead));
       stm.close();
     } finally {
       fileSys.setWorkingDirectory(origDir);
-      fileSys.delete(subdir, true);
     }
   }
 
@@ -115,6 +106,7 @@ public class TestLocalFileSystem extends
    * test Syncable interface on raw local file system
    * @throws IOException
    */
+  @Test
   public void testSyncable() throws IOException {
     Configuration conf = new Configuration();
     FileSystem fs = FileSystem.getLocal(conf).getRawFileSystem();
@@ -148,12 +140,13 @@ public class TestLocalFileSystem extends
     }
   }
   
+  @Test
   public void testCopy() throws IOException {
     Configuration conf = new Configuration();
     LocalFileSystem fs = FileSystem.getLocal(conf);
     Path src = new Path(TEST_ROOT_DIR, "dingo");
     Path dst = new Path(TEST_ROOT_DIR, "yak");
-    writeFile(fs, src);
+    writeFile(fs, src, 1);
     assertTrue(FileUtil.copy(fs, src, fs, dst, true, false, conf));
     assertTrue(!fs.exists(src) && fs.exists(dst));
     assertTrue(FileUtil.copy(fs, dst, fs, src, false, false, conf));
@@ -170,9 +163,12 @@ public class TestLocalFileSystem extends
     try {
       FileUtil.copy(fs, dst, fs, src, true, true, conf);
       fail("Failed to detect existing dir");
-    } catch (IOException e) { }
+    } catch (IOException e) {
+      // Expected
+    }
   }
 
+  @Test
   public void testHomeDirectory() throws IOException {
     Configuration conf = new Configuration();
     FileSystem fileSys = FileSystem.getLocal(conf);
@@ -182,16 +178,18 @@ public class TestLocalFileSystem extends
     assertEquals(home, fsHome);
   }
 
+  @Test
   public void testPathEscapes() throws IOException {
     Configuration conf = new Configuration();
     FileSystem fs = FileSystem.getLocal(conf);
     Path path = new Path(TEST_ROOT_DIR, "foo%bar");
-    writeFile(fs, path);
+    writeFile(fs, path, 1);
     FileStatus status = fs.getFileStatus(path);
     assertEquals(path.makeQualified(fs), status.getPath());
     cleanupFile(fs, path);
   }
   
+  @Test
   public void testMkdirs() throws IOException {
     Configuration conf = new Configuration();
     LocalFileSystem fs = FileSystem.getLocal(conf);
@@ -199,18 +197,40 @@ public class TestLocalFileSystem extends
     Path test_file = new Path(TEST_ROOT_DIR, "file1");
     assertTrue(fs.mkdirs(test_dir));
    
-    writeFile(fs, test_file);
+    writeFile(fs, test_file, 1);
     // creating dir over a file
     Path bad_dir = new Path(test_file, "another_dir");
     
     try {
       fs.mkdirs(bad_dir);
       fail("Failed to detect existing file in path");
-    } catch (FileAlreadyExistsException e) { }
+    } catch (FileAlreadyExistsException e) { 
+      // Expected
+    }
     
     try {
       fs.mkdirs(null);
       fail("Failed to detect null in mkdir arg");
-    } catch (IllegalArgumentException e) { }
+    } catch (IllegalArgumentException e) {
+      // Expected
+    }
+  }
+
+  /** Test deleting a file, directory, and non-existent path */
+  @Test
+  public void testBasicDelete() throws IOException {
+    Configuration conf = new Configuration();
+    LocalFileSystem fs = FileSystem.getLocal(conf);
+    Path dir1 = new Path(TEST_ROOT_DIR, "dir1");
+    Path file1 = new Path(TEST_ROOT_DIR, "file1");
+    Path file2 = new Path(TEST_ROOT_DIR+"/dir1", "file2");
+    Path file3 = new Path(TEST_ROOT_DIR, "does-not-exist");
+    assertTrue(fs.mkdirs(dir1));
+    writeFile(fs, file1, 1);
+    writeFile(fs, file2, 1);
+    assertFalse("Returned true deleting non-existant path", 
+        fs.delete(file3));
+    assertTrue("Did not delete file", fs.delete(file1));
+    assertTrue("Did not delete non-empty dir", fs.delete(dir1));
   }
 }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java Fri Sep  2 03:20:12 2011
@@ -19,9 +19,9 @@ package org.apache.hadoop.fs;
 
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.*;
+import static org.apache.hadoop.fs.FileSystemTestHelper.*;
 
 import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
@@ -42,14 +42,6 @@ public class TestTrash extends TestCase 
     new Path(new File(System.getProperty("test.build.data","/tmp")
           ).toURI().toString().replace(' ', '+'), "testTrash");
 
-  protected static Path writeFile(FileSystem fs, Path f) throws IOException {
-    DataOutputStream out = fs.create(f);
-    out.writeBytes("dhruba: " + f);
-    out.close();
-    assertTrue(fs.exists(f));
-    return f;
-  }
-
   protected static Path mkdir(FileSystem fs, Path p) throws IOException {
     assertTrue(fs.mkdirs(p));
     assertTrue(fs.exists(p));
@@ -139,7 +131,7 @@ public class TestTrash extends TestCase 
 
     // Second, create a file in that directory.
     Path myFile = new Path(base, "test/mkdirs/myFile");
-    writeFile(fs, myFile);
+    writeFile(fs, myFile, 10);
 
     // Verify that expunge without Trash directory
     // won't throw Exception
@@ -176,7 +168,7 @@ public class TestTrash extends TestCase 
     }
 
     // Verify that we can recreate the file
-    writeFile(fs, myFile);
+    writeFile(fs, myFile, 10);
 
     // Verify that we succeed in removing the file we re-created
     {
@@ -194,7 +186,7 @@ public class TestTrash extends TestCase 
     }
 
     // Verify that we can recreate the file
-    writeFile(fs, myFile);
+    writeFile(fs, myFile, 10);
     
     // Verify that we succeed in removing the whole directory
     // along with the file inside it.
@@ -234,7 +226,7 @@ public class TestTrash extends TestCase 
     {
         Path toErase = new Path(trashRoot, "toErase");
         int retVal = -1;
-        writeFile(trashRootFs, toErase);
+        writeFile(trashRootFs, toErase, 10);
         try {
           retVal = shell.run(new String[] {"-rm", toErase.toString()});
         } catch (Exception e) {
@@ -265,7 +257,7 @@ public class TestTrash extends TestCase 
 
     // recreate directory and file
     mkdir(fs, myPath);
-    writeFile(fs, myFile);
+    writeFile(fs, myFile, 10);
 
     // remove file first, then remove directory
     {
@@ -316,7 +308,7 @@ public class TestTrash extends TestCase 
     
     // recreate directory and file
     mkdir(fs, myPath);
-    writeFile(fs, myFile);
+    writeFile(fs, myFile, 10);
     
     // Verify that skip trash option really skips the trash for files (rm)
     {
@@ -346,7 +338,7 @@ public class TestTrash extends TestCase 
     
     // recreate directory and file
     mkdir(fs, myPath);
-    writeFile(fs, myFile);
+    writeFile(fs, myFile, 10);
     
     // Verify that skip trash option really skips the trash for rmr
     {
@@ -392,7 +384,7 @@ public class TestTrash extends TestCase 
       for(int i=0;i<num_runs; i++) {
         
         //create file
-        writeFile(fs, myFile);
+        writeFile(fs, myFile, 10);
          
         // delete file
         try {
@@ -452,8 +444,7 @@ public class TestTrash extends TestCase 
         lfs.delete(p, true);
       }
       try {
-        f = writeFile(lfs, f);
-
+        writeFile(lfs, f, 10);
         FileSystem.closeAll();
         FileSystem localFs = FileSystem.get(URI.create("file:///"), conf);
         Trash lTrash = new Trash(localFs, conf);
@@ -515,7 +506,7 @@ public class TestTrash extends TestCase 
     while (true)  {
       // Create a file with a new name
       Path myFile = new Path(TEST_DIR, "test/mkdirs/myFile" + fileIndex++);
-      writeFile(fs, myFile);
+      writeFile(fs, myFile, 10);
 
       // Delete the file to trash
       String[] args = new String[2];
@@ -606,7 +597,7 @@ public class TestTrash extends TestCase 
     int iters = 1000;
     for(int i=0;i<iters; i++) {
       
-      writeFile(fs, myFile);
+      writeFile(fs, myFile, 10);
       
       start = System.currentTimeMillis();
       

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java Fri Sep  2 03:20:12 2011
@@ -50,7 +50,7 @@ public class TestNativeIO {
   }
 
   @Before
-  public void setupTestDir() throws IOException {
+  public void setupTestDir() {
     FileUtil.fullyDelete(TEST_DIR);
     TEST_DIR.mkdirs();
   }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java?rev=1164339&r1=1164338&r2=1164339&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestRunJar.java Fri Sep  2 03:20:12 2011
@@ -49,8 +49,7 @@ public class TestRunJar extends TestCase
   }
 
   @After
-  protected void tearDown()
-      throws Exception {
+  protected void tearDown() {
     FileUtil.fullyDelete(TEST_ROOT_DIR);
   }