Procházet zdrojové kódy

HADOOP-4940. Remove a deprecated FileSystem.delete(Path f). (Enis Soztutar via szetszwo)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@732833 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze před 16 roky
rodič
revize
3ee0d67e70
28 změnil soubory, kde provedl 118 přidání a 124 odebrání
  1. 3 0
      CHANGES.txt
  2. 2 2
      src/benchmarks/gridmix2/src/java/org/apache/hadoop/mapred/GridMixRunner.java
  3. 4 3
      src/c++/libhdfs/hdfs.c
  4. 4 2
      src/c++/libhdfs/hdfs.h
  5. 4 4
      src/c++/libhdfs/hdfs_test.c
  6. 1 1
      src/contrib/data_join/src/java/org/apache/hadoop/contrib/utils/join/DataJoinJob.java
  7. 1 1
      src/contrib/failmon/src/java/org/apache/hadoop/contrib/failmon/HDFSMerger.java
  8. 1 1
      src/contrib/fuse-dfs/src/fuse_trash.c
  9. 3 3
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/FileSystemDirectory.java
  10. 3 3
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/ShardWriter.java
  11. 7 7
      src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestDistributionPolicy.java
  12. 4 4
      src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestIndexUpdater.java
  13. 11 4
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java
  14. 3 3
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java
  15. 11 4
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java
  16. 0 4
      src/core/org/apache/hadoop/fs/FileSystem.java
  17. 0 5
      src/core/org/apache/hadoop/fs/FilterFileSystem.java
  18. 11 10
      src/core/org/apache/hadoop/fs/RawLocalFileSystem.java
  19. 0 7
      src/core/org/apache/hadoop/fs/ftp/FTPFileSystem.java
  20. 28 10
      src/core/org/apache/hadoop/fs/kfs/KosmosFileSystem.java
  21. 1 7
      src/core/org/apache/hadoop/fs/s3/S3FileSystem.java
  22. 0 6
      src/core/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
  23. 0 8
      src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java
  24. 0 9
      src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java
  25. 10 7
      src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  26. 2 2
      src/mapred/org/apache/hadoop/mapred/JobConf.java
  27. 1 1
      src/test/org/apache/hadoop/fs/TestLocalDirAllocator.java
  28. 3 6
      src/test/org/apache/hadoop/mapred/lib/TestTotalOrderPartitioner.java

+ 3 - 0
CHANGES.txt

@@ -13,6 +13,9 @@ Trunk (unreleased changes)
     HADOOP-4648. Remove obsolete, deprecated InMemoryFileSystem and
     HADOOP-4648. Remove obsolete, deprecated InMemoryFileSystem and
     ChecksumDistributedFileSystem.  (cdouglas via szetszwo)
     ChecksumDistributedFileSystem.  (cdouglas via szetszwo)
 
 
+    HADOOP-4940. Remove a deprecated method FileSystem.delete(Path f).  (Enis
+    Soztutar via szetszwo)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HADOOP-4268. Change fsck to use ClientProtocol methods so that the
     HADOOP-4268. Change fsck to use ClientProtocol methods so that the

+ 2 - 2
src/benchmarks/gridmix2/src/java/org/apache/hadoop/mapred/GridMixRunner.java

