瀏覽代碼

Revert HADOOP-4779.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@729354 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 16 年之前
父節點
當前提交
d3a0ded6e8

+ 1 - 5
CHANGES.txt

@@ -4,13 +4,9 @@ Trunk (unreleased changes)
 
   INCOMPATIBLE CHANGES
 
-    HADOOP-4895. Remove deprecated methods DFSClient.getHints(..) and
+    HADOOP-4895. Removes deprecated methods DFSClient.getHints(..) and
     DFSClient.isDirectory(..).  (szetszwo)
 
-    HADOOP-4779. Remove deprecated FileSystem methods: delete(Path f),
-    getBlockSize(Path f), getLength(Path f) and getReplication(Path src).
-    (szetszwo)
-
   NEW FEATURES
 
   IMPROVEMENTS

+ 1 - 1
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamUtil.java

@@ -319,7 +319,7 @@ public class StreamUtil {
   }
 
   static String slurpHadoop(Path p, FileSystem fs) throws IOException {
-    int len = (int) fs.getFileStatus(p).getLen();
+    int len = (int) fs.getLength(p);
     byte[] buf = new byte[len];
     FSDataInputStream in = fs.open(p);
     String contents = null;

+ 41 - 19
src/core/org/apache/hadoop/fs/FileSystem.java

@@ -17,33 +17,22 @@
  */
 package org.apache.hadoop.fs;
 
-import java.io.Closeable;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.IdentityHashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
+import java.io.*;
+import java.net.*;
+import java.util.*;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
 import javax.security.auth.login.LoginException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
+import org.apache.commons.logging.*;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.*;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.ReflectionUtils;
 
 /****************************************************************
  * An abstract base class for a fairly generic filesystem.  It
@@ -540,6 +529,19 @@ public abstract class FileSystem extends Configured implements Closeable {
    */
   public abstract FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException;
+  
+  /**
+   * Get replication.
+   * 
+   * @deprecated Use getFileStatus() instead
+   * @param src file name
+   * @return file replication
+   * @throws IOException
+   */ 
+  @Deprecated
+  public short getReplication(Path src) throws IOException {
+    return getFileStatus(src).getReplication();
+  }
 
   /**
    * Set replication for an existing file.
@@ -561,6 +563,10 @@ 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.
@@ -644,6 +650,12 @@ public abstract class FileSystem extends Configured implements Closeable {
     }
   }
     
+  /** The number of bytes in a file. */
+  /** @deprecated Use getFileStatus() instead */ @Deprecated
+  public long getLength(Path f) throws IOException {
+    return getFileStatus(f).getLen();
+  }
+    
   /** Return the {@link ContentSummary} of a given {@link Path}. */
   public ContentSummary getContentSummary(Path f) throws IOException {
     FileStatus status = getFileStatus(f);
@@ -1212,6 +1224,16 @@ public abstract class FileSystem extends Configured implements Closeable {
     return used;
   }
 
+  /**
+   * Get the block size for a particular file.
+   * @param f the filename
+   * @return the number of bytes in a block
+   */
+  /** @deprecated Use getFileStatus() instead */ @Deprecated
+  public long getBlockSize(Path f) throws IOException {
+    return getFileStatus(f).getBlockSize();
+  }
+    
   /** Return the number of bytes that large input files should be optimally
    * be split into to minimize i/o time. */
   public long getDefaultBlockSize() {

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

@@ -138,6 +138,11 @@ 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);

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

@@ -18,23 +18,17 @@
 
 package org.apache.hadoop.fs;
 
-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.io.*;
 import java.net.URI;
 import java.nio.ByteBuffer;
-import java.util.StringTokenizer;
+import java.nio.channels.FileLock;
+import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Shell;
 
 /****************************************************************
  * Implement the FileSystem API for the raw local filesystem.
@@ -258,6 +252,11 @@ 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()) {

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

@@ -264,6 +264,13 @@ 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();

+ 20 - 1
src/core/org/apache/hadoop/fs/kfs/KosmosFileSystem.java

@@ -176,7 +176,7 @@ public class KosmosFileSystem extends FileSystem {
 
         if (exists(file)) {
             if (overwrite) {
-                delete(file, true);
+                delete(file);
             } else {
                 throw new IOException("File already exists: " + file);
             }
@@ -234,6 +234,25 @@ public class KosmosFileSystem extends FileSystem {
       }
       return kfsImpl.rmdir(srep) == 0;
     }
+    
+    @Deprecated
+    public boolean delete(Path path) throws IOException {
+      return delete(path, true);
+    }
+    
+    @Deprecated
+    public long getLength(Path path) throws IOException {
+	Path absolute = makeAbsolute(path);
+        String srep = absolute.toUri().getPath();
+        return kfsImpl.filesize(srep);
+    }
+
+    @Deprecated
+    public short getReplication(Path path) throws IOException {
+	Path absolute = makeAbsolute(path);
+        String srep = absolute.toUri().getPath();
+        return kfsImpl.getReplication(srep);
+    }
 
     public short getDefaultReplication() {
 	return 3;

+ 7 - 1
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, true);
+        delete(file);
       } else {
         throw new IOException("File already exists: " + file);
       }
@@ -317,6 +317,12 @@ 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. 
    */

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

@@ -279,6 +279,12 @@ 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;

+ 7 - 8
src/core/org/apache/hadoop/io/SequenceFile.java

@@ -1414,7 +1414,7 @@ public class SequenceFile {
 
     private Reader(FileSystem fs, Path file, int bufferSize,
                    Configuration conf, boolean tempReader) throws IOException {
-      this(fs, file, bufferSize, 0, fs.getFileStatus(file).getLen(), conf, tempReader);
+      this(fs, file, bufferSize, 0, fs.getLength(file), conf, tempReader);
     }
     
     private Reader(FileSystem fs, Path file, int bufferSize, long start,
@@ -2605,8 +2605,8 @@ public class SequenceFile {
       //get the segments from inNames
       ArrayList <SegmentDescriptor> a = new ArrayList <SegmentDescriptor>();
       for (int i = 0; i < inNames.length; i++) {
-        SegmentDescriptor s = new SegmentDescriptor(0,
-            fs.getFileStatus(inNames[i]).getLen(), inNames[i]);
+        SegmentDescriptor s = new SegmentDescriptor(0, 
+                                                    fs.getLength(inNames[i]), inNames[i]);
         s.preserveInput(!deleteInputs);
         s.doSync();
         a.add(s);
@@ -2634,8 +2634,8 @@ public class SequenceFile {
       //get the segments from inNames
       ArrayList <SegmentDescriptor> a = new ArrayList <SegmentDescriptor>();
       for (int i = 0; i < inNames.length; i++) {
-        SegmentDescriptor s = new SegmentDescriptor(0,
-            fs.getFileStatus(inNames[i]).getLen(), inNames[i]);
+        SegmentDescriptor s = new SegmentDescriptor(0, 
+                                                    fs.getLength(inNames[i]), inNames[i]);
         s.preserveInput(!deleteInputs);
         s.doSync();
         a.add(s);
@@ -2951,8 +2951,7 @@ public class SequenceFile {
             this.close();
             
             SegmentDescriptor tempSegment = 
-              new SegmentDescriptor(0,
-                  fs.getFileStatus(outputFile).getLen(), outputFile);
+              new SegmentDescriptor(0, fs.getLength(outputFile), outputFile);
             //put the segment back in the TreeMap
             sortedSegmentSizes.put(tempSegment, null);
             numSegments = sortedSegmentSizes.size();
@@ -3169,7 +3168,7 @@ public class SequenceFile {
       public SegmentContainer(Path inName, Path indexIn) throws IOException {
         //get the segments from indexIn
         FSDataInputStream fsIndexIn = fs.open(indexIn);
-        long end = fs.getFileStatus(indexIn).getLen();
+        long end = fs.getLength(indexIn);
         while (fsIndexIn.getPos() < end) {
           long segmentOffset = WritableUtils.readVLong(fsIndexIn);
           long segmentLength = WritableUtils.readVLong(fsIndexIn);

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

@@ -210,6 +210,14 @@ 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.

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

@@ -328,6 +328,15 @@ 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");

+ 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]), true);
+      FileSystem.getLocal(this).delete(new Path(localDirs[i]));
     }
   }
 
   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), true);
+      FileSystem.getLocal(this).delete(new Path(localDirs[i], subdir));
     }
   }
 

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

@@ -156,7 +156,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
                 Path mapOut = this.mapoutputFile.getOutputFile(mapId);
                 Path reduceIn = this.mapoutputFile.getInputFileForWrite(
                                   mapId.getTaskID(),reduceId,
-                                  localFs.getFileStatus(mapOut).getLen());
+                                  localFs.getLength(mapOut));
                 if (!localFs.mkdirs(reduceIn.getParent())) {
                   throw new IOException("Mkdirs failed to create "
                       + reduceIn.getParent().toString());

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

@@ -110,7 +110,7 @@ public class DistributedFSCheck extends TestCase {
       nrFiles++;
       // For a regular file generate <fName,offset> pairs
       long blockSize = fs.getDefaultBlockSize();
-      long fileLength = fs.getFileStatus(rootFile).getLen();
+      long fileLength = fs.getLength(rootFile);
       for(long offset = 0; offset < fileLength; offset += blockSize)
         writer.append(new UTF8(rootFile.toString()), new LongWritable(offset));
       return;

+ 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, true));
+        localFs.delete(BUFFER_PATH_ROOT));
   }
   
   private void validateTempDirCreation(int i) throws IOException {

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

@@ -17,12 +17,14 @@
  */
 package org.apache.hadoop.fs;
 
-import java.io.DataOutputStream;
-import java.io.IOException;
-
 import junit.framework.TestCase;
-
+import java.io.*;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * test for the input truncation bug when mark/reset is used.
@@ -63,7 +65,7 @@ public class TestTruncatedInputBug extends TestCase {
       Path testFile = new Path(TEST_ROOT_DIR, "HADOOP-1489");
       writeFile(fileSys, testFile, fileSize);
       assertTrue(fileSys.exists(testFile));
-      assertTrue(fileSys.getFileStatus(testFile).getLen() == fileSize);
+      assertTrue(fileSys.getLength(testFile) == fileSize);
 
       // Now read the file for ioBufSize bytes
       FSDataInputStream in = fileSys.open(testFile, ioBufSize);

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

@@ -21,15 +21,17 @@
 
 package org.apache.hadoop.fs.kfs;
 
-import java.io.IOException;
+import java.io.*;
 
 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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.BlockLocation;
 
 
 public class KFSEmulationImpl implements IFSImpl {
@@ -97,7 +99,7 @@ public class KFSEmulationImpl implements IFSImpl {
     }
 
     public long filesize(String path) throws IOException {
-        return localFS.getFileStatus(new Path(path)).getLen();
+        return localFS.getLength(new Path(path));
     }
     public short getReplication(String path) throws IOException {
         return 1;

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

@@ -18,18 +18,15 @@
 
 package org.apache.hadoop.mapred;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
 import java.text.DecimalFormat;
+import java.io.*;
 import java.util.Arrays;
 import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.examples.RandomWriter;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -39,6 +36,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSequenceFileInputFormat;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 
@@ -157,7 +155,7 @@ public class UtilsForTests {
   }
 
   static String slurpHadoop(Path p, FileSystem fs) throws IOException {
-    int len = (int) fs.getFileStatus(p).getLen();
+    int len = (int) fs.getLength(p);
     byte[] buf = new byte[len];
     InputStream in = fs.open(p);
     String contents = null;

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

@@ -22,7 +22,10 @@ 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;
@@ -114,7 +117,7 @@ public class TestTotalOrderPartitioner extends TestCase {
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
     } finally {
-      p.getFileSystem(job).delete(p, true);
+      p.getFileSystem(job).delete(p);
     }
   }
 
@@ -134,7 +137,7 @@ public class TestTotalOrderPartitioner extends TestCase {
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
     } finally {
-      p.getFileSystem(job).delete(p, true);
+      p.getFileSystem(job).delete(p);
     }
   }
 
@@ -181,7 +184,7 @@ public class TestTotalOrderPartitioner extends TestCase {
             partitioner.getPartition(chk.data, nw, splitStrings.length + 1));
       }
     } finally {
-      p.getFileSystem(job).delete(p, true);
+      p.getFileSystem(job).delete(p);
     }
   }