浏览代码

First version that compiles.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@374738 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 19 年之前
父节点
当前提交
1e3ec8df8a
共有 100 个文件被更改,包括 389 次插入385 次删除
  1. 1 1
      bin/hadoop
  2. 二进制
      lib/lucene-core-1.9-rc1-dev.jar
  3. 2 0
      src/java/org/apache/hadoop/conf/Configuration.java
  4. 1 1
      src/java/org/apache/hadoop/dfs/Block.java
  5. 1 1
      src/java/org/apache/hadoop/dfs/BlockCommand.java
  6. 1 1
      src/java/org/apache/hadoop/dfs/ClientProtocol.java
  7. 1 1
      src/java/org/apache/hadoop/dfs/DF.java
  8. 6 6
      src/java/org/apache/hadoop/dfs/DFSClient.java
  9. 1 1
      src/java/org/apache/hadoop/dfs/DFSFile.java
  10. 1 1
      src/java/org/apache/hadoop/dfs/DFSFileInfo.java
  11. 25 25
      src/java/org/apache/hadoop/dfs/DFSShell.java
  12. 2 2
      src/java/org/apache/hadoop/dfs/DataNode.java
  13. 1 1
      src/java/org/apache/hadoop/dfs/DatanodeInfo.java
  14. 1 1
      src/java/org/apache/hadoop/dfs/DatanodeProtocol.java
  15. 16 16
      src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
  16. 1 1
      src/java/org/apache/hadoop/dfs/FSConstants.java
  17. 1 1
      src/java/org/apache/hadoop/dfs/FSDataset.java
  18. 1 1
      src/java/org/apache/hadoop/dfs/FSDirectory.java
  19. 2 2
      src/java/org/apache/hadoop/dfs/FSNamesystem.java
  20. 1 1
      src/java/org/apache/hadoop/dfs/LocatedBlock.java
  21. 1 1
      src/java/org/apache/hadoop/dfs/NameNode.java
  22. 1 1
      src/java/org/apache/hadoop/fs/ChecksumException.java
  23. 14 14
      src/java/org/apache/hadoop/fs/FSDataInputStream.java
  24. 9 9
      src/java/org/apache/hadoop/fs/FSDataOutputStream.java
  25. 1 1
      src/java/org/apache/hadoop/fs/FSError.java
  26. 3 3
      src/java/org/apache/hadoop/fs/FSInputStream.java
  27. 3 3
      src/java/org/apache/hadoop/fs/FSOutputStream.java
  28. 50 50
      src/java/org/apache/hadoop/fs/FileSystem.java
  29. 22 22
      src/java/org/apache/hadoop/fs/FileUtil.java
  30. 19 19
      src/java/org/apache/hadoop/fs/LocalFileSystem.java
  31. 1 1
      src/java/org/apache/hadoop/fs/Seekable.java
  32. 5 5
      src/java/org/apache/hadoop/io/ArrayFile.java
  33. 1 1
      src/java/org/apache/hadoop/io/ArrayWritable.java
  34. 1 1
      src/java/org/apache/hadoop/io/BooleanWritable.java
  35. 1 1
      src/java/org/apache/hadoop/io/BytesWritable.java
  36. 1 1
      src/java/org/apache/hadoop/io/CompressedWritable.java
  37. 1 1
      src/java/org/apache/hadoop/io/DataInputBuffer.java
  38. 1 1
      src/java/org/apache/hadoop/io/DataOutputBuffer.java
  39. 1 1
      src/java/org/apache/hadoop/io/FloatWritable.java
  40. 1 1
      src/java/org/apache/hadoop/io/IntWritable.java
  41. 1 1
      src/java/org/apache/hadoop/io/LongWritable.java
  42. 1 1
      src/java/org/apache/hadoop/io/MD5Hash.java
  43. 31 31
      src/java/org/apache/hadoop/io/MapFile.java
  44. 1 1
      src/java/org/apache/hadoop/io/NullWritable.java
  45. 1 1
      src/java/org/apache/hadoop/io/ObjectWritable.java
  46. 38 38
      src/java/org/apache/hadoop/io/SequenceFile.java
  47. 9 9
      src/java/org/apache/hadoop/io/SetFile.java
  48. 1 1
      src/java/org/apache/hadoop/io/TwoDArrayWritable.java
  49. 1 1
      src/java/org/apache/hadoop/io/UTF8.java
  50. 1 1
      src/java/org/apache/hadoop/io/VersionMismatchException.java
  51. 1 1
      src/java/org/apache/hadoop/io/VersionedWritable.java
  52. 1 1
      src/java/org/apache/hadoop/io/Writable.java
  53. 1 1
      src/java/org/apache/hadoop/io/WritableComparable.java
  54. 1 1
      src/java/org/apache/hadoop/io/WritableComparator.java
  55. 1 1
      src/java/org/apache/hadoop/io/WritableName.java
  56. 1 1
      src/java/org/apache/hadoop/io/WritableUtils.java
  57. 1 1
      src/java/org/apache/hadoop/ipc/Client.java
  58. 1 1
      src/java/org/apache/hadoop/ipc/RPC.java
  59. 1 1
      src/java/org/apache/hadoop/ipc/Server.java
  60. 1 1
      src/java/org/apache/hadoop/mapred/CombiningCollector.java
  61. 4 4
      src/java/org/apache/hadoop/mapred/FileSplit.java
  62. 6 6
      src/java/org/apache/hadoop/mapred/InputFormat.java
  63. 6 6
      src/java/org/apache/hadoop/mapred/InputFormatBase.java
  64. 1 1
      src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java
  65. 5 5
      src/java/org/apache/hadoop/mapred/JobClient.java
  66. 2 2
      src/java/org/apache/hadoop/mapred/JobConf.java
  67. 1 1
      src/java/org/apache/hadoop/mapred/JobConfigurable.java
  68. 4 4
      src/java/org/apache/hadoop/mapred/JobInProgress.java
  69. 1 1
      src/java/org/apache/hadoop/mapred/JobProfile.java
  70. 1 1
      src/java/org/apache/hadoop/mapred/JobStatus.java
  71. 1 1
      src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java
  72. 3 3
      src/java/org/apache/hadoop/mapred/JobTracker.java
  73. 1 1
      src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java
  74. 4 4
      src/java/org/apache/hadoop/mapred/LocalJobRunner.java
  75. 1 1
      src/java/org/apache/hadoop/mapred/MRConstants.java
  76. 4 4
      src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java
  77. 3 3
      src/java/org/apache/hadoop/mapred/MapOutputFile.java
  78. 1 1
      src/java/org/apache/hadoop/mapred/MapOutputLocation.java
  79. 1 1
      src/java/org/apache/hadoop/mapred/MapOutputProtocol.java
  80. 1 1
      src/java/org/apache/hadoop/mapred/MapRunnable.java
  81. 1 1
      src/java/org/apache/hadoop/mapred/MapRunner.java
  82. 4 4
      src/java/org/apache/hadoop/mapred/MapTask.java
  83. 1 1
      src/java/org/apache/hadoop/mapred/MapTaskRunner.java
  84. 1 1
      src/java/org/apache/hadoop/mapred/Mapper.java
  85. 1 1
      src/java/org/apache/hadoop/mapred/OutputCollector.java
  86. 4 4
      src/java/org/apache/hadoop/mapred/OutputFormat.java
  87. 1 1
      src/java/org/apache/hadoop/mapred/Partitioner.java
  88. 1 1
      src/java/org/apache/hadoop/mapred/RecordReader.java
  89. 1 1
      src/java/org/apache/hadoop/mapred/RecordWriter.java
  90. 4 3
      src/java/org/apache/hadoop/mapred/ReduceTask.java
  91. 2 2
      src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java
  92. 1 1
      src/java/org/apache/hadoop/mapred/Reducer.java
  93. 1 1
      src/java/org/apache/hadoop/mapred/Reporter.java
  94. 1 1
      src/java/org/apache/hadoop/mapred/RunningJob.java
  95. 4 4
      src/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java
  96. 4 4
      src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java
  97. 3 3
      src/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java
  98. 2 1
      src/java/org/apache/hadoop/mapred/Task.java
  99. 2 2
      src/java/org/apache/hadoop/mapred/TaskInProgress.java
  100. 1 1
      src/java/org/apache/hadoop/mapred/TaskRunner.java

+ 1 - 1
bin/hadoop

@@ -115,7 +115,7 @@ if [ "$COMMAND" = "namenode" ] ; then
 elif [ "$COMMAND" = "datanode" ] ; then
   CLASS='org.apache.hadoop.dfs.DataNode'
 elif [ "$COMMAND" = "dfs" ] ; then
-  CLASS=org.apache.hadoop.fs.DFSShell
+  CLASS=org.apache.hadoop.dfs.DFSShell
 elif [ "$COMMAND" = "jobtracker" ] ; then
   CLASS=org.apache.hadoop.mapred.JobTracker
 elif [ "$COMMAND" = "tasktracker" ] ; then

二进制
lib/lucene-core-1.9-rc1-dev.jar


+ 2 - 0
src/java/org/apache/hadoop/conf/Configuration.java

@@ -29,6 +29,8 @@ import javax.xml.transform.Transformer;
 import javax.xml.transform.dom.DOMSource;
 import javax.xml.transform.stream.StreamResult;
 
+import org.apache.hadoop.util.LogFormatter;
+
 /** Provides access to configuration parameters.
  * <p>An ordered list of configuration parameter files with
  * default and always-overrides site parameters.

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import java.io.*;
 import org.apache.hadoop.io.*;

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import java.io.File;
 import java.io.IOException;

+ 6 - 6
src/java/org/apache/hadoop/dfs/NDFSClient.java → src/java/org/apache/hadoop/dfs/DFSClient.java

@@ -13,13 +13,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
+import org.apache.hadoop.util.*;
 
 import java.io.*;
 import java.net.*;
@@ -75,12 +75,12 @@ public class DFSClient implements FSConstants {
      * inner subclass of InputStream that does the right out-of-band
      * work.
      */