@@ -255,7 +255,7 @@ public class GridMixRunner {
           String[] args = sb.toString().split(" ");
           String[] args = sb.toString().split(" ");
 
 
           try {
           try {
-            fs.delete(outfile);
+            fs.delete(outfile, true);
           } catch (IOException ex) {
           } catch (IOException ex) {
             System.out.println(ex.toString());
             System.out.println(ex.toString());
           }
           }
@@ -377,7 +377,7 @@ public class GridMixRunner {
   private static void clearDir(String dir) {
   private static void clearDir(String dir) {
     try {
     try {
       Path outfile = new Path(dir);
       Path outfile = new Path(dir);
-      fs.delete(outfile);
+      fs.delete(outfile, true);
     } catch (IOException ex) {
     } catch (IOException ex) {
       ex.printStackTrace();
       ex.printStackTrace();
       System.out.println("delete file error:");
       System.out.println("delete file error:");

+ 4 - 3
src/c++/libhdfs/hdfs.c

@@ -1090,7 +1090,7 @@ int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst)
 
 
 
 
 
 
-int hdfsDelete(hdfsFS fs, const char* path)
+int hdfsDelete(hdfsFS fs, const char* path, int recursive)
 {
 {
     // JAVA EQUIVALENT:
     // JAVA EQUIVALENT:
     //  File f = new File(path);
     //  File f = new File(path);
@@ -1114,9 +1114,10 @@ int hdfsDelete(hdfsFS fs, const char* path)
     //Delete the file
     //Delete the file
     jvalue jVal;
     jvalue jVal;
     jthrowable jExc = NULL;
     jthrowable jExc = NULL;
+    jboolean jRecursive = recursive;
     if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
     if (invokeMethod(env, &jVal, &jExc, INSTANCE, jFS, HADOOP_FS,
-                     "delete", "(Lorg/apache/hadoop/fs/Path;)Z",
-                     jPath) != 0) {
+                     "delete", "(Lorg/apache/hadoop/fs/Path;Z)Z",
+                     jPath, jRecursive) != 0) {
         errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
         errno = errnoFromException(jExc, env, "org.apache.hadoop.fs."
                                    "FileSystem::delete");
                                    "FileSystem::delete");
         return -1;
         return -1;

+ 4 - 2
src/c++/libhdfs/hdfs.h

@@ -268,10 +268,12 @@ extern  "C" {
      * hdfsDelete - Delete file. 
      * hdfsDelete - Delete file. 
      * @param fs The configured filesystem handle.
      * @param fs The configured filesystem handle.
      * @param path The path of the file. 
      * @param path The path of the file. 
+     * @param recursive if path is a directory and set to 
+     * non-zero, the directory is deleted else throws an exception. In
+     * case of a file the recursive argument is irrelevant.
      * @return Returns 0 on success, -1 on error. 
      * @return Returns 0 on success, -1 on error. 
      */
      */
-    int hdfsDelete(hdfsFS fs, const char* path);
-
+    int hdfsDelete(hdfsFS fs, const char* path, int recursive);
 
 
     /**
     /**
      * hdfsRename - Rename file. 
      * hdfsRename - Rename file. 

+ 4 - 4
src/c++/libhdfs/hdfs_test.c

@@ -305,13 +305,13 @@ int main(int argc, char **argv) {
         hdfsFreeFileInfo(finfo, 1);
         hdfsFreeFileInfo(finfo, 1);
 
 
         // Clean up
         // Clean up
-        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, newDirectory)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, newDirectory, 1)) ? "Failed!" : "Success!"));
         totalResult += result;
         totalResult += result;
-        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, srcPath)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(fs, srcPath, 1)) ? "Failed!" : "Success!"));
         totalResult += result;
         totalResult += result;
-        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(lfs, srcPath)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(lfs, srcPath, 1)) ? "Failed!" : "Success!"));
         totalResult += result;
         totalResult += result;
-        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(lfs, dstPath)) ? "Failed!" : "Success!"));
+        fprintf(stderr, "hdfsDelete: %s\n", ((result = hdfsDelete(lfs, dstPath, 1)) ? "Failed!" : "Success!"));
         totalResult += result;
         totalResult += result;
         fprintf(stderr, "hdfsExists: %s\n", ((result = hdfsExists(fs, newDirectory)) ? "Success!" : "Failed!"));
         fprintf(stderr, "hdfsExists: %s\n", ((result = hdfsExists(fs, newDirectory)) ? "Success!" : "Failed!"));
         totalResult += (result ? 0 : 1);
         totalResult += (result ? 0 : 1);

+ 1 - 1
src/contrib/data_join/src/java/org/apache/hadoop/contrib/utils/join/DataJoinJob.java

@@ -93,7 +93,7 @@ public class DataJoinJob {
     job.setJobName("DataJoinJob: " + jobName);
     job.setJobName("DataJoinJob: " + jobName);
 
 
     FileSystem fs = FileSystem.get(defaults);
     FileSystem fs = FileSystem.get(defaults);
-    fs.delete(new Path(outputDir));
+    fs.delete(new Path(outputDir), true);
     FileInputFormat.setInputPaths(job, inputDir);
     FileInputFormat.setInputPaths(job, inputDir);
 
 
     job.setInputFormat(inputFormat);
     job.setInputFormat(inputFormat);

+ 1 - 1
src/contrib/failmon/src/java/org/apache/hadoop/contrib/failmon/HDFSMerger.java

@@ -105,7 +105,7 @@ public class HDFSMerger {
     
     
     for (FileStatus fstatus : inputFiles) {
     for (FileStatus fstatus : inputFiles) {
       appendFile(fstatus.getPath());
       appendFile(fstatus.getPath());
-      hdfs.delete(fstatus.getPath());
+      hdfs.delete(fstatus.getPath(), true);
     }
     }
 
 
     outputFile.close();
     outputFile.close();

+ 1 - 1
src/contrib/fuse-dfs/src/fuse_trash.c

@@ -116,7 +116,7 @@ int hdfsDeleteWithTrash(hdfsFS userFS, const char *path, int useTrash) {
     return ret;
     return ret;
   }
   }
 
 
-  if (hdfsDelete(userFS, path)) {
+  if (hdfsDelete(userFS, path, 1)) {
     syslog(LOG_ERR,"ERROR: hdfs trying to delete the file %s",path);
     syslog(LOG_ERR,"ERROR: hdfs trying to delete the file %s",path);
     return -EIO;
     return -EIO;
   }
   }

+ 3 - 3
src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/FileSystemDirectory.java

@@ -98,7 +98,7 @@ public class FileSystemDirectory extends Directory {
     FileStatus[] fileStatus =
     FileStatus[] fileStatus =
         fs.listStatus(directory, LuceneIndexFileNameFilter.getFilter());
         fs.listStatus(directory, LuceneIndexFileNameFilter.getFilter());
     for (int i = 0; i < fileStatus.length; i++) {
     for (int i = 0; i < fileStatus.length; i++) {
-      if (!fs.delete(fileStatus[i].getPath())) {
+      if (!fs.delete(fileStatus[i].getPath(), true)) {
         throw new IOException("Cannot delete index file "
         throw new IOException("Cannot delete index file "
             + fileStatus[i].getPath());
             + fileStatus[i].getPath());
       }
       }
@@ -150,7 +150,7 @@ public class FileSystemDirectory extends Directory {
    * @see org.apache.lucene.store.Directory#deleteFile(java.lang.String)
    * @see org.apache.lucene.store.Directory#deleteFile(java.lang.String)
    */
    */
   public void deleteFile(String name) throws IOException {
   public void deleteFile(String name) throws IOException {
-    if (!fs.delete(new Path(directory, name))) {
+    if (!fs.delete(new Path(directory, name), true)) {
       throw new IOException("Cannot delete index file " + name);
       throw new IOException("Cannot delete index file " + name);
     }
     }
   }
   }
@@ -167,7 +167,7 @@ public class FileSystemDirectory extends Directory {
    */
    */
   public IndexOutput createOutput(String name) throws IOException {
   public IndexOutput createOutput(String name) throws IOException {
     Path file = new Path(directory, name);
     Path file = new Path(directory, name);
-    if (fs.exists(file) && !fs.delete(file)) {
+    if (fs.exists(file) && !fs.delete(file, true)) {
       // delete the existing one if applicable
       // delete the existing one if applicable
       throw new IOException("Cannot overwrite index file " + file);
       throw new IOException("Cannot overwrite index file " + file);
     }
     }

+ 3 - 3
src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/ShardWriter.java

@@ -176,7 +176,7 @@ public class ShardWriter {
     for (int i = 0; i < fileStatus.length; i++) {
     for (int i = 0; i < fileStatus.length; i++) {
       Path path = fileStatus[i].getPath();
       Path path = fileStatus[i].getPath();
       if (startGen < LuceneUtil.generationFromSegmentsFileName(path.getName())) {
       if (startGen < LuceneUtil.generationFromSegmentsFileName(path.getName())) {
-        fs.delete(path);
+        fs.delete(path, true);
       }
       }
     }
     }
 
 
@@ -184,7 +184,7 @@ public class ShardWriter {
     // but not segments.gen, and segments.gen will be overwritten anyway.
     // but not segments.gen, and segments.gen will be overwritten anyway.
     Path segmentsGenFile = new Path(LuceneUtil.IndexFileNames.SEGMENTS_GEN);
     Path segmentsGenFile = new Path(LuceneUtil.IndexFileNames.SEGMENTS_GEN);
     if (fs.exists(segmentsGenFile)) {
     if (fs.exists(segmentsGenFile)) {
-      fs.delete(segmentsGenFile);
+      fs.delete(segmentsGenFile, true);
     }
     }
   }
   }
 
 
@@ -226,7 +226,7 @@ public class ShardWriter {
       }
       }
     } finally {
     } finally {
       // finally delete the temp dir (files should have been deleted)
       // finally delete the temp dir (files should have been deleted)
-      localFs.delete(temp);
+      localFs.delete(temp, true);
     }
     }
   }
   }
 
 

+ 7 - 7
src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestDistributionPolicy.java

@@ -88,21 +88,21 @@ public class TestDistributionPolicy extends TestCase {
 
 
       fs = dfsCluster.getFileSystem();
       fs = dfsCluster.getFileSystem();
       if (fs.exists(inputPath)) {
       if (fs.exists(inputPath)) {
-        fs.delete(inputPath);
+        fs.delete(inputPath, true);
       }
       }
       fs.copyFromLocalFile(localInputPath, inputPath);
       fs.copyFromLocalFile(localInputPath, inputPath);
       if (fs.exists(updatePath)) {
       if (fs.exists(updatePath)) {
-        fs.delete(updatePath);
+        fs.delete(updatePath, true);
       }
       }
       fs.copyFromLocalFile(localUpdatePath, updatePath);
       fs.copyFromLocalFile(localUpdatePath, updatePath);
 
 
       if (fs.exists(outputPath)) {
       if (fs.exists(outputPath)) {
         // do not create, mapred will create
         // do not create, mapred will create
-        fs.delete(outputPath);
+        fs.delete(outputPath, true);
       }
       }
 
 
       if (fs.exists(indexPath)) {
       if (fs.exists(indexPath)) {
-        fs.delete(indexPath);
+        fs.delete(indexPath, true);
       }
       }
 
 
       mrCluster =
       mrCluster =
@@ -156,7 +156,7 @@ public class TestDistributionPolicy extends TestCase {
     onetest();
     onetest();
 
 
     if (fs.exists(indexPath)) {
     if (fs.exists(indexPath)) {
-      fs.delete(indexPath);
+      fs.delete(indexPath, true);
     }
     }
 
 
     // test round-robin distribution policy
     // test round-robin distribution policy
@@ -177,7 +177,7 @@ public class TestDistributionPolicy extends TestCase {
     }
     }
 
 
     if (fs.exists(outputPath)) {
     if (fs.exists(outputPath)) {
-      fs.delete(outputPath);
+      fs.delete(outputPath, true);
     }
     }
 
 
     IIndexUpdater updater = new IndexUpdater();
     IIndexUpdater updater = new IndexUpdater();
@@ -185,7 +185,7 @@ public class TestDistributionPolicy extends TestCase {
         shards);
         shards);
 
 
     if (fs.exists(outputPath)) {
     if (fs.exists(outputPath)) {
-      fs.delete(outputPath);
+      fs.delete(outputPath, true);
     }
     }
 
 
     // delete docs w/ even docids, update docs w/ odd docids
     // delete docs w/ even docids, update docs w/ odd docids

+ 4 - 4
src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestIndexUpdater.java

@@ -88,17 +88,17 @@ public class TestIndexUpdater extends TestCase {
 
 
       fs = dfsCluster.getFileSystem();
       fs = dfsCluster.getFileSystem();
       if (fs.exists(inputPath)) {
       if (fs.exists(inputPath)) {
-        fs.delete(inputPath);
+        fs.delete(inputPath, true);
       }
       }
       fs.copyFromLocalFile(localInputPath, inputPath);
       fs.copyFromLocalFile(localInputPath, inputPath);
 
 
       if (fs.exists(outputPath)) {
       if (fs.exists(outputPath)) {
         // do not create, mapred will create
         // do not create, mapred will create
-        fs.delete(outputPath);
+        fs.delete(outputPath, true);
       }
       }
 
 
       if (fs.exists(indexPath)) {
       if (fs.exists(indexPath)) {
-        fs.delete(indexPath);
+        fs.delete(indexPath, true);
       }
       }
 
 
       mrCluster =
       mrCluster =
@@ -157,7 +157,7 @@ public class TestIndexUpdater extends TestCase {
 
 
     for (int i = 0; i < numRuns; i++) {
     for (int i = 0; i < numRuns; i++) {
       if (fs.exists(outputPath)) {
       if (fs.exists(outputPath)) {
-        fs.delete(outputPath);
+        fs.delete(outputPath, true);
       }
       }
 
 
       Shard[] shards = new Shard[initNumShards + i];
       Shard[] shards = new Shard[initNumShards + i];

+ 11 - 4
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java

@@ -18,15 +18,22 @@
 
 
 package org.apache.hadoop.streaming;
 package org.apache.hadoop.streaming;
 
 
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+
 import junit.framework.TestCase;
 import junit.framework.TestCase;
-import java.io.*;
-import java.util.*;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.OutputLogFilter;
 /**
 /**
  * This test case tests the symlink creation
  * This test case tests the symlink creation
  * utility provided by distributed caching 
  * utility provided by distributed caching 
@@ -83,7 +90,7 @@ public class TestMultipleCachefiles extends TestCase
           "-cacheFile", "hdfs://"+fileSys.getName()+CACHE_FILE_2 + "#" + mapString2
           "-cacheFile", "hdfs://"+fileSys.getName()+CACHE_FILE_2 + "#" + mapString2
         };
         };
 
 
-        fileSys.delete(new Path(OUTPUT_DIR));
+        fileSys.delete(new Path(OUTPUT_DIR), true);
         
         
         DataOutputStream file = fileSys.create(new Path(INPUT_FILE));
         DataOutputStream file = fileSys.create(new Path(INPUT_FILE));
         file.writeBytes(mapString + "\n");
         file.writeBytes(mapString + "\n");

+ 3 - 3
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamedMerge.java

@@ -31,10 +31,10 @@ import java.util.Arrays;
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
@@ -71,7 +71,7 @@ public class TestStreamedMerge extends TestCase {
   }
   }
 
 
   String createInputs(boolean tag) throws IOException {
   String createInputs(boolean tag) throws IOException {
-    fs_.delete(new Path("/input/"));
+    fs_.delete(new Path("/input/"), true);
 
 
     // i18n() replaces some ASCII with multibyte UTF-8 chars
     // i18n() replaces some ASCII with multibyte UTF-8 chars
     addInput("/input/part-00", i18n("k1\tv1\n" + "k3\tv5\n"));
     addInput("/input/part-00", i18n("k1\tv1\n" + "k3\tv5\n"));
@@ -192,7 +192,7 @@ public class TestStreamedMerge extends TestCase {
     public void run() {
     public void run() {
       try {
       try {
         in_ = connectInputStream();
         in_ = connectInputStream();
-        LineReader lineReader = new LineReader((InputStream)in_, conf_);
+        LineReader lineReader = new LineReader(in_, conf_);
         Text line = new Text();
         Text line = new Text();
         while (lineReader.readLine(line) > 0) {
         while (lineReader.readLine(line) > 0) {
           buf_.append(line.toString());
           buf_.append(line.toString());

+ 11 - 4
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java

@@ -18,15 +18,22 @@
 
 
 package org.apache.hadoop.streaming;
 package org.apache.hadoop.streaming;
 
 
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+
 import junit.framework.TestCase;
 import junit.framework.TestCase;
-import java.io.*;
-import java.util.*;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.OutputLogFilter;
 /**
 /**
  * This test case tests the symlink creation
  * This test case tests the symlink creation
  * utility provided by distributed caching 
  * utility provided by distributed caching 
@@ -79,7 +86,7 @@ public class TestSymLink extends TestCase
           "-cacheFile", "hdfs://"+fileSys.getName()+CACHE_FILE + "#testlink"
           "-cacheFile", "hdfs://"+fileSys.getName()+CACHE_FILE + "#testlink"
         };
         };
 
 
-        fileSys.delete(new Path(OUTPUT_DIR));
+        fileSys.delete(new Path(OUTPUT_DIR), true);
         
         
         DataOutputStream file = fileSys.create(new Path(INPUT_FILE));
         DataOutputStream file = fileSys.create(new Path(INPUT_FILE));
         file.writeBytes(mapString);
         file.writeBytes(mapString);

+ 0 - 4
src/core/org/apache/hadoop/fs/FileSystem.java

@@ -561,10 +561,6 @@ 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. */
-  /** @deprecated Use delete(Path, boolean) instead */ @Deprecated 
-  public abstract boolean delete(Path f) throws IOException;
-  
   /** Delete a file.
   /** Delete a file.
    *
    *
    * @param f the path to delete.
    * @param f the path to delete.

+ 0 - 5
src/core/org/apache/hadoop/fs/FilterFileSystem.java

@@ -138,11 +138,6 @@ public class FilterFileSystem extends FileSystem {
     return fs.rename(src, dst);
     return fs.rename(src, dst);
   }
   }
   
   
-  /** Delete a file */@Deprecated
-  public boolean delete(Path f) throws IOException {
-    return delete(f, true);
-  }
-  
   /** Delete a file */
   /** Delete a file */
   public boolean delete(Path f, boolean recursive) throws IOException {
   public boolean delete(Path f, boolean recursive) throws IOException {
     return fs.delete(f, recursive);
     return fs.delete(f, recursive);

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

@@ -18,17 +18,23 @@
 
 
 package org.apache.hadoop.fs;
 package org.apache.hadoop.fs;
 
 
-import java.io.*;
+import java.io.BufferedOutputStream;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
 import java.net.URI;
 import java.net.URI;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileLock;
-import java.util.*;
+import java.util.StringTokenizer;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
 
 
 /****************************************************************
 /****************************************************************
  * Implement the FileSystem API for the raw local filesystem.
  * Implement the FileSystem API for the raw local filesystem.
@@ -252,11 +258,6 @@ 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 {
-    return delete(p, true);
-  }
-  
   public boolean delete(Path p, boolean recursive) throws IOException {
   public boolean delete(Path p, boolean recursive) throws IOException {
     File f = pathToFile(p);
     File f = pathToFile(p);
     if (f.isFile()) {
     if (f.isFile()) {

+ 0 - 7
src/core/org/apache/hadoop/fs/ftp/FTPFileSystem.java

@@ -264,13 +264,6 @@ public class FTPFileSystem extends FileSystem {
     }
     }
   }
   }
 
 
-  /** @deprecated Use delete(Path, boolean) instead */
-  @Override
-  @Deprecated
-  public boolean delete(Path file) throws IOException {
-    return delete(file, false);
-  }
-
   @Override
   @Override
   public boolean delete(Path file, boolean recursive) throws IOException {
   public boolean delete(Path file, boolean recursive) throws IOException {
     FTPClient client = connect();
     FTPClient client = connect();

+ 28 - 10
src/core/org/apache/hadoop/fs/kfs/KosmosFileSystem.java

@@ -20,19 +20,20 @@
 
 
 package org.apache.hadoop.fs.kfs;
 package org.apache.hadoop.fs.kfs;
 
 
-import java.io.*;
-import java.net.*;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.fs.BlockLocation;
 
 
 /**
 /**
  * A FileSystem backed by KFS.
  * A FileSystem backed by KFS.
@@ -54,10 +55,12 @@ public class KosmosFileSystem extends FileSystem {
         this.kfsImpl = fsimpl;
         this.kfsImpl = fsimpl;
     }
     }
 
 
+    @Override
     public URI getUri() {
     public URI getUri() {
 	return uri;
 	return uri;
     }
     }
 
 
+    @Override
     public void initialize(URI uri, Configuration conf) throws IOException {
     public void initialize(URI uri, Configuration conf) throws IOException {
         try {
         try {
 	    if (kfsImpl == null) {
 	    if (kfsImpl == null) {
@@ -82,15 +85,18 @@ public class KosmosFileSystem extends FileSystem {
         }
         }
     }
     }
 
 
+    @Override
     @Deprecated
     @Deprecated
     public String getName() {
     public String getName() {
 	return getUri().toString();
 	return getUri().toString();
     }
     }
 
 
+    @Override
     public Path getWorkingDirectory() {
     public Path getWorkingDirectory() {
 	return workingDir;
 	return workingDir;
     }
     }
 
 
+    @Override
     public void setWorkingDirectory(Path dir) {
     public void setWorkingDirectory(Path dir) {
 	workingDir = makeAbsolute(dir);
 	workingDir = makeAbsolute(dir);
     }
     }
@@ -102,6 +108,7 @@ public class KosmosFileSystem extends FileSystem {
 	return new Path(workingDir, path);
 	return new Path(workingDir, path);
     }
     }
 
 
+    @Override
     public boolean mkdirs(Path path, FsPermission permission
     public boolean mkdirs(Path path, FsPermission permission
         ) throws IOException {
         ) throws IOException {
 	Path absolute = makeAbsolute(path);
 	Path absolute = makeAbsolute(path);
@@ -116,6 +123,7 @@ public class KosmosFileSystem extends FileSystem {
 	return res == 0;
 	return res == 0;
     }
     }
 
 
+    @Override
     @Deprecated
     @Deprecated
     public boolean isDirectory(Path path) throws IOException {
     public boolean isDirectory(Path path) throws IOException {
 	Path absolute = makeAbsolute(path);
 	Path absolute = makeAbsolute(path);
@@ -126,6 +134,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.isDirectory(srep);
         return kfsImpl.isDirectory(srep);
     }
     }
 
 
+    @Override
     @Deprecated
     @Deprecated
     public boolean isFile(Path path) throws IOException {
     public boolean isFile(Path path) throws IOException {
 	Path absolute = makeAbsolute(path);
 	Path absolute = makeAbsolute(path);
@@ -133,6 +142,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.isFile(srep);
         return kfsImpl.isFile(srep);
     }
     }
 
 
+    @Override
     public FileStatus[] listStatus(Path path) throws IOException {
     public FileStatus[] listStatus(Path path) throws IOException {
         Path absolute = makeAbsolute(path);
         Path absolute = makeAbsolute(path);
         String srep = absolute.toUri().getPath();
         String srep = absolute.toUri().getPath();
@@ -143,6 +153,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.readdirplus(absolute);
         return kfsImpl.readdirplus(absolute);
     }
     }
 
 
+    @Override
     public FileStatus getFileStatus(Path path) throws IOException {
     public FileStatus getFileStatus(Path path) throws IOException {
 	Path absolute = makeAbsolute(path);
 	Path absolute = makeAbsolute(path);
         String srep = absolute.toUri().getPath();
         String srep = absolute.toUri().getPath();
@@ -164,11 +175,13 @@ public class KosmosFileSystem extends FileSystem {
     }
     }
     
     
     /** This optional operation is not yet supported. */
     /** This optional operation is not yet supported. */
+    @Override
     public FSDataOutputStream append(Path f, int bufferSize,
     public FSDataOutputStream append(Path f, int bufferSize,
         Progressable progress) throws IOException {
         Progressable progress) throws IOException {
       throw new IOException("Not supported");
       throw new IOException("Not supported");
     }
     }
 
 
+    @Override
     public FSDataOutputStream create(Path file, FsPermission permission,
     public FSDataOutputStream create(Path file, FsPermission permission,
                                      boolean overwrite, int bufferSize,
                                      boolean overwrite, int bufferSize,
 				     short replication, long blockSize, Progressable progress)
 				     short replication, long blockSize, Progressable progress)
@@ -176,7 +189,7 @@ public class KosmosFileSystem extends FileSystem {
 
 
         if (exists(file)) {
         if (exists(file)) {
             if (overwrite) {
             if (overwrite) {
-                delete(file);
+                delete(file, true);
             } else {
             } else {
                 throw new IOException("File already exists: " + file);
                 throw new IOException("File already exists: " + file);
             }
             }
@@ -193,6 +206,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.create(srep, replication, bufferSize);
         return kfsImpl.create(srep, replication, bufferSize);
     }
     }
 
 
+    @Override
     public FSDataInputStream open(Path path, int bufferSize) throws IOException {
     public FSDataInputStream open(Path path, int bufferSize) throws IOException {
         if (!exists(path))
         if (!exists(path))
             throw new IOException("File does not exist: " + path);
             throw new IOException("File does not exist: " + path);
@@ -203,6 +217,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.open(srep, bufferSize);
         return kfsImpl.open(srep, bufferSize);
     }
     }
 
 
+    @Override
     public boolean rename(Path src, Path dst) throws IOException {
     public boolean rename(Path src, Path dst) throws IOException {
 	Path absoluteS = makeAbsolute(src);
 	Path absoluteS = makeAbsolute(src);
         String srepS = absoluteS.toUri().getPath();
         String srepS = absoluteS.toUri().getPath();
@@ -215,6 +230,7 @@ public class KosmosFileSystem extends FileSystem {
     }
     }
 
 
     // recursively delete the directory and its contents
     // recursively delete the directory and its contents
+    @Override
     public boolean delete(Path path, boolean recursive) throws IOException {
     public boolean delete(Path path, boolean recursive) throws IOException {
       Path absolute = makeAbsolute(path);
       Path absolute = makeAbsolute(path);
       String srep = absolute.toUri().getPath();
       String srep = absolute.toUri().getPath();
@@ -235,15 +251,12 @@ public class KosmosFileSystem extends FileSystem {
       return kfsImpl.rmdir(srep) == 0;
       return kfsImpl.rmdir(srep) == 0;
     }
     }
     
     
-    @Deprecated
-    public boolean delete(Path path) throws IOException {
-      return delete(path, true);
-    }
-    
+    @Override
     public short getDefaultReplication() {
     public short getDefaultReplication() {
 	return 3;
 	return 3;
     }
     }
 
 
+    @Override
     public boolean setReplication(Path path, short replication)
     public boolean setReplication(Path path, short replication)
 	throws IOException {
 	throws IOException {
 
 
@@ -256,6 +269,7 @@ public class KosmosFileSystem extends FileSystem {
 
 
     // 64MB is the KFS block size
     // 64MB is the KFS block size
 
 
+    @Override
     public long getDefaultBlockSize() {
     public long getDefaultBlockSize() {
 	return 1 << 26;
 	return 1 << 26;
     }
     }
@@ -296,19 +310,23 @@ public class KosmosFileSystem extends FileSystem {
       return result;
       return result;
     }
     }
 
 
+    @Override
     public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
     public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
 	FileUtil.copy(localFs, src, this, dst, delSrc, getConf());
 	FileUtil.copy(localFs, src, this, dst, delSrc, getConf());
     }
     }
 
 
+    @Override
     public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
     public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
 	FileUtil.copy(this, src, localFs, dst, delSrc, getConf());
 	FileUtil.copy(this, src, localFs, dst, delSrc, getConf());
     }
     }
 
 
+    @Override
     public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
     public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
 	throws IOException {
 	throws IOException {
 	return tmpLocalFile;
 	return tmpLocalFile;
     }
     }
 
 
+    @Override
     public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
     public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
 	throws IOException {
 	throws IOException {
 	moveFromLocalFile(tmpLocalFile, fsOutputFile);
 	moveFromLocalFile(tmpLocalFile, fsOutputFile);

+ 1 - 7
src/core/org/apache/hadoop/fs/s3/S3FileSystem.java

@@ -212,7 +212,7 @@ public class S3FileSystem extends FileSystem {
     INode inode = store.retrieveINode(makeAbsolute(file));
     INode inode = store.retrieveINode(makeAbsolute(file));
     if (inode != null) {
     if (inode != null) {
       if (overwrite) {
       if (overwrite) {
-        delete(file);
+        delete(file, true);
       } else {
       } else {
         throw new IOException("File already exists: " + file);
         throw new IOException("File already exists: " + file);
       }
       }
@@ -317,12 +317,6 @@ public class S3FileSystem extends FileSystem {
    return true;
    return true;
   }
   }
   
   
-  @Override
-  @Deprecated
-  public boolean delete(Path path) throws IOException {
-    return delete(path, true);
-  }
-
   /**
   /**
    * FileStatus for S3 file systems. 
    * FileStatus for S3 file systems. 
    */
    */

+ 0 - 6
src/core/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java

@@ -279,12 +279,6 @@ public class NativeS3FileSystem extends FileSystem {
         key, progress, bufferSize), statistics);
         key, progress, bufferSize), statistics);
   }
   }
   
   
-  @Override
-  @Deprecated
-  public boolean delete(Path path) throws IOException {
-    return delete(path, true);
-  }
-
   @Override
   @Override
   public boolean delete(Path f, boolean recursive) throws IOException {
   public boolean delete(Path f, boolean recursive) throws IOException {
     FileStatus status;
     FileStatus status;

+ 0 - 8
src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -210,14 +210,6 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.rename(getPathName(src), getPathName(dst));
     return dfs.rename(getPathName(src), getPathName(dst));
   }
   }
 
 
-  /**
-   * Get rid of Path f, whether a true file or dir.
-   */
-  @Deprecated
-  public boolean delete(Path f) throws IOException {
-    return dfs.delete(getPathName(f));
-  }
-  
   /**
   /**
    * requires a boolean check to delete a non 
    * requires a boolean check to delete a non 
    * empty directory recursively.
    * empty directory recursively.

+ 0 - 9
src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -328,15 +328,6 @@ public class HftpFileSystem extends FileSystem {
     throw new IOException("Not supported");
     throw new IOException("Not supported");
   }
   }
 
 
-  @Override
-  /*
-   * @deprecated Use delete(path, boolean)
-   */
-  @Deprecated
-  public boolean delete(Path f) throws IOException {
-    throw new IOException("Not supported");
-  }
-  
   @Override 
   @Override 
   public boolean delete(Path f, boolean recursive) throws IOException {
   public boolean delete(Path f, boolean recursive) throws IOException {
     throw new IOException("Not supported");
     throw new IOException("Not supported");

+ 10 - 7
src/hdfs/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -17,15 +17,16 @@
  */
  */
 package org.apache.hadoop.hdfs.protocol;
 package org.apache.hadoop.hdfs.protocol;
 
 
-import java.io.*;
+import java.io.FileNotFoundException;
+import java.io.IOException;
 
 
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.AccessControlException;
 
 
 /**********************************************************************
 /**********************************************************************
  * ClientProtocol is used by user code via 
  * ClientProtocol is used by user code via 
@@ -78,7 +79,7 @@ public interface ClientProtocol extends VersionedProtocol {
    * The name-node does not have a notion of "current" directory for a client.
    * The name-node does not have a notion of "current" directory for a client.
    * <p>
    * <p>
    * Once created, the file is visible and available for read to other clients.
    * Once created, the file is visible and available for read to other clients.
-   * Although, other clients cannot {@link #delete(String)}, re-create or 
+   * Although, other clients cannot {@link #delete(String, boolean)}, re-create or 
    * {@link #rename(String, String)} it until the file is completed
    * {@link #rename(String, String)} it until the file is completed
    * or explicitly as a result of lease expiration.
    * or explicitly as a result of lease expiration.
    * <p>
    * <p>
@@ -220,7 +221,9 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src existing name.
    * @param src existing name.
    * @return true only if the existing file or directory was actually removed 
    * @return true only if the existing file or directory was actually removed 
    * from the file system. 
    * from the file system. 
+   * @deprecated use {@link #delete(String, boolean)} istead.
    */
    */
+  @Deprecated
   public boolean delete(String src) throws IOException;
   public boolean delete(String src) throws IOException;
 
 
   /**
   /**

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

@@ -326,14 +326,14 @@ public class JobConf extends Configuration {
   public void deleteLocalFiles() throws IOException {
   public void deleteLocalFiles() throws IOException {
     String[] localDirs = getLocalDirs();
     String[] localDirs = getLocalDirs();
     for (int i = 0; i < localDirs.length; i++) {
     for (int i = 0; i < localDirs.length; i++) {
-      FileSystem.getLocal(this).delete(new Path(localDirs[i]));
+      FileSystem.getLocal(this).delete(new Path(localDirs[i]), true);
     }
     }
   }
   }
 
 
   public void deleteLocalFiles(String subdir) throws IOException {
   public void deleteLocalFiles(String subdir) throws IOException {
     String[] localDirs = getLocalDirs();
     String[] localDirs = getLocalDirs();
     for (int i = 0; i < localDirs.length; i++) {
     for (int i = 0; i < localDirs.length; i++) {
-      FileSystem.getLocal(this).delete(new Path(localDirs[i], subdir));
+      FileSystem.getLocal(this).delete(new Path(localDirs[i], subdir), true);
     }
     }
   }
   }
 
 

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

@@ -66,7 +66,7 @@ public class TestLocalDirAllocator extends TestCase {
 
 
   private static void rmBufferDirs() throws IOException {
   private static void rmBufferDirs() throws IOException {
     assertTrue(!localFs.exists(BUFFER_PATH_ROOT) ||
     assertTrue(!localFs.exists(BUFFER_PATH_ROOT) ||
-        localFs.delete(BUFFER_PATH_ROOT));
+        localFs.delete(BUFFER_PATH_ROOT, true));
   }
   }
   
   
   private void validateTempDirCreation(int i) throws IOException {
   private void validateTempDirCreation(int i) throws IOException {

+ 3 - 6
src/test/org/apache/hadoop/mapred/lib/TestTotalOrderPartitioner.java

@@ -22,10 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 
 
-import junit.framework.Test;
 import junit.framework.TestCase;
 import junit.framework.TestCase;
-import junit.framework.TestSuite;
-import junit.extensions.TestSetup;
 
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -117,7 +114,7 @@ public class TestTotalOrderPartitioner extends TestCase {
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
       }
     } finally {
     } finally {
-      p.getFileSystem(job).delete(p);
+      p.getFileSystem(job).delete(p, true);
     }
     }
   }
   }
 
 
@@ -137,7 +134,7 @@ public class TestTotalOrderPartitioner extends TestCase {
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
       }
     } finally {
     } finally {
-      p.getFileSystem(job).delete(p);
+      p.getFileSystem(job).delete(p, true);
     }
     }
   }
   }
 
 
@@ -184,7 +181,7 @@ public class TestTotalOrderPartitioner extends TestCase {
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
       }
     } finally {
     } finally {
-      p.getFileSystem(job).delete(p);
+      p.getFileSystem(job).delete(p, true);
     }
     }
   }
   }