You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2006/02/03 21:34:55 UTC
svn commit: r374738 [1/3] - in /lucene/hadoop/trunk: bin/ lib/
src/java/org/apache/hadoop/conf/ src/java/org/apache/hadoop/dfs/
src/java/org/apache/hadoop/fs/ src/java/org/apache/hadoop/io/
src/java/org/apache/hadoop/ipc/ src/java/org/apache/hadoop/map...
Author: cutting
Date: Fri Feb 3 12:34:32 2006
New Revision: 374738
URL: http://svn.apache.org/viewcvs?rev=374738&view=rev
Log:
First version that compiles.
Added:
lucene/hadoop/trunk/lib/lucene-core-1.9-rc1-dev.jar
- copied unchanged from r374710, lucene/nutch/trunk/lib/lucene-core-1.9-rc1-dev.jar
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSClient.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFile.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFileInfo.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSShell.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSFileSystem.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataInputStream.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataOutputStream.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataOutputStream.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSInputStream.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSInputStream.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSOutputStream.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSOutputStream.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
- copied, changed from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NutchFileSystem.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/
lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestDFS.java
- copied, changed from r374735, lucene/hadoop/trunk/src/test/org/apache/hadoop/ndfs/TestNDFS.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestFileSystem.java
- copied, changed from r374735, lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestNutchFileSystem.java
lucene/hadoop/trunk/src/webapps/
- copied from r374710, lucene/nutch/trunk/src/webapps/
Removed:
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSClient.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFileInfo.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSFileSystem.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSShell.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataInputStream.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataOutputStream.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSInputStream.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSOutputStream.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NutchFileSystem.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestNutchFileSystem.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/ndfs/
Modified:
lucene/hadoop/trunk/bin/hadoop
lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/Block.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/BlockCommand.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DF.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeProtocol.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/LocatedBlock.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumException.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSError.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/Seekable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BooleanWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BytesWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/CompressedWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataInputBuffer.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataOutputBuffer.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/FloatWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/LongWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MD5Hash.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/NullWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ObjectWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SetFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/TwoDArrayWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/UTF8.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/VersionMismatchException.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/VersionedWritable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/Writable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/WritableComparable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/WritableComparator.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/WritableName.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/WritableUtils.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/RPC.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/CombiningCollector.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/FileSplit.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/InputFormat.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobConf.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobConfigurable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobProfile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobStatus.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MRConstants.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MapOutputFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MapOutputLocation.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MapOutputProtocol.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MapRunnable.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MapRunner.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MapTask.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MapTaskRunner.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Mapper.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/OutputCollector.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/OutputFormat.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Partitioner.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/RecordReader.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/RecordWriter.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/ReduceTask.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Reducer.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Reporter.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/RunningJob.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/SequenceFileInputFormat.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/SequenceFileOutputFormat.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/SequenceFileRecordReader.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Task.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskInProgress.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskRunner.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskStatus.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTrackerStatus.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TextInputFormat.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TextOutputFormat.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/RandomDatum.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestArrayFile.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestMD5Hash.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSequenceFile.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSetFile.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestUTF8.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestVersionedWritable.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestWritable.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/ipc/TestIPC.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/ipc/TestRPC.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/MapredLoadTest.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestTextInputFormat.java
Modified: lucene/hadoop/trunk/bin/hadoop
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/bin/hadoop?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/bin/hadoop (original)
+++ lucene/hadoop/trunk/bin/hadoop Fri Feb 3 12:34:32 2006
@@ -115,7 +115,7 @@
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
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java Fri Feb 3 12:34:32 2006
@@ -29,6 +29,8 @@
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.
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/Block.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/Block.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/Block.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/Block.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/BlockCommand.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/BlockCommand.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/BlockCommand.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/BlockCommand.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DF.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DF.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DF.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DF.java Fri Feb 3 12:34:32 2006
@@ -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;
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSClient.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSClient.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSClient.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java Fri Feb 3 12:34:32 2006
@@ -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 @@
* 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 @@
* 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 @@
/****************************************************************
* 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];
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFile.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFile.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFile.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFile.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFile.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFile.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFile.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFileInfo.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFileInfo.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NDFSFileInfo.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSFileInfo.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSShell.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSShell.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSShell.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSShell.java Fri Feb 3 12:34:32 2006
@@ -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 @@
* 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 @@
* 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 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 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 @@
* 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 @@
* 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 @@
}
/**
- * 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 @@
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 @@
}
System.exit(0);
} finally {
- nfs.close();
+ fs.close();
}
}
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeProtocol.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeProtocol.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeProtocol.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeProtocol.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSFileSystem.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSFileSystem.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NDFSFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java Fri Feb 3 12:34:32 2006
@@ -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 @@
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 @@
}
/**
- * 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 @@
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 @@
* 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 @@
/**
* 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 @@
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
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java Fri Feb 3 12:34:32 2006
@@ -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;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/LocatedBlock.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/LocatedBlock.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/LocatedBlock.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/LocatedBlock.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumException.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumException.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumException.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumException.java Fri Feb 3 12:34:32 2006
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.fs.
+package org.apache.hadoop.fs;
import java.io.IOException;
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataInputStream.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataInputStream.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataInputStream.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java Fri Feb 3 12:34:32 2006
@@ -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 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 @@
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 @@
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 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 @@
}
/** 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);
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataOutputStream.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataOutputStream.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataOutputStream.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataOutputStream.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataOutputStream.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSDataOutputStream.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataOutputStream.java Fri Feb 3 12:34:32 2006
@@ -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 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 @@
}
/** 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));
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSError.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSError.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSError.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSError.java Fri Feb 3 12:34:32 2006
@@ -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. */
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSInputStream.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSInputStream.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSInputStream.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSInputStream.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSInputStream.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSInputStream.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSInputStream.java Fri Feb 3 12:34:32 2006
@@ -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
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSOutputStream.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSOutputStream.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSOutputStream.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSOutputStream.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSOutputStream.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NFSOutputStream.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSOutputStream.java Fri Feb 3 12:34:32 2006
@@ -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
*/
Copied: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java (from r374733, lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NutchFileSystem.java)
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java?p2=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java&p1=lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NutchFileSystem.java&r1=374733&r2=374738&rev=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/NutchFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java Fri Feb 3 12:34:32 2006
@@ -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 @@
* <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 @@
* 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 @@
*
* @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 @@
}
///////////////////////////////////////////////////////////////
- // NutchFileSystem
+ // FileSystem
///////////////////////////////////////////////////////////////
/**
*/
- public NutchFileSystem(Configuration conf) {
+ public FileSystem(Configuration conf) {
this.conf = conf;
}
@@ -135,56 +135,56 @@
* 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 @@
* @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 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 @@
/**
* 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 @@
* @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);
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java Fri Feb 3 12:34:32 2006
@@ -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 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 @@
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 @@
* 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);
}
}
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java Fri Feb 3 12:34:32 2006
@@ -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.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 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 @@
/**
* 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 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 @@
if (parent != null)
parent.mkdirs();
- return new LocalNFSFileOutputStream(f);
+ return new LocalFSFileOutputStream(f);
}
/**
@@ -316,21 +316,21 @@
/**
* 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 @@
/** 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
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/Seekable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/Seekable.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/Seekable.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/Seekable.java Fri Feb 3 12:34:32 2006
@@ -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.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayFile.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayFile.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayFile.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayFile.java Fri Feb 3 12:34:32 2006
@@ -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 @@
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 @@
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. */
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayWritable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayWritable.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayWritable.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/ArrayWritable.java Fri Feb 3 12:34:32 2006
@@ -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;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BooleanWritable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BooleanWritable.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BooleanWritable.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BooleanWritable.java Fri Feb 3 12:34:32 2006
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
import java.io.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BytesWritable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BytesWritable.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BytesWritable.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/BytesWritable.java Fri Feb 3 12:34:32 2006
@@ -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;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/CompressedWritable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/CompressedWritable.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/CompressedWritable.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/CompressedWritable.java Fri Feb 3 12:34:32 2006
@@ -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;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataInputBuffer.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataInputBuffer.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataInputBuffer.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataInputBuffer.java Fri Feb 3 12:34:32 2006
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
import java.io.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataOutputBuffer.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataOutputBuffer.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataOutputBuffer.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/DataOutputBuffer.java Fri Feb 3 12:34:32 2006
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
import java.io.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/FloatWritable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/FloatWritable.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/FloatWritable.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/FloatWritable.java Fri Feb 3 12:34:32 2006
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
import java.io.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java Fri Feb 3 12:34:32 2006
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
import java.io.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/LongWritable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/LongWritable.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/LongWritable.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/LongWritable.java Fri Feb 3 12:34:32 2006
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
import java.io.*;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MD5Hash.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MD5Hash.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MD5Hash.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MD5Hash.java Fri Feb 3 12:34:32 2006
@@ -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;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java Fri Feb 3 12:34:32 2006
@@ -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 @@
/** 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 @@
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 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 @@
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 @@
}
/** 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 @@
* @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 @@
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 @@
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();
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/NullWritable.java
URL: http://svn.apache.org/viewcvs/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/NullWritable.java?rev=374738&r1=374737&r2=374738&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/NullWritable.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/NullWritable.java Fri Feb 3 12:34:32 2006
@@ -14,7 +14,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.io.
+package org.apache.hadoop.io;
import java.io.*;