Просмотр исходного кода

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 16 лет назад
Родитель
Сommit
3ee0d67e70
28 измененных файлов с 118 добавлено и 124 удалено
  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
     ChecksumDistributedFileSystem.  (cdouglas via szetszwo)
 
+    HADOOP-4940. Remove a deprecated method FileSystem.delete(Path f).  (Enis
+    Soztutar via szetszwo)
+
   NEW FEATURES
 
     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(" ");
 
           try {
-            fs.delete(outfile);
+            fs.delete(outfile, true);
           } catch (IOException ex) {
             System.out.println(ex.toString());
           }
@@ -377,7 +377,7 @@ public class GridMixRunner {
   private static void clearDir(String dir) {
     try {
       Path outfile = new Path(dir);
-      fs.delete(outfile);
+      fs.delete(outfile, true);
     } catch (IOException ex) {
       ex.printStackTrace();
       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:
     //  File f = new File(path);
@@ -1114,9 +1114,10 @@ int hdfsDelete(hdfsFS fs, const char* path)
     //Delete the file
     jvalue jVal;
     jthrowable jExc = NULL;
+    jboolean jRecursive = recursive;
     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."
                                    "FileSystem::delete");
         return -1;

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

@@ -268,10 +268,12 @@ extern  "C" {
      * hdfsDelete - Delete file. 
      * @param fs The configured filesystem handle.
      * @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. 
      */
-    int hdfsDelete(hdfsFS fs, const char* path);
-
+    int hdfsDelete(hdfsFS fs, const char* path, int recursive);
 
     /**
      * hdfsRename - Rename file. 

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

@@ -305,13 +305,13 @@ int main(int argc, char **argv) {
         hdfsFreeFileInfo(finfo, 1);
 
         // 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;
-        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;
-        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;
-        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;
         fprintf(stderr, "hdfsExists: %s\n", ((result = hdfsExists(fs, newDirectory)) ? "Success!" : "Failed!"));
         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);
 
     FileSystem fs = FileSystem.get(defaults);
-    fs.delete(new Path(outputDir));
+    fs.delete(new Path(outputDir), true);
     FileInputFormat.setInputPaths(job, inputDir);
 
     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) {
       appendFile(fstatus.getPath());
-      hdfs.delete(fstatus.getPath());
+      hdfs.delete(fstatus.getPath(), true);
     }
 
     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;
   }
 
-  if (hdfsDelete(userFS, path)) {
+  if (hdfsDelete(userFS, path, 1)) {
     syslog(LOG_ERR,"ERROR: hdfs trying to delete the file %s",path);
     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 =
         fs.listStatus(directory, LuceneIndexFileNameFilter.getFilter());
     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 "
             + fileStatus[i].getPath());
       }
@@ -150,7 +150,7 @@ public class FileSystemDirectory extends Directory {
    * @see org.apache.lucene.store.Directory#deleteFile(java.lang.String)
    */
   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);
     }
   }
