Browse Source

HADOOP-4941. Remove deprecated FileSystem methods: getBlockSize(Path f), getLength(Path f) and getReplication(Path src). (szetszwo)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@730575 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 16 years ago
parent
commit
7fdcac0504

+ 3 - 0
CHANGES.txt

@@ -7,6 +7,9 @@ Trunk (unreleased changes)
     HADOOP-4895. Remove deprecated methods DFSClient.getHints(..) and
     HADOOP-4895. Remove deprecated methods DFSClient.getHints(..) and
     DFSClient.isDirectory(..).  (szetszwo)
     DFSClient.isDirectory(..).  (szetszwo)
 
 
+    HADOOP-4941. Remove deprecated FileSystem methods: getBlockSize(Path f),
+    getLength(Path f) and getReplication(Path src).  (szetszwo)
+
   NEW FEATURES
   NEW FEATURES
 
 
   IMPROVEMENTS
   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 {
   static String slurpHadoop(Path p, FileSystem fs) throws IOException {
-    int len = (int) fs.getLength(p);
+    int len = (int) fs.getFileStatus(p).getLen();
     byte[] buf = new byte[len];
     byte[] buf = new byte[len];
     FSDataInputStream in = fs.open(p);
     FSDataInputStream in = fs.open(p);
     String contents = null;
     String contents = null;

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

@@ -17,22 +17,33 @@
  */
  */
 package org.apache.hadoop.fs;
 package org.apache.hadoop.fs;
 
 
-import java.io.*;
-import java.net.*;
-import java.util.*;
+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.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
 import javax.security.auth.login.LoginException;
 import javax.security.auth.login.LoginException;
 
 
-import org.apache.commons.logging.*;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.util.*;
+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.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.UserGroupInformation;
 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
  * An abstract base class for a fairly generic filesystem.  It
@@ -529,19 +540,6 @@ public abstract class FileSystem extends Configured implements Closeable {
    */
    */
   public abstract FSDataOutputStream append(Path f, int bufferSize,
   public abstract FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException;
       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.
    * Set replication for an existing file.
@@ -650,12 +648,6 @@ 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}. */
   /** Return the {@link ContentSummary} of a given {@link Path}. */
   public ContentSummary getContentSummary(Path f) throws IOException {
   public ContentSummary getContentSummary(Path f) throws IOException {
     FileStatus status = getFileStatus(f);
     FileStatus status = getFileStatus(f);
@@ -1224,16 +1216,6 @@ public abstract class FileSystem extends Configured implements Closeable {
     return used;
     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
   /** Return the number of bytes that large input files should be optimally
    * be split into to minimize i/o time. */
    * be split into to minimize i/o time. */
   public long getDefaultBlockSize() {
   public long getDefaultBlockSize() {

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

@@ -240,20 +240,6 @@ public class KosmosFileSystem extends FileSystem {
       return delete(path, true);
       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() {
     public short getDefaultReplication() {
 	return 3;
 	return 3;
     }
     }

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

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

+ 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 mapOut = this.mapoutputFile.getOutputFile(mapId);
                 Path reduceIn = this.mapoutputFile.getInputFileForWrite(
                 Path reduceIn = this.mapoutputFile.getInputFileForWrite(
                                   mapId.getTaskID(),reduceId,
                                   mapId.getTaskID(),reduceId,
-                                  localFs.getLength(mapOut));
+                                  localFs.getFileStatus(mapOut).getLen());
                 if (!localFs.mkdirs(reduceIn.getParent())) {
                 if (!localFs.mkdirs(reduceIn.getParent())) {
                   throw new IOException("Mkdirs failed to create "
                   throw new IOException("Mkdirs failed to create "
                       + reduceIn.getParent().toString());
                       + reduceIn.getParent().toString());

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

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

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

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

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

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

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

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