-    public NFSInputStream open(UTF8 src) throws IOException {
+    public FSInputStream open(UTF8 src) throws IOException {
         // Get block info from namenode
         return new DFSInputStream(src.toString());
     }
 
-    public NFSOutputStream create(UTF8 src, boolean overwrite) throws IOException {
+    public FSOutputStream create(UTF8 src, boolean overwrite) throws IOException {
         return new DFSOutputStream(src, overwrite);
     }
 
@@ -224,7 +224,7 @@ public class DFSClient implements FSConstants {
      * DFSInputStream provides bytes from a named file.  It handles 
      * negotiation of the namenode and various datanodes as necessary.
      ****************************************************************/
-    class DFSInputStream extends NFSInputStream {
+    class DFSInputStream extends FSInputStream {
         boolean closed = false;
 
         private String src;
@@ -489,7 +489,7 @@ public class DFSClient implements FSConstants {
     /****************************************************************
      * DFSOutputStream creates files from a stream of bytes.
      ****************************************************************/
-    class DFSOutputStream extends NFSOutputStream {
+    class DFSOutputStream extends FSOutputStream {
         boolean closed = false;
 
         private byte outBuf[] = new byte[BUFFER_SIZE];

+ 1 - 1
src/java/org/apache/hadoop/dfs/NDFSFile.java → src/java/org/apache/hadoop/dfs/DFSFile.java

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import java.io.*;
 

+ 1 - 1
src/java/org/apache/hadoop/dfs/NDFSFileInfo.java → src/java/org/apache/hadoop/dfs/DFSFileInfo.java

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 

+ 25 - 25
src/java/org/apache/hadoop/fs/NDFSShell.java → src/java/org/apache/hadoop/dfs/DFSShell.java

@@ -13,43 +13,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.fs.
-
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.ipc.*;
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.dfs.*;
+package org.apache.hadoop.dfs;
 
 import java.io.*;
 import java.net.*;
 import java.util.*;
 
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.ipc.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+
 /**************************************************
  * This class provides some DFS administrative access.
  *
  * @author Mike Cafarella
  **************************************************/
 public class DFSShell {
-    NutchFileSystem nfs;
+    FileSystem fs;
 
     /**
      */
-    public DFSShell(NutchFileSystem nfs) {
-        this.nfs = nfs;
+    public DFSShell(FileSystem fs) {
+        this.fs = fs;
     }
 
     /**
      * Add a local file to the indicated name in DFS. src is kept.
      */
     void copyFromLocal(File src, String dstf) throws IOException {
-        nfs.copyFromLocalFile(src, new File(dstf));
+        fs.copyFromLocalFile(src, new File(dstf));
     }
 
     /**
      * Add a local file to the indicated name in DFS. src is removed.
      */
     void moveFromLocal(File src, String dstf) throws IOException {
-        nfs.moveFromLocalFile(src, new File(dstf));
+        fs.moveFromLocalFile(src, new File(dstf));
     }
 
     /**
@@ -57,7 +57,7 @@ public class DFSShell {
      * srcf is kept.
      */
     void copyToLocal(String srcf, File dst) throws IOException {
-        nfs.copyToLocalFile(new File(srcf), dst);
+        fs.copyToLocalFile(new File(srcf), dst);
     }
 
     /**
@@ -72,7 +72,7 @@ public class DFSShell {
      * Get a listing of all files in DFS at the indicated name
      */
     public void ls(String src) throws IOException {
-        File items[] = nfs.listFiles(new File(src));
+        File items[] = fs.listFiles(new File(src));
         if (items == null) {
             System.out.println("Could not get listing for " + src);
         } else {
@@ -87,7 +87,7 @@ public class DFSShell {
     /**
      */
     public void du(String src) throws IOException {
-        File items[] = nfs.listFiles(new File(src));
+        File items[] = fs.listFiles(new File(src));
         if (items == null) {
             System.out.println("Could not get listing for " + src);
         } else {
@@ -104,14 +104,14 @@ public class DFSShell {
      */
     public void mkdir(String src) throws IOException {
         File f = new File(src);
-        nfs.mkdirs(f);
+        fs.mkdirs(f);
     }
     
     /**
      * Rename an DFS file
      */
     public void rename(String srcf, String dstf) throws IOException {
-        if (nfs.rename(new File(srcf), new File(dstf))) {
+        if (fs.rename(new File(srcf), new File(dstf))) {
             System.out.println("Renamed " + srcf + " to " + dstf);
         } else {
             System.out.println("Rename failed");
@@ -122,7 +122,7 @@ public class DFSShell {
      * Copy an DFS file
      */
     public void copy(String srcf, String dstf, Configuration conf) throws IOException {
-        if (FileUtil.copyContents(nfs, new File(srcf), new File(dstf), true, conf)) {
+        if (FileUtil.copyContents(fs, new File(srcf), new File(dstf), true, conf)) {
             System.out.println("Copied " + srcf + " to " + dstf);
         } else {
             System.out.println("Copy failed");
@@ -133,7 +133,7 @@ public class DFSShell {
      * Delete an DFS file
      */
     public void delete(String srcf) throws IOException {
-        if (nfs.delete(new File(srcf))) {
+        if (fs.delete(new File(srcf))) {
             System.out.println("Deleted " + srcf);
         } else {
             System.out.println("Delete failed");
@@ -169,18 +169,18 @@ public class DFSShell {
     }
 
     /**
-     * Gives a report on how the NutchFileSystem is doing
+     * Gives a report on how the FileSystem is doing
      */
     public void report() throws IOException {
-        if (nfs instanceof DistributedFileSystem) {
-            DistributedFileSystem dfsfs = (DistributedFileSystem) nfs;
+        if (fs instanceof DistributedFileSystem) {
+            DistributedFileSystem dfsfs = (DistributedFileSystem) fs;
             DFSClient dfs = dfsfs.getClient();
             long total = dfs.totalRawCapacity();
             long used = dfs.totalRawUsed();
             DatanodeInfo info[] = dfs.datanodeReport();
 
             long totalEffectiveBytes = 0;
-            File topItems[] = nfs.listFiles(new File("/"));
+            File topItems[] = fs.listFiles(new File("/"));
             for (int i = 0; i < topItems.length; i++) {
                 DFSFile cur = (DFSFile) topItems[i];
                 totalEffectiveBytes += cur.getContentsLength();
@@ -225,9 +225,9 @@ public class DFSShell {
 
         Configuration conf = new Configuration();
         int i = 0;
-        NutchFileSystem nfs = NutchFileSystem.parseArgs(argv, i, conf);
+        FileSystem fs = FileSystem.parseArgs(argv, i, conf);
         try {
-            DFSShell tc = new DFSShell(nfs);
+            DFSShell tc = new DFSShell(fs);
 
             String cmd = argv[i++];
             if ("-put".equals(cmd) || "-copyFromLocal".equals(cmd)) {
@@ -257,7 +257,7 @@ public class DFSShell {
             }
             System.exit(0);
         } finally {
-            nfs.close();
+            fs.close();
         }
     }
 }

+ 2 - 2
src/java/org/apache/hadoop/dfs/DataNode.java

@@ -13,12 +13,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
+import org.apache.hadoop.util.*;
 
 import java.io.*;
 import java.net.*;

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import java.io.*;
 import org.apache.hadoop.io.*;

+ 16 - 16
src/java/org/apache/hadoop/fs/NDFSFileSystem.java → src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -14,23 +14,23 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.
+package org.apache.hadoop.dfs;
 
 import java.io.*;
 import java.net.*;
 import java.util.*;
 
 import org.apache.hadoop.io.*;
-import org.apache.hadoop.dfs.*;
+import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
 
 /****************************************************************
- * Implementation of the abstract NutchFileSystem for the DFS system.
+ * Implementation of the abstract FileSystem for the DFS system.
  * This is the distributed file system.  It can be distributed over
  * 1 or more machines 
  * @author Mike Cafarella
  *****************************************************************/
-public class DistributedFileSystem extends NutchFileSystem {
+public class DistributedFileSystem extends FileSystem {
     private static final String HOME_DIR =
       "/user/" + System.getProperty("user.name") + "/";
 
@@ -63,11 +63,11 @@ public class DistributedFileSystem extends NutchFileSystem {
       return dfs.getHints(getPath(f), start, len);
     }
 
-    public NFSInputStream openRaw(File f) throws IOException {
+    public FSInputStream openRaw(File f) throws IOException {
       return dfs.open(getPath(f));
     }
 
-    public NFSOutputStream createRaw(File f, boolean overwrite)
+    public FSOutputStream createRaw(File f, boolean overwrite)
       throws IOException {
       return dfs.create(getPath(f), overwrite);
     }
@@ -195,7 +195,7 @@ public class DistributedFileSystem extends NutchFileSystem {
     }
 
     /**
-     * Takes a hierarchy of files from the NFS system and writes to
+     * Takes a hierarchy of files from the FS system and writes to
      * the given local target.
      */
     public void copyToLocalFile(File src, File dst) throws IOException {
@@ -221,7 +221,7 @@ public class DistributedFileSystem extends NutchFileSystem {
             byte buf[] = new byte[this.conf.getInt("io.file.buffer.size", 4096)];
             InputStream in = open(src);
             try {
-                OutputStream out = NutchFileSystem.getNamed("local", this.conf).create(dst);
+                OutputStream out = FileSystem.getNamed("local", this.conf).create(dst);
                 try {
                     int bytesRead = in.read(buf);
                     while (bytesRead >= 0) {
@@ -241,9 +241,9 @@ public class DistributedFileSystem extends NutchFileSystem {
      * Output will go to the tmp working area.  There may be some source
      * material that we obtain first.
      */
-    public File startLocalOutput(File nfsOutputFile, File tmpLocalFile) throws IOException {
-        if (exists(nfsOutputFile)) {
-            copyToLocalFile(nfsOutputFile, tmpLocalFile);
+    public File startLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException {
+        if (exists(fsOutputFile)) {
+            copyToLocalFile(fsOutputFile, tmpLocalFile);
         }
         return tmpLocalFile;
     }
@@ -251,15 +251,15 @@ public class DistributedFileSystem extends NutchFileSystem {
     /**
      * Move completed local data to DFS destination
      */
-    public void completeLocalOutput(File nfsOutputFile, File tmpLocalFile) throws IOException {
-        moveFromLocalFile(tmpLocalFile, nfsOutputFile);
+    public void completeLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException {
+        moveFromLocalFile(tmpLocalFile, fsOutputFile);
     }
 
     /**
      * Fetch remote DFS file, place at tmpLocalFile
      */
-    public File startLocalInput(File nfsInputFile, File tmpLocalFile) throws IOException {
-        copyToLocalFile(nfsInputFile, tmpLocalFile);
+    public File startLocalInput(File fsInputFile, File tmpLocalFile) throws IOException {
+        copyToLocalFile(fsInputFile, tmpLocalFile);
         return tmpLocalFile;
     }
 
@@ -307,7 +307,7 @@ public class DistributedFileSystem extends NutchFileSystem {
       return path.toString();
     }
 
-    public void reportChecksumFailure(File f, NFSInputStream in,
+    public void reportChecksumFailure(File f, FSInputStream in,
                                       long start, long length, int crc) {
       
       // ignore for now, causing task to fail, and hope that when task is

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.conf.Configuration;
 

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import java.io.*;
 import java.util.*;

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 

+ 2 - 2
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -13,11 +13,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
+import org.apache.hadoop.util.*;
 
 import java.io.*;
 import java.util.*;

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.dfs.
+package org.apache.hadoop.dfs;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.*;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 import java.io.IOException;
 

+ 14 - 14
src/java/org/apache/hadoop/fs/NFSDataInputStream.java → src/java/org/apache/hadoop/fs/FSDataInputStream.java

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 import java.io.*;
 import java.util.Arrays;
@@ -22,26 +22,26 @@ import java.util.zip.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.LogFormatter;
 
-/** Utility that wraps a {@link NFSInputStream} in a {@link DataInputStream}
+/** Utility that wraps a {@link FSInputStream} in a {@link DataInputStream}
  * and buffers input through a {@link BufferedInputStream}. */
-public class NFSDataInputStream extends DataInputStream {
+public class FSDataInputStream extends DataInputStream {
   private static final Logger LOG =
     LogFormatter.getLogger("org.apache.hadoop.fs.DataInputStream");
 
-  private static final byte[] VERSION = NFSDataOutputStream.CHECKSUM_VERSION;
+  private static final byte[] VERSION = FSDataOutputStream.CHECKSUM_VERSION;
   private static final int HEADER_LENGTH = 8;
   
   private int bytesPerSum = 1;
   
   /** Verify that data matches checksums. */
   private class Checker extends FilterInputStream implements Seekable {
-    private NutchFileSystem fs;
+    private FileSystem fs;
     private File file;
-    private NFSDataInputStream sums;
+    private FSDataInputStream sums;
     private Checksum sum = new CRC32();
     private int inSum;
 
-    public Checker(NutchFileSystem fs, File file, Configuration conf)
+    public Checker(FileSystem fs, File file, Configuration conf)
       throws IOException {
       super(fs.openRaw(file));
       
@@ -49,7 +49,7 @@ public class NFSDataInputStream extends DataInputStream {
       this.file = file;
       File sumFile = fs.getChecksumFile(file);
       try {
-        this.sums = new NFSDataInputStream(fs.openRaw(sumFile), conf);
+        this.sums = new FSDataInputStream(fs.openRaw(sumFile), conf);
         byte[] version = new byte[VERSION.length];
         sums.readFully(version);
         if (!Arrays.equals(version, VERSION))
@@ -117,14 +117,14 @@ public class NFSDataInputStream extends DataInputStream {
       inSum = 0;
       if (crc != sumValue) {
         long pos = getPos() - delta;
-        fs.reportChecksumFailure(file, (NFSInputStream)in,
+        fs.reportChecksumFailure(file, (FSInputStream)in,
                                  pos, bytesPerSum, crc);
         throw new ChecksumException("Checksum error: "+file+" at "+pos);
       }
     }
 
     public long getPos() throws IOException {
-      return ((NFSInputStream)in).getPos();
+      return ((FSInputStream)in).getPos();
     }
 
     public void close() throws IOException {
@@ -213,14 +213,14 @@ public class NFSDataInputStream extends DataInputStream {
 }
   
   
-  public NFSDataInputStream(NutchFileSystem fs, File file, int bufferSize, Configuration conf)
+  public FSDataInputStream(FileSystem fs, File file, int bufferSize, Configuration conf)
       throws IOException {
     super(null);
     this.in = new Buffer(new PositionCache(new Checker(fs, file, conf)), bufferSize);
   }
   
   
-  public NFSDataInputStream(NutchFileSystem fs, File file, Configuration conf)
+  public FSDataInputStream(FileSystem fs, File file, Configuration conf)
     throws IOException {
     super(null);
     int bufferSize = conf.getInt("io.file.buffer.size", 4096);
@@ -228,11 +228,11 @@ public class NFSDataInputStream extends DataInputStream {
   }
     
   /** Construct without checksums. */
-  public NFSDataInputStream(NFSInputStream in, Configuration conf) throws IOException {
+  public FSDataInputStream(FSInputStream in, Configuration conf) throws IOException {
     this(in, conf.getInt("io.file.buffer.size", 4096));
   }
   /** Construct without checksums. */
-  public NFSDataInputStream(NFSInputStream in, int bufferSize)
+  public FSDataInputStream(FSInputStream in, int bufferSize)
     throws IOException {
     super(null);
     this.in = new Buffer(new PositionCache(in), bufferSize);

+ 9 - 9
src/java/org/apache/hadoop/fs/NFSDataOutputStream.java → src/java/org/apache/hadoop/fs/FSDataOutputStream.java

@@ -13,33 +13,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 import java.io.*;
 import java.util.zip.Checksum;
 import java.util.zip.CRC32;
 import org.apache.hadoop.conf.Configuration;
 
-/** Utility that wraps a {@link NFSOutputStream} in a {@link DataOutputStream},
+/** Utility that wraps a {@link FSOutputStream} in a {@link DataOutputStream},
  * buffers output through a {@link BufferedOutputStream} and creates a checksum
  * file. */
-public class NFSDataOutputStream extends DataOutputStream {
+public class FSDataOutputStream extends DataOutputStream {
   public static final byte[] CHECKSUM_VERSION = new byte[] {'c', 'r', 'c', 0};
   
   /** Store checksums for data. */
   private static class Summer extends FilterOutputStream {
 
-    private NFSDataOutputStream sums;
+    private FSDataOutputStream sums;
     private Checksum sum = new CRC32();
     private int inSum;
     private int bytesPerSum;
 
-    public Summer(NutchFileSystem fs, File file, boolean overwrite, Configuration conf)
+    public Summer(FileSystem fs, File file, boolean overwrite, Configuration conf)
       throws IOException {
       super(fs.createRaw(file, overwrite));
       this.bytesPerSum = conf.getInt("io.bytes.per.checksum", 512);
       this.sums =
-        new NFSDataOutputStream(fs.createRaw(fs.getChecksumFile(file), true), conf);
+        new FSDataOutputStream(fs.createRaw(fs.getChecksumFile(file), true), conf);
 
       sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
       sums.writeInt(this.bytesPerSum);
@@ -121,7 +121,7 @@ public class NFSDataOutputStream extends DataOutputStream {
 
   }
 
-  public NFSDataOutputStream(NutchFileSystem fs, File file,
+  public FSDataOutputStream(FileSystem fs, File file,
                              boolean overwrite, Configuration conf)
     throws IOException {
     super(new Buffer(new PositionCache(new Summer(fs, file, overwrite, conf)),
@@ -129,12 +129,12 @@ public class NFSDataOutputStream extends DataOutputStream {
   }
 
   /** Construct without checksums. */
-  public NFSDataOutputStream(NFSOutputStream out, Configuration conf) throws IOException {
+  public FSDataOutputStream(FSOutputStream out, Configuration conf) throws IOException {
     this(out, conf.getInt("io.file.buffer.size", 4096));
   }
 
   /** Construct without checksums. */
-  public NFSDataOutputStream(NFSOutputStream out, int bufferSize)
+  public FSDataOutputStream(FSOutputStream out, int bufferSize)
     throws IOException {
     super(new Buffer(new PositionCache(out), bufferSize));
   }

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 /** Thrown for unexpected filesystem errors, presumed to reflect disk errors
  * in the native filesystem. */

+ 3 - 3
src/java/org/apache/hadoop/fs/NFSInputStream.java → src/java/org/apache/hadoop/fs/FSInputStream.java

@@ -13,17 +13,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 import java.io.*;
 
 /****************************************************************
- * NFSInputStream is a generic old InputStream with a little bit
+ * FSInputStream is a generic old InputStream with a little bit
  * of RAF-style seek ability.
  *
  * @author Mike Cafarella
  *****************************************************************/
-public abstract class NFSInputStream extends InputStream implements Seekable {
+public abstract class FSInputStream extends InputStream implements Seekable {
     /**
      * Seek to the given offset from the start of the file.
      * The next read() will be from that location.  Can't

+ 3 - 3
src/java/org/apache/hadoop/fs/NFSOutputStream.java → src/java/org/apache/hadoop/fs/FSOutputStream.java

@@ -13,16 +13,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 import java.io.*;
 
 /****************************************************************
- * NFSOutputStream is an OutputStream that can track its position.
+ * FSOutputStream is an OutputStream that can track its position.
  *
  * @author Mike Cafarella
  *****************************************************************/
-public abstract class NFSOutputStream extends OutputStream {
+public abstract class FSOutputStream extends OutputStream {
     /**
      * Return the current offset from the start of the file
      */

+ 50 - 50
src/java/org/apache/hadoop/fs/NutchFileSystem.java → src/java/org/apache/hadoop/fs/FileSystem.java

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 import java.io.*;
 import java.net.*;
@@ -34,7 +34,7 @@ import org.apache.hadoop.util.LogFormatter;
  * <p>
  * A local implementation exists for testing and for small Nutch instances.
  * <p>
- * The standard job of NutchFileSystem is to take the location-
+ * The standard job of FileSystem is to take the location-
  * independent NutchFile objects, and resolve them using local
  * knowledge and local instances of ShareGroup.
  * <p>
@@ -42,7 +42,7 @@ import org.apache.hadoop.util.LogFormatter;
  * implementation is {@link DistributedFileSystem}.
  * @author Mike Cafarella
  *****************************************************************/
-public abstract class NutchFileSystem {
+public abstract class FileSystem {
     public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.dfs.DistributedFileSystem");
 
     private static final HashMap NAME_TO_FS = new HashMap();
@@ -53,48 +53,48 @@ public abstract class NutchFileSystem {
      *
      * @deprecated use fs.default.name config option instead
      */
-    public static NutchFileSystem parseArgs(String argv[], int i, Configuration conf) throws IOException {
+    public static FileSystem parseArgs(String argv[], int i, Configuration conf) throws IOException {
         /**
         if (argv.length - i < 1) {
             throw new IOException("Must indicate filesystem type for DFS");
         }
         */
         int orig = i;
-        NutchFileSystem nfs = null;
+        FileSystem fs = null;
         String cmd = argv[i];
         if ("-dfs".equals(cmd)) {
             i++;
             InetSocketAddress addr = DataNode.createSocketAddr(argv[i++]);
-            nfs = new DistributedFileSystem(addr, conf);
+            fs = new DistributedFileSystem(addr, conf);
         } else if ("-local".equals(cmd)) {
             i++;
-            nfs = new LocalFileSystem(conf);
+            fs = new LocalFileSystem(conf);
         } else {
-            nfs = get(conf);                          // using default
-            LOG.info("No FS indicated, using default:"+nfs.getName());
+            fs = get(conf);                          // using default
+            LOG.info("No FS indicated, using default:"+fs.getName());
 
         }
         System.arraycopy(argv, i, argv, orig, argv.length - i);
         for (int j = argv.length - i; j < argv.length; j++) {
             argv[j] = null;
         }
-        return nfs;
+        return fs;
     }
 
     /** Returns the configured filesystem implementation.*/
-    public static NutchFileSystem get(Configuration conf) throws IOException {
+    public static FileSystem get(Configuration conf) throws IOException {
       return getNamed(conf.get("fs.default.name", "local"), conf);
     }
 
     protected Configuration conf;
     /** Returns a name for this filesystem, suitable to pass to {@link
-     * NutchFileSystem#getNamed(String).*/
+     * FileSystem#getNamed(String).*/
     public abstract String getName();
   
     /** Returns a named filesystem.  Names are either the string "local" or a
      * host:port pair, naming an DFS name server.*/
-    public static NutchFileSystem getNamed(String name, Configuration conf) throws IOException {
-      NutchFileSystem fs = (NutchFileSystem)NAME_TO_FS.get(name);
+    public static FileSystem getNamed(String name, Configuration conf) throws IOException {
+      FileSystem fs = (FileSystem)NAME_TO_FS.get(name);
       int ioFileBufferSize = conf.getInt("io.file.buffer.size", 4096);
       if (fs == null) {
         if ("local".equals(name)) {
@@ -119,11 +119,11 @@ public abstract class NutchFileSystem {
     }
 
     ///////////////////////////////////////////////////////////////
-    // NutchFileSystem
+    // FileSystem
     ///////////////////////////////////////////////////////////////
     /**
      */
-    public NutchFileSystem(Configuration conf) {
+    public FileSystem(Configuration conf) {
         this.conf = conf;
     }
 
@@ -135,56 +135,56 @@ public abstract class NutchFileSystem {
      * This call is most helpful with DFS, where it returns 
      * hostnames of machines that contain the given file.
      *
-     * The NutchFileSystem will simply return an elt containing 'localhost'.
+     * The FileSystem will simply return an elt containing 'localhost'.
      */
     public abstract String[][] getFileCacheHints(File f, long start, long len) throws IOException;
 
     /**
-     * Opens an NFSDataInputStream at the indicated File.
+     * Opens an FSDataInputStream at the indicated File.
      * @param f the file name to open
      * @param overwrite if a file with this name already exists, then if true,
      *   the file will be overwritten, and if false an error will be thrown.
      * @param bufferSize the size of the buffer to be used.
      */
-    public NFSDataInputStream open(File f, int bufferSize) throws IOException {
-      return new NFSDataInputStream(this, f, bufferSize, this.conf);
+    public FSDataInputStream open(File f, int bufferSize) throws IOException {
+      return new FSDataInputStream(this, f, bufferSize, this.conf);
     }
     
     /**
-     * Opens an NFSDataInputStream at the indicated File.
+     * Opens an FSDataInputStream at the indicated File.
      * @param f the file name to open
      * @param overwrite if a file with this name already exists, then if true,
      *   the file will be overwritten, and if false an error will be thrown.
      * @param bufferSize the size of the buffer to be used.
      */
-    public NFSDataInputStream open(File f) throws IOException {
-      return new NFSDataInputStream(this, f, conf);
+    public FSDataInputStream open(File f) throws IOException {
+      return new FSDataInputStream(this, f, conf);
     }
 
     /**
      * Opens an InputStream for the indicated File, whether local
      * or via DFS.
      */
-    public abstract NFSInputStream openRaw(File f) throws IOException;
+    public abstract FSInputStream openRaw(File f) throws IOException;
 
     /**
-     * Opens an NFSDataOutputStream at the indicated File.
+     * Opens an FSDataOutputStream at the indicated File.
      * Files are overwritten by default.
      */
-    public NFSDataOutputStream create(File f) throws IOException {
+    public FSDataOutputStream create(File f) throws IOException {
       return create(f, true,this.conf.getInt("io.file.buffer.size", 4096));
     }
 
     /**
-     * Opens an NFSDataOutputStream at the indicated File.
+     * Opens an FSDataOutputStream at the indicated File.
      * @param f the file name to open
      * @param overwrite if a file with this name already exists, then if true,
      *   the file will be overwritten, and if false an error will be thrown.
      * @param bufferSize the size of the buffer to be used.
      */
-    public NFSDataOutputStream create(File f, boolean overwrite,
+    public FSDataOutputStream create(File f, boolean overwrite,
                                       int bufferSize) throws IOException {
-      return new NFSDataOutputStream(this, f, overwrite, this.conf);
+      return new FSDataOutputStream(this, f, overwrite, this.conf);
     }
 
     /** Opens an OutputStream at the indicated File.
@@ -192,7 +192,7 @@ public abstract class NutchFileSystem {
      * @param overwrite if a file with this name already exists, then if true,
      *   the file will be overwritten, and if false an error will be thrown.
      */
-    public abstract NFSOutputStream createRaw(File f, boolean overwrite)
+    public abstract FSOutputStream createRaw(File f, boolean overwrite)
       throws IOException;
 
     /**
@@ -312,21 +312,21 @@ public abstract class NutchFileSystem {
     public abstract void release(File f) throws IOException;
 
     /**
-     * The src file is on the local disk.  Add it to NFS at
+     * The src file is on the local disk.  Add it to FS at
      * the given dst name and the source is kept intact afterwards
      */
     // not implemneted yet
     public abstract void copyFromLocalFile(File src, File dst) throws IOException;
 
     /**
-     * The src file is on the local disk.  Add it to NFS at
+     * The src file is on the local disk.  Add it to FS at
      * the given dst name, removing the source afterwards.
      */
     public abstract void moveFromLocalFile(File src, File dst) throws IOException;
 
     /**
-     * The src file is under NFS2, and the dst is on the local disk.
-     * Copy it from NFS control to the local dst name.
+     * The src file is under FS2, and the dst is on the local disk.
+     * Copy it from FS control to the local dst name.
      */
     public abstract void copyToLocalFile(File src, File dst) throws IOException;
 
@@ -339,33 +339,33 @@ public abstract class NutchFileSystem {
 
     /**
      * Returns a local File that the user can write output to.  The caller
-     * provides both the eventual NFS target name and the local working
-     * file.  If the NFS is local, we write directly into the target.  If
-     * the NFS is remote, we write into the tmp local area.
+     * provides both the eventual FS target name and the local working
+     * file.  If the FS is local, we write directly into the target.  If
+     * the FS is remote, we write into the tmp local area.
      */
-    public abstract File startLocalOutput(File nfsOutputFile, File tmpLocalFile) throws IOException;
+    public abstract File startLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException;
 
     /**
-     * Called when we're all done writing to the target.  A local NFS will
+     * Called when we're all done writing to the target.  A local FS will
      * do nothing, because we've written to exactly the right place.  A remote
-     * NFS will copy the contents of tmpLocalFile to the correct target at
-     * nfsOutputFile.
+     * FS will copy the contents of tmpLocalFile to the correct target at
+     * fsOutputFile.
      */
-    public abstract void completeLocalOutput(File nfsOutputFile, File tmpLocalFile) throws IOException;
+    public abstract void completeLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException;
 
     /**
      * Returns a local File that the user can read from.  The caller 
-     * provides both the eventual NFS target name and the local working
-     * file.  If the NFS is local, we read directly from the source.  If
-     * the NFS is remote, we write data into the tmp local area.
+     * provides both the eventual FS target name and the local working
+     * file.  If the FS is local, we read directly from the source.  If
+     * the FS is remote, we write data into the tmp local area.
      */
-    public abstract File startLocalInput(File nfsInputFile, File tmpLocalFile) throws IOException;
+    public abstract File startLocalInput(File fsInputFile, File tmpLocalFile) throws IOException;
 
     /**
-     * Called when we're all done writing to the target.  A local NFS will
+     * Called when we're all done writing to the target.  A local FS will
      * do nothing, because we've written to exactly the right place.  A remote
-     * NFS will copy the contents of tmpLocalFile to the correct target at
-     * nfsOutputFile.
+     * FS will copy the contents of tmpLocalFile to the correct target at
+     * fsOutputFile.
      */
     public abstract void completeLocalInput(File localFile) throws IOException;
 
@@ -383,7 +383,7 @@ public abstract class NutchFileSystem {
      * @param length the length of the bad data in the file
      * @param crc the expected CRC32 of the data
      */
-    public abstract void reportChecksumFailure(File f, NFSInputStream in,
+    public abstract void reportChecksumFailure(File f, FSInputStream in,
                                                long start, long length,
                                                int crc);
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 import java.io.*;
 
@@ -31,32 +31,32 @@ public class FileUtil {
     public static boolean fullyDelete(File dir, Configuration conf) throws IOException {
         return fullyDelete(new LocalFileSystem(conf), dir);
     }
-    public static boolean fullyDelete(NutchFileSystem nfs, File dir) throws IOException {
+    public static boolean fullyDelete(FileSystem fs, File dir) throws IOException {
         // 20041022, xing.
-        // Currently nfs.detele(File) means fully delete for both
+        // Currently fs.detele(File) means fully delete for both
         // LocalFileSystem.java and DistributedFileSystem.java. So we are okay now.
         // If implementation changes in future, it should be modified too.
-        return nfs.delete(dir);
+        return fs.delete(dir);
     }
 
     /**
      * Copy a file's contents to a new location.
      * Returns whether a target file was overwritten
      */
-    public static boolean copyContents(NutchFileSystem nfs, File src, File dst, boolean overwrite, Configuration conf) throws IOException {
-        if (nfs.exists(dst) && !overwrite) {
+    public static boolean copyContents(FileSystem fs, File src, File dst, boolean overwrite, Configuration conf) throws IOException {
+        if (fs.exists(dst) && !overwrite) {
             return false;
         }
 
         File dstParent = dst.getParentFile();
-        if ((dstParent != null) && (!nfs.exists(dstParent))) {
-            nfs.mkdirs(dstParent);
+        if ((dstParent != null) && (!fs.exists(dstParent))) {
+            fs.mkdirs(dstParent);
         }
 
-        if (nfs.isFile(src)) {
-            NFSInputStream in = nfs.openRaw(src);
+        if (fs.isFile(src)) {
+            FSInputStream in = fs.openRaw(src);
             try {
-                NFSOutputStream out = nfs.createRaw(dst, true);
+                FSOutputStream out = fs.createRaw(dst, true);
                 byte buf[] = new byte[conf.getInt("io.file.buffer.size", 4096)];
                 try {
                     int readBytes = in.read(buf);
@@ -72,12 +72,12 @@ public class FileUtil {
                 in.close();
             }
         } else {
-            nfs.mkdirs(dst);
-            File contents[] = nfs.listFilesRaw(src);
+            fs.mkdirs(dst);
+            File contents[] = fs.listFilesRaw(src);
             if (contents != null) {
                 for (int i = 0; i < contents.length; i++) {
                     File newDst = new File(dst, contents[i].getName());
-                    if (! copyContents(nfs, contents[i], newDst, overwrite, conf)) {
+                    if (! copyContents(fs, contents[i], newDst, overwrite, conf)) {
                         return false;
                     }
                 }
@@ -90,32 +90,32 @@ public class FileUtil {
      * Copy a file and/or directory and all its contents (whether
      * data or other files/dirs)
      */
-    public static void recursiveCopy(NutchFileSystem nfs, File src, File dst, Configuration conf) throws IOException {
+    public static void recursiveCopy(FileSystem fs, File src, File dst, Configuration conf) throws IOException {
         //
         // Resolve the real target.
         //
-        if (nfs.exists(dst) && nfs.isDirectory(dst)) {
+        if (fs.exists(dst) && fs.isDirectory(dst)) {
             dst = new File(dst, src.getName());
-        } else if (nfs.exists(dst)) {
+        } else if (fs.exists(dst)) {
             throw new IOException("Destination " + dst + " already exists");
         }
 
         //
         // Copy the items
         //
-        if (! nfs.isDirectory(src)) {
+        if (! fs.isDirectory(src)) {
             //
             // If the source is a file, then just copy the contents
             //
-            copyContents(nfs, src, dst, true, conf);
+            copyContents(fs, src, dst, true, conf);
         } else {
             //
             // If the source is a dir, then we need to copy all the subfiles.
             //
-            nfs.mkdirs(dst);
-            File contents[] = nfs.listFiles(src);
+            fs.mkdirs(dst);
+            File contents[] = fs.listFiles(src);
             for (int i = 0; i < contents.length; i++) {
-                recursiveCopy(nfs, contents[i], new File(dst, contents[i].getName()), conf);
+                recursiveCopy(fs, contents[i], new File(dst, contents[i].getName()), conf);
             }
         }
     }

+ 19 - 19
src/java/org/apache/hadoop/fs/LocalFileSystem.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 import java.io.*;
 import java.util.*;
@@ -27,13 +27,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.UTF8;
 
 /****************************************************************
- * Implement the NutchFileSystem interface for the local disk.
+ * Implement the FileSystem interface for the local disk.
  * This is pretty easy.  The interface exists so we can use either
  * remote or local Files very easily.
  *
  * @author Mike Cafarella
  *****************************************************************/
-public class LocalFileSystem extends NutchFileSystem {
+public class LocalFileSystem extends FileSystem {
     TreeMap sharedLockDataSet = new TreeMap();
     TreeMap nonsharedLockDataSet = new TreeMap();
     TreeMap lockObjSet = new TreeMap();
@@ -70,12 +70,12 @@ public class LocalFileSystem extends NutchFileSystem {
     public String getName() { return "local"; }
 
     /*******************************************************
-     * For open()'s NFSInputStream
+     * For open()'s FSInputStream
      *******************************************************/
-    class LocalNFSFileInputStream extends NFSInputStream {
+    class LocalFSFileInputStream extends FSInputStream {
         FileInputStream fis;
 
-        public LocalNFSFileInputStream(File f) throws IOException {
+        public LocalFSFileInputStream(File f) throws IOException {
           this.fis = new FileInputStream(f);
         }
 
@@ -116,20 +116,20 @@ public class LocalFileSystem extends NutchFileSystem {
     /**
      * Open the file at f
      */
-    public NFSInputStream openRaw(File f) throws IOException {
+    public FSInputStream openRaw(File f) throws IOException {
         if (! f.exists()) {
             throw new FileNotFoundException(f.toString());
         }
-        return new LocalNFSFileInputStream(f);
+        return new LocalFSFileInputStream(f);
     }
 
     /*********************************************************
-     * For create()'s NFSOutputStream.
+     * For create()'s FSOutputStream.
      *********************************************************/
-    class LocalNFSFileOutputStream extends NFSOutputStream {
+    class LocalFSFileOutputStream extends FSOutputStream {
       FileOutputStream fos;
 
-      public LocalNFSFileOutputStream(File f) throws IOException {
+      public LocalFSFileOutputStream(File f) throws IOException {
         this.fos = new FileOutputStream(f);
       }
 
@@ -159,7 +159,7 @@ public class LocalFileSystem extends NutchFileSystem {
       }
     }
 
-    public NFSOutputStream createRaw(File f, boolean overwrite)
+    public FSOutputStream createRaw(File f, boolean overwrite)
       throws IOException {
         if (f.exists() && ! overwrite) {
             throw new IOException("File already exists:"+f);
@@ -168,7 +168,7 @@ public class LocalFileSystem extends NutchFileSystem {
         if (parent != null)
           parent.mkdirs();
 
-        return new LocalNFSFileOutputStream(f);
+        return new LocalFSFileOutputStream(f);
     }
 
     /**
@@ -316,21 +316,21 @@ public class LocalFileSystem extends NutchFileSystem {
     /**
      * We can write output directly to the final location
      */
-    public File startLocalOutput(File nfsOutputFile, File tmpLocalFile) throws IOException {
-        return nfsOutputFile;
+    public File startLocalOutput(File fsOutputFile, File tmpLocalFile) throws IOException {
+        return fsOutputFile;
     }
 
     /**
      * It's in the right place - nothing to do.
      */
-    public void completeLocalOutput(File nfsWorkingFile, File tmpLocalFile) throws IOException {
+    public void completeLocalOutput(File fsWorkingFile, File tmpLocalFile) throws IOException {
     }
 
     /**
      * We can read directly from the real local fs.
      */
-    public File startLocalInput(File nfsInputFile, File tmpLocalFile) throws IOException {
-        return nfsInputFile;
+    public File startLocalInput(File fsInputFile, File tmpLocalFile) throws IOException {
+        return fsInputFile;
     }
 
     /**
@@ -379,7 +379,7 @@ public class LocalFileSystem extends NutchFileSystem {
 
     /** Moves files to a bad file directory on the same device, so that their
      * storage will not be reused. */
-    public void reportChecksumFailure(File f, NFSInputStream in,
+    public void reportChecksumFailure(File f, FSInputStream in,
                                       long start, long length, int crc) {
       try {
         // canonicalize f   

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
 
 import java.io.*;
 

+ 5 - 5
src/java/org/apache/hadoop/io/ArrayFile.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 import org.apache.hadoop.fs.*;
@@ -30,8 +30,8 @@ public class ArrayFile extends MapFile {
     private LongWritable count = new LongWritable(0);
 
     /** Create the named file for values of the named class. */
-    public Writer(NutchFileSystem nfs, String file, Class valClass) throws IOException {
-      super(nfs, file, LongWritable.class, valClass);
+    public Writer(FileSystem fs, String file, Class valClass) throws IOException {
+      super(fs, file, LongWritable.class, valClass);
     }
 
     /** Append a value to the file. */
@@ -46,8 +46,8 @@ public class ArrayFile extends MapFile {
     private LongWritable key = new LongWritable();
 
     /** Construct an array reader for the named file.*/
-    public Reader(NutchFileSystem nfs, String file, Configuration conf) throws IOException {
-      super(nfs, file, conf);
+    public Reader(FileSystem fs, String file, Configuration conf) throws IOException {
+      super(fs, file, conf);
     }
 
     /** Positions the reader before its <code>n</code>th value. */

+ 1 - 1
src/java/org/apache/hadoop/io/ArrayWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 import java.lang.reflect.Array;

+ 1 - 1
src/java/org/apache/hadoop/io/BooleanWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 

+ 1 - 1
src/java/org/apache/hadoop/io/BytesWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.IOException;
 import java.io.DataInput;

+ 1 - 1
src/java/org/apache/hadoop/io/CompressedWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.IOException;
 import java.io.DataInput;

+ 1 - 1
src/java/org/apache/hadoop/io/DataInputBuffer.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 

+ 1 - 1
src/java/org/apache/hadoop/io/DataOutputBuffer.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 

+ 1 - 1
src/java/org/apache/hadoop/io/FloatWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 

+ 1 - 1
src/java/org/apache/hadoop/io/IntWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 

+ 1 - 1
src/java/org/apache/hadoop/io/LongWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 

+ 1 - 1
src/java/org/apache/hadoop/io/MD5Hash.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.IOException;
 import java.io.DataInput;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 import java.util.Arrays;
@@ -64,27 +64,27 @@ public class MapFile {
 
 
     /** Create the named map for keys of the named class. */
-    public Writer(NutchFileSystem nfs, String dirName,
+    public Writer(FileSystem fs, String dirName,
                   Class keyClass, Class valClass)
       throws IOException {
-      this(nfs, dirName, WritableComparator.get(keyClass), valClass, false);
+      this(fs, dirName, WritableComparator.get(keyClass), valClass, false);
     }
 
     /** Create the named map for keys of the named class. */
-    public Writer(NutchFileSystem nfs, String dirName,
+    public Writer(FileSystem fs, String dirName,
                   Class keyClass, Class valClass, boolean compress)
       throws IOException {
-      this(nfs, dirName, WritableComparator.get(keyClass), valClass, compress);
+      this(fs, dirName, WritableComparator.get(keyClass), valClass, compress);
     }
 
     /** Create the named map using the named key comparator. */
-    public Writer(NutchFileSystem nfs, String dirName,
+    public Writer(FileSystem fs, String dirName,
                   WritableComparator comparator, Class valClass)
       throws IOException {
-      this(nfs, dirName, comparator, valClass, false);
+      this(fs, dirName, comparator, valClass, false);
     }
     /** Create the named map using the named key comparator. */
-    public Writer(NutchFileSystem nfs, String dirName,
+    public Writer(FileSystem fs, String dirName,
                   WritableComparator comparator, Class valClass,
                   boolean compress)
       throws IOException {
@@ -93,17 +93,17 @@ public class MapFile {
       this.lastKey = comparator.newKey();
 
       File dir = new File(dirName);
-      nfs.mkdirs(dir);
+      fs.mkdirs(dir);
 
       File dataFile = new File(dir, DATA_FILE_NAME);
       File indexFile = new File(dir, INDEX_FILE_NAME);
 
       Class keyClass = comparator.getKeyClass();
       this.data =
-        new SequenceFile.Writer(nfs, dataFile.getPath(), keyClass, valClass,
+        new SequenceFile.Writer(fs, dataFile.getPath(), keyClass, valClass,
                                 compress);
       this.index =
-        new SequenceFile.Writer(nfs, indexFile.getPath(),
+        new SequenceFile.Writer(fs, indexFile.getPath(),
                                 keyClass, LongWritable.class);
     }
     
@@ -190,20 +190,20 @@ public class MapFile {
     public Class getValueClass() { return data.getValueClass(); }
 
     /** Construct a map reader for the named map.*/
-    public Reader(NutchFileSystem nfs, String dirName, Configuration conf) throws IOException {
-      this(nfs, dirName, null, conf);
+    public Reader(FileSystem fs, String dirName, Configuration conf) throws IOException {
+      this(fs, dirName, null, conf);
       INDEX_SKIP = conf.getInt("io.map.index.skip", 0);
     }
 
     /** Construct a map reader for the named map using the named comparator.*/
-    public Reader(NutchFileSystem nfs, String dirName, WritableComparator comparator, Configuration conf)
+    public Reader(FileSystem fs, String dirName, WritableComparator comparator, Configuration conf)
       throws IOException {
       File dir = new File(dirName);
       File dataFile = new File(dir, DATA_FILE_NAME);
       File indexFile = new File(dir, INDEX_FILE_NAME);
 
       // open the data
-      this.data = new SequenceFile.Reader(nfs, dataFile.getPath(),  conf);
+      this.data = new SequenceFile.Reader(fs, dataFile.getPath(),  conf);
       this.firstPosition = data.getPosition();
 
       if (comparator == null)
@@ -214,7 +214,7 @@ public class MapFile {
       this.getKey = this.comparator.newKey();
 
       // open the index
-      this.index = new SequenceFile.Reader(nfs, indexFile.getPath(), conf);
+      this.index = new SequenceFile.Reader(fs, indexFile.getPath(), conf);
     }
 
     private void readIndex() throws IOException {
@@ -386,29 +386,29 @@ public class MapFile {
   }
 
   /** Renames an existing map directory. */
-  public static void rename(NutchFileSystem nfs, String oldName, String newName)
+  public static void rename(FileSystem fs, String oldName, String newName)
     throws IOException {
     File oldDir = new File(oldName);
     File newDir = new File(newName);
-    if (!nfs.rename(oldDir, newDir)) {
+    if (!fs.rename(oldDir, newDir)) {
       throw new IOException("Could not rename " + oldDir + " to " + newDir);
     }
   }
 
   /** Deletes the named map file. */
-  public static void delete(NutchFileSystem nfs, String name) throws IOException {
+  public static void delete(FileSystem fs, String name) throws IOException {
     File dir = new File(name);
     File data = new File(dir, DATA_FILE_NAME);
     File index = new File(dir, INDEX_FILE_NAME);
 
-    nfs.delete(data);
-    nfs.delete(index);
-    nfs.delete(dir);
+    fs.delete(data);
+    fs.delete(index);
+    fs.delete(dir);
   }
 
   /**
    * This method attempts to fix a corrupt MapFile by re-creating its index.
-   * @param nfs filesystem
+   * @param fs filesystem
    * @param dir directory containing the MapFile data and index
    * @param keyClass key class (has to be a subclass of Writable)
    * @param valueClass value class (has to be a subclass of Writable)
@@ -416,21 +416,21 @@ public class MapFile {
    * @return number of valid entries in this MapFile, or -1 if no fixing was needed
    * @throws Exception
    */
-  public static long fix(NutchFileSystem nfs, File dir,
+  public static long fix(FileSystem fs, File dir,
           Class keyClass, Class valueClass, boolean dryrun, Configuration conf) throws Exception {
     String dr = (dryrun ? "[DRY RUN ] " : "");
     File data = new File(dir, DATA_FILE_NAME);
     File index = new File(dir, INDEX_FILE_NAME);
     int indexInterval = 128;
-    if (!nfs.exists(data)) {
+    if (!fs.exists(data)) {
       // there's nothing we can do to fix this!
       throw new Exception(dr + "Missing data file in " + dir + ", impossible to fix this.");
     }
-    if (nfs.exists(index)) {
+    if (fs.exists(index)) {
       // no fixing needed
       return -1;
     }
-    SequenceFile.Reader dataReader = new SequenceFile.Reader(nfs, data.toString(), conf);
+    SequenceFile.Reader dataReader = new SequenceFile.Reader(fs, data.toString(), conf);
     if (!dataReader.getKeyClass().equals(keyClass)) {
       throw new Exception(dr + "Wrong key class in " + dir + ", expected" + keyClass.getName() +
               ", got " + dataReader.getKeyClass().getName());
@@ -443,7 +443,7 @@ public class MapFile {
     Writable key = (Writable)keyClass.getConstructor(new Class[0]).newInstance(new Object[0]);
     Writable value = (Writable)valueClass.getConstructor(new Class[0]).newInstance(new Object[0]);
     SequenceFile.Writer indexWriter = null;
-    if (!dryrun) indexWriter = new SequenceFile.Writer(nfs, index.toString(), keyClass, LongWritable.class);
+    if (!dryrun) indexWriter = new SequenceFile.Writer(fs, index.toString(), keyClass, LongWritable.class);
     try {
       long pos = 0L;
       LongWritable position = new LongWritable();
@@ -477,10 +477,10 @@ public class MapFile {
 
     Configuration conf = new Configuration();
     int ioFileBufferSize = conf.getInt("io.file.buffer.size", 4096);
-    NutchFileSystem nfs = new LocalFileSystem(conf);
-    MapFile.Reader reader = new MapFile.Reader(nfs, in, conf);
+    FileSystem fs = new LocalFileSystem(conf);
+    MapFile.Reader reader = new MapFile.Reader(fs, in, conf);
     MapFile.Writer writer =
-      new MapFile.Writer(nfs, out, reader.getKeyClass(), reader.getValueClass());
+      new MapFile.Writer(fs, out, reader.getKeyClass(), reader.getValueClass());
 
     WritableComparable key =
       (WritableComparable)reader.getKeyClass().newInstance();

+ 1 - 1
src/java/org/apache/hadoop/io/NullWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 

+ 1 - 1
src/java/org/apache/hadoop/io/ObjectWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.lang.reflect.Proxy;
 import java.lang.reflect.Method;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 import java.util.*;
@@ -49,9 +49,9 @@ public class SequenceFile {
 
   /** Write key/value pairs to a sequence-format file. */
   public static class Writer {
-    private NFSDataOutputStream out;
+    private FSDataOutputStream out;
     private DataOutputBuffer buffer = new DataOutputBuffer();
-    private NutchFileSystem nfs = null;
+    private FileSystem fs = null;
     private File target = null;
 
     private Class keyClass;
@@ -78,32 +78,32 @@ public class SequenceFile {
     }
 
     /** Create the named file. */
-    public Writer(NutchFileSystem nfs, String name,
+    public Writer(FileSystem fs, String name,
                   Class keyClass, Class valClass)
       throws IOException {
-      this(nfs, name, keyClass, valClass, false);
+      this(fs, name, keyClass, valClass, false);
     }
     
     /** Create the named file.
      * @param compress if true, values are compressed.
      */
-    public Writer(NutchFileSystem nfs, String name,
+    public Writer(FileSystem fs, String name,
                   Class keyClass, Class valClass, boolean compress)
       throws IOException {
-      this.nfs = nfs;
+      this.fs = fs;
       this.target = new File(name);
-      init(nfs.create(target), keyClass, valClass, compress);
+      init(fs.create(target), keyClass, valClass, compress);
     }
     
     /** Write to an arbitrary stream using a specified buffer size. */
-    private Writer(NFSDataOutputStream out,
+    private Writer(FSDataOutputStream out,
                    Class keyClass, Class valClass, boolean compress)
       throws IOException {
       init(out, keyClass, valClass, compress);
     }
     
     /** Write and flush the file header. */
-    private void init(NFSDataOutputStream out,
+    private void init(FSDataOutputStream out,
                       Class keyClass, Class valClass,
                       boolean compress) throws IOException {
       this.out = out;
@@ -202,10 +202,10 @@ public class SequenceFile {
   /** Writes key/value pairs from a sequence-format file. */
   public static class Reader {
     private String file;
-    private NFSDataInputStream in;
+    private FSDataInputStream in;
     private DataOutputBuffer outBuf = new DataOutputBuffer();
     private DataInputBuffer inBuf = new DataInputBuffer();
-    private NutchFileSystem nfs = null;
+    private FileSystem fs = null;
 
     private byte[] version = new byte[VERSION.length];
 
@@ -226,25 +226,25 @@ public class SequenceFile {
     private Configuration conf;
 
     /** Open the named file. */
-    public Reader(NutchFileSystem nfs, String file, Configuration conf) throws IOException {
-      this(nfs, file, conf.getInt("io.file.buffer.size", 4096));
+    public Reader(FileSystem fs, String file, Configuration conf) throws IOException {
+      this(fs, file, conf.getInt("io.file.buffer.size", 4096));
       this.conf = conf;
     }
 
-    private Reader(NutchFileSystem nfs, String name, int bufferSize) throws IOException {
-      this.nfs = nfs;
+    private Reader(FileSystem fs, String name, int bufferSize) throws IOException {
+      this.fs = fs;
       this.file = name;
       File file = new File(name);
-      this.in = nfs.open(file, bufferSize);
-      this.end = nfs.getLength(file);
+      this.in = fs.open(file, bufferSize);
+      this.end = fs.getLength(file);
       init();
     }
     
-    private Reader(NutchFileSystem nfs, String file, int bufferSize, long start, long length)
+    private Reader(FileSystem fs, String file, int bufferSize, long start, long length)
       throws IOException {
-      this.nfs = nfs;
+      this.fs = fs;
       this.file = file;
-      this.in = nfs.open(new File(file), bufferSize);
+      this.in = fs.open(new File(file), bufferSize);
       seek(start);
       init();
 
@@ -465,7 +465,7 @@ public class SequenceFile {
     private int memory; // bytes
     private int factor; // merged per pass
 
-    private NutchFileSystem nfs = null;
+    private FileSystem fs = null;
 
     private Class keyClass;
     private Class valClass;
@@ -473,13 +473,13 @@ public class SequenceFile {
     private Configuration conf;
 
     /** Sort and merge files containing the named classes. */
-    public Sorter(NutchFileSystem nfs, Class keyClass, Class valClass, Configuration conf)  {
-      this(nfs, new WritableComparator(keyClass), valClass, conf);
+    public Sorter(FileSystem fs, Class keyClass, Class valClass, Configuration conf)  {
+      this(fs, new WritableComparator(keyClass), valClass, conf);
     }
 
     /** Sort and merge using an arbitrary {@link WritableComparator}. */
-    public Sorter(NutchFileSystem nfs, WritableComparator comparator, Class valClass, Configuration conf) {
-      this.nfs = nfs;
+    public Sorter(FileSystem fs, WritableComparator comparator, Class valClass, Configuration conf) {
+      this.fs = fs;
       this.comparator = comparator;
       this.keyClass = comparator.getKeyClass();
       this.valClass = valClass;
@@ -502,7 +502,7 @@ public class SequenceFile {
 
     /** Perform a file sort.*/
     public void sort(String inFile, String outFile) throws IOException {
-      if (nfs.exists(new File(outFile))) {
+      if (fs.exists(new File(outFile))) {
         throw new IOException("already exists: " + outFile);
       }
 
@@ -539,11 +539,11 @@ public class SequenceFile {
       private int[] lengths = new int[starts.length];
       
       private Reader in;
-      private NFSDataOutputStream out;
+      private FSDataOutputStream out;
         private String outName;
 
       public SortPass(Configuration conf) throws IOException {
-        in = new Reader(nfs, inFile, conf);
+        in = new Reader(fs, inFile, conf);
       }
       
       public int run() throws IOException {
@@ -610,7 +610,7 @@ public class SequenceFile {
       private void flush(int count, boolean done) throws IOException {
         if (out == null) {
           outName = done ? outFile : outFile+".0";
-          out = nfs.create(new File(outName));
+          out = fs.create(new File(outName));
         }
 
         if (!done) {                              // an intermediate file
@@ -697,7 +697,7 @@ public class SequenceFile {
       private boolean last;
 
       private MergeQueue queue;
-      private NFSDataInputStream in;
+      private FSDataInputStream in;
       private String inName;
 
       public MergePass(int pass, boolean last) throws IOException {
@@ -708,19 +708,19 @@ public class SequenceFile {
           new MergeQueue(factor, last ? outFile : outFile+"."+pass, last);
 
         this.inName = outFile+"."+(pass-1);
-        this.in = nfs.open(new File(inName));
+        this.in = fs.open(new File(inName));
       }
 
       public void close() throws IOException {
         in.close();                               // close and delete input
-        nfs.delete(new File(inName));
+        fs.delete(new File(inName));
 
         queue.close();                            // close queue
       }
 
       public int run() throws IOException {
         int segments = 0;
-        long end = nfs.getLength(new File(inName));
+        long end = fs.getLength(new File(inName));
 
         while (in.getPos() < end) {
           LOG.finer("merging segment " + segments);
@@ -734,7 +734,7 @@ public class SequenceFile {
 
             totalCount+= count;
 
-            Reader reader = new Reader(nfs, inName, memory/(factor+1),
+            Reader reader = new Reader(fs, inName, memory/(factor+1),
                                        in.getPos(), length);
             reader.sync = null;                   // disable sync on temp files
 
@@ -794,7 +794,7 @@ public class SequenceFile {
         for (int i = 0; i < inFiles.length; i++) {
           String inFile = inFiles[i];
           MergeStream ms =
-            new MergeStream(new Reader(nfs, inFile, memory/(factor+1)));
+            new MergeStream(new Reader(fs, inFile, memory/(factor+1)));
           if (ms.next())
             queue.put(ms);
         }
@@ -827,7 +827,7 @@ public class SequenceFile {
     }
 
     private class MergeQueue extends PriorityQueue {
-      private NFSDataOutputStream out;
+      private FSDataOutputStream out;
       private boolean done;
       private boolean compress;
 
@@ -843,7 +843,7 @@ public class SequenceFile {
       public MergeQueue(int size, String outName, boolean done)
         throws IOException {
         initialize(size);
-        this.out = nfs.create(new File(outName), true, memory/(factor+1));
+        this.out = fs.create(new File(outName), true, memory/(factor+1));
         this.done = done;
       }
 

+ 9 - 9
src/java/org/apache/hadoop/io/SetFile.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 
@@ -30,14 +30,14 @@ public class SetFile extends MapFile {
   public static class Writer extends MapFile.Writer {
 
     /** Create the named set for keys of the named class. */
-    public Writer(NutchFileSystem nfs, String dirName, Class keyClass) throws IOException {
-      super(nfs, dirName, keyClass, NullWritable.class);
+    public Writer(FileSystem fs, String dirName, Class keyClass) throws IOException {
+      super(fs, dirName, keyClass, NullWritable.class);
     }
 
     /** Create the named set using the named key comparator. */
-    public Writer(NutchFileSystem nfs, String dirName, WritableComparator comparator)
+    public Writer(FileSystem fs, String dirName, WritableComparator comparator)
       throws IOException {
-      super(nfs, dirName, comparator, NullWritable.class);
+      super(fs, dirName, comparator, NullWritable.class);
     }
 
     /** Append a key to a set.  The key must be strictly greater than the
@@ -51,14 +51,14 @@ public class SetFile extends MapFile {
   public static class Reader extends MapFile.Reader {
 
     /** Construct a set reader for the named set.*/
-    public Reader(NutchFileSystem nfs, String dirName, Configuration conf) throws IOException {
-      super(nfs, dirName, conf);
+    public Reader(FileSystem fs, String dirName, Configuration conf) throws IOException {
+      super(fs, dirName, conf);
     }
 
     /** Construct a set reader for the named set using the named comparator.*/
-    public Reader(NutchFileSystem nfs, String dirName, WritableComparator comparator, Configuration conf)
+    public Reader(FileSystem fs, String dirName, WritableComparator comparator, Configuration conf)
       throws IOException {
-      super(nfs, dirName, comparator, conf);
+      super(fs, dirName, comparator, conf);
     }
 
     // javadoc inherited

+ 1 - 1
src/java/org/apache/hadoop/io/TwoDArrayWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 import java.lang.reflect.Array;

+ 1 - 1
src/java/org/apache/hadoop/io/UTF8.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.IOException;
 import java.io.DataInput;

+ 1 - 1
src/java/org/apache/hadoop/io/VersionMismatchException.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.IOException;
 

+ 1 - 1
src/java/org/apache/hadoop/io/VersionedWritable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.DataOutput;
 import java.io.DataInput;

+ 1 - 1
src/java/org/apache/hadoop/io/Writable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.DataOutput;
 import java.io.DataInput;

+ 1 - 1
src/java/org/apache/hadoop/io/WritableComparable.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 /** An interface which extends both {@link Writable} and {@link Comparable}.
  *

+ 1 - 1
src/java/org/apache/hadoop/io/WritableComparator.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 import java.util.*;

+ 1 - 1
src/java/org/apache/hadoop/io/WritableName.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.util.HashMap;
 import java.io.IOException;

+ 1 - 1
src/java/org/apache/hadoop/io/WritableUtils.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
 
 import java.io.*;
 

+ 1 - 1
src/java/org/apache/hadoop/ipc/Client.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.ipc.
+package org.apache.hadoop.ipc;
 
 import java.net.Socket;
 import java.net.InetSocketAddress;

+ 1 - 1
src/java/org/apache/hadoop/ipc/RPC.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.ipc.
+package org.apache.hadoop.ipc;
 
 import java.lang.reflect.Proxy;
 import java.lang.reflect.Method;

+ 1 - 1
src/java/org/apache/hadoop/ipc/Server.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.ipc.
+package org.apache.hadoop.ipc;
 
 import java.io.IOException;
 import java.io.EOFException;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.*;
 import java.net.*;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.File;
@@ -23,11 +23,11 @@ import java.io.DataOutput;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.UTF8;
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 
 /** A section of an input file.  Returned by {@link
- * InputFormat#getSplits(NutchFileSystem, JobConf, int)} and passed to
- * InputFormat#getRecordReader(NutchFileSystem,FileSplit,JobConf,Reporter). */
+ * InputFormat#getSplits(FileSystem, JobConf, int)} and passed to
+ * InputFormat#getRecordReader(FileSystem,FileSplit,JobConf,Reporter). */
 public class FileSplit implements Writable {
   private File file;
   private long start;

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

@@ -14,14 +14,14 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.File;
 
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 
-/** An input data format.  Input files are stored in a {@link NutchFileSystem}.
+/** An input data format.  Input files are stored in a {@link FileSystem}.
  * The processing of an input file may be split across multiple machines.
  * Files are processed as sequences of records, implementing {@link
  * RecordReader}.  Files must thus be split on record boundaries. */
@@ -34,17 +34,17 @@ public interface InputFormat {
    * @param numSplits the desired number of splits
    * @return the splits
    */
-  FileSplit[] getSplits(NutchFileSystem fs, JobConf job, int numSplits)
+  FileSplit[] getSplits(FileSystem fs, JobConf job, int numSplits)
     throws IOException;
 
   /** Construct a {@link RecordReader} for a {@link FileSplit}.
    *
-   * @param fs the {@link NutchFileSystem}
+   * @param fs the {@link FileSystem}
    * @param split the {@link FileSplit}
    * @param job the job that this split belongs to
    * @return a {@link RecordReader}
    */
-  RecordReader getRecordReader(NutchFileSystem fs, FileSplit split,
+  RecordReader getRecordReader(FileSystem fs, FileSplit split,
                                JobConf job, Reporter reporter)
     throws IOException;
 }

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.File;
@@ -23,7 +23,7 @@ import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.logging.Logger;
 
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.util.LogFormatter;
 
 /** A base class for {@link InputFormat}. */
@@ -40,7 +40,7 @@ public abstract class InputFormatBase implements InputFormat {
     this.minSplitSize = minSplitSize;
   }
 
-  public abstract RecordReader getRecordReader(NutchFileSystem fs,
+  public abstract RecordReader getRecordReader(FileSystem fs,
                                                FileSplit split,
                                                JobConf job,
                                                Reporter reporter)
@@ -57,7 +57,7 @@ public abstract class InputFormatBase implements InputFormat {
    * @return array of File objects, never zero length.
    * @throws IOException if zero items.
    */
-  protected File[] listFiles(NutchFileSystem fs, JobConf job)
+  protected File[] listFiles(FileSystem fs, JobConf job)
     throws IOException {
     File[] dirs = job.getInputDirs();
     String subdir = job.get("mapred.input.subdir");
@@ -87,9 +87,9 @@ public abstract class InputFormatBase implements InputFormat {
     return (File[])result.toArray(new File[result.size()]);
   }
 
-  /** Splits files returned by {#listFiles(NutchFileSystem,JobConf) when
+  /** Splits files returned by {#listFiles(FileSystem,JobConf) when
    * they're too big.*/ 
-  public FileSplit[] getSplits(NutchFileSystem fs, JobConf job, int numSplits)
+  public FileSplit[] getSplits(FileSystem fs, JobConf job, int numSplits)
     throws IOException {
 
     File[] files = listFiles(fs, job);

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.*;
 import org.apache.hadoop.io.*;

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.fs.*;
@@ -165,7 +165,7 @@ public class JobClient implements MRConstants {
     }
 
     JobSubmissionProtocol jobSubmitClient;
-    NutchFileSystem fs = null;
+    FileSystem fs = null;
 
     private Configuration conf;
     static Random r = new Random();
@@ -207,10 +207,10 @@ public class JobClient implements MRConstants {
      * Get a filesystem handle.  We need this to prepare jobs
      * for submission to the MapReduce system.
      */
-    public synchronized NutchFileSystem getFs() throws IOException {
+    public synchronized FileSystem getFs() throws IOException {
       if (this.fs == null) {
         String fsName = jobSubmitClient.getFilesystemName();
-        this.fs = NutchFileSystem.getNamed(fsName, this.conf);
+        this.fs = FileSystem.getNamed(fsName, this.conf);
       }
       return fs;
     }
@@ -249,7 +249,7 @@ public class JobClient implements MRConstants {
         }
 
         // Write job file to JobTracker's fs
-        NFSDataOutputStream out = getFs().create(submitJobFile);
+        FSDataOutputStream out = getFs().create(submitJobFile);
         try {
           job.write(out);
         } finally {

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
@@ -31,7 +31,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Collections;
 
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.conf.Configuration;
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 /** That what may be configured. */
 public interface JobConfigurable {

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.fs.*;
@@ -67,7 +67,7 @@ public class JobInProgress {
         this.startTime = System.currentTimeMillis();
 
         this.localJobFile = new JobConf(conf).getLocalFile(JobTracker.SUBDIR, jobid + ".xml");
-        NutchFileSystem fs = NutchFileSystem.get(conf);
+        FileSystem fs = FileSystem.get(conf);
         fs.copyToLocalFile(new File(jobFile), localJobFile);
 
         JobConf jd = new JobConf(localJobFile);
@@ -99,7 +99,7 @@ public class JobInProgress {
         String jobFile = profile.getJobFile();
 
         JobConf jd = new JobConf(localJobFile);
-        NutchFileSystem fs = NutchFileSystem.get(conf);
+        FileSystem fs = FileSystem.get(conf);
         FileSplit[] splits =
             jd.getInputFormat().getSplits(fs, jd, numMapTasks);
 
@@ -427,7 +427,7 @@ public class JobInProgress {
         //
         if (deleteUponCompletion != null) {
             JobConf jd = new JobConf(deleteUponCompletion);
-            NutchFileSystem fs = NutchFileSystem.get(conf);
+            FileSystem fs = FileSystem.get(conf);
             fs.delete(new File(jd.getJar()));
             fs.delete(new File(deleteUponCompletion));
             deleteUponCompletion = null;

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.*;
 import java.util.*;

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 
 import org.apache.hadoop.io.*;
@@ -257,7 +257,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     // Some jobs are stored in a local system directory.  We can delete
     // the files when we're done with the job.
     static final String SUBDIR = "jobTracker";
-    NutchFileSystem fs;
+    FileSystem fs;
     File systemDir;
     private Configuration conf;
 
@@ -280,7 +280,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         this.conf = conf;
         JobConf jobConf = new JobConf(conf);
         this.systemDir = jobConf.getSystemDir();
-        this.fs = NutchFileSystem.get(conf);
+        this.fs = FileSystem.get(conf);
         FileUtil.fullyDelete(fs, systemDir);
         fs.mkdirs(systemDir);
 

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.*;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.*;
 import java.util.*;
@@ -30,7 +30,7 @@ public class LocalJobRunner implements JobSubmissionProtocol {
   public static final Logger LOG =
     LogFormatter.getLogger("org.apache.hadoop.mapred.LocalJobRunner");
 
-  private NutchFileSystem fs;
+  private FileSystem fs;
   private HashMap jobs = new HashMap();
   private Configuration conf;
 
@@ -84,7 +84,7 @@ public class LocalJobRunner implements JobSubmissionProtocol {
           File mapOut = this.mapoutputFile.getOutputFile(mapId, 0);
           File reduceIn = this.mapoutputFile.getInputFile(mapId, reduceId);
           reduceIn.getParentFile().mkdirs();
-          if (!NutchFileSystem.getNamed("local", this.job).rename(mapOut, reduceIn))
+          if (!FileSystem.getNamed("local", this.job).rename(mapOut, reduceIn))
             throw new IOException("Couldn't rename " + mapOut);
           this.mapoutputFile.removeAll(mapId);
         }
@@ -151,7 +151,7 @@ public class LocalJobRunner implements JobSubmissionProtocol {
   }
 
   public LocalJobRunner(Configuration conf) throws IOException {
-    this.fs = NutchFileSystem.get(conf);
+    this.fs = FileSystem.get(conf);
     this.conf = conf;
   }
 

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 /*******************************
  * Some handy constants

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

@@ -14,13 +14,13 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.File;
 import java.util.Arrays;
 
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 
 import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.WritableComparable;
@@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 
 public class MapFileOutputFormat implements OutputFormat {
 
-  public RecordWriter getRecordWriter(NutchFileSystem fs, JobConf job,
+  public RecordWriter getRecordWriter(FileSystem fs, JobConf job,
                                       String name) throws IOException {
 
     File file = new File(job.getOutputDir(), name);
@@ -53,7 +53,7 @@ public class MapFileOutputFormat implements OutputFormat {
   }
 
   /** Open the output generated by this format. */
-  public static MapFile.Reader[] getReaders(NutchFileSystem fs, File dir, Configuration conf)
+  public static MapFile.Reader[] getReaders(FileSystem fs, File dir, Configuration conf)
     throws IOException {
     File[] names = fs.listFiles(dir);
     

+ 3 - 3
src/java/org/apache/hadoop/mapred/MapOutputFile.java

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 
@@ -94,7 +94,7 @@ public class MapOutputFile implements Writable, Configurable {
     // write the length-prefixed file content to the wire
     File file = getOutputFile(mapTaskId, partition);
     out.writeLong(file.length());
-    NFSDataInputStream in = NutchFileSystem.getNamed("local", this.jobConf).open(file);
+    FSDataInputStream in = FileSystem.getNamed("local", this.jobConf).open(file);
     try {
       byte[] buffer = new byte[8192];
       int l;
@@ -118,7 +118,7 @@ public class MapOutputFile implements Writable, Configurable {
     long length = in.readLong();
     float progPerByte = 1.0f / length;
     long unread = length;
-    NFSDataOutputStream out = NutchFileSystem.getNamed("local", this.jobConf).create(file);
+    FSDataOutputStream out = FileSystem.getNamed("local", this.jobConf).create(file);
     try {
       byte[] buffer = new byte[8192];
       while (unread > 0) {

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.*;
 import java.net.*;
@@ -22,7 +22,7 @@ import java.util.*;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 
 /** A Map task. */
 public class MapTask extends Task {
@@ -68,7 +68,7 @@ public class MapTask extends Task {
     try {
       for (int i = 0; i < partitions; i++) {
         outs[i] =
-          new SequenceFile.Writer(NutchFileSystem.getNamed("local", job),
+          new SequenceFile.Writer(FileSystem.getNamed("local", job),
                                   this.mapOutputFile.getOutputFile(getTaskId(), i).toString(),
                                   job.getOutputKeyClass(),
                                   job.getOutputValueClass());
@@ -97,7 +97,7 @@ public class MapTask extends Task {
 
       final RecordReader rawIn =                  // open input
         job.getInputFormat().getRecordReader
-        (NutchFileSystem.get(job), split, job, reporter);
+        (FileSystem.get(job), split, job, reporter);
 
       RecordReader in = new RecordReader() {      // wrap in progress reporter
           private float perByte = 1.0f /(float)split.getLength();

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.*;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.DataInput;

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

@@ -14,15 +14,15 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.File;
 
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 
 /** An output data format.  Output files are stored in a {@link
- * NutchFileSystem}. */
+ * FileSystem}. */
 public interface OutputFormat {
 
   /** Construct a {@link RecordWriter}.
@@ -32,7 +32,7 @@ public interface OutputFormat {
    * @param name the unique name for this part of the output
    * @return a {@link RecordWriter}
    */
-  RecordWriter getRecordWriter(NutchFileSystem fs, JobConf job, String name)
+  RecordWriter getRecordWriter(FileSystem fs, JobConf job, String name)
     throws IOException;
 }
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.DataInput;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.DataOutput;

+ 4 - 3
src/java/org/apache/hadoop/mapred/ReduceTask.java

@@ -14,11 +14,12 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.*;
 
 import java.io.*;
 import java.net.*;
@@ -170,7 +171,7 @@ public class ReduceTask extends Task {
     Class valueClass = job.getOutputValueClass();
     Reducer reducer = (Reducer)job.newInstance(job.getReducerClass());
     reducer.configure(job);
-    NutchFileSystem lfs = NutchFileSystem.getNamed("local", job);
+    FileSystem lfs = FileSystem.getNamed("local", job);
 
     copyPhase.complete();                         // copy is already complete
 
@@ -254,7 +255,7 @@ public class ReduceTask extends Task {
     // make output collector
     String name = getOutputName(getPartition());
     final RecordWriter out =
-      job.getOutputFormat().getRecordWriter(NutchFileSystem.get(job), job, name);
+      job.getOutputFormat().getRecordWriter(FileSystem.get(job), job, name);
     OutputCollector collector = new OutputCollector() {
         public void collect(WritableComparable key, Writable value)
           throws IOException {

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

@@ -13,12 +13,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
+import org.apache.hadoop.util.*;
 
 import java.io.*;
 import java.net.*;

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 

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

@@ -14,7 +14,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 

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

@@ -14,12 +14,12 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.File;
 
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.MapFile;
@@ -35,7 +35,7 @@ public class SequenceFileInputFormat extends InputFormatBase {
     setMinSplitSize(SequenceFile.SYNC_INTERVAL);
   }
 
-  protected File[] listFiles(NutchFileSystem fs, JobConf job)
+  protected File[] listFiles(FileSystem fs, JobConf job)
     throws IOException {
 
     File[] files = super.listFiles(fs, job);
@@ -48,7 +48,7 @@ public class SequenceFileInputFormat extends InputFormatBase {
     return files;
   }
 
-  public RecordReader getRecordReader(NutchFileSystem fs, FileSplit split,
+  public RecordReader getRecordReader(FileSystem fs, FileSplit split,
                                       JobConf job, Reporter reporter)
     throws IOException {
 

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

@@ -14,13 +14,13 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.File;
 import java.util.Arrays;
 
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 
 import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.SequenceFile;
@@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 
 public class SequenceFileOutputFormat implements OutputFormat {
 
-  public RecordWriter getRecordWriter(NutchFileSystem fs, JobConf job,
+  public RecordWriter getRecordWriter(FileSystem fs, JobConf job,
                                       String name) throws IOException {
 
     File file = new File(job.getOutputDir(), name);
@@ -56,7 +56,7 @@ public class SequenceFileOutputFormat implements OutputFormat {
   /** Open the output generated by this format. */
   public static SequenceFile.Reader[] getReaders(Configuration conf, File dir)
     throws IOException {
-    NutchFileSystem fs = NutchFileSystem.get(conf);
+    FileSystem fs = FileSystem.get(conf);
     File[] names = fs.listFiles(dir);
     
     // sort names, so that hash partitioning works

+ 3 - 3
src/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java

@@ -14,12 +14,12 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.io.File;
 
-import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileSystem;
 
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Writable;
@@ -36,7 +36,7 @@ public class SequenceFileRecordReader implements RecordReader {
 
   public SequenceFileRecordReader(Configuration conf, FileSplit split)
     throws IOException {
-    NutchFileSystem fs = NutchFileSystem.get(conf);
+    FileSystem fs = FileSystem.get(conf);
     this.in = new SequenceFile.Reader(fs, split.getFile().toString(), conf);
     this.end = split.getStart() + split.getLength();
 

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

@@ -14,10 +14,11 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
+import org.apache.hadoop.util.*;
 
 import java.io.*;
 import java.net.*;

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.fs.*;
@@ -363,7 +363,7 @@ class TaskInProgress {
         } else {
             try {
                 if (isMapTask()) {
-                    NutchFileSystem fs = NutchFileSystem.get(conf);
+                    FileSystem fs = FileSystem.get(conf);
                     String hints[][] = fs.getFileCacheHints(split.getFile(), split.getStart(), split.getLength());
                     for (int i = 0; i < hints.length; i++) {
                         for (int j = 0; j < hints[i].length; j++) {

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

@@ -13,7 +13,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.
+package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.*;

部分文件因为文件数量过多而无法显示