Browse Source

HADOOP-771. FileSystem.delete() takes an explicit parameter that
specifies whether a recursive delete is intended.
(Mahadev Konar via dhruba)



git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@635650 13f79535-47bb-0310-9956-ffa450edef68

Dhruba Borthakur 17 years ago
parent
commit
10bd0d8dc0
100 changed files with 405 additions and 251 deletions
  1. 4 0
      CHANGES.txt
  2. 1 1
      src/examples/org/apache/hadoop/examples/Grep.java
  3. 2 2
      src/examples/org/apache/hadoop/examples/PiEstimator.java
  4. 1 1
      src/examples/org/apache/hadoop/examples/SleepJob.java
  5. 1 1
      src/examples/org/apache/hadoop/examples/dancing/DistributedPentomino.java
  6. 1 1
      src/java/org/apache/hadoop/dfs/Balancer.java
  7. 15 1
      src/java/org/apache/hadoop/dfs/ClientProtocol.java
  8. 12 1
      src/java/org/apache/hadoop/dfs/DFSClient.java
  9. 10 1
      src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
  10. 16 1
      src/java/org/apache/hadoop/dfs/FSDirectory.java
  11. 15 3
      src/java/org/apache/hadoop/dfs/FSNamesystem.java
  12. 10 1
      src/java/org/apache/hadoop/dfs/HftpFileSystem.java
  13. 5 0
      src/java/org/apache/hadoop/dfs/NameNode.java
  14. 1 1
      src/java/org/apache/hadoop/dfs/NamenodeFsck.java
  15. 3 3
      src/java/org/apache/hadoop/filecache/DistributedCache.java
  16. 4 4
      src/java/org/apache/hadoop/fs/ChecksumFileSystem.java
  17. 14 2
      src/java/org/apache/hadoop/fs/FileSystem.java
  18. 5 5
      src/java/org/apache/hadoop/fs/FileUtil.java
  19. 6 1
      src/java/org/apache/hadoop/fs/FilterFileSystem.java
  20. 1 1
      src/java/org/apache/hadoop/fs/FsShell.java
  21. 6 1
      src/java/org/apache/hadoop/fs/InMemoryFileSystem.java
  22. 10 1
      src/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  23. 1 1
      src/java/org/apache/hadoop/fs/Trash.java
  24. 23 14
      src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
  25. 32 23
      src/java/org/apache/hadoop/fs/s3/S3FileSystem.java
  26. 3 3
      src/java/org/apache/hadoop/io/MapFile.java
  27. 4 4
      src/java/org/apache/hadoop/io/SequenceFile.java
  28. 2 2
      src/java/org/apache/hadoop/mapred/CompletedJobStatusStore.java
  29. 1 1
      src/java/org/apache/hadoop/mapred/JobClient.java
  30. 4 4
      src/java/org/apache/hadoop/mapred/JobInProgress.java
  31. 1 1
      src/java/org/apache/hadoop/mapred/JobTracker.java
  32. 2 2
      src/java/org/apache/hadoop/mapred/LocalJobRunner.java
  33. 2 2
      src/java/org/apache/hadoop/mapred/MapOutputLocation.java
  34. 2 2
      src/java/org/apache/hadoop/mapred/MapTask.java
  35. 6 6
      src/java/org/apache/hadoop/mapred/PhasedFileSystem.java
  36. 5 5
      src/java/org/apache/hadoop/mapred/ReduceTask.java
  37. 2 2
      src/java/org/apache/hadoop/mapred/Task.java
  38. 1 1
      src/java/org/apache/hadoop/mapred/TaskRunner.java
  39. 1 1
      src/java/org/apache/hadoop/mapred/TaskTracker.java
  40. 2 2
      src/java/org/apache/hadoop/util/CopyFiles.java
  41. 2 2
      src/test/org/apache/hadoop/dfs/BenchmarkThroughput.java
  42. 1 1
      src/test/org/apache/hadoop/dfs/ClusterTestDFS.java
  43. 2 2
      src/test/org/apache/hadoop/dfs/ClusterTestDFSNamespaceLogging.java
  44. 1 1
      src/test/org/apache/hadoop/dfs/DFSTestUtil.java
  45. 4 4
      src/test/org/apache/hadoop/dfs/NNBench.java
  46. 1 1
      src/test/org/apache/hadoop/dfs/NNBenchWithoutMR.java
  47. 1 1
      src/test/org/apache/hadoop/dfs/NNThroughputBenchmark.java
  48. 1 1
      src/test/org/apache/hadoop/dfs/TestCheckpoint.java
  49. 1 1
      src/test/org/apache/hadoop/dfs/TestDFSMkdirs.java
  50. 5 5
      src/test/org/apache/hadoop/dfs/TestDFSPermission.java
  51. 32 6
      src/test/org/apache/hadoop/dfs/TestDFSShell.java
  52. 1 1
      src/test/org/apache/hadoop/dfs/TestDFSShellGenericOptions.java
  53. 2 2
      src/test/org/apache/hadoop/dfs/TestDecommission.java
  54. 1 1
      src/test/org/apache/hadoop/dfs/TestFSInputChecker.java
  55. 1 1
      src/test/org/apache/hadoop/dfs/TestFSOutputSummer.java
  56. 1 1
      src/test/org/apache/hadoop/dfs/TestFileCorruption.java
  57. 2 2
      src/test/org/apache/hadoop/dfs/TestFileLimit.java
  58. 2 2
      src/test/org/apache/hadoop/dfs/TestLocalDFS.java
  59. 2 2
      src/test/org/apache/hadoop/dfs/TestModTime.java
  60. 2 2
      src/test/org/apache/hadoop/dfs/TestPread.java
  61. 1 1
      src/test/org/apache/hadoop/dfs/TestReplication.java
  62. 1 1
      src/test/org/apache/hadoop/dfs/TestSeekBug.java
  63. 1 1
      src/test/org/apache/hadoop/dfs/TestSmallBlock.java
  64. 9 9
      src/test/org/apache/hadoop/fs/DFSCIOTest.java
  65. 1 1
      src/test/org/apache/hadoop/fs/DistributedFSCheck.java
  66. 1 1
      src/test/org/apache/hadoop/fs/TestCopyFiles.java
  67. 5 5
      src/test/org/apache/hadoop/fs/TestDFSIO.java
  68. 9 9
      src/test/org/apache/hadoop/fs/TestFileSystem.java
  69. 1 1
      src/test/org/apache/hadoop/fs/TestGlobPaths.java
  70. 4 4
      src/test/org/apache/hadoop/fs/TestLocalFileSystem.java
  71. 1 1
      src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java
  72. 2 2
      src/test/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java
  73. 7 7
      src/test/org/apache/hadoop/fs/kfs/TestKosmosFileSystem.java
  74. 3 3
      src/test/org/apache/hadoop/fs/s3/S3FileSystemBaseTest.java
  75. 6 6
      src/test/org/apache/hadoop/io/TestSequenceFile.java
  76. 1 1
      src/test/org/apache/hadoop/mapred/BigMapOutput.java
  77. 1 1
      src/test/org/apache/hadoop/mapred/MRBench.java
  78. 3 3
      src/test/org/apache/hadoop/mapred/MRCaching.java
  79. 1 1
      src/test/org/apache/hadoop/mapred/NotificationTestCase.java
  80. 2 2
      src/test/org/apache/hadoop/mapred/PiEstimator.java
  81. 2 2
      src/test/org/apache/hadoop/mapred/SortValidator.java
  82. 1 1
      src/test/org/apache/hadoop/mapred/TestCollect.java
  83. 1 1
      src/test/org/apache/hadoop/mapred/TestComparators.java
  84. 1 1
      src/test/org/apache/hadoop/mapred/TestEmptyJobWithDFS.java
  85. 4 4
      src/test/org/apache/hadoop/mapred/TestFieldSelection.java
  86. 2 2
      src/test/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java
  87. 1 1
      src/test/org/apache/hadoop/mapred/TestMapOutputType.java
  88. 7 7
      src/test/org/apache/hadoop/mapred/TestMapRed.java
  89. 2 2
      src/test/org/apache/hadoop/mapred/TestMiniMRClasspath.java
  90. 2 2
      src/test/org/apache/hadoop/mapred/TestMiniMRLocalFS.java
  91. 1 1
      src/test/org/apache/hadoop/mapred/TestMiniMRMapRedDebugScript.java
  92. 4 4
      src/test/org/apache/hadoop/mapred/TestMiniMRTaskTempDir.java
  93. 1 1
      src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java
  94. 2 2
      src/test/org/apache/hadoop/mapred/TestMultiFileInputFormat.java
  95. 2 2
      src/test/org/apache/hadoop/mapred/TestRackAwareTaskPlacement.java
  96. 1 1
      src/test/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java
  97. 1 1
      src/test/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java
  98. 6 6
      src/test/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java
  99. 1 1
      src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java
  100. 1 1
      src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java

+ 4 - 0
CHANGES.txt

@@ -26,6 +26,10 @@ Trunk (unreleased changes)
     HADOOP-1986.  Add support for a general serialization mechanism for
     HADOOP-1986.  Add support for a general serialization mechanism for
     Map Reduce. (tomwhite)
     Map Reduce. (tomwhite)
 
 
+    HADOOP-771. FileSystem.delete() takes an explicit parameter that
+    specifies whether a recursive delete is intended.
+    (Mahadev Konar via dhruba)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HADOOP-1398.  Add HBase in-memory block cache.  (tomwhite)
     HADOOP-1398.  Add HBase in-memory block cache.  (tomwhite)

+ 1 - 1
src/examples/org/apache/hadoop/examples/Grep.java

@@ -84,7 +84,7 @@ public class Grep extends Configured implements Tool {
       JobClient.runJob(sortJob);
       JobClient.runJob(sortJob);
     }
     }
     finally {
     finally {
-      FileSystem.get(grepJob).delete(tempDir);
+      FileSystem.get(grepJob).delete(tempDir, true);
     }
     }
     return 0;
     return 0;
   }
   }

+ 2 - 2
src/examples/org/apache/hadoop/examples/PiEstimator.java

@@ -183,7 +183,7 @@ public class PiEstimator extends Configured implements Tool {
     Path inDir = new Path(tmpDir, "in");
     Path inDir = new Path(tmpDir, "in");
     Path outDir = new Path(tmpDir, "out");
     Path outDir = new Path(tmpDir, "out");
     FileSystem fileSys = FileSystem.get(jobConf);
     FileSystem fileSys = FileSystem.get(jobConf);
-    fileSys.delete(tmpDir);
+    fileSys.delete(tmpDir, true);
     if (!fileSys.mkdirs(inDir)) {
     if (!fileSys.mkdirs(inDir)) {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     }
@@ -219,7 +219,7 @@ public class PiEstimator extends Configured implements Tool {
       reader.close();
       reader.close();
       estimate = (numInside.get()*4.0)/(numMaps*numPoints);
       estimate = (numInside.get()*4.0)/(numMaps*numPoints);
     } finally {
     } finally {
-      fileSys.delete(tmpDir);
+      fileSys.delete(tmpDir, true);
     }
     }
     
     
     return estimate;
     return estimate;

+ 1 - 1
src/examples/org/apache/hadoop/examples/SleepJob.java

@@ -155,7 +155,7 @@ public class SleepJob extends Configured implements Tool,
       JobClient.runJob(job);
       JobClient.runJob(job);
     } 
     } 
     finally {
     finally {
-      fs.delete(tempPath);
+      fs.delete(tempPath, true);
     }
     }
     return 0;
     return 0;
   }
   }

+ 1 - 1
src/examples/org/apache/hadoop/examples/dancing/DistributedPentomino.java

@@ -199,7 +199,7 @@ public class DistributedPentomino extends Configured implements Tool {
       
       
       JobClient.runJob(conf);
       JobClient.runJob(conf);
       } finally {
       } finally {
-      fileSys.delete(input);
+      fileSys.delete(input, true);
     }
     }
     return 0;
     return 0;
   }
   }

+ 1 - 1
src/java/org/apache/hadoop/dfs/Balancer.java