@@ -167,7 +167,7 @@ public class FileSystemDirectory extends Directory {
    */
   public IndexOutput createOutput(String name) throws IOException {
     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
       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++) {
       Path path = fileStatus[i].getPath();
       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.
     Path segmentsGenFile = new Path(LuceneUtil.IndexFileNames.SEGMENTS_GEN);
     if (fs.exists(segmentsGenFile)) {
-      fs.delete(segmentsGenFile);
+      fs.delete(segmentsGenFile, true);
     }
   }
 
@@ -226,7 +226,7 @@ public class ShardWriter {
       }
     } finally {
       // 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();
       if (fs.exists(inputPath)) {
-        fs.delete(inputPath);
+        fs.delete(inputPath, true);
       }
       fs.copyFromLocalFile(localInputPath, inputPath);
       if (fs.exists(updatePath)) {
-        fs.delete(updatePath);
+        fs.delete(updatePath, true);
       }
       fs.copyFromLocalFile(localUpdatePath, updatePath);
 
       if (fs.exists(outputPath)) {
         // do not create, mapred will create
-        fs.delete(outputPath);
+        fs.delete(outputPath, true);
       }
 
       if (fs.exists(indexPath)) {
-        fs.delete(indexPath);
+        fs.delete(indexPath, true);
       }
 
       mrCluster =
@@ -156,7 +156,7 @@ public class TestDistributionPolicy extends TestCase {
     onetest();
 
     if (fs.exists(indexPath)) {
-      fs.delete(indexPath);
+      fs.delete(indexPath, true);
     }
 
     // test round-robin distribution policy
@@ -177,7 +177,7 @@ public class TestDistributionPolicy extends TestCase {
     }
 
     if (fs.exists(outputPath)) {
-      fs.delete(outputPath);
+      fs.delete(outputPath, true);
     }
 
     IIndexUpdater updater = new IndexUpdater();
@@ -185,7 +185,7 @@ public class TestDistributionPolicy extends TestCase {
         shards);
 
     if (fs.exists(outputPath)) {
-      fs.delete(outputPath);
+      fs.delete(outputPath, true);
     }
 
     // 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();
       if (fs.exists(inputPath)) {
-        fs.delete(inputPath);
+        fs.delete(inputPath, true);
       }
       fs.copyFromLocalFile(localInputPath, inputPath);
 
       if (fs.exists(outputPath)) {
         // do not create, mapred will create
-        fs.delete(outputPath);
+        fs.delete(outputPath, true);
       }
 
       if (fs.exists(indexPath)) {
-        fs.delete(indexPath);
+        fs.delete(indexPath, true);
       }
 
       mrCluster =
@@ -157,7 +157,7 @@ public class TestIndexUpdater extends TestCase {
 
     for (int i = 0; i < numRuns; i++) {
       if (fs.exists(outputPath)) {
-        fs.delete(outputPath);
+        fs.delete(outputPath, true);
       }
 
       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;
 
+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 java.io.*;
-import java.util.*;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.*;
 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
  * utility provided by distributed caching 
@@ -83,7 +90,7 @@ public class TestMultipleCachefiles extends TestCase
           "-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));
         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 org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.ToolRunner;
@@ -71,7 +71,7 @@ public class TestStreamedMerge extends TestCase {
   }
 
   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
     addInput("/input/part-00", i18n("k1\tv1\n" + "k3\tv5\n"));
@@ -192,7 +192,7 @@ public class TestStreamedMerge extends TestCase {
     public void run() {
       try {
         in_ = connectInputStream();
-        LineReader lineReader = new LineReader((InputStream)in_, conf_);
+        LineReader lineReader = new LineReader(in_, conf_);
         Text line = new Text();
         while (lineReader.readLine(line) > 0) {
           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;
 
+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 java.io.*;
-import java.util.*;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.*;
 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
  * utility provided by distributed caching 
@@ -79,7 +86,7 @@ public class TestSymLink extends TestCase
           "-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));
         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;
     
-  /** Delete a file. */
-  /** @deprecated Use delete(Path, boolean) instead */ @Deprecated 
-  public abstract boolean delete(Path f) throws IOException;
-  
   /** Delete a file.
    *
    * @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);
   }
   
-  /** Delete a file */@Deprecated
-  public boolean delete(Path f) throws IOException {
-    return delete(f, true);
-  }
-  
   /** Delete a file */
   public boolean delete(Path f, boolean recursive) throws IOException {
     return fs.delete(f, recursive);

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

@@ -18,17 +18,23 @@
 
 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.nio.ByteBuffer;
-import java.nio.channels.FileLock;
-import java.util.*;
+import java.util.StringTokenizer;
 
 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.StringUtils;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.StringUtils;
 
 /****************************************************************
  * 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());
   }
   
-  @Deprecated
-  public boolean delete(Path p) throws IOException {
-    return delete(p, true);
-  }
-  
   public boolean delete(Path p, boolean recursive) throws IOException {
     File f = pathToFile(p);
     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
   public boolean delete(Path file, boolean recursive) throws IOException {
     FTPClient client = connect();

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

@@ -20,19 +20,20 @@
 
 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.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.fs.BlockLocation;
 
 /**
  * A FileSystem backed by KFS.
@@ -54,10 +55,12 @@ public class KosmosFileSystem extends FileSystem {
         this.kfsImpl = fsimpl;
     }
 
+    @Override
     public URI getUri() {
 	return uri;
     }
 
+    @Override
     public void initialize(URI uri, Configuration conf) throws IOException {
         try {
 	    if (kfsImpl == null) {
@@ -82,15 +85,18 @@ public class KosmosFileSystem extends FileSystem {
         }
     }
 
+    @Override
     @Deprecated
     public String getName() {
 	return getUri().toString();
     }
 
+    @Override
     public Path getWorkingDirectory() {
 	return workingDir;
     }
 
+    @Override
     public void setWorkingDirectory(Path dir) {
 	workingDir = makeAbsolute(dir);
     }
@@ -102,6 +108,7 @@ public class KosmosFileSystem extends FileSystem {
 	return new Path(workingDir, path);
     }
 
+    @Override
     public boolean mkdirs(Path path, FsPermission permission
         ) throws IOException {
 	Path absolute = makeAbsolute(path);
@@ -116,6 +123,7 @@ public class KosmosFileSystem extends FileSystem {
 	return res == 0;
     }
 
+    @Override
     @Deprecated
     public boolean isDirectory(Path path) throws IOException {
 	Path absolute = makeAbsolute(path);
@@ -126,6 +134,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.isDirectory(srep);
     }
 
+    @Override
     @Deprecated
     public boolean isFile(Path path) throws IOException {
 	Path absolute = makeAbsolute(path);
@@ -133,6 +142,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.isFile(srep);
     }
 
+    @Override
     public FileStatus[] listStatus(Path path) throws IOException {
         Path absolute = makeAbsolute(path);
         String srep = absolute.toUri().getPath();
@@ -143,6 +153,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.readdirplus(absolute);
     }
 
+    @Override
     public FileStatus getFileStatus(Path path) throws IOException {
 	Path absolute = makeAbsolute(path);
         String srep = absolute.toUri().getPath();
@@ -164,11 +175,13 @@ public class KosmosFileSystem extends FileSystem {
     }
     
     /** This optional operation is not yet supported. */
+    @Override
     public FSDataOutputStream append(Path f, int bufferSize,
         Progressable progress) throws IOException {
       throw new IOException("Not supported");
     }
 
+    @Override
     public FSDataOutputStream create(Path file, FsPermission permission,
                                      boolean overwrite, int bufferSize,
 				     short replication, long blockSize, Progressable progress)
@@ -176,7 +189,7 @@ public class KosmosFileSystem extends FileSystem {
 
         if (exists(file)) {
             if (overwrite) {
-                delete(file);
+                delete(file, true);
             } else {
                 throw new IOException("File already exists: " + file);
             }
@@ -193,6 +206,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.create(srep, replication, bufferSize);
     }
 
+    @Override
     public FSDataInputStream open(Path path, int bufferSize) throws IOException {
         if (!exists(path))
             throw new IOException("File does not exist: " + path);
@@ -203,6 +217,7 @@ public class KosmosFileSystem extends FileSystem {
         return kfsImpl.open(srep, bufferSize);
     }
 
+    @Override
     public boolean rename(Path src, Path dst) throws IOException {
 	Path absoluteS = makeAbsolute(src);
         String srepS = absoluteS.toUri().getPath();
@@ -215,6 +230,7 @@ public class KosmosFileSystem extends FileSystem {
     }
 
     // recursively delete the directory and its contents
+    @Override
     public boolean delete(Path path, boolean recursive) throws IOException {
       Path absolute = makeAbsolute(path);
       String srep = absolute.toUri().getPath();
@@ -235,15 +251,12 @@ public class KosmosFileSystem extends FileSystem {
       return kfsImpl.rmdir(srep) == 0;
     }
     
-    @Deprecated
-    public boolean delete(Path path) throws IOException {
-      return delete(path, true);
-    }
-    
+    @Override
     public short getDefaultReplication() {
 	return 3;
     }
 
+    @Override
     public boolean setReplication(Path path, short replication)
 	throws IOException {
 
@@ -256,6 +269,7 @@ public class KosmosFileSystem extends FileSystem {
 
     // 64MB is the KFS block size
 
+    @Override
     public long getDefaultBlockSize() {
 	return 1 << 26;
     }
@@ -296,19 +310,23 @@ public class KosmosFileSystem extends FileSystem {
       return result;
     }
 
+    @Override
     public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
 	FileUtil.copy(localFs, src, this, dst, delSrc, getConf());
     }
 
+    @Override
     public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException {
 	FileUtil.copy(this, src, localFs, dst, delSrc, getConf());
     }
 
+    @Override
     public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
 	throws IOException {
 	return tmpLocalFile;
     }
 
+    @Override
     public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
 	throws IOException {
 	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));
     if (inode != null) {
       if (overwrite) {
-        delete(file);
+        delete(file, true);
       } else {
         throw new IOException("File already exists: " + file);
       }
@@ -317,12 +317,6 @@ public class S3FileSystem extends FileSystem {
    return true;
   }
   
-  @Override
-  @Deprecated
-  public boolean delete(Path path) throws IOException {
-    return delete(path, true);
-  }
-
   /**
    * 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);
   }
   
-  @Override
-  @Deprecated
-  public boolean delete(Path path) throws IOException {
-    return delete(path, true);
-  }
-
   @Override
   public boolean delete(Path f, boolean recursive) throws IOException {
     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));
   }
 
-  /**
-   * 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 
    * 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");
   }
 
-  @Override
-  /*
-   * @deprecated Use delete(path, boolean)
-   */
-  @Deprecated
-  public boolean delete(Path f) throws IOException {
-    throw new IOException("Not supported");
-  }
-  
   @Override 
   public boolean delete(Path f, boolean recursive) throws IOException {
     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;
 
-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.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 
@@ -78,7 +79,7 @@ public interface ClientProtocol extends VersionedProtocol {
    * The name-node does not have a notion of "current" directory for a client.
    * <p>
    * 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
    * or explicitly as a result of lease expiration.
    * <p>
@@ -220,7 +221,9 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src existing name.
    * @return true only if the existing file or directory was actually removed 
    * from the file system. 
+   * @deprecated use {@link #delete(String, boolean)} istead.
    */
+  @Deprecated
   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 {
     String[] localDirs = getLocalDirs();
     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 {
     String[] localDirs = getLocalDirs();
     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 {
     assertTrue(!localFs.exists(BUFFER_PATH_ROOT) ||
-        localFs.delete(BUFFER_PATH_ROOT));
+        localFs.delete(BUFFER_PATH_ROOT, true));
   }
   
   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.Arrays;
 
-import junit.framework.Test;
 import junit.framework.TestCase;
-import junit.framework.TestSuite;
-import junit.extensions.TestSetup;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -117,7 +114,7 @@ public class TestTotalOrderPartitioner extends TestCase {
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
     } 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));
       }
     } 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));
       }
     } finally {
-      p.getFileSystem(job).delete(p);
+      p.getFileSystem(job).delete(p, true);
     }
   }