@@ -1452,7 +1452,7 @@ public class Balancer implements Tool {
     } finally {
     } finally {
       IOUtils.closeStream(out); 
       IOUtils.closeStream(out); 
       try {
       try {
-        fs.delete(BALANCER_ID_PATH);
+        fs.delete(BALANCER_ID_PATH, true);
       } catch(IOException ignored) {
       } catch(IOException ignored) {
       }
       }
       System.out.println("Balancing took " + 
       System.out.println("Balancing took " + 

+ 15 - 1
src/java/org/apache/hadoop/dfs/ClientProtocol.java

@@ -37,8 +37,9 @@ interface ClientProtocol extends VersionedProtocol {
    * (Only the latest change is reflected.
    * (Only the latest change is reflected.
    * The log of historical changes can be retrieved from the svn).
    * The log of historical changes can be retrieved from the svn).
    * 25 : added {@link #getContentSummary(String path)}
    * 25 : added {@link #getContentSummary(String path)}
+   * 26 : added delete(src, boolean) for recursive deletes on the namenode
    */
    */
-  public static final long versionID = 25L;
+  public static final long versionID = 26L;
   
   
   ///////////////////////////////////////
   ///////////////////////////////////////
   // File contents
   // File contents
@@ -236,6 +237,19 @@ interface ClientProtocol extends VersionedProtocol {
    */
    */
   public boolean delete(String src) throws IOException;
   public boolean delete(String src) throws IOException;
 
 
+  /**
+   * Delete the given file or directory from the file system.
+   * <p>
+   * same as delete but provides a way to avoid accidentally 
+   * deleting non empty directories programmatically. 
+   * @param src existing name
+   * @param recursive if true deletes a non empty directory recursively,
+   * else throws an exception.
+   * @return true only if the existing file or directory was actually removed 
+   * from the file system. 
+   */
+  public boolean delete(String src, boolean recursive) throws IOException;
+  
   /**
   /**
    * Check whether the given file exists.
    * Check whether the given file exists.
    */
    */

+ 12 - 1
src/java/org/apache/hadoop/dfs/DFSClient.java

@@ -438,11 +438,22 @@ class DFSClient implements FSConstants {
    * Delete file or directory.
    * Delete file or directory.
    * See {@link ClientProtocol#delete(String)}. 
    * See {@link ClientProtocol#delete(String)}. 
    */
    */
+  @Deprecated
   public boolean delete(String src) throws IOException {
   public boolean delete(String src) throws IOException {
     checkOpen();
     checkOpen();
-    return namenode.delete(src);
+    return namenode.delete(src, true);
   }
   }
 
 
+  /**
+   * delete file or directory.
+   * delete contents of the directory if non empty and recursive 
+   * set to true
+   */
+  public boolean delete(String src, boolean recursive) throws IOException {
+    checkOpen();
+    return namenode.delete(src, recursive);
+  }
+  
   /**
   /**
    */
    */
   public boolean exists(String src) throws IOException {
   public boolean exists(String src) throws IOException {

+ 10 - 1
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -166,10 +166,19 @@ public class DistributedFileSystem extends FileSystem {
   /**
   /**
    * Get rid of Path f, whether a true file or dir.
    * Get rid of Path f, whether a true file or dir.
    */
    */
+  @Deprecated
   public boolean delete(Path f) throws IOException {
   public boolean delete(Path f) throws IOException {
     return dfs.delete(getPathName(f));
     return dfs.delete(getPathName(f));
   }
   }
-
+  
+  /**
+   * requires a boolean check to delete a non 
+   * empty directory recursively.
+   */
+  public boolean delete(Path f, boolean recursive) throws IOException {
+   return dfs.delete(getPathName(f), recursive);
+  }
+  
   public boolean exists(Path f) throws IOException {
   public boolean exists(Path f) throws IOException {
     return dfs.exists(getPathName(f));
     return dfs.exists(getPathName(f));
   }
   }

+ 16 - 1
src/java/org/apache/hadoop/dfs/FSDirectory.java

@@ -453,7 +453,22 @@ class FSDirectory implements FSConstants {
     }
     }
     return deletedNode;
     return deletedNode;
   }
   }
-
+  
+  /** Return if a directory is empty or not **/
+  public boolean isDirEmpty(String src) {
+	boolean dirNotEmpty = true;
+    if (!isDir(src)) {
+      return true;
+    }
+    synchronized(rootDir) {
+      INode targetNode = rootDir.getNode(src);
+      if (((INodeDirectory)targetNode).getChildren().size() != 0) {
+        dirNotEmpty = false;
+      }
+    }
+    return dirNotEmpty;
+  }
+  
   /**
   /**
    */
    */
   INode unprotectedDelete(String src, long modificationTime, 
   INode unprotectedDelete(String src, long modificationTime, 

+ 15 - 3
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -1458,12 +1458,24 @@ class FSNamesystem implements FSConstants, FSNamesystemMBean {
    * Remove the indicated filename from the namespace.  This may
    * Remove the indicated filename from the namespace.  This may
    * invalidate some blocks that make up the file.
    * invalidate some blocks that make up the file.
    */
    */
+  @Deprecated
   public boolean delete(String src) throws IOException {
   public boolean delete(String src) throws IOException {
-    boolean status = deleteInternal(src, true, true);
-    getEditLog().logSync();
-    return status;
+    return delete(src, true);
   }
   }
 
 
+  /**
+   * Remove the indicated filename from namespace. If the filename 
+   * is a directory (non empty) and recursive is set to false then throw exception.
+   */
+    public boolean delete(String src, boolean recursive) throws IOException {
+      if (isDir(src) && (!recursive) && (!dir.isDirEmpty(src))) {
+        throw new IOException(src + " is non empty");
+      }
+      boolean status = deleteInternal(src, true, true);
+      getEditLog().logSync();
+      return status;
+    }
+    
   /**
   /**
    * An internal delete function that does not enforce safe mode
    * An internal delete function that does not enforce safe mode
    */
    */

+ 10 - 1
src/java/org/apache/hadoop/dfs/HftpFileSystem.java

@@ -255,10 +255,19 @@ public class HftpFileSystem extends FileSystem {
   }
   }
 
 
   @Override
   @Override
+  /*
+   * @deprecated Use delete(path, boolean)
+   */
+  @Deprecated
   public boolean delete(Path f) throws IOException {
   public boolean delete(Path f) throws IOException {
     throw new IOException("Not supported");
     throw new IOException("Not supported");
   }
   }
-
+  
+  @Override 
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    throw new IOException("Not supported");
+  }
+  
   @Override
   @Override
   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
     throw new IOException("Not supported");
     throw new IOException("Not supported");

+ 5 - 0
src/java/org/apache/hadoop/dfs/NameNode.java

@@ -385,11 +385,16 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
 
 
   /**
   /**
    */
    */
+  @Deprecated
   public boolean delete(String src) throws IOException {
   public boolean delete(String src) throws IOException {
     stateChangeLog.debug("*DIR* NameNode.delete: " + src);
     stateChangeLog.debug("*DIR* NameNode.delete: " + src);
     return namesystem.delete(src);
     return namesystem.delete(src);
   }
   }
 
 
+  public boolean delete(String src, boolean recursive) throws IOException {
+    stateChangeLog.debug("*DIR* Namenode.delete:  " + src);
+    return namesystem.delete(src, recursive);
+  }
   /**
   /**
    */
    */
   public boolean exists(String src) throws IOException {
   public boolean exists(String src) throws IOException {

+ 1 - 1
src/java/org/apache/hadoop/dfs/NamenodeFsck.java

@@ -323,7 +323,7 @@ public class NamenodeFsck {
       }
       }
       if (fos != null) fos.close();
       if (fos != null) fos.close();
       LOG.warn("\n - moved corrupted file " + file.getPath() + " to /lost+found");
       LOG.warn("\n - moved corrupted file " + file.getPath() + " to /lost+found");
-      dfs.delete(file.getPath().toString());
+      dfs.delete(file.getPath().toString(), true);
     }  catch (Exception e) {
     }  catch (Exception e) {
       e.printStackTrace();
       e.printStackTrace();
       LOG.warn(errmsg + ": " + e.getMessage());
       LOG.warn(errmsg + ": " + e.getMessage());

+ 3 - 3
src/java/org/apache/hadoop/filecache/DistributedCache.java

@@ -239,7 +239,7 @@ public class DistributedCache {
         synchronized (lcacheStatus) {
         synchronized (lcacheStatus) {
           if (lcacheStatus.refcount == 0) {
           if (lcacheStatus.refcount == 0) {
             // delete this cache entry
             // delete this cache entry
-            FileSystem.getLocal(conf).delete(lcacheStatus.localLoadPath);
+            FileSystem.getLocal(conf).delete(lcacheStatus.localLoadPath, true);
             it.remove();
             it.remove();
           }
           }
         }
         }
@@ -314,7 +314,7 @@ public class DistributedCache {
                               + " is in use and cannot be refreshed");
                               + " is in use and cannot be refreshed");
       
       
       FileSystem localFs = FileSystem.getLocal(conf);
       FileSystem localFs = FileSystem.getLocal(conf);
-      localFs.delete(cacheStatus.localLoadPath);
+      localFs.delete(cacheStatus.localLoadPath, true);
       Path parchive = new Path(cacheStatus.localLoadPath,
       Path parchive = new Path(cacheStatus.localLoadPath,
                                new Path(cacheStatus.localLoadPath.getName()));
                                new Path(cacheStatus.localLoadPath.getName()));
       
       
@@ -779,7 +779,7 @@ public class DistributedCache {
       FileSystem localFs = FileSystem.getLocal(conf);
       FileSystem localFs = FileSystem.getLocal(conf);
       for (Map.Entry<String,CacheStatus> f: cachedArchives.entrySet()) {
       for (Map.Entry<String,CacheStatus> f: cachedArchives.entrySet()) {
         try {
         try {
-          localFs.delete(f.getValue().localLoadPath);
+          localFs.delete(f.getValue().localLoadPath, true);
         } catch (IOException ie) {
         } catch (IOException ie) {
           LOG.debug("Error cleaning up cache", ie);
           LOG.debug("Error cleaning up cache", ie);
         }
         }

+ 4 - 4
src/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -413,14 +413,14 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
    */
    */
   public boolean delete(Path f) throws IOException {
   public boolean delete(Path f) throws IOException {
     if (fs.isDirectory(f)) {
     if (fs.isDirectory(f)) {
-      return fs.delete(f);
+      return fs.delete(f, true);
     } else {
     } else {
       Path checkFile = getChecksumFile(f);
       Path checkFile = getChecksumFile(f);
       if (fs.exists(checkFile)) {
       if (fs.exists(checkFile)) {
-        fs.delete(checkFile);
+        fs.delete(checkFile, true);
       }
       }
 
 
-      return fs.delete(f);
+      return fs.delete(f, true);
     }
     }
   }
   }
 
 
@@ -508,7 +508,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
       }
       }
       dst = getChecksumFile(dst);
       dst = getChecksumFile(dst);
       if (localFs.exists(dst)) { //remove old local checksum file
       if (localFs.exists(dst)) { //remove old local checksum file
-        localFs.delete(dst);
+        localFs.delete(dst, true);
       }
       }
       Path checksumFile = getChecksumFile(src);
       Path checksumFile = getChecksumFile(src);
       if (copyCrc && fs.exists(checksumFile)) { //copy checksum file
       if (copyCrc && fs.exists(checksumFile)) { //copy checksum file

+ 14 - 2
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -531,9 +531,21 @@ public abstract class FileSystem extends Configured implements Closeable {
    */
    */
   public abstract boolean rename(Path src, Path dst) throws IOException;
   public abstract boolean rename(Path src, Path dst) throws IOException;
     
     
-  /** Delete a file */
+  /** Delete a file. */
+  /** @deprecated Use delete(Path, boolean) instead */ @Deprecated 
   public abstract boolean delete(Path f) throws IOException;
   public abstract boolean delete(Path f) throws IOException;
-    
+  
+  /** Delete a file.
+   *
+   * @param f the path to delete.
+   * @param recursive if path is a directory and set to 
+   * true, the directory is deleted else throws an exception. In
+   * case of a file the recursive can be set to either true or false. 
+   * @return  true if delete is successful else false. 
+   * @throws IOException
+   */
+  public abstract boolean delete(Path f, boolean recursive) throws IOException;
+  
   /** Check if exists.
   /** Check if exists.
    * @param f source file
    * @param f source file
    */
    */

+ 5 - 5
src/java/org/apache/hadoop/fs/FileUtil.java

@@ -93,13 +93,13 @@ public class FileUtil {
     if (paths != null) {
     if (paths != null) {
       for (Path p : paths) {
       for (Path p : paths) {
         if (fs.isFile(p))  {
         if (fs.isFile(p))  {
-          fs.delete(p);
+          fs.delete(p, true);
         } else {
         } else {
           fullyDelete(fs, p);
           fullyDelete(fs, p);
         }
         }
       }
       }
     }
     }
-    fs.delete(dir);
+    fs.delete(dir, true);
   }
   }
 
 
   //
   //
@@ -159,7 +159,7 @@ public class FileUtil {
       throw new IOException(src.toString() + ": No such file or directory");
       throw new IOException(src.toString() + ": No such file or directory");
     }
     }
     if (deleteSource) {
     if (deleteSource) {
-      return srcFS.delete(src);
+      return srcFS.delete(src, true);
     } else {
     } else {
       return true;
       return true;
     }
     }
@@ -199,7 +199,7 @@ public class FileUtil {
     
     
 
 
     if (deleteSource) {
     if (deleteSource) {
-      return srcFS.delete(srcDir);
+      return srcFS.delete(srcDir, true);
     } else {
     } else {
       return true;
       return true;
     }
     }
@@ -256,7 +256,7 @@ public class FileUtil {
                             ": No such file or directory");
                             ": No such file or directory");
     }
     }
     if (deleteSource) {
     if (deleteSource) {
-      return srcFS.delete(src);
+      return srcFS.delete(src, true);
     } else {
     } else {
       return true;
       return true;
     }
     }

+ 6 - 1
src/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -135,11 +135,16 @@ public class FilterFileSystem extends FileSystem {
     return fs.rename(src, dst);
     return fs.rename(src, dst);
   }
   }
   
   
-  /** Delete a file */
+  /** Delete a file */@Deprecated
   public boolean delete(Path f) throws IOException {
   public boolean delete(Path f) throws IOException {
     return fs.delete(f);
     return fs.delete(f);
   }
   }
   
   
+  /** Delete a file */
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    return fs.delete(f, recursive);
+  }
+  
   /** Check if exists.
   /** Check if exists.
    * @param f source file
    * @param f source file
    */
    */

+ 1 - 1
src/java/org/apache/hadoop/fs/FsShell.java

@@ -1047,7 +1047,7 @@ public class FsShell extends Configured implements Tool {
       System.out.println("Moved to trash: " + src);
       System.out.println("Moved to trash: " + src);
       return;
       return;
     }
     }
-    if (srcFs.delete(src)) {
+    if (srcFs.delete(src, true)) {
       System.out.println("Deleted " + src);
       System.out.println("Deleted " + src);
     } else {
     } else {
       if (!srcFs.exists(src)) {
       if (!srcFs.exists(src)) {

+ 6 - 1
src/java/org/apache/hadoop/fs/InMemoryFileSystem.java

@@ -245,8 +245,13 @@ public class InMemoryFileSystem extends ChecksumFileSystem {
         return true;
         return true;
       }
       }
     }
     }
-
+    
+    @Deprecated
     public boolean delete(Path f) throws IOException {
     public boolean delete(Path f) throws IOException {
+      return delete(f, true);
+    }
+    
+    public boolean delete(Path f, boolean recursive) throws IOException {
       synchronized (this) {
       synchronized (this) {
         FileAttributes fAttr = pathToFileAttribs.remove(getPath(f));
         FileAttributes fAttr = pathToFileAttribs.remove(getPath(f));
         if (fAttr != null) {
         if (fAttr != null) {

+ 10 - 1
src/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -211,11 +211,20 @@ public class RawLocalFileSystem extends FileSystem {
     return FileUtil.copy(this, src, this, dst, true, getConf());
     return FileUtil.copy(this, src, this, dst, true, getConf());
   }
   }
   
   
+  @Deprecated
   public boolean delete(Path p) throws IOException {
   public boolean delete(Path p) throws IOException {
+    return delete(p, true);
+  }
+  
+  public boolean delete(Path p, boolean recursive) throws IOException {
     File f = pathToFile(p);
     File f = pathToFile(p);
     if (f.isFile()) {
     if (f.isFile()) {
       return f.delete();
       return f.delete();
-    } else return FileUtil.fullyDelete(f);
+    } else if ((!recursive) && f.isDirectory() && 
+        (f.listFiles().length != 0)) {
+      throw new IOException("Directory " + f.toString() + " is not empty");
+    }
+    return FileUtil.fullyDelete(f);
   }
   }
   
   
   public boolean exists(Path f) throws IOException {
   public boolean exists(Path f) throws IOException {

+ 1 - 1
src/java/org/apache/hadoop/fs/Trash.java

@@ -161,7 +161,7 @@ public class Trash extends Configured {
       }
       }
 
 
       if ((now - interval) > time) {
       if ((now - interval) > time) {
-        if (fs.delete(path)) {
+        if (fs.delete(path, true)) {
           LOG.info("Deleted trash checkpoint: "+dir);
           LOG.info("Deleted trash checkpoint: "+dir);
         } else {
         } else {
           LOG.warn("Couldn't delete checkpoint: "+dir+" Ignoring.");
           LOG.warn("Couldn't delete checkpoint: "+dir+" Ignoring.");

+ 23 - 14
src/java/org/apache/hadoop/fs/kfs/KosmosFileSystem.java

@@ -294,22 +294,31 @@ public class KosmosFileSystem extends FileSystem {
     }
     }
 
 
     // recursively delete the directory and its contents
     // recursively delete the directory and its contents
-    public boolean delete(Path path) throws IOException {
-	Path absolute = makeAbsolute(path);
-        String srep = absolute.toUri().getPath();
-
-        if (kfsImpl.isFile(srep))
-            return kfsImpl.remove(srep) == 0;
-
-        Path[] dirEntries = listPaths(absolute);
-        if (dirEntries != null) {
-            for (int i = 0; i < dirEntries.length; i++) {
-                delete(new Path(absolute, dirEntries[i]));
-            }
+    public boolean delete(Path path, boolean recursive) throws IOException {
+      Path absolute = makeAbsolute(path);
+      String srep = absolute.toUri().getPath();
+      if (kfsImpl.isFile(srep))
+        return kfsImpl.remove(srep) == 0;
+
+      Path[] dirEntries = listPaths(absolute);
+      if ((!recursive) && (dirEntries != null) && 
+            (dirEntries.length != 0)) {
+        throw new IOException("Directory " + path.toString() + 
+        " is not empty.");
+      }
+      if (dirEntries != null) {
+        for (int i = 0; i < dirEntries.length; i++) {
+          delete(new Path(absolute, dirEntries[i]), recursive);
         }
         }
-        return kfsImpl.rmdir(srep) == 0;
+      }
+      return kfsImpl.rmdir(srep) == 0;
     }
     }
-
+    
+    @Deprecated
+    public boolean delete(Path path) throws IOException {
+      return delete(path, true);
+    }
+    
     @Deprecated
     @Deprecated
     public long getLength(Path path) throws IOException {
     public long getLength(Path path) throws IOException {
 	Path absolute = makeAbsolute(path);
 	Path absolute = makeAbsolute(path);

+ 32 - 23
src/java/org/apache/hadoop/fs/s3/S3FileSystem.java

@@ -265,31 +265,40 @@ public class S3FileSystem extends FileSystem {
     return true;
     return true;
   }
   }
 
 
+  public boolean delete(Path path, boolean recursive) throws IOException {
+   Path absolutePath = makeAbsolute(path);
+   INode inode = store.retrieveINode(absolutePath);
+   if (inode == null) {
+     return false;
+   }
+   if (inode.isFile()) {
+     store.deleteINode(absolutePath);
+     for (Block block: inode.getBlocks()) {
+       store.deleteBlock(block);
+     }
+   } else {
+     Path[] contents = listPaths(absolutePath);
+     if (contents == null) {
+       return false;
+     }
+     if ((contents.length !=0) && (!recursive)) {
+       throw new IOException("Directory " + path.toString() 
+           + " is not empty.");
+     }
+     for (Path p:contents) {
+       if (!delete(p, recursive)) {
+         return false;
+       }
+     }
+     store.deleteINode(absolutePath);
+   }
+   return true;
+  }
+  
   @Override
   @Override
+  @Deprecated
   public boolean delete(Path path) throws IOException {
   public boolean delete(Path path) throws IOException {
-    Path absolutePath = makeAbsolute(path);
-    INode inode = store.retrieveINode(absolutePath);
-    if (inode == null) {
-      return false;
-    }
-    if (inode.isFile()) {
-      store.deleteINode(absolutePath);
-      for (Block block : inode.getBlocks()) {
-        store.deleteBlock(block);
-      }
-    } else {
-      Path[] contents = listPaths(absolutePath);
-      if (contents == null) {
-        return false;
-      }
-      for (Path p : contents) {
-        if (!delete(p)) {
-          return false;
-        }
-      }
-      store.deleteINode(absolutePath);
-    }
-    return true;
+    return delete(path, true);
   }
   }
 
 
   /**
   /**

+ 3 - 3
src/java/org/apache/hadoop/io/MapFile.java

@@ -561,9 +561,9 @@ public class MapFile {
     Path data = new Path(dir, DATA_FILE_NAME);
     Path data = new Path(dir, DATA_FILE_NAME);
     Path index = new Path(dir, INDEX_FILE_NAME);
     Path index = new Path(dir, INDEX_FILE_NAME);
 
 
-    fs.delete(data);
-    fs.delete(index);
-    fs.delete(dir);
+    fs.delete(data, true);
+    fs.delete(index, true);
+    fs.delete(dir, true);
   }
   }
 
 
   /**
   /**

+ 4 - 4
src/java/org/apache/hadoop/io/SequenceFile.java

@@ -2340,7 +2340,7 @@ public class SequenceFile {
             if (recordLength == -1) {
             if (recordLength == -1) {
               in.close();
               in.close();
               if (deleteInput) {
               if (deleteInput) {
-                fs.delete(inFiles[currentFile]);
+                fs.delete(inFiles[currentFile], true);
               }
               }
               currentFile += 1;
               currentFile += 1;
               atEof = currentFile >= inFiles.length;
               atEof = currentFile >= inFiles.length;
@@ -3034,7 +3034,7 @@ public class SequenceFile {
       public void cleanup() throws IOException {
       public void cleanup() throws IOException {
         close();
         close();
         if (!preserveInput) {
         if (!preserveInput) {
-          fs.delete(segmentPathName);
+          fs.delete(segmentPathName, true);
         }
         }
       }
       }
     } // SequenceFile.Sorter.SegmentDescriptor
     } // SequenceFile.Sorter.SegmentDescriptor
@@ -3092,7 +3092,7 @@ public class SequenceFile {
                                                     segmentLength, segmentName, this));
                                                     segmentLength, segmentName, this));
         }
         }
         fsIndexIn.close();
         fsIndexIn.close();
-        fs.delete(indexIn);
+        fs.delete(indexIn, true);
         numSegmentsContained = segments.size();
         numSegmentsContained = segments.size();
         this.inName = inName;
         this.inName = inName;
       }
       }
@@ -3103,7 +3103,7 @@ public class SequenceFile {
       public void cleanup() throws IOException {
       public void cleanup() throws IOException {
         numSegmentsCleanedUp++;
         numSegmentsCleanedUp++;
         if (numSegmentsCleanedUp == numSegmentsContained) {
         if (numSegmentsCleanedUp == numSegmentsContained) {
-          fs.delete(inName);
+          fs.delete(inName, true);
         }
         }
       }
       }
     } //SequenceFile.Sorter.SegmentContainer
     } //SequenceFile.Sorter.SegmentContainer

+ 2 - 2
src/java/org/apache/hadoop/mapred/CompletedJobStatusStore.java

@@ -105,7 +105,7 @@ public class CompletedJobStatusStore implements Runnable {
         try {
         try {
           FileStatus status = fs.getFileStatus(jobInfo);
           FileStatus status = fs.getFileStatus(jobInfo);
           if ((currentTime - status.getModificationTime()) > retainTime) {
           if ((currentTime - status.getModificationTime()) > retainTime) {
-            fs.delete(jobInfo);
+            fs.delete(jobInfo, true);
           }
           }
         }
         }
         catch (IOException ie) {
         catch (IOException ie) {
@@ -153,7 +153,7 @@ public class CompletedJobStatusStore implements Runnable {
         LOG.warn("Could not store [" + jobId + "] job info : " +
         LOG.warn("Could not store [" + jobId + "] job info : " +
                  ex.getMessage(), ex);
                  ex.getMessage(), ex);
         try {
         try {
-          fs.delete(jobStatusFile);
+          fs.delete(jobStatusFile, true);
         }
         }
         catch (IOException ex1) {
         catch (IOException ex1) {
           //ignore
           //ignore

+ 1 - 1
src/java/org/apache/hadoop/mapred/JobClient.java

@@ -485,7 +485,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     Path submitJobDir = new Path(job.getSystemDir(), jobId);
     Path submitJobDir = new Path(job.getSystemDir(), jobId);
     FileSystem fs = getFs();
     FileSystem fs = getFs();
     LOG.debug("default FileSystem: " + fs.getUri());
     LOG.debug("default FileSystem: " + fs.getUri());
-    fs.delete(submitJobDir);    
+    fs.delete(submitJobDir, true);    
     FileSystem.mkdirs(fs, submitJobDir, new FsPermission(SYSTEM_DIR_PERMISSION));
     FileSystem.mkdirs(fs, submitJobDir, new FsPermission(SYSTEM_DIR_PERMISSION));
     Path submitJobFile = new Path(submitJobDir, "job.xml");
     Path submitJobFile = new Path(submitJobDir, "job.xml");
     Path submitJarFile = new Path(submitJobDir, "job.jar");
     Path submitJarFile = new Path(submitJobDir, "job.jar");

+ 4 - 4
src/java/org/apache/hadoop/mapred/JobInProgress.java

@@ -1255,23 +1255,23 @@ class JobInProgress {
     try {
     try {
       // Definitely remove the local-disk copy of the job file
       // Definitely remove the local-disk copy of the job file
       if (localJobFile != null) {
       if (localJobFile != null) {
-        localFs.delete(localJobFile);
+        localFs.delete(localJobFile, true);
         localJobFile = null;
         localJobFile = null;
       }
       }
       if (localJarFile != null) {
       if (localJarFile != null) {
-        localFs.delete(localJarFile);
+        localFs.delete(localJarFile, true);
         localJarFile = null;
         localJarFile = null;
       }
       }
 
 
       // JobClient always creates a new directory with job files
       // JobClient always creates a new directory with job files
       // so we remove that directory to cleanup
       // so we remove that directory to cleanup
       FileSystem fs = FileSystem.get(conf);
       FileSystem fs = FileSystem.get(conf);
-      fs.delete(new Path(profile.getJobFile()).getParent());
+      fs.delete(new Path(profile.getJobFile()).getParent(), true);
         
         
       // Delete temp dfs dirs created if any, like in case of 
       // Delete temp dfs dirs created if any, like in case of 
       // speculative exn of reduces.  
       // speculative exn of reduces.  
       Path tempDir = new Path(conf.getSystemDir(), jobId); 
       Path tempDir = new Path(conf.getSystemDir(), jobId); 
-      fs.delete(tempDir); 
+      fs.delete(tempDir, true); 
 
 
       // delete the temporary directory in output directory
       // delete the temporary directory in output directory
       Path outputPath = conf.getOutputPath();
       Path outputPath = conf.getOutputPath();

+ 1 - 1
src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -703,7 +703,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         }
         }
         // clean up the system dir, which will only work if hdfs is out of 
         // clean up the system dir, which will only work if hdfs is out of 
         // safe mode
         // safe mode
-        fs.delete(systemDir);
+        fs.delete(systemDir, true);
         if (FileSystem.mkdirs(fs, systemDir, 
         if (FileSystem.mkdirs(fs, systemDir, 
             new FsPermission(JobClient.SYSTEM_DIR_PERMISSION))) {
             new FsPermission(JobClient.SYSTEM_DIR_PERMISSION))) {
           break;
           break;

+ 2 - 2
src/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -226,8 +226,8 @@ class LocalJobRunner implements JobSubmissionProtocol {
 
 
       } finally {
       } finally {
         try {
         try {
-          fs.delete(file.getParent());  // delete submit dir
-          localFs.delete(localFile);              // delete local copy
+          fs.delete(file.getParent(), true);  // delete submit dir
+          localFs.delete(localFile, true);              // delete local copy
         } catch (IOException e) {
         } catch (IOException e) {
           LOG.warn("Error cleaning up "+id+": "+e);
           LOG.warn("Error cleaning up "+id+": "+e);
         }
         }

+ 2 - 2
src/java/org/apache/hadoop/mapred/MapOutputLocation.java

@@ -160,7 +160,7 @@ class MapOutputLocation implements Writable, MRConstants {
     } finally {
     } finally {
       if (!good) {
       if (!good) {
         try {
         try {
-          fileSys.delete(localFilename);
+          fileSys.delete(localFilename, true);
           totalBytes = 0;
           totalBytes = 0;
         } catch (Throwable th) {
         } catch (Throwable th) {
           // IGNORED because we are cleaning up
           // IGNORED because we are cleaning up
@@ -264,7 +264,7 @@ class MapOutputLocation implements Writable, MRConstants {
     } finally {
     } finally {
       if (!good) {
       if (!good) {
         try {
         try {
-          fileSys.delete(localFilename);
+          fileSys.delete(localFilename, true);
           totalBytes = 0;
           totalBytes = 0;
         } catch (Throwable th) {
         } catch (Throwable th) {
           // IGNORED because we are cleaning up
           // IGNORED because we are cleaning up

+ 2 - 2
src/java/org/apache/hadoop/mapred/MapTask.java

@@ -671,8 +671,8 @@ class MapTask extends Task {
         finalIndexOut.close();
         finalIndexOut.close();
         //cleanup
         //cleanup
         for(int i = 0; i < numSpills; i++) {
         for(int i = 0; i < numSpills; i++) {
-          localFs.delete(filename[i]);
-          localFs.delete(indexFileName[i]);
+          localFs.delete(filename[i], true);
+          localFs.delete(indexFileName[i], true);
         }
         }
       }
       }
     }
     }

+ 6 - 6
src/java/org/apache/hadoop/mapred/PhasedFileSystem.java

@@ -104,7 +104,7 @@ public class PhasedFileSystem extends FilterFileSystem {
           // ignore if already closed
           // ignore if already closed
         }
         }
         if (fs.exists(fInfo.getTempPath())){
         if (fs.exists(fInfo.getTempPath())){
-          fs.delete(fInfo.getTempPath());
+          fs.delete(fInfo.getTempPath(), true);
         }
         }
         finalNameToFileInfo.remove(finalFile); 
         finalNameToFileInfo.remove(finalFile); 
       }
       }
@@ -166,23 +166,23 @@ public class PhasedFileSystem extends FilterFileSystem {
         }
         }
         
         
         if (fs.exists(fPath) && fInfo.isOverwrite()){
         if (fs.exists(fPath) && fInfo.isOverwrite()){
-          fs.delete(fPath); 
+          fs.delete(fPath, true); 
         }
         }
         
         
         try {
         try {
           if (!fs.rename(fInfo.getTempPath(), fPath)){
           if (!fs.rename(fInfo.getTempPath(), fPath)){
             // delete the temp file if rename failed
             // delete the temp file if rename failed
-            fs.delete(fInfo.getTempPath());
+            fs.delete(fInfo.getTempPath(), true);
           }
           }
         }catch(IOException ioe){
         }catch(IOException ioe){
           // rename failed, log error and delete temp files
           // rename failed, log error and delete temp files
           LOG.error("PhasedFileSystem failed to commit file : " + fPath 
           LOG.error("PhasedFileSystem failed to commit file : " + fPath 
                     + " error : " + ioe.getMessage()); 
                     + " error : " + ioe.getMessage()); 
-          fs.delete(fInfo.getTempPath());
+          fs.delete(fInfo.getTempPath(), true);
         }
         }
       }else{
       }else{
         // delete temp file
         // delete temp file
-        fs.delete(fInfo.getTempPath());
+        fs.delete(fInfo.getTempPath(), true);
       }
       }
       // done with the file
       // done with the file
       if (removeFromMap){
       if (removeFromMap){
@@ -224,7 +224,7 @@ public class PhasedFileSystem extends FilterFileSystem {
       }catch(IOException ioe){
       }catch(IOException ioe){
         // ignore if already closed
         // ignore if already closed
       }
       }
-      fs.delete(fInfo.getTempPath()); 
+      fs.delete(fInfo.getTempPath(), true); 
       if (removeFromMap){
       if (removeFromMap){
         finalNameToFileInfo.remove(p);
         finalNameToFileInfo.remove(p);
       }
       }

+ 5 - 5
src/java/org/apache/hadoop/mapred/ReduceTask.java

@@ -791,7 +791,7 @@ class ReduceTask extends Task {
         if (!neededOutputs.contains(loc.getMapId())) {
         if (!neededOutputs.contains(loc.getMapId())) {
           if (tmpFilename != null) {
           if (tmpFilename != null) {
             FileSystem fs = tmpFilename.getFileSystem(conf);
             FileSystem fs = tmpFilename.getFileSystem(conf);
-            fs.delete(tmpFilename);
+            fs.delete(tmpFilename, true);
           }
           }
           return CopyResult.OBSOLETE;
           return CopyResult.OBSOLETE;
         }
         }
@@ -805,7 +805,7 @@ class ReduceTask extends Task {
         // lock the ReduceTask while we do the rename
         // lock the ReduceTask while we do the rename
         synchronized (ReduceTask.this) {
         synchronized (ReduceTask.this) {
           if (!neededOutputs.contains(loc.getMapId())) {
           if (!neededOutputs.contains(loc.getMapId())) {
-            fs.delete(tmpFilename);
+            fs.delete(tmpFilename, true);
             return CopyResult.OBSOLETE;
             return CopyResult.OBSOLETE;
           }
           }
           
           
@@ -816,7 +816,7 @@ class ReduceTask extends Task {
           // if we can't rename the file, something is broken (and IOException
           // if we can't rename the file, something is broken (and IOException
           // will be thrown). 
           // will be thrown). 
           if (!fs.rename(tmpFilename, filename)) {
           if (!fs.rename(tmpFilename, filename)) {
-            fs.delete(tmpFilename);
+            fs.delete(tmpFilename, true);
             bytes = -1;
             bytes = -1;
             throw new IOException("failure to rename map output " + 
             throw new IOException("failure to rename map output " + 
                                   tmpFilename);
                                   tmpFilename);
@@ -1340,7 +1340,7 @@ class ReduceTask extends Task {
               //make sure that we delete the ondisk file that we created earlier
               //make sure that we delete the ondisk file that we created earlier
               //when we invoked cloneFileAttributes
               //when we invoked cloneFileAttributes
               writer.close();
               writer.close();
-              localFileSys.delete(inMemClosedFiles[0]);
+              localFileSys.delete(inMemClosedFiles[0], true);
               throw new IOException (StringUtils.stringifyException(e));
               throw new IOException (StringUtils.stringifyException(e));
             }
             }
             sorter.writeFile(rIter, writer);
             sorter.writeFile(rIter, writer);
@@ -1599,7 +1599,7 @@ class ReduceTask extends Task {
               //make sure that we delete the ondisk file that we created 
               //make sure that we delete the ondisk file that we created 
               //earlier when we invoked cloneFileAttributes
               //earlier when we invoked cloneFileAttributes
               writer.close();
               writer.close();
-              localFileSys.delete(outputPath);
+              localFileSys.delete(outputPath, true);
               throw new IOException (StringUtils.stringifyException(e));
               throw new IOException (StringUtils.stringifyException(e));
             }
             }
             sorter.writeFile(rIter, writer);
             sorter.writeFile(rIter, writer);

+ 2 - 2
src/java/org/apache/hadoop/mapred/Task.java

@@ -439,7 +439,7 @@ abstract class Task implements Writable, Configurable {
         moveTaskOutputs(fs, jobOutputPath, taskOutputPath);
         moveTaskOutputs(fs, jobOutputPath, taskOutputPath);
 
 
         // Delete the temporary task-specific output directory
         // Delete the temporary task-specific output directory
-        if (!fs.delete(taskOutputPath)) {
+        if (!fs.delete(taskOutputPath, true)) {
           LOG.info("Failed to delete the temporary output directory of task: " + 
           LOG.info("Failed to delete the temporary output directory of task: " + 
                   getTaskId() + " - " + taskOutputPath);
                   getTaskId() + " - " + taskOutputPath);
         }
         }
@@ -463,7 +463,7 @@ abstract class Task implements Writable, Configurable {
     if (fs.isFile(taskOutput)) {
     if (fs.isFile(taskOutput)) {
       Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput);
       Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput);
       if (!fs.rename(taskOutput, finalOutputPath)) {
       if (!fs.rename(taskOutput, finalOutputPath)) {
-        if (!fs.delete(finalOutputPath)) {
+        if (!fs.delete(finalOutputPath, true)) {
           throw new IOException("Failed to delete earlier output of task: " + 
           throw new IOException("Failed to delete earlier output of task: " + 
                   getTaskId());
                   getTaskId());
         }
         }

+ 1 - 1
src/java/org/apache/hadoop/mapred/TaskRunner.java

@@ -171,7 +171,7 @@ abstract class TaskRunner extends Thread {
         }
         }
         Path localTaskFile = new Path(t.getJobFile());
         Path localTaskFile = new Path(t.getJobFile());
         FileSystem localFs = FileSystem.getLocal(conf);
         FileSystem localFs = FileSystem.getLocal(conf);
-        localFs.delete(localTaskFile);
+        localFs.delete(localTaskFile, true);
         OutputStream out = localFs.create(localTaskFile);
         OutputStream out = localFs.create(localTaskFile);
         try {
         try {
           conf.write(out);
           conf.write(out);

+ 1 - 1
src/java/org/apache/hadoop/mapred/TaskTracker.java

@@ -656,7 +656,7 @@ public class TaskTracker
         // we should clean up and then try again
         // we should clean up and then try again
         Path jobDir = localJobFile.getParent();
         Path jobDir = localJobFile.getParent();
         if (localFs.exists(jobDir)){
         if (localFs.exists(jobDir)){
-          localFs.delete(jobDir);
+          localFs.delete(jobDir, true);
           boolean b = localFs.mkdirs(jobDir);
           boolean b = localFs.mkdirs(jobDir);
           if (!b)
           if (!b)
             throw new IOException("Not able to create job directory "
             throw new IOException("Not able to create job directory "

+ 2 - 2
src/java/org/apache/hadoop/util/CopyFiles.java

@@ -350,7 +350,7 @@ public class CopyFiles implements Tool {
     private void rename(FileSystem fs, Path tmp, Path dst) throws IOException {
     private void rename(FileSystem fs, Path tmp, Path dst) throws IOException {
       try {
       try {
         if (fs.exists(dst)) {
         if (fs.exists(dst)) {
-          fs.delete(dst);
+          fs.delete(dst, true);
         }
         }
         fs.rename(tmp, dst);
         fs.rename(tmp, dst);
       }
       }
@@ -413,7 +413,7 @@ public class CopyFiles implements Tool {
         try {
         try {
           for (int i = 0; i < 3; ++i) {
           for (int i = 0; i < 3; ++i) {
             try {
             try {
-              if (destFileSys.delete(dstpath))
+              if (destFileSys.delete(dstpath, true))
                 break;
                 break;
             } catch (Throwable ex) {
             } catch (Throwable ex) {
               // ignore, we are just cleaning up
               // ignore, we are just cleaning up

+ 2 - 2
src/test/org/apache/hadoop/dfs/BenchmarkThroughput.java

@@ -152,7 +152,7 @@ public class BenchmarkThroughput extends Configured implements Tool {
     } finally {
     } finally {
       try {
       try {
         if (f != null) {
         if (f != null) {
-          fs.delete(f);
+          fs.delete(f, true);
         }
         }
       } catch (IOException ie) {
       } catch (IOException ie) {
         // IGNORE
         // IGNORE
@@ -216,7 +216,7 @@ public class BenchmarkThroughput extends Configured implements Tool {
       if (cluster != null) {
       if (cluster != null) {
         cluster.shutdown();
         cluster.shutdown();
         // clean up minidfs junk
         // clean up minidfs junk
-        rawLocal.delete(new Path(localDir, "dfs"));
+        rawLocal.delete(new Path(localDir, "dfs"), true);
       }
       }
     }
     }
     return 0;
     return 0;

+ 1 - 1
src/test/org/apache/hadoop/dfs/ClusterTestDFS.java

@@ -336,7 +336,7 @@ public class ClusterTestDFS extends TestCase implements FSConstants {
         li = testfilesList.listIterator();
         li = testfilesList.listIterator();
         while (li.hasNext()) {
         while (li.hasNext()) {
           testFileName = (UTF8) li.next();
           testFileName = (UTF8) li.next();
-          assertTrue(dfsClient.delete(testFileName.toString()));
+          assertTrue(dfsClient.delete(testFileName.toString(), true));
         }
         }
 
 
         //
         //

+ 2 - 2
src/test/org/apache/hadoop/dfs/ClusterTestDFSNamespaceLogging.java

@@ -175,7 +175,7 @@ public class ClusterTestDFSNamespaceLogging extends TestCase implements FSConsta
     
     
       // delete the file
       // delete the file
       try {
       try {
-        dfsClient.delete("/data/yy");
+        dfsClient.delete("/data/yy", true);
         assertDelete("/data/yy", false);
         assertDelete("/data/yy", false);
       } catch(IOException ioe) {
       } catch(IOException ioe) {
         ioe.printStackTrace();
         ioe.printStackTrace();
@@ -191,7 +191,7 @@ public class ClusterTestDFSNamespaceLogging extends TestCase implements FSConsta
       }
       }
 
 
       try {
       try {
-        dfsClient.delete("/data/xx");
+        dfsClient.delete("/data/xx", true);
         assertDelete("/data/xx", true);
         assertDelete("/data/xx", true);
       } catch(IOException ioe) {
       } catch(IOException ioe) {
     	ioe.printStackTrace();
     	ioe.printStackTrace();

+ 1 - 1
src/test/org/apache/hadoop/dfs/DFSTestUtil.java

@@ -211,7 +211,7 @@ public class DFSTestUtil extends TestCase {
   /** delete directory and everything underneath it.*/
   /** delete directory and everything underneath it.*/
   void cleanup(FileSystem fs, String topdir) throws IOException {
   void cleanup(FileSystem fs, String topdir) throws IOException {
     Path root = new Path(topdir);
     Path root = new Path(topdir);
-    fs.delete(root);
+    fs.delete(root, true);
     files = null;
     files = null;
   }
   }
   
   

+ 4 - 4
src/test/org/apache/hadoop/dfs/NNBench.java

@@ -123,10 +123,10 @@ public class NNBench {
     // Delete the data directory only if it is the create/write operation
     // Delete the data directory only if it is the create/write operation
     if (operation.equals(OP_CREATE_WRITE)) {
     if (operation.equals(OP_CREATE_WRITE)) {
       LOG.info("Deleting data directory");
       LOG.info("Deleting data directory");
-      tempFS.delete(new Path(baseDir, DATA_DIR_NAME));
+      tempFS.delete(new Path(baseDir, DATA_DIR_NAME), true);
     }
     }
-    tempFS.delete(new Path(baseDir, CONTROL_DIR_NAME));
-    tempFS.delete(new Path(baseDir, OUTPUT_DIR_NAME));
+    tempFS.delete(new Path(baseDir, CONTROL_DIR_NAME), true);
+    tempFS.delete(new Path(baseDir, OUTPUT_DIR_NAME), true);
   }
   }
   
   
   /**
   /**
@@ -871,7 +871,7 @@ public class NNBench {
           try {
           try {
             // Set up timer for measuring AL
             // Set up timer for measuring AL
             startTimeAL = System.currentTimeMillis();
             startTimeAL = System.currentTimeMillis();
-            filesystem.delete(filePath);
+            filesystem.delete(filePath, true);
             totalTimeAL1 += (System.currentTimeMillis() - startTimeAL);
             totalTimeAL1 += (System.currentTimeMillis() - startTimeAL);
             
             
             successfulOp = true;
             successfulOp = true;

+ 1 - 1
src/test/org/apache/hadoop/dfs/NNBenchWithoutMR.java

@@ -208,7 +208,7 @@ public class NNBenchWithoutMR {
       int singleFileExceptions = 0;
       int singleFileExceptions = 0;
       do { // delete file until is succeeds
       do { // delete file until is succeeds
         try {
         try {
-          boolean result = fileSys.delete(new Path(taskDir, "A" + index));
+          boolean result = fileSys.delete(new Path(taskDir, "A" + index), true);
           success = true;
           success = true;
         } catch (IOException ioe) { 
         } catch (IOException ioe) { 
           success=false; 
           success=false; 

+ 1 - 1
src/test/org/apache/hadoop/dfs/NNThroughputBenchmark.java

@@ -211,7 +211,7 @@ public class NNThroughputBenchmark implements FSConstants {
 
 
     void cleanUp() throws IOException {
     void cleanUp() throws IOException {
       nameNode.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_LEAVE);
       nameNode.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_LEAVE);
-      nameNode.delete(getBaseDir());
+      nameNode.delete(getBaseDir(), true);
     }
     }
 
 
     int getNumOpsExecuted() {
     int getNumOpsExecuted() {

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestCheckpoint.java

@@ -61,7 +61,7 @@ public class TestCheckpoint extends TestCase {
   private void cleanupFile(FileSystem fileSys, Path name)
   private void cleanupFile(FileSystem fileSys, Path name)
     throws IOException {
     throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    fileSys.delete(name);
+    fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
 
 

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestDFSMkdirs.java

@@ -66,7 +66,7 @@ public class TestDFSMkdirs extends TestCase {
       }
       }
       assertFalse(exist);
       assertFalse(exist);
       assertFalse(fileSys.exists(myIllegalPath));
       assertFalse(fileSys.exists(myIllegalPath));
-      fileSys.delete(myFile);
+      fileSys.delete(myFile, true);
     	
     	
     } finally {
     } finally {
       fileSys.close();
       fileSys.close();

+ 5 - 5
src/test/org/apache/hadoop/dfs/TestDFSPermission.java

@@ -207,7 +207,7 @@ public class TestDFSPermission extends TestCase {
     } finally {
     } finally {
       // delete the file
       // delete the file
       if (delete) {
       if (delete) {
-        fs.delete(name);
+        fs.delete(name, true);
       }
       }
     }
     }
   }
   }
@@ -278,7 +278,7 @@ public class TestDFSPermission extends TestCase {
 
 
     // delete the file/directory
     // delete the file/directory
     login(SUPERUSER);
     login(SUPERUSER);
-    fs.delete(FILE_DIR_PATH);
+    fs.delete(FILE_DIR_PATH, true);
   }
   }
 
 
   /* Return the group owner of the file/directory */
   /* Return the group owner of the file/directory */
@@ -596,7 +596,7 @@ public class TestDFSPermission extends TestCase {
     void call() throws IOException {
     void call() throws IOException {
       create(opType, path);
       create(opType, path);
       if (cleanup) {
       if (cleanup) {
-        fs.delete(path);
+        fs.delete(path, true);
       }
       }
     }
     }
   }
   }
@@ -849,7 +849,7 @@ public class TestDFSPermission extends TestCase {
 
 
     @Override
     @Override
     void call() throws IOException {
     void call() throws IOException {
-      fs.delete(path);
+      fs.delete(path, true);
     }
     }
   }
   }
 
 
@@ -954,7 +954,7 @@ public class TestDFSPermission extends TestCase {
       checkNoPermissionDeny(e);
       checkNoPermissionDeny(e);
     }
     }
     try {
     try {
-      fs.delete(NON_EXISTENT_FILE);
+      fs.delete(NON_EXISTENT_FILE, true);
     } catch (IOException e) {
     } catch (IOException e) {
       checkNoPermissionDeny(e);
       checkNoPermissionDeny(e);
     }
     }

+ 32 - 6
src/test/org/apache/hadoop/dfs/TestDFSShell.java

@@ -108,7 +108,33 @@ public class TestDFSShell extends TestCase {
       cluster.shutdown();
       cluster.shutdown();
     }
     }
   }
   }
-
+  
+  public void testRecrusiveRm() throws IOException {
+	  Configuration conf = new Configuration();
+	  MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+	  FileSystem fs = cluster.getFileSystem();
+	  assertTrue("Not a HDFS: " + fs.getUri(), 
+			  fs instanceof DistributedFileSystem);
+	  try {
+      fs.mkdirs(new Path(new Path("parent"), "child"));
+      try {
+        fs.delete(new Path("parent"), false);
+        assert(false); // should never reach here.
+      } catch(IOException e) {
+         //should have thrown an exception
+      }
+      try {
+        fs.delete(new Path("parent"), true);
+      } catch(IOException e) {
+        assert(false);
+      }
+    } finally {  
+      try { fs.close();}catch(IOException e){};
+      cluster.shutdown();
+    }
+  }
+    
+  
   public void testPut() throws IOException {
   public void testPut() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
@@ -372,8 +398,8 @@ public class TestDFSShell extends TestCase {
       ret = ToolRunner.run(shell, argv);
       ret = ToolRunner.run(shell, argv);
       assertTrue(" cat is working ", (ret == 0));
       assertTrue(" cat is working ", (ret == 0));
       //check chown
       //check chown
-      dstFs.delete(new Path("/furi"));
-      dstFs.delete(new Path("/hadoopdir"));
+      dstFs.delete(new Path("/furi"), true);
+      dstFs.delete(new Path("/hadoopdir"), true);
       String file = "/tmp/chownTest";
       String file = "/tmp/chownTest";
       Path path = new Path(file);
       Path path = new Path(file);
       Path parent = new Path("/tmp");
       Path parent = new Path("/tmp");
@@ -613,7 +639,7 @@ public class TestDFSShell extends TestCase {
     try {
     try {
      //first make dir
      //first make dir
      Path dir = new Path(chmodDir);
      Path dir = new Path(chmodDir);
-     fs.delete(dir);
+     fs.delete(dir, true);
      fs.mkdirs(dir);
      fs.mkdirs(dir);
 
 
      runCmd(shell, "-chmod", "u+rwx,g=rw,o-rwx", chmodDir);
      runCmd(shell, "-chmod", "u+rwx,g=rw,o-rwx", chmodDir);
@@ -636,7 +662,7 @@ public class TestDFSShell extends TestCase {
      assertEquals("rw-rw-rw-",
      assertEquals("rw-rw-rw-",
                   fs.getFileStatus(file).getPermission().toString());
                   fs.getFileStatus(file).getPermission().toString());
      
      
-     fs.delete(dir);     
+     fs.delete(dir, true);     
     } finally {
     } finally {
       try {
       try {
         fs.close();
         fs.close();
@@ -775,7 +801,7 @@ public class TestDFSShell extends TestCase {
         }
         }
         assertTrue(val == 0);
         assertTrue(val == 0);
       }
       }
-      fileSys.delete(myFile2);
+      fileSys.delete(myFile2, true);
 
 
       // Verify that we can get with and without crc
       // Verify that we can get with and without crc
       {
       {

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestDFSShellGenericOptions.java

@@ -105,7 +105,7 @@ public class TestDFSShellGenericOptions extends TestCase {
                                      shell.getConf());
                                      shell.getConf());
       assertTrue("Directory does not get created", 
       assertTrue("Directory does not get created", 
                  fs.isDirectory(new Path("/data")));
                  fs.isDirectory(new Path("/data")));
-      fs.delete(new Path("/data"));
+      fs.delete(new Path("/data"), true);
     } catch (Exception e) {
     } catch (Exception e) {
       System.err.println(e.getMessage());
       System.err.println(e.getMessage());
       e.printStackTrace();
       e.printStackTrace();

+ 2 - 2
src/test/org/apache/hadoop/dfs/TestDecommission.java

@@ -54,7 +54,7 @@ public class TestDecommission extends TestCase {
 
 
     // delete if it already exists
     // delete if it already exists
     if (fs.exists(name)) {
     if (fs.exists(name)) {
-      fs.delete(name);
+      fs.delete(name, true);
     }
     }
 
 
     FSDataOutputStream stm = fs.create(name);
     FSDataOutputStream stm = fs.create(name);
@@ -150,7 +150,7 @@ public class TestDecommission extends TestCase {
   
   
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    fileSys.delete(name);
+    fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
 
 

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestFSInputChecker.java

@@ -191,7 +191,7 @@ public class TestFSInputChecker extends TestCase {
 
 
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    fileSys.delete(name);
+    fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
   
   

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestFSOutputSummer.java

@@ -102,7 +102,7 @@ public class TestFSOutputSummer extends TestCase {
 
 
   private void cleanupFile(Path name) throws IOException {
   private void cleanupFile(Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    fileSys.delete(name);
+    fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
   
   

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestFileCorruption.java

@@ -91,6 +91,6 @@ public class TestFileCorruption extends TestCase {
     } catch (ChecksumException ignore) {
     } catch (ChecksumException ignore) {
       //expect this exception but let any NPE get thrown
       //expect this exception but let any NPE get thrown
     }
     }
-    fs.delete(file);
+    fs.delete(file, true);
   }
   }
 }
 }

+ 2 - 2
src/test/org/apache/hadoop/dfs/TestFileLimit.java

@@ -132,7 +132,7 @@ public class TestFileLimit extends TestCase {
 
 
       // delete one file
       // delete one file
       Path file0 = new Path("/filestatus0");
       Path file0 = new Path("/filestatus0");
-      fs.delete(file0);
+      fs.delete(file0, true);
       System.out.println("Deleted file " + file0);
       System.out.println("Deleted file " + file0);
       currentNodes -= 2;
       currentNodes -= 2;
 
 
@@ -146,7 +146,7 @@ public class TestFileLimit extends TestCase {
 
 
       // delete the file again
       // delete the file again
       file0 = new Path("/filestatus0");
       file0 = new Path("/filestatus0");
-      fs.delete(file0);
+      fs.delete(file0, true);
       System.out.println("Deleted file " + file0 + " again.");
       System.out.println("Deleted file " + file0 + " again.");
       currentNodes -= 2;
       currentNodes -= 2;
 
 

+ 2 - 2
src/test/org/apache/hadoop/dfs/TestLocalDFS.java

@@ -45,7 +45,7 @@ public class TestLocalDFS extends TestCase {
   
   
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    fileSys.delete(name);
+    fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
   
   
@@ -62,7 +62,7 @@ public class TestLocalDFS extends TestCase {
       Path file1 = new Path("somewhat/random.txt");
       Path file1 = new Path("somewhat/random.txt");
       writeFile(fileSys, file1);
       writeFile(fileSys, file1);
       assertTrue(fileSys.exists(new Path(orig_path, file1.toString())));
       assertTrue(fileSys.exists(new Path(orig_path, file1.toString())));
-      fileSys.delete(file1);
+      fileSys.delete(file1, true);
       Path subdir1 = new Path("/somewhere");
       Path subdir1 = new Path("/somewhere");
       fileSys.setWorkingDirectory(subdir1);
       fileSys.setWorkingDirectory(subdir1);
       writeFile(fileSys, file1);
       writeFile(fileSys, file1);

+ 2 - 2
src/test/org/apache/hadoop/dfs/TestModTime.java

@@ -58,7 +58,7 @@ public class TestModTime extends TestCase {
   
   
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    fileSys.delete(name);
+    fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
 
 
@@ -154,7 +154,7 @@ public class TestModTime extends TestCase {
      // delete newfile
      // delete newfile
      //
      //
      System.out.println("Deleting testdir2/testnew.dat.");
      System.out.println("Deleting testdir2/testnew.dat.");
-     assertTrue(fileSys.delete(newfile));
+     assertTrue(fileSys.delete(newfile, true));
      //
      //
      // verify that modification time of testdir1 has not changed.
      // verify that modification time of testdir1 has not changed.
      //
      //

+ 2 - 2
src/test/org/apache/hadoop/dfs/TestPread.java

@@ -52,7 +52,7 @@ public class TestPread extends TestCase {
     }
     }
     assertTrue("Error reading beyond file boundary.", res != null);
     assertTrue("Error reading beyond file boundary.", res != null);
     in.close();
     in.close();
-    if (!fileSys.delete(name))
+    if (!fileSys.delete(name, true))
       assertTrue("Cannot delete file", false);
       assertTrue("Cannot delete file", false);
     
     
     // now create the real file
     // now create the real file
@@ -153,7 +153,7 @@ public class TestPread extends TestCase {
   
   
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    assertTrue(fileSys.delete(name));
+    assertTrue(fileSys.delete(name, true));
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
   
   

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestReplication.java

@@ -124,7 +124,7 @@ public class TestReplication extends TestCase {
   
   
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    fileSys.delete(name);
+    fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
   
   

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestSeekBug.java

@@ -113,7 +113,7 @@ public class TestSeekBug extends TestCase {
   
   
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    fileSys.delete(name);
+    fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
   
   

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestSmallBlock.java

@@ -79,7 +79,7 @@ public class TestSmallBlock extends TestCase {
   
   
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
   private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
     assertTrue(fileSys.exists(name));
     assertTrue(fileSys.exists(name));
-    fileSys.delete(name);
+    fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
     assertTrue(!fileSys.exists(name));
   }
   }
   
   

+ 9 - 9
src/test/org/apache/hadoop/fs/DFSCIOTest.java

@@ -116,7 +116,7 @@ public class DFSCIOTest extends TestCase {
                                         ) throws IOException {
                                         ) throws IOException {
     LOG.info("creating control file: "+fileSize+" mega bytes, "+nrFiles+" files");
     LOG.info("creating control file: "+fileSize+" mega bytes, "+nrFiles+" files");
 
 
-    fs.delete(CONTROL_DIR);
+    fs.delete(CONTROL_DIR, true);
 
 
     for(int i=0; i < nrFiles; i++) {
     for(int i=0; i < nrFiles; i++) {
       String name = getFileName(i);
       String name = getFileName(i);
@@ -204,7 +204,7 @@ public class DFSCIOTest extends TestCase {
           
           
         // copy the dso and executable from dfs and chmod them
         // copy the dso and executable from dfs and chmod them
         synchronized (this) {
         synchronized (this) {
-          localFS.delete(HDFS_TEST_DIR);
+          localFS.delete(HDFS_TEST_DIR, true);
           if (!(localFS.mkdirs(HDFS_TEST_DIR))) {
           if (!(localFS.mkdirs(HDFS_TEST_DIR))) {
             throw new IOException("Failed to create " +	HDFS_TEST_DIR + " on local filesystem");
             throw new IOException("Failed to create " +	HDFS_TEST_DIR + " on local filesystem");
           }
           }
@@ -256,8 +256,8 @@ public class DFSCIOTest extends TestCase {
   private static void writeTest(FileSystem fs)
   private static void writeTest(FileSystem fs)
     throws IOException {
     throws IOException {
 
 
-    fs.delete(DATA_DIR);
-    fs.delete(WRITE_DIR);
+    fs.delete(DATA_DIR, true);
+    fs.delete(WRITE_DIR, true);
     
     
     runIOTest(WriteMapper.class, WRITE_DIR);
     runIOTest(WriteMapper.class, WRITE_DIR);
   }
   }
@@ -304,7 +304,7 @@ public class DFSCIOTest extends TestCase {
         
         
         // copy the dso and executable from dfs
         // copy the dso and executable from dfs
         synchronized (this) {
         synchronized (this) {
-          localFS.delete(HDFS_TEST_DIR);
+          localFS.delete(HDFS_TEST_DIR, true);
           if (!(localFS.mkdirs(HDFS_TEST_DIR))) {
           if (!(localFS.mkdirs(HDFS_TEST_DIR))) {
             throw new IOException("Failed to create " +	HDFS_TEST_DIR + " on local filesystem");
             throw new IOException("Failed to create " +	HDFS_TEST_DIR + " on local filesystem");
           }
           }
@@ -361,7 +361,7 @@ public class DFSCIOTest extends TestCase {
   }
   }
 
 
   private static void readTest(FileSystem fs) throws IOException {
   private static void readTest(FileSystem fs) throws IOException {
-    fs.delete(READ_DIR);
+    fs.delete(READ_DIR, true);
     runIOTest(ReadMapper.class, READ_DIR);
     runIOTest(ReadMapper.class, READ_DIR);
   }
   }
 
 
@@ -429,7 +429,7 @@ public class DFSCIOTest extends TestCase {
       FileSystem fs = FileSystem.get(fsConfig);
       FileSystem fs = FileSystem.get(fsConfig);
       
       
       if (testType != TEST_TYPE_CLEANUP) {
       if (testType != TEST_TYPE_CLEANUP) {
-        fs.delete(HDFS_TEST_DIR);
+        fs.delete(HDFS_TEST_DIR, true);
         if (!fs.mkdirs(HDFS_TEST_DIR)) {
         if (!fs.mkdirs(HDFS_TEST_DIR)) {
           throw new IOException("Mkdirs failed to create " + 
           throw new IOException("Mkdirs failed to create " + 
                                 HDFS_TEST_DIR.toString());
                                 HDFS_TEST_DIR.toString());
@@ -532,7 +532,7 @@ public class DFSCIOTest extends TestCase {
 
 
   private static void cleanup(FileSystem fs) throws Exception {
   private static void cleanup(FileSystem fs) throws Exception {
     LOG.info("Cleaning up test files");
     LOG.info("Cleaning up test files");
-    fs.delete(new Path(TEST_ROOT_DIR));
-    fs.delete(HDFS_TEST_DIR);
+    fs.delete(new Path(TEST_ROOT_DIR), true);
+    fs.delete(HDFS_TEST_DIR, true);
   }
   }
 }
 }

+ 1 - 1
src/test/org/apache/hadoop/fs/DistributedFSCheck.java

@@ -326,6 +326,6 @@ public class DistributedFSCheck extends TestCase {
 
 
   private void cleanup() throws IOException {
   private void cleanup() throws IOException {
     LOG.info("Cleaning up test files");
     LOG.info("Cleaning up test files");
-    fs.delete(TEST_ROOT_DIR);
+    fs.delete(TEST_ROOT_DIR, true);
   }
   }
 }
 }

+ 1 - 1
src/test/org/apache/hadoop/fs/TestCopyFiles.java

@@ -208,7 +208,7 @@ public class TestCopyFiles extends TestCase {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     FileSystem fs = FileSystem.getNamed(fsname, conf);
     FileSystem fs = FileSystem.getNamed(fsname, conf);
     Path root = new Path(topdir);
     Path root = new Path(topdir);
-    fs.delete(root);
+    fs.delete(root, true);
   }
   }
   
   
   /** copy files from local file system to local file system */
   /** copy files from local file system to local file system */

+ 5 - 5
src/test/org/apache/hadoop/fs/TestDFSIO.java

@@ -111,7 +111,7 @@ public class TestDFSIO extends TestCase {
                                         ) throws IOException {
                                         ) throws IOException {
     LOG.info("creating control file: "+fileSize+" mega bytes, "+nrFiles+" files");
     LOG.info("creating control file: "+fileSize+" mega bytes, "+nrFiles+" files");
 
 
-    fs.delete(CONTROL_DIR);
+    fs.delete(CONTROL_DIR, true);
 
 
     for(int i=0; i < nrFiles; i++) {
     for(int i=0; i < nrFiles; i++) {
       String name = getFileName(i);
       String name = getFileName(i);
@@ -212,8 +212,8 @@ public class TestDFSIO extends TestCase {
   private static void writeTest(FileSystem fs)
   private static void writeTest(FileSystem fs)
     throws IOException {
     throws IOException {
 
 
-    fs.delete(DATA_DIR);
-    fs.delete(WRITE_DIR);
+    fs.delete(DATA_DIR, true);
+    fs.delete(WRITE_DIR, true);
     
     
     runIOTest(WriteMapper.class, WRITE_DIR);
     runIOTest(WriteMapper.class, WRITE_DIR);
   }
   }
@@ -269,7 +269,7 @@ public class TestDFSIO extends TestCase {
   }
   }
 
 
   private static void readTest(FileSystem fs) throws IOException {
   private static void readTest(FileSystem fs) throws IOException {
-    fs.delete(READ_DIR);
+    fs.delete(READ_DIR, true);
     runIOTest(ReadMapper.class, READ_DIR);
     runIOTest(ReadMapper.class, READ_DIR);
   }
   }
 
 
@@ -425,6 +425,6 @@ public class TestDFSIO extends TestCase {
 
 
   private static void cleanup(FileSystem fs) throws IOException {
   private static void cleanup(FileSystem fs) throws IOException {
     LOG.info("Cleaning up test files");
     LOG.info("Cleaning up test files");
-    fs.delete(new Path(TEST_ROOT_DIR));
+    fs.delete(new Path(TEST_ROOT_DIR), true);
   }
   }
 }
 }

+ 9 - 9
src/test/org/apache/hadoop/fs/TestFileSystem.java

@@ -77,10 +77,10 @@ public class TestFileSystem extends TestCase {
     writeTest(fs, false);
     writeTest(fs, false);
     readTest(fs, false);
     readTest(fs, false);
     seekTest(fs, false);
     seekTest(fs, false);
-    fs.delete(CONTROL_DIR);
-    fs.delete(DATA_DIR);
-    fs.delete(WRITE_DIR);
-    fs.delete(READ_DIR);
+    fs.delete(CONTROL_DIR, true);
+    fs.delete(DATA_DIR, true);
+    fs.delete(WRITE_DIR, true);
+    fs.delete(READ_DIR, true);
   }
   }
 
 
   public static void createControlFile(FileSystem fs,
   public static void createControlFile(FileSystem fs,
@@ -90,7 +90,7 @@ public class TestFileSystem extends TestCase {
     LOG.info("creating control file: "+megaBytes+" bytes, "+numFiles+" files");
     LOG.info("creating control file: "+megaBytes+" bytes, "+numFiles+" files");
 
 
     Path controlFile = new Path(CONTROL_DIR, "files");
     Path controlFile = new Path(CONTROL_DIR, "files");
-    fs.delete(controlFile);
+    fs.delete(controlFile, true);
     Random random = new Random(seed);
     Random random = new Random(seed);
 
 
     SequenceFile.Writer writer =
     SequenceFile.Writer writer =
@@ -197,8 +197,8 @@ public class TestFileSystem extends TestCase {
   public static void writeTest(FileSystem fs, boolean fastCheck)
   public static void writeTest(FileSystem fs, boolean fastCheck)
     throws Exception {
     throws Exception {
 
 
-    fs.delete(DATA_DIR);
-    fs.delete(WRITE_DIR);
+    fs.delete(DATA_DIR, true);
+    fs.delete(WRITE_DIR, true);
     
     
     JobConf job = new JobConf(conf, TestFileSystem.class);
     JobConf job = new JobConf(conf, TestFileSystem.class);
     job.setBoolean("fs.test.fastCheck", fastCheck);
     job.setBoolean("fs.test.fastCheck", fastCheck);
@@ -295,7 +295,7 @@ public class TestFileSystem extends TestCase {
   public static void readTest(FileSystem fs, boolean fastCheck)
   public static void readTest(FileSystem fs, boolean fastCheck)
     throws Exception {
     throws Exception {
 
 
-    fs.delete(READ_DIR);
+    fs.delete(READ_DIR, true);
 
 
     JobConf job = new JobConf(conf, TestFileSystem.class);
     JobConf job = new JobConf(conf, TestFileSystem.class);
     job.setBoolean("fs.test.fastCheck", fastCheck);
     job.setBoolean("fs.test.fastCheck", fastCheck);
@@ -392,7 +392,7 @@ public class TestFileSystem extends TestCase {
   public static void seekTest(FileSystem fs, boolean fastCheck)
   public static void seekTest(FileSystem fs, boolean fastCheck)
     throws Exception {
     throws Exception {
 
 
-    fs.delete(READ_DIR);
+    fs.delete(READ_DIR, true);
 
 
     JobConf job = new JobConf(conf, TestFileSystem.class);
     JobConf job = new JobConf(conf, TestFileSystem.class);
     job.setBoolean("fs.test.fastCheck", fastCheck);
     job.setBoolean("fs.test.fastCheck", fastCheck);

+ 1 - 1
src/test/org/apache/hadoop/fs/TestGlobPaths.java

@@ -345,7 +345,7 @@ public class TestGlobPaths extends TestCase {
   }
   }
   
   
   private void cleanupDFS() throws IOException {
   private void cleanupDFS() throws IOException {
-    fs.delete(new Path("/user"));
+    fs.delete(new Path("/user"), true);
   }
   }
   
   
 }
 }

+ 4 - 4
src/test/org/apache/hadoop/fs/TestLocalFileSystem.java

@@ -37,7 +37,7 @@ public class TestLocalFileSystem extends TestCase {
   
   
   private void cleanupFile(FileSystem fs, Path name) throws IOException {
   private void cleanupFile(FileSystem fs, Path name) throws IOException {
     assertTrue(fs.exists(name));
     assertTrue(fs.exists(name));
-    fs.delete(name);
+    fs.delete(name, true);
     assertTrue(!fs.exists(name));
     assertTrue(!fs.exists(name));
   }
   }
   
   
@@ -64,7 +64,7 @@ public class TestLocalFileSystem extends TestCase {
       assertTrue(fileSys.isDirectory(dir1));
       assertTrue(fileSys.isDirectory(dir1));
       
       
       // delete the directory and make sure it went away
       // delete the directory and make sure it went away
-      fileSys.delete(dir1);
+      fileSys.delete(dir1, true);
       assertTrue(!fileSys.exists(dir1));
       assertTrue(!fileSys.exists(dir1));
       
       
       // create files and manipulate them.
       // create files and manipulate them.
@@ -92,7 +92,7 @@ public class TestLocalFileSystem extends TestCase {
       stm.close();
       stm.close();
     } finally {
     } finally {
       fileSys.setWorkingDirectory(origDir);
       fileSys.setWorkingDirectory(origDir);
-      fileSys.delete(subdir);
+      fileSys.delete(subdir, true);
     }
     }
   }
   }
 
 
@@ -113,7 +113,7 @@ public class TestLocalFileSystem extends TestCase {
     Path tmp = new Path(src, dst.getName());
     Path tmp = new Path(src, dst.getName());
     assertTrue(fs.exists(tmp) && fs.exists(dst));
     assertTrue(fs.exists(tmp) && fs.exists(dst));
     assertTrue(FileUtil.copy(fs, dst, fs, src, false, true, conf));
     assertTrue(FileUtil.copy(fs, dst, fs, src, false, true, conf));
-    assertTrue(fs.delete(tmp));
+    assertTrue(fs.delete(tmp, true));
     fs.mkdirs(tmp);
     fs.mkdirs(tmp);
     try {
     try {
       FileUtil.copy(fs, dst, fs, src, true, true, conf);
       FileUtil.copy(fs, dst, fs, src, true, true, conf);

+ 1 - 1
src/test/org/apache/hadoop/fs/TestLocalFileSystemPermission.java

@@ -56,7 +56,7 @@ public class TestLocalFileSystemPermission extends TestCase {
 
 
   private void cleanupFile(FileSystem fs, Path name) throws IOException {
   private void cleanupFile(FileSystem fs, Path name) throws IOException {
     assertTrue(fs.exists(name));
     assertTrue(fs.exists(name));
-    fs.delete(name);
+    fs.delete(name, true);
     assertTrue(!fs.exists(name));
     assertTrue(!fs.exists(name));
   }
   }
 
 

+ 2 - 2
src/test/org/apache/hadoop/fs/kfs/KFSEmulationImpl.java

@@ -98,14 +98,14 @@ public class KFSEmulationImpl implements IFSImpl {
         if (isDirectory(path)) {
         if (isDirectory(path)) {
             // the directory better be empty
             // the directory better be empty
             String[] dirEntries = readdir(path);
             String[] dirEntries = readdir(path);
-            if ((dirEntries.length <= 2) && (localFS.delete(new Path(path))))
+            if ((dirEntries.length <= 2) && (localFS.delete(new Path(path), true)))
                 return 0;
                 return 0;
         }
         }
         return -1;
         return -1;
     }
     }
 
 
     public int remove(String path) throws IOException {
     public int remove(String path) throws IOException {
-        if (isFile(path) && (localFS.delete(new Path(path))))
+        if (isFile(path) && (localFS.delete(new Path(path), true)))
             return 0;
             return 0;
         return -1;
         return -1;
     }
     }

+ 7 - 7
src/test/org/apache/hadoop/fs/kfs/TestKosmosFileSystem.java

@@ -73,7 +73,7 @@ public class TestKosmosFileSystem extends TestCase {
         Path[] p = kosmosFileSystem.listPaths(baseDir);
         Path[] p = kosmosFileSystem.listPaths(baseDir);
         assertEquals(p.length, 1);
         assertEquals(p.length, 1);
 
 
-        kosmosFileSystem.delete(baseDir);
+        kosmosFileSystem.delete(baseDir, true);
         assertFalse(kosmosFileSystem.exists(baseDir));
         assertFalse(kosmosFileSystem.exists(baseDir));
     }
     }
 
 
@@ -100,15 +100,15 @@ public class TestKosmosFileSystem extends TestCase {
         Path[] p = kosmosFileSystem.listPaths(subDir1);
         Path[] p = kosmosFileSystem.listPaths(subDir1);
         assertEquals(p.length, 2);
         assertEquals(p.length, 2);
 
 
-        kosmosFileSystem.delete(file1);
+        kosmosFileSystem.delete(file1, true);
         p = kosmosFileSystem.listPaths(subDir1);
         p = kosmosFileSystem.listPaths(subDir1);
         assertEquals(p.length, 1);
         assertEquals(p.length, 1);
 
 
-        kosmosFileSystem.delete(file2);
+        kosmosFileSystem.delete(file2, true);
         p = kosmosFileSystem.listPaths(subDir1);
         p = kosmosFileSystem.listPaths(subDir1);
         assertEquals(p, null);
         assertEquals(p, null);
 
 
-        kosmosFileSystem.delete(baseDir);
+        kosmosFileSystem.delete(baseDir, true);
         assertFalse(kosmosFileSystem.exists(baseDir));
         assertFalse(kosmosFileSystem.exists(baseDir));
     }
     }
 
 
@@ -158,11 +158,11 @@ public class TestKosmosFileSystem extends TestCase {
 
 
         s2.close();
         s2.close();
 
 
-        kosmosFileSystem.delete(file1);
+        kosmosFileSystem.delete(file1, true);
         assertFalse(kosmosFileSystem.exists(file1));        
         assertFalse(kosmosFileSystem.exists(file1));        
-        kosmosFileSystem.delete(subDir1);
+        kosmosFileSystem.delete(subDir1, true);
         assertFalse(kosmosFileSystem.exists(subDir1));        
         assertFalse(kosmosFileSystem.exists(subDir1));        
-        kosmosFileSystem.delete(baseDir);
+        kosmosFileSystem.delete(baseDir, true);
         assertFalse(kosmosFileSystem.exists(baseDir));        
         assertFalse(kosmosFileSystem.exists(baseDir));        
     }
     }
     
     

+ 3 - 3
src/test/org/apache/hadoop/fs/s3/S3FileSystemBaseTest.java

@@ -179,7 +179,7 @@ public abstract class S3FileSystemBaseTest extends TestCase {
       assertEquals("Position " + i, data[i], buf[i]);
       assertEquals("Position " + i, data[i], buf[i]);
     }
     }
     
     
-    assertTrue("Deleted", s3FileSystem.delete(path));
+    assertTrue("Deleted", s3FileSystem.delete(path, true));
     
     
     assertFalse("No longer exists", s3FileSystem.exists(path));
     assertFalse("No longer exists", s3FileSystem.exists(path));
 
 
@@ -227,7 +227,7 @@ public abstract class S3FileSystemBaseTest extends TestCase {
   public void testDeleteNonExistentFile() throws IOException {
   public void testDeleteNonExistentFile() throws IOException {
     Path path = new Path("/test/hadoop/file");    
     Path path = new Path("/test/hadoop/file");    
     assertFalse("Doesn't exist", s3FileSystem.exists(path));
     assertFalse("Doesn't exist", s3FileSystem.exists(path));
-    assertFalse("No deletion", s3FileSystem.delete(path));
+    assertFalse("No deletion", s3FileSystem.delete(path, true));
   }
   }
 
 
   public void testDeleteDirectory() throws IOException {
   public void testDeleteDirectory() throws IOException {
@@ -247,7 +247,7 @@ public abstract class S3FileSystemBaseTest extends TestCase {
     assertTrue("subdir exists", s3FileSystem.exists(subdir));
     assertTrue("subdir exists", s3FileSystem.exists(subdir));
     assertTrue("file2 exists", s3FileSystem.exists(file2));
     assertTrue("file2 exists", s3FileSystem.exists(file2));
     
     
-    assertTrue("Delete", s3FileSystem.delete(dir));
+    assertTrue("Delete", s3FileSystem.delete(dir, true));
 
 
     assertTrue("root exists", s3FileSystem.exists(root));
     assertTrue("root exists", s3FileSystem.exists(root));
     assertFalse("dir exists", s3FileSystem.exists(dir));
     assertFalse("dir exists", s3FileSystem.exists(dir));

+ 6 - 6
src/test/org/apache/hadoop/io/TestSequenceFile.java

@@ -143,7 +143,7 @@ public class TestSequenceFile extends TestCase {
   private static void writeTest(FileSystem fs, int count, int seed, Path file, 
   private static void writeTest(FileSystem fs, int count, int seed, Path file, 
                                 CompressionType compressionType, CompressionCodec codec)
                                 CompressionType compressionType, CompressionCodec codec)
     throws IOException {
     throws IOException {
-    fs.delete(file);
+    fs.delete(file, true);
     LOG.info("creating " + count + " records with " + compressionType +
     LOG.info("creating " + count + " records with " + compressionType +
              " compression");
              " compression");
     SequenceFile.Writer writer = 
     SequenceFile.Writer writer = 
@@ -219,7 +219,7 @@ public class TestSequenceFile extends TestCase {
   private static void sortTest(FileSystem fs, int count, int megabytes, 
   private static void sortTest(FileSystem fs, int count, int megabytes, 
                                int factor, boolean fast, Path file)
                                int factor, boolean fast, Path file)
     throws IOException {
     throws IOException {
-    fs.delete(new Path(file+".sorted"));
+    fs.delete(new Path(file+".sorted"), true);
     SequenceFile.Sorter sorter = newSorter(fs, fast, megabytes, factor);
     SequenceFile.Sorter sorter = newSorter(fs, fast, megabytes, factor);
     LOG.debug("sorting " + count + " records");
     LOG.debug("sorting " + count + " records");
     sorter.sort(file, file.suffix(".sorted"));
     sorter.sort(file, file.suffix(".sorted"));
@@ -277,8 +277,8 @@ public class TestSequenceFile extends TestCase {
     for (int i = 0; i < factor; i++) {
     for (int i = 0; i < factor; i++) {
       names[i] = file.suffix("."+i);
       names[i] = file.suffix("."+i);
       sortedNames[i] = names[i].suffix(".sorted");
       sortedNames[i] = names[i].suffix(".sorted");
-      fs.delete(names[i]);
-      fs.delete(sortedNames[i]);
+      fs.delete(names[i], true);
+      fs.delete(sortedNames[i], true);
       writers[i] = SequenceFile.createWriter(fs, conf, names[i], 
       writers[i] = SequenceFile.createWriter(fs, conf, names[i], 
                                              RandomDatum.class, RandomDatum.class, compressionType);
                                              RandomDatum.class, RandomDatum.class, compressionType);
     }
     }
@@ -302,7 +302,7 @@ public class TestSequenceFile extends TestCase {
     }
     }
 
 
     LOG.info("merging " + factor + " files with " + count/factor + " debug");
     LOG.info("merging " + factor + " files with " + count/factor + " debug");
-    fs.delete(new Path(file+".sorted"));
+    fs.delete(new Path(file+".sorted"), true);
     newSorter(fs, fast, megabytes, factor)
     newSorter(fs, fast, megabytes, factor)
       .merge(sortedNames, file.suffix(".sorted"));
       .merge(sortedNames, file.suffix(".sorted"));
   }
   }
@@ -388,7 +388,7 @@ public class TestSequenceFile extends TestCase {
   private static void writeMetadataTest(FileSystem fs, int count, int seed, Path file, 
   private static void writeMetadataTest(FileSystem fs, int count, int seed, Path file, 
                                         CompressionType compressionType, CompressionCodec codec, SequenceFile.Metadata metadata)
                                         CompressionType compressionType, CompressionCodec codec, SequenceFile.Metadata metadata)
     throws IOException {
     throws IOException {
-    fs.delete(file);
+    fs.delete(file, true);
     LOG.info("creating " + count + " records with metadata and with" + compressionType +
     LOG.info("creating " + count + " records with metadata and with" + compressionType +
              " compression");
              " compression");
     SequenceFile.Writer writer = 
     SequenceFile.Writer writer = 

+ 1 - 1
src/test/org/apache/hadoop/mapred/BigMapOutput.java

@@ -132,7 +132,7 @@ public class BigMapOutput extends Configured implements Tool {
     jobConf.setOutputFormat(SequenceFileOutputFormat.class);
     jobConf.setOutputFormat(SequenceFileOutputFormat.class);
     jobConf.setInputPath(bigMapInput);
     jobConf.setInputPath(bigMapInput);
     if (fs.exists(outputPath)) {
     if (fs.exists(outputPath)) {
-      fs.delete(outputPath);
+      fs.delete(outputPath, true);
     }
     }
     jobConf.setOutputPath(outputPath);
     jobConf.setOutputPath(outputPath);
     jobConf.setMapperClass(IdentityMapper.class);
     jobConf.setMapperClass(IdentityMapper.class);

+ 1 - 1
src/test/org/apache/hadoop/mapred/MRBench.java

@@ -282,7 +282,7 @@ public class MRBench {
       execTimes = runJobInSequence(jobConf, numRuns);
       execTimes = runJobInSequence(jobConf, numRuns);
     } finally {
     } finally {
       // delete output -- should we really do this?
       // delete output -- should we really do this?
-      fs.delete(BASE_DIR);
+      fs.delete(BASE_DIR, true);
     }
     }
     
     
     if (verbose) {
     if (verbose) {

+ 3 - 3
src/test/org/apache/hadoop/mapred/MRCaching.java

@@ -71,7 +71,7 @@ public class MRCaching {
           throw new IOException("Mkdirs failed to create " + file.toString());
           throw new IOException("Mkdirs failed to create " + file.toString());
         }
         }
         Path fileOut = new Path(file, "test.txt");
         Path fileOut = new Path(file, "test.txt");
-        fs.delete(fileOut);
+        fs.delete(fileOut, true);
         DataOutputStream out = fs.create(fileOut);
         DataOutputStream out = fs.create(fileOut);
         for (int i = 0; i < localArchives.length; i++) {
         for (int i = 0; i < localArchives.length; i++) {
           // read out the files from these archives
           // read out the files from these archives
@@ -149,7 +149,7 @@ public class MRCaching {
     final Path inDir = new Path(indir);
     final Path inDir = new Path(indir);
     final Path outDir = new Path(outdir);
     final Path outDir = new Path(outdir);
     FileSystem fs = FileSystem.get(conf);
     FileSystem fs = FileSystem.get(conf);
-    fs.delete(outDir);
+    fs.delete(outDir, true);
     if (!fs.mkdirs(inDir)) {
     if (!fs.mkdirs(inDir)) {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     }
@@ -179,7 +179,7 @@ public class MRCaching {
     Path jarPath = new Path(localPath, new Path("test.jar"));
     Path jarPath = new Path(localPath, new Path("test.jar"));
     Path zipPath = new Path(localPath, new Path("test.zip"));
     Path zipPath = new Path(localPath, new Path("test.zip"));
     Path cachePath = new Path(cacheDir);
     Path cachePath = new Path(cacheDir);
-    fs.delete(cachePath);
+    fs.delete(cachePath, true);
     if (!fs.mkdirs(cachePath)) {
     if (!fs.mkdirs(cachePath)) {
       throw new IOException("Mkdirs failed to create " + cachePath.toString());
       throw new IOException("Mkdirs failed to create " + cachePath.toString());
     }
     }

+ 1 - 1
src/test/org/apache/hadoop/mapred/NotificationTestCase.java

@@ -185,7 +185,7 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     }
     }
 
 
     FileSystem fs = FileSystem.get(conf);
     FileSystem fs = FileSystem.get(conf);
-    fs.delete(outDir);
+    fs.delete(outDir, true);
     if (!fs.mkdirs(inDir)) {
     if (!fs.mkdirs(inDir)) {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     }

+ 2 - 2
src/test/org/apache/hadoop/mapred/PiEstimator.java

@@ -153,7 +153,7 @@ public class PiEstimator {
     Path inDir = new Path(tmpDir, "in");
     Path inDir = new Path(tmpDir, "in");
     Path outDir = new Path(tmpDir, "out");
     Path outDir = new Path(tmpDir, "out");
     FileSystem fileSys = FileSystem.get(jobConf);
     FileSystem fileSys = FileSystem.get(jobConf);
-    fileSys.delete(tmpDir);
+    fileSys.delete(tmpDir, true);
     if (!fileSys.mkdirs(inDir)) {
     if (!fileSys.mkdirs(inDir)) {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     }
@@ -183,7 +183,7 @@ public class PiEstimator {
       reader.close();
       reader.close();
       estimate = (double) (numInside.get()*4.0)/(numMaps*numPoints);
       estimate = (double) (numInside.get()*4.0)/(numMaps*numPoints);
     } finally {
     } finally {
-      FileSystem.get(jobConf).delete(tmpDir);
+      FileSystem.get(jobConf).delete(tmpDir, true);
     }
     }
     
     
     return estimate;
     return estimate;

+ 2 - 2
src/test/org/apache/hadoop/mapred/SortValidator.java

@@ -340,7 +340,7 @@ public class SortValidator extends Configured implements Tool {
       jobConf.addInputPath(sortOutput);
       jobConf.addInputPath(sortOutput);
       Path outputPath = new Path("/tmp/sortvalidate/recordstatschecker");
       Path outputPath = new Path("/tmp/sortvalidate/recordstatschecker");
       if (fs.exists(outputPath)) {
       if (fs.exists(outputPath)) {
-        fs.delete(outputPath);
+        fs.delete(outputPath, true);
       }
       }
       jobConf.setOutputPath(outputPath);
       jobConf.setOutputPath(outputPath);
       
       
@@ -482,7 +482,7 @@ public class SortValidator extends Configured implements Tool {
       Path outputPath = new Path("/tmp/sortvalidate/recordchecker");
       Path outputPath = new Path("/tmp/sortvalidate/recordchecker");
       FileSystem fs = FileSystem.get(defaults);
       FileSystem fs = FileSystem.get(defaults);
       if (fs.exists(outputPath)) {
       if (fs.exists(outputPath)) {
-        fs.delete(outputPath);
+        fs.delete(outputPath, true);
       }
       }
       jobConf.setOutputPath(outputPath);
       jobConf.setOutputPath(outputPath);
       
       

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestCollect.java

@@ -139,7 +139,7 @@ public class TestCollect extends TestCase
       throw ioe;
       throw ioe;
     } finally {
     } finally {
       FileSystem fs = FileSystem.get(conf);
       FileSystem fs = FileSystem.get(conf);
-      fs.delete(OUTPUT_DIR);
+      fs.delete(OUTPUT_DIR, true);
     }
     }
   }
   }
   
   

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestComparators.java

@@ -295,7 +295,7 @@ public class TestComparators extends TestCase
     Path inDir = new Path(testdir, "in");
     Path inDir = new Path(testdir, "in");
     Path outDir = new Path(testdir, "out");
     Path outDir = new Path(testdir, "out");
     FileSystem fs = FileSystem.get(conf);
     FileSystem fs = FileSystem.get(conf);
-    fs.delete(testdir);
+    fs.delete(testdir, true);
     conf.setInputFormat(SequenceFileInputFormat.class);
     conf.setInputFormat(SequenceFileInputFormat.class);
     conf.setInputPath(inDir);
     conf.setInputPath(inDir);
     conf.setOutputPath(outDir);
     conf.setOutputPath(outDir);

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestEmptyJobWithDFS.java

@@ -60,7 +60,7 @@ public class TestEmptyJobWithDFS extends TestCase {
     final Path inDir = new Path("/testing/empty/input");
     final Path inDir = new Path("/testing/empty/input");
     final Path outDir = new Path("/testing/empty/output");
     final Path outDir = new Path("/testing/empty/output");
     FileSystem fs = FileSystem.getNamed(fileSys, conf);
     FileSystem fs = FileSystem.getNamed(fileSys, conf);
-    fs.delete(outDir);
+    fs.delete(outDir, true);
     if (!fs.mkdirs(inDir)) {
     if (!fs.mkdirs(inDir)) {
       LOG.warn("Can't create " + inDir);
       LOG.warn("Can't create " + inDir);
       return false;
       return false;

+ 4 - 4
src/test/org/apache/hadoop/mapred/TestFieldSelection.java

@@ -45,9 +45,9 @@ private static NumberFormat idFormat = NumberFormat.getInstance();
     Path OUTPUT_DIR = new Path("build/test/output_for_field_selection_test");
     Path OUTPUT_DIR = new Path("build/test/output_for_field_selection_test");
     Path INPUT_DIR = new Path("build/test/input_for_field_selection_test");
     Path INPUT_DIR = new Path("build/test/input_for_field_selection_test");
     String inputFile = "input.txt";
     String inputFile = "input.txt";
-    fs.delete(INPUT_DIR);
+    fs.delete(INPUT_DIR, true);
     fs.mkdirs(INPUT_DIR);
     fs.mkdirs(INPUT_DIR);
-    fs.delete(OUTPUT_DIR);
+    fs.delete(OUTPUT_DIR, true);
 
 
     StringBuffer inputData = new StringBuffer();
     StringBuffer inputData = new StringBuffer();
     StringBuffer expectedOutput = new StringBuffer();
     StringBuffer expectedOutput = new StringBuffer();
@@ -118,8 +118,8 @@ private static NumberFormat idFormat = NumberFormat.getInstance();
     String outdata = TestMiniMRWithDFS.readOutput(outPath,job);
     String outdata = TestMiniMRWithDFS.readOutput(outPath,job);
 
 
     assertEquals(expectedOutput.toString(),outdata);
     assertEquals(expectedOutput.toString(),outdata);
-    fs.delete(OUTPUT_DIR);
-    fs.delete(INPUT_DIR);
+    fs.delete(OUTPUT_DIR, true);
+    fs.delete(INPUT_DIR, true);
   }
   }
 
 
   /**
   /**

+ 2 - 2
src/test/org/apache/hadoop/mapred/TestKeyValueTextInputFormat.java

@@ -58,7 +58,7 @@ public class TestKeyValueTextInputFormat extends TestCase {
     LOG.info("seed = "+seed);
     LOG.info("seed = "+seed);
     Random random = new Random(seed);
     Random random = new Random(seed);
 
 
-    localFs.delete(workDir);
+    localFs.delete(workDir, true);
     job.setInputPath(workDir);
     job.setInputPath(workDir);
 
 
     // for a variety of lengths
     // for a variety of lengths
@@ -209,7 +209,7 @@ public class TestKeyValueTextInputFormat extends TestCase {
     JobConf job = new JobConf();
     JobConf job = new JobConf();
     CompressionCodec gzip = new GzipCodec();
     CompressionCodec gzip = new GzipCodec();
     ReflectionUtils.setConf(gzip, job);
     ReflectionUtils.setConf(gzip, job);
-    localFs.delete(workDir);
+    localFs.delete(workDir, true);
     writeFile(localFs, new Path(workDir, "part1.txt.gz"), gzip, 
     writeFile(localFs, new Path(workDir, "part1.txt.gz"), gzip, 
               "line-1\tthe quick\nline-2\tbrown\nline-3\tfox jumped\nline-4\tover\nline-5\t the lazy\nline-6\t dog\n");
               "line-1\tthe quick\nline-2\tbrown\nline-3\tfox jumped\nline-4\tover\nline-5\t the lazy\nline-6\t dog\n");
     writeFile(localFs, new Path(workDir, "part2.txt.gz"), gzip,
     writeFile(localFs, new Path(workDir, "part2.txt.gz"), gzip,

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMapOutputType.java

@@ -80,7 +80,7 @@ public class TestMapOutputType extends TestCase
     Path inDir = new Path(testdir, "in");
     Path inDir = new Path(testdir, "in");
     Path outDir = new Path(testdir, "out");
     Path outDir = new Path(testdir, "out");
     FileSystem fs = FileSystem.get(conf);
     FileSystem fs = FileSystem.get(conf);
-    fs.delete(testdir);
+    fs.delete(testdir, true);
     conf.setInt("io.sort.mb", 1);
     conf.setInt("io.sort.mb", 1);
     conf.setInputFormat(SequenceFileInputFormat.class);
     conf.setInputFormat(SequenceFileInputFormat.class);
     conf.setInputPath(inDir);
     conf.setInputPath(inDir);

+ 7 - 7
src/test/org/apache/hadoop/mapred/TestMapRed.java

@@ -292,7 +292,7 @@ public class TestMapRed extends TestCase {
     Path inDir = new Path(testdir, "in");
     Path inDir = new Path(testdir, "in");
     Path outDir = new Path(testdir, "out");
     Path outDir = new Path(testdir, "out");
     FileSystem fs = FileSystem.get(conf);
     FileSystem fs = FileSystem.get(conf);
-    fs.delete(testdir);
+    fs.delete(testdir, true);
     conf.setInputPath(inDir);
     conf.setInputPath(inDir);
     conf.setOutputPath(outDir);
     conf.setOutputPath(outDir);
     conf.setMapperClass(MyMap.class);
     conf.setMapperClass(MyMap.class);
@@ -332,7 +332,7 @@ public class TestMapRed extends TestCase {
                    rdr.isCompressed());
                    rdr.isCompressed());
       rdr.close();
       rdr.close();
     } finally {
     } finally {
-      fs.delete(testdir);
+      fs.delete(testdir, true);
     }
     }
   }
   }
     
     
@@ -415,7 +415,7 @@ public class TestMapRed extends TestCase {
     // file of random numbers.
     // file of random numbers.
     //
     //
     Path randomOuts = new Path(testdir, "genouts");
     Path randomOuts = new Path(testdir, "genouts");
-    fs.delete(randomOuts);
+    fs.delete(randomOuts, true);
 
 
 
 
     JobConf genJob = new JobConf(conf, TestMapRed.class);
     JobConf genJob = new JobConf(conf, TestMapRed.class);
@@ -462,7 +462,7 @@ public class TestMapRed extends TestCase {
     //
     //
     int intermediateReduces = 10;
     int intermediateReduces = 10;
     Path intermediateOuts = new Path(testdir, "intermediateouts");
     Path intermediateOuts = new Path(testdir, "intermediateouts");
-    fs.delete(intermediateOuts);
+    fs.delete(intermediateOuts, true);
     JobConf checkJob = new JobConf(conf, TestMapRed.class);
     JobConf checkJob = new JobConf(conf, TestMapRed.class);
     checkJob.setInputPath(randomOuts);
     checkJob.setInputPath(randomOuts);
     checkJob.setInputFormat(TextInputFormat.class);
     checkJob.setInputFormat(TextInputFormat.class);
@@ -485,7 +485,7 @@ public class TestMapRed extends TestCase {
     // all the files.
     // all the files.
     //
     //
     Path finalOuts = new Path(testdir, "finalouts");        
     Path finalOuts = new Path(testdir, "finalouts");        
-    fs.delete(finalOuts);
+    fs.delete(finalOuts, true);
     JobConf mergeJob = new JobConf(conf, TestMapRed.class);
     JobConf mergeJob = new JobConf(conf, TestMapRed.class);
     mergeJob.setInputPath(intermediateOuts);
     mergeJob.setInputPath(intermediateOuts);
     mergeJob.setInputFormat(SequenceFileInputFormat.class);
     mergeJob.setInputFormat(SequenceFileInputFormat.class);
@@ -556,7 +556,7 @@ public class TestMapRed extends TestCase {
     } finally {
     } finally {
       bw.close();
       bw.close();
     }
     }
-    fs.delete(testdir);
+    fs.delete(testdir, true);
   }
   }
 
 
   /**
   /**
@@ -591,7 +591,7 @@ public class TestMapRed extends TestCase {
       Path inDir = new Path(testdir, "in");
       Path inDir = new Path(testdir, "in");
       Path outDir = new Path(testdir, "out");
       Path outDir = new Path(testdir, "out");
       FileSystem fs = FileSystem.get(conf);
       FileSystem fs = FileSystem.get(conf);
-      fs.delete(testdir);
+      fs.delete(testdir, true);
       conf.setInt("io.sort.mb", 1);
       conf.setInt("io.sort.mb", 1);
       conf.setInputFormat(SequenceFileInputFormat.class);
       conf.setInputFormat(SequenceFileInputFormat.class);
       conf.setInputPath(inDir);
       conf.setInputPath(inDir);

+ 2 - 2
src/test/org/apache/hadoop/mapred/TestMiniMRClasspath.java

@@ -45,7 +45,7 @@ public class TestMiniMRClasspath extends TestCase {
     final Path inDir = new Path("/testing/wc/input");
     final Path inDir = new Path("/testing/wc/input");
     final Path outDir = new Path("/testing/wc/output");
     final Path outDir = new Path("/testing/wc/output");
     FileSystem fs = FileSystem.getNamed(fileSys, conf);
     FileSystem fs = FileSystem.getNamed(fileSys, conf);
-    fs.delete(outDir);
+    fs.delete(outDir, true);
     if (!fs.mkdirs(inDir)) {
     if (!fs.mkdirs(inDir)) {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     }
@@ -101,7 +101,7 @@ public class TestMiniMRClasspath extends TestCase {
     final Path inDir = new Path("/testing/ext/input");
     final Path inDir = new Path("/testing/ext/input");
     final Path outDir = new Path("/testing/ext/output");
     final Path outDir = new Path("/testing/ext/output");
     FileSystem fs = FileSystem.getNamed(fileSys, conf);
     FileSystem fs = FileSystem.getNamed(fileSys, conf);
-    fs.delete(outDir);
+    fs.delete(outDir, true);
     if (!fs.mkdirs(inDir)) {
     if (!fs.mkdirs(inDir)) {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     }

+ 2 - 2
src/test/org/apache/hadoop/mapred/TestMiniMRLocalFS.java

@@ -86,7 +86,7 @@ public class TestMiniMRLocalFS extends TestCase {
     Path testDir = new Path(TEST_ROOT_DIR + "/test_mini_mr_local");
     Path testDir = new Path(TEST_ROOT_DIR + "/test_mini_mr_local");
     Path outDir = new Path(testDir, "out");
     Path outDir = new Path(testDir, "out");
     System.out.println("testDir= " + testDir);
     System.out.println("testDir= " + testDir);
-    fileSys.delete(testDir);
+    fileSys.delete(testDir, true);
     
     
     job.setInputFormat(MyInputFormat.class);
     job.setInputFormat(MyInputFormat.class);
     job.setOutputFormat(MyOutputFormat.class);
     job.setOutputFormat(MyOutputFormat.class);
@@ -111,7 +111,7 @@ public class TestMiniMRLocalFS extends TestCase {
                               "eggs\t2\n" + 
                               "eggs\t2\n" + 
                               "finagle the agent\t3\n"), result);
                               "finagle the agent\t3\n"), result);
     } finally {
     } finally {
-      fileSys.delete(testDir);
+      fileSys.delete(testDir, true);
     }
     }
     
     
   }
   }

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMiniMRMapRedDebugScript.java

@@ -115,7 +115,7 @@ public class TestMiniMRMapRedDebugScript extends TestCase {
     // set up the input file system and write input text.
     // set up the input file system and write input text.
     FileSystem inFs = inDir.getFileSystem(conf);
     FileSystem inFs = inDir.getFileSystem(conf);
     FileSystem outFs = outDir.getFileSystem(conf);
     FileSystem outFs = outDir.getFileSystem(conf);
-    outFs.delete(outDir);
+    outFs.delete(outDir, true);
     if (!inFs.mkdirs(inDir)) {
     if (!inFs.mkdirs(inDir)) {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     }

+ 4 - 4
src/test/org/apache/hadoop/mapred/TestMiniMRTaskTempDir.java

@@ -96,7 +96,7 @@ public class TestMiniMRTaskTempDir extends TestCase {
     // set up the input file system and write input text.
     // set up the input file system and write input text.
     FileSystem inFs = inDir.getFileSystem(conf);
     FileSystem inFs = inDir.getFileSystem(conf);
     FileSystem outFs = outDir.getFileSystem(conf);
     FileSystem outFs = outDir.getFileSystem(conf);
-    outFs.delete(outDir);
+    outFs.delete(outDir, true);
     if (!inFs.mkdirs(inDir)) {
     if (!inFs.mkdirs(inDir)) {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     }
@@ -122,17 +122,17 @@ public class TestMiniMRTaskTempDir extends TestCase {
     // Launch job with default option for temp dir. 
     // Launch job with default option for temp dir. 
     // i.e. temp dir is ./tmp 
     // i.e. temp dir is ./tmp 
     JobClient.runJob(conf);
     JobClient.runJob(conf);
-    outFs.delete(outDir);
+    outFs.delete(outDir, true);
 
 
     // Launch job by giving relative path to temp dir.
     // Launch job by giving relative path to temp dir.
     conf.set("mapred.child.tmp", "../temp");
     conf.set("mapred.child.tmp", "../temp");
     JobClient.runJob(conf);
     JobClient.runJob(conf);
-    outFs.delete(outDir);
+    outFs.delete(outDir, true);
 
 
     // Launch job by giving absolute path to temp dir
     // Launch job by giving absolute path to temp dir
     conf.set("mapred.child.tmp", "/tmp");
     conf.set("mapred.child.tmp", "/tmp");
     JobClient.runJob(conf);
     JobClient.runJob(conf);
-    outFs.delete(outDir);
+    outFs.delete(outDir, true);
   }
   }
 
 
   /**
   /**

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java

@@ -59,7 +59,7 @@ public class TestMiniMRWithDFS extends TestCase {
                                            int numReduces) throws IOException {
                                            int numReduces) throws IOException {
     FileSystem inFs = inDir.getFileSystem(conf);
     FileSystem inFs = inDir.getFileSystem(conf);
     FileSystem outFs = outDir.getFileSystem(conf);
     FileSystem outFs = outDir.getFileSystem(conf);
-    outFs.delete(outDir);
+    outFs.delete(outDir, true);
     if (!inFs.mkdirs(inDir)) {
     if (!inFs.mkdirs(inDir)) {
       throw new IOException("Mkdirs failed to create " + inDir.toString());
       throw new IOException("Mkdirs failed to create " + inDir.toString());
     }
     }

+ 2 - 2
src/test/org/apache/hadoop/mapred/TestMultiFileInputFormat.java

@@ -57,7 +57,7 @@ public class TestMultiFileInputFormat extends TestCase{
   private Path initFiles(FileSystem fs, int numFiles, int numBytes) throws IOException{
   private Path initFiles(FileSystem fs, int numFiles, int numBytes) throws IOException{
     Path dir = new Path(System.getProperty("test.build.data",".") + "/mapred");
     Path dir = new Path(System.getProperty("test.build.data",".") + "/mapred");
     Path multiFileDir = new Path(dir, "test.multifile");
     Path multiFileDir = new Path(dir, "test.multifile");
-    fs.delete(multiFileDir);
+    fs.delete(multiFileDir, true);
     fs.mkdirs(multiFileDir);
     fs.mkdirs(multiFileDir);
     LOG.info("Creating " + numFiles + " file(s) in " + multiFileDir);
     LOG.info("Creating " + numFiles + " file(s) in " + multiFileDir);
     for(int i=0; i<numFiles ;i++) {
     for(int i=0; i<numFiles ;i++) {
@@ -119,7 +119,7 @@ public class TestMultiFileInputFormat extends TestCase{
         }
         }
       }
       }
       assertEquals(bits.cardinality(), numFiles);
       assertEquals(bits.cardinality(), numFiles);
-      fs.delete(dir);
+      fs.delete(dir, true);
     }
     }
     LOG.info("Test Finished");
     LOG.info("Test Finished");
   }
   }

+ 2 - 2
src/test/org/apache/hadoop/mapred/TestRackAwareTaskPlacement.java

@@ -96,7 +96,7 @@ public class TestRackAwareTaskPlacement extends TestCase {
       mr = new MiniMRCluster(taskTrackers, namenode, 1, rack2, hosts4);
       mr = new MiniMRCluster(taskTrackers, namenode, 1, rack2, hosts4);
       JobConf jobConf = mr.createJobConf();
       JobConf jobConf = mr.createJobConf();
       if (fileSys.exists(outputPath)) {
       if (fileSys.exists(outputPath)) {
-        fileSys.delete(outputPath);
+        fileSys.delete(outputPath, true);
       }
       }
       /* The job is configured with three maps since there are three 
       /* The job is configured with three maps since there are three 
        * (non-splittable) files. On rack2, there are two files and both
        * (non-splittable) files. On rack2, there are two files and both
@@ -118,7 +118,7 @@ public class TestRackAwareTaskPlacement extends TestCase {
       jobConf = mr.createJobConf();
       jobConf = mr.createJobConf();
       fileSys = dfs.getFileSystem();
       fileSys = dfs.getFileSystem();
       if (fileSys.exists(outputPath)) {
       if (fileSys.exists(outputPath)) {
-        fileSys.delete(outputPath);
+        fileSys.delete(outputPath, true);
       }
       }
       /* The job is configured with three maps since there are three 
       /* The job is configured with three maps since there are three 
        * (non-splittable) files. On rack1, because of the way in which repl
        * (non-splittable) files. On rack1, because of the way in which repl

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestSequenceFileAsBinaryInputFormat.java

@@ -41,7 +41,7 @@ public class TestSequenceFileAsBinaryInputFormat extends TestCase {
     long seed = r.nextLong();
     long seed = r.nextLong();
     r.setSeed(seed);
     r.setSeed(seed);
 
 
-    fs.delete(dir);
+    fs.delete(dir, true);
     job.setInputPath(dir);
     job.setInputPath(dir);
 
 
     Text tkey = new Text();
     Text tkey = new Text();

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestSequenceFileAsTextInputFormat.java

@@ -46,7 +46,7 @@ public class TestSequenceFileAsTextInputFormat extends TestCase {
     //LOG.info("seed = "+seed);
     //LOG.info("seed = "+seed);
     Random random = new Random(seed);
     Random random = new Random(seed);
 
 
-    fs.delete(dir);
+    fs.delete(dir, true);
 
 
     job.setInputPath(dir);
     job.setInputPath(dir);
 
 

+ 6 - 6
src/test/org/apache/hadoop/mapred/TestSequenceFileInputFilter.java

@@ -106,7 +106,7 @@ public class TestSequenceFileInputFilter extends TestCase {
     SequenceFileInputFilter.RegexFilter.setPattern(job, "\\A10*");
     SequenceFileInputFilter.RegexFilter.setPattern(job, "\\A10*");
     
     
     // clean input dir
     // clean input dir
-    fs.delete(inDir);
+    fs.delete(inDir, true);
   
   
     // for a variety of lengths
     // for a variety of lengths
     for (int length = 1; length < MAX_LENGTH;
     for (int length = 1; length < MAX_LENGTH;
@@ -118,7 +118,7 @@ public class TestSequenceFileInputFilter extends TestCase {
     }
     }
     
     
     // clean up
     // clean up
-    fs.delete(inDir);
+    fs.delete(inDir, true);
   }
   }
 
 
   public void testPercentFilter() throws Exception {
   public void testPercentFilter() throws Exception {
@@ -129,7 +129,7 @@ public class TestSequenceFileInputFilter extends TestCase {
     SequenceFileInputFilter.PercentFilter.setFrequency(job, 1000);
     SequenceFileInputFilter.PercentFilter.setFrequency(job, 1000);
       
       
     // clean input dir
     // clean input dir
-    fs.delete(inDir);
+    fs.delete(inDir, true);
     
     
     // for a variety of lengths
     // for a variety of lengths
     for (int length = 0; length < MAX_LENGTH;
     for (int length = 0; length < MAX_LENGTH;
@@ -145,7 +145,7 @@ public class TestSequenceFileInputFilter extends TestCase {
     }
     }
       
       
     // clean up
     // clean up
-    fs.delete(inDir);
+    fs.delete(inDir, true);
   }
   }
   
   
   public void testMD5Filter() throws Exception {
   public void testMD5Filter() throws Exception {
@@ -156,7 +156,7 @@ public class TestSequenceFileInputFilter extends TestCase {
     SequenceFileInputFilter.MD5Filter.setFrequency(job, 1000);
     SequenceFileInputFilter.MD5Filter.setFrequency(job, 1000);
       
       
     // clean input dir
     // clean input dir
-    fs.delete(inDir);
+    fs.delete(inDir, true);
     
     
     // for a variety of lengths
     // for a variety of lengths
     for (int length = 0; length < MAX_LENGTH;
     for (int length = 0; length < MAX_LENGTH;
@@ -166,7 +166,7 @@ public class TestSequenceFileInputFilter extends TestCase {
       LOG.info("Accepted "+countRecords(0)+" records");
       LOG.info("Accepted "+countRecords(0)+" records");
     }
     }
     // clean up
     // clean up
-    fs.delete(inDir);
+    fs.delete(inDir, true);
   }
   }
 
 
   public static void main(String[] args) throws Exception {
   public static void main(String[] args) throws Exception {

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java

@@ -46,7 +46,7 @@ public class TestSequenceFileInputFormat extends TestCase {
     //LOG.info("seed = "+seed);
     //LOG.info("seed = "+seed);
     Random random = new Random(seed);
     Random random = new Random(seed);
 
 
-    fs.delete(dir);
+    fs.delete(dir, true);
 
 
     job.setInputPath(dir);
     job.setInputPath(dir);
 
 

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java

@@ -53,7 +53,7 @@ public class TestSpecialCharactersInOutputPath extends TestCase {
     final Path inDir = new Path("/testing/input");
     final Path inDir = new Path("/testing/input");
     final Path outDir = new Path("/testing/output");
     final Path outDir = new Path("/testing/output");
     FileSystem fs = FileSystem.getNamed(fileSys, conf);
     FileSystem fs = FileSystem.getNamed(fileSys, conf);
-    fs.delete(outDir);
+    fs.delete(outDir, true);
     if (!fs.mkdirs(inDir)) {
     if (!fs.mkdirs(inDir)) {
       LOG.warn("Can't create " + inDir);
       LOG.warn("Can't create " + inDir);
       return false;
       return false;

Some files were not shown because too many files changed in this diff