瀏覽代碼

Initial commit of code copied from Nutch.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@374733 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 19 年之前
父節點
當前提交
dc372af412
共有 100 個文件被更改,包括 17150 次插入0 次删除
  1. 135 0
      bin/hadoop
  2. 101 0
      bin/hadoop-daemon.sh
  3. 16 0
      bin/hadoop-daemons.sh
  4. 27 0
      bin/slaves.sh
  5. 11 0
      bin/start-all.sh
  6. 11 0
      bin/stop-all.sh
  7. 277 0
      build.xml
  8. 237 0
      conf/hadoop-default.xml
  9. 8 0
      conf/hadoop-site.xml.template
  10. 8 0
      conf/mapred-default.xml.template
  11. 202 0
      lib/jetty-5.1.4.LICENSE.txt
  12. 二進制
      lib/jetty-5.1.4.jar
  13. 二進制
      lib/jetty-ext/ant.jar
  14. 二進制
      lib/jetty-ext/commons-el.jar
  15. 二進制
      lib/jetty-ext/jasper-compiler.jar
  16. 二進制
      lib/jetty-ext/jasper-runtime.jar
  17. 二進制
      lib/jetty-ext/jsp-api.jar
  18. 100 0
      lib/junit-3.8.1.LICENSE.txt
  19. 二進制
      lib/junit-3.8.1.jar
  20. 二進制
      lib/servlet-api.jar
  21. 27 0
      src/java/org/apache/hadoop/conf/Configurable.java
  22. 445 0
      src/java/org/apache/hadoop/conf/Configuration.java
  23. 39 0
      src/java/org/apache/hadoop/conf/Configured.java
  24. 126 0
      src/java/org/apache/hadoop/dfs/Block.java
  25. 108 0
      src/java/org/apache/hadoop/dfs/BlockCommand.java
  26. 140 0
      src/java/org/apache/hadoop/dfs/ClientProtocol.java
  27. 95 0
      src/java/org/apache/hadoop/dfs/DF.java
  28. 754 0
      src/java/org/apache/hadoop/dfs/DataNode.java
  29. 167 0
      src/java/org/apache/hadoop/dfs/DatanodeInfo.java
  30. 36 0
      src/java/org/apache/hadoop/dfs/DatanodeProtocol.java
  31. 114 0
      src/java/org/apache/hadoop/dfs/FSConstants.java
  32. 429 0
      src/java/org/apache/hadoop/dfs/FSDataset.java
  33. 744 0
      src/java/org/apache/hadoop/dfs/FSDirectory.java
  34. 1342 0
      src/java/org/apache/hadoop/dfs/FSNamesystem.java
  35. 79 0
      src/java/org/apache/hadoop/dfs/LocatedBlock.java
  36. 856 0
      src/java/org/apache/hadoop/dfs/NDFSClient.java
  37. 98 0
      src/java/org/apache/hadoop/dfs/NDFSFile.java
  38. 102 0
      src/java/org/apache/hadoop/dfs/NDFSFileInfo.java
  39. 352 0
      src/java/org/apache/hadoop/dfs/NameNode.java
  40. 26 0
      src/java/org/apache/hadoop/fs/ChecksumException.java
  41. 25 0
      src/java/org/apache/hadoop/fs/FSError.java
  42. 122 0
      src/java/org/apache/hadoop/fs/FileUtil.java
  43. 415 0
      src/java/org/apache/hadoop/fs/LocalFileSystem.java
  44. 320 0
      src/java/org/apache/hadoop/fs/NDFSFileSystem.java
  45. 263 0
      src/java/org/apache/hadoop/fs/NDFSShell.java
  46. 249 0
      src/java/org/apache/hadoop/fs/NFSDataInputStream.java
  47. 146 0
      src/java/org/apache/hadoop/fs/NFSDataOutputStream.java
  48. 38 0
      src/java/org/apache/hadoop/fs/NFSInputStream.java
  49. 30 0
      src/java/org/apache/hadoop/fs/NFSOutputStream.java
  50. 390 0
      src/java/org/apache/hadoop/fs/NutchFileSystem.java
  51. 28 0
      src/java/org/apache/hadoop/fs/Seekable.java
  52. 79 0
      src/java/org/apache/hadoop/io/ArrayFile.java
  53. 102 0
      src/java/org/apache/hadoop/io/ArrayWritable.java
  54. 107 0
      src/java/org/apache/hadoop/io/BooleanWritable.java
  55. 50 0
      src/java/org/apache/hadoop/io/BytesWritable.java
  56. 83 0
      src/java/org/apache/hadoop/io/CompressedWritable.java
  57. 87 0
      src/java/org/apache/hadoop/io/DataInputBuffer.java
  58. 91 0
      src/java/org/apache/hadoop/io/DataOutputBuffer.java
  59. 85 0
      src/java/org/apache/hadoop/io/FloatWritable.java
  60. 84 0
      src/java/org/apache/hadoop/io/IntWritable.java
  61. 95 0
      src/java/org/apache/hadoop/io/LongWritable.java
  62. 199 0
      src/java/org/apache/hadoop/io/MD5Hash.java
  63. 495 0
      src/java/org/apache/hadoop/io/MapFile.java
  64. 34 0
      src/java/org/apache/hadoop/io/NullWritable.java
  65. 265 0
      src/java/org/apache/hadoop/io/ObjectWritable.java
  66. 887 0
      src/java/org/apache/hadoop/io/SequenceFile.java
  67. 89 0
      src/java/org/apache/hadoop/io/SetFile.java
  68. 89 0
      src/java/org/apache/hadoop/io/TwoDArrayWritable.java
  69. 287 0
      src/java/org/apache/hadoop/io/UTF8.java
  70. 39 0
      src/java/org/apache/hadoop/io/VersionMismatchException.java
  71. 50 0
      src/java/org/apache/hadoop/io/VersionedWritable.java
  72. 41 0
      src/java/org/apache/hadoop/io/Writable.java
  73. 24 0
      src/java/org/apache/hadoop/io/WritableComparable.java
  74. 153 0
      src/java/org/apache/hadoop/io/WritableComparator.java
  75. 71 0
      src/java/org/apache/hadoop/io/WritableName.java
  76. 193 0
      src/java/org/apache/hadoop/io/WritableUtils.java
  77. 6 0
      src/java/org/apache/hadoop/io/package.html
  78. 370 0
      src/java/org/apache/hadoop/ipc/Client.java
  79. 238 0
      src/java/org/apache/hadoop/ipc/RPC.java
  80. 291 0
      src/java/org/apache/hadoop/ipc/Server.java
  81. 5 0
      src/java/org/apache/hadoop/ipc/package.html
  82. 83 0
      src/java/org/apache/hadoop/mapred/CombiningCollector.java
  83. 77 0
      src/java/org/apache/hadoop/mapred/FileSplit.java
  84. 51 0
      src/java/org/apache/hadoop/mapred/InputFormat.java
  85. 134 0
      src/java/org/apache/hadoop/mapred/InputFormatBase.java
  86. 64 0
      src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java
  87. 378 0
      src/java/org/apache/hadoop/mapred/JobClient.java
  88. 284 0
      src/java/org/apache/hadoop/mapred/JobConf.java
  89. 26 0
      src/java/org/apache/hadoop/mapred/JobConfigurable.java
  90. 437 0
      src/java/org/apache/hadoop/mapred/JobInProgress.java
  91. 86 0
      src/java/org/apache/hadoop/mapred/JobProfile.java
  92. 79 0
      src/java/org/apache/hadoop/mapred/JobStatus.java
  93. 63 0
      src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java
  94. 816 0
      src/java/org/apache/hadoop/mapred/JobTracker.java
  95. 128 0
      src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java
  96. 188 0
      src/java/org/apache/hadoop/mapred/LocalJobRunner.java
  97. 35 0
      src/java/org/apache/hadoop/mapred/MRConstants.java
  98. 80 0
      src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java
  99. 146 0
      src/java/org/apache/hadoop/mapred/MapOutputFile.java
  100. 68 0
      src/java/org/apache/hadoop/mapred/MapOutputLocation.java

+ 135 - 0
bin/hadoop

@@ -0,0 +1,135 @@
+#!/bin/bash
+# 
+# The Hadoop command script
+#
+# Environment Variables
+#
+#   JAVA_HOME The java implementation to use.  Overrides JAVA_HOME.
+#
+#   HADOOP_HEAPSIZE  The maximum amount of heap to use, in MB. 
+#                   Default is 1000.
+#
+#   HADOOP_OPTS      Extra Java runtime options.
+#
+
+# resolve links - $0 may be a softlink
+THIS="$0"
+while [ -h "$THIS" ]; do
+  ls=`ls -ld "$THIS"`
+  link=`expr "$ls" : '.*-> \(.*\)$'`
+  if expr "$link" : '.*/.*' > /dev/null; then
+    THIS="$link"
+  else
+    THIS=`dirname "$THIS"`/"$link"
+  fi
+done
+
+# if no args specified, show usage
+if [ $# = 0 ]; then
+  echo "Usage: hadoop COMMAND"
+  echo "where COMMAND is one of:"
+  echo "  namenode          run the DFS namenode"
+  echo "  datanode          run an DFS datanode"
+  echo "  dfs              run an DFS admin client"
+  echo "  jobtracker        run the MapReduce job Tracker node" 
+  echo "  tasktracker       run a MapReduce task Tracker node" 
+  echo "  job               manipulate MapReduce jobs" 
+  echo " or"
+  echo "  CLASSNAME         run the class named CLASSNAME"
+  echo "Most commands print help when invoked w/o parameters."
+  exit 1
+fi
+
+# get arguments
+COMMAND=$1
+shift
+
+# some directories
+THIS_DIR=`dirname "$THIS"`
+HADOOP_HOME=`cd "$THIS_DIR/.." ; pwd`
+
+# some Java parameters
+if [ "$JAVA_HOME" != "" ]; then
+  #echo "run java in $JAVA_HOME"
+  JAVA_HOME=$JAVA_HOME
+fi
+  
+if [ "$JAVA_HOME" = "" ]; then
+  echo "Error: JAVA_HOME is not set."
+  exit 1
+fi
+
+JAVA=$JAVA_HOME/bin/java
+JAVA_HEAP_MAX=-Xmx1000m 
+
+# check envvars which might override default args
+if [ "$HADOOP_HEAPSIZE" != "" ]; then
+  #echo "run with heapsize $HADOOP_HEAPSIZE"
+  JAVA_HEAP_MAX="-Xmx""$HADOOP_HEAPSIZE""m"
+  #echo $JAVA_HEAP_MAX
+fi
+
+# CLASSPATH initially contains $HADOOP_CONF_DIR, or defaults to $HADOOP_HOME/conf
+CLASSPATH=${HADOOP_CONF_DIR:=$HADOOP_HOME/conf}
+CLASSPATH=${CLASSPATH}:$JAVA_HOME/lib/tools.jar
+
+# for developers, add Hadoop classes to CLASSPATH
+if [ -d "$HADOOP_HOME/build/classes" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/classes
+fi
+if [ -d "$HADOOP_HOME/build/plugins" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build
+fi
+if [ -d "$HADOOP_HOME/build/test/classes" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/test/classes
+fi
+
+# so that filenames w/ spaces are handled correctly in loops below
+IFS=
+
+# for releases, add Hadoop jar to CLASSPATH
+for f in $HADOOP_HOME/hadoop-*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+# add plugins to classpath
+if [ -d "$HADOOP_HOME/plugins" ]; then
+  CLASSPATH=${CLASSPATH}:$HADOOP_HOME
+fi
+
+# add libs to CLASSPATH
+for f in $HADOOP_HOME/lib/*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+for f in $HADOOP_HOME/lib/jetty-ext/*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+# restore ordinary behaviour
+unset IFS
+
+# figure out which class to run
+if [ "$COMMAND" = "namenode" ] ; then
+  CLASS='org.apache.hadoop.dfs.NameNode'
+elif [ "$COMMAND" = "datanode" ] ; then
+  CLASS='org.apache.hadoop.dfs.DataNode'
+elif [ "$COMMAND" = "dfs" ] ; then
+  CLASS=org.apache.hadoop.fs.DFSShell
+elif [ "$COMMAND" = "jobtracker" ] ; then
+  CLASS=org.apache.hadoop.mapred.JobTracker
+elif [ "$COMMAND" = "tasktracker" ] ; then
+  CLASS=org.apache.hadoop.mapred.TaskTracker
+elif [ "$COMMAND" = "job" ] ; then
+  CLASS=org.apache.hadoop.mapred.JobClient
+else
+  CLASS=$COMMAND
+fi
+
+# cygwin path translation
+if expr `uname` : 'CYGWIN*' > /dev/null; then
+  CLASSPATH=`cygpath -p -w "$CLASSPATH"`
+fi
+
+# run it
+exec "$JAVA" $JAVA_HEAP_MAX $HADOOP_OPTS -classpath "$CLASSPATH" $CLASS "$@"

+ 101 - 0
bin/hadoop-daemon.sh

@@ -0,0 +1,101 @@
+#!/bin/bash
+# 
+# Runs a Hadoop command as a daemon.
+#
+# Environment Variables
+#
+#   HADOOP_LOG_DIR   Where log files are stored.  PWD by default.
+#   HADOOP_MASTER    host:path where hadoop code should be rsync'd from
+#   HADOOP_PID_DIR   The pid files are stored. /tmp by default.
+#   HADOOP_IDENT_STRING   A string representing this instance of hadoop. $USER by default
+##
+
+usage="Usage: hadoop-daemon [start|stop] [hadoop-command] [args...]"
+
+# if no args specified, show usage
+if [ $# -le 1 ]; then
+  echo $usage
+  exit 1
+fi
+
+# get arguments
+startStop=$1
+shift
+command=$1
+shift
+
+# resolve links - $0 may be a softlink
+this="$0"
+while [ -h "$this" ]; do
+  ls=`ls -ld "$this"`
+  link=`expr "$ls" : '.*-> \(.*\)$'`
+  if expr "$link" : '.*/.*' > /dev/null; then
+    this="$link"
+  else
+    this=`dirname "$this"`/"$link"
+  fi
+done
+
+# get log directory
+if [ "$HADOOP_LOG_DIR" = "" ]; then
+  HADOOP_LOG_DIR=$PWD
+fi
+
+if [ "$HADOOP_PID_DIR" = "" ]; then
+  HADOOP_PID_DIR=/tmp
+fi
+
+if [ "$HADOOP_IDENT_STRING" = "" ]; then
+  HADOOP_IDENT_STRING=$USER
+fi
+
+# some variables
+log=$HADOOP_LOG_DIR/hadoop-$HADOOP_IDENT_STRING-$command-`hostname`.log
+pid=$HADOOP_PID_DIR/hadoop-$HADOOP_IDENT_STRING-$command.pid
+
+case $startStop in
+
+  (start)
+
+    if [ -f $pid ]; then
+      if [ -a /proc/`cat $pid` ]; then
+        echo $command running as process `cat $pid`.  Stop it first.
+        exit 1
+      fi
+    fi
+
+    root=`dirname $this`/..
+    if [ "$HADOOP_MASTER" != "" ]; then
+      echo rsync from $HADOOP_MASTER
+      rsync -a --delete --exclude=.svn $HADOOP_MASTER/ $root
+    fi
+
+    cd $root
+    echo starting $command, logging to $log
+    nohup bin/hadoop $command "$@" >& $log < /dev/null &
+    echo $! > $pid
+    sleep 1; head $log
+    ;;
+          
+  (stop)
+
+    if [ -f $pid ]; then
+      if [ -a /proc/`cat $pid` ]; then
+        echo stopping $command
+        kill `cat $pid`
+      else
+        echo no $command to stop
+      fi
+    else
+      echo no $command to stop
+    fi
+    ;;
+
+  (*)
+    echo $usage
+    exit 1
+    ;;
+
+esac
+
+

+ 16 - 0
bin/hadoop-daemons.sh

@@ -0,0 +1,16 @@
+#!/bin/bash
+# 
+# Run a Hadoop command on all slave hosts.
+
+usage="Usage: hadoop-daemons.sh [start|stop] command args..."
+
+# if no args specified, show usage
+if [ $# -le 1 ]; then
+  echo $usage
+  exit 1
+fi
+
+bin=`dirname $0`
+bin=`cd $bin; pwd`
+
+exec $bin/slaves.sh $bin/hadoop-daemon.sh "$@"

+ 27 - 0
bin/slaves.sh

@@ -0,0 +1,27 @@
+#!/bin/bash
+# 
+# Run a shell command on all slave hosts.
+#
+# Environment Variables
+#
+#   HADOOP_SLAVES    File naming remote hosts.  Default is ~/.slaves
+##
+
+usage="Usage: slaves.sh command..."
+
+# if no args specified, show usage
+if [ $# -le 0 ]; then
+  echo $usage
+  exit 1
+fi
+
+if [ "$HADOOP_SLAVES" = "" ]; then
+  export HADOOP_SLAVES=$HOME/.slaves
+fi
+
+for slave in `cat $HADOOP_SLAVES`; do
+ ssh -o ConnectTimeout=1 $slave "$@" \
+   2>&1 | sed "s/^/$slave: /" &
+done
+
+wait

+ 11 - 0
bin/start-all.sh

@@ -0,0 +1,11 @@
+#!/bin/bash
+
+# Start all hadoop daemons.  Run this on master node.
+
+bin=`dirname $0`
+bin=`cd $bin; pwd`
+
+$bin/hadoop-daemons.sh start datanode
+$bin/hadoop-daemon.sh start namenode
+$bin/hadoop-daemon.sh start jobtracker
+$bin/hadoop-daemons.sh start tasktracker

+ 11 - 0
bin/stop-all.sh

@@ -0,0 +1,11 @@
+#!/bin/bash
+
+# Stop all hadoop daemons.  Run this on master node.
+
+bin=`dirname $0`
+bin=`cd $bin; pwd`
+
+$bin/hadoop-daemon.sh stop jobtracker
+$bin/hadoop-daemons.sh stop tasktracker
+$bin/hadoop-daemon.sh stop namenode
+$bin/hadoop-daemons.sh stop datanode

+ 277 - 0
build.xml

@@ -0,0 +1,277 @@
+<?xml version="1.0"?>
+
+<project name="Hadoop" default="compile">
+
+  <!-- Load all the default properties, and any the user wants    -->
+  <!-- to contribute (without having to type -D or edit this file -->
+  <property file="${user.home}/build.properties" />
+  <property file="${basedir}/build.properties" />
+ 
+  <property name="Name" value="Hadoop"/>
+  <property name="name" value="hadoop"/>
+  <property name="version" value="0.1-dev"/>
+  <property name="final.name" value="${name}-${version}"/>
+  <property name="year" value="2006"/>
+
+  <property name="basedir" value="./"/>
+  <property name="src.dir" value="src/java"/>
+  <property name="lib.dir" value="lib"/>
+  <property name="conf.dir" value="conf"/>
+  <property name="docs.dir" value="docs"/>
+  <property name="docs.src" value="${basedir}/src/web"/>
+
+  <property name="build.dir" value="build"/>
+  <property name="build.classes" value="${build.dir}/classes"/>
+  <property name="build.webapps" value="${build.dir}/webapps"/>
+  <property name="build.docs" value="${build.dir}/docs"/>
+  <property name="build.javadoc" value="${build.docs}/api"/>
+  <property name="build.encoding" value="ISO-8859-1"/>
+
+  <property name="test.src.dir" value="src/test"/>
+  <property name="test.build.dir" value="${build.dir}/test"/>
+  <property name="test.build.data" value=" ${test.build.dir}/data"/>
+  <property name="test.build.classes" value="${test.build.dir}/classes"/>
+  <property name="test.build.javadoc" value="${test.build.dir}/docs/api"/>
+
+  <property name="web.src.dir" value="src/web"/>
+  <property name="src.webapps" value="src/webapps"/>
+
+  <property name="javadoc.link.java"
+	    value="http://java.sun.com/j2se/1.4.2/docs/api/"/>
+  <property name="javadoc.packages" value="org.apache.hadoop.*"/>
+
+  <property name="dist.dir" value="${build.dir}/${final.name}"/>
+
+  <property name="javac.debug" value="on"/>
+  <property name="javac.optimize" value="on"/>
+  <property name="javac.deprecation" value="off"/>
+  <property name="javac.version" value="1.4"/>
+
+  <!-- the normal classpath -->
+  <path id="classpath">
+    <pathelement location="${build.classes}"/>
+    <fileset dir="${lib.dir}">
+      <include name="*.jar" />
+    </fileset>
+  </path>
+
+  <!-- the unit test classpath -->
+  <path id="test.classpath">
+    <pathelement location="${test.build.classes}" />
+    <pathelement location="${conf.dir}"/>
+    <pathelement location="${test.src.dir}"/>
+    <path refid="classpath"/>
+  </path>
+
+  <!-- ====================================================== -->
+  <!-- Stuff needed by all targets                            -->
+  <!-- ====================================================== -->
+  <target name="init">
+    <mkdir dir="${build.dir}"/>
+    <mkdir dir="${build.classes}"/>
+    <mkdir dir="${build.webapps}"/>
+
+    <mkdir dir="${test.build.dir}"/>
+    <mkdir dir="${test.build.classes}"/>
+
+    <touch datetime="01/25/1971 2:00 pm">
+      <fileset dir="${conf.dir}" includes="**/*.template"/>
+    </touch>
+
+    <copy todir="${build.webapps}/jobtracker">
+      <fileset dir="${src.webapps}/jobtracker"/>
+    </copy>
+
+    <copy todir="${conf.dir}" verbose="true">
+      <fileset dir="${conf.dir}" includes="**/*.template"/>
+      <mapper type="glob" from="*.template" to="*"/>
+    </copy>
+  </target>
+
+  <!-- ====================================================== -->
+  <!-- Compile the Java files                                 -->
+  <!-- ====================================================== -->
+  <target name="compile" depends="init">
+    <javac 
+     encoding="${build.encoding}" 
+     srcdir="${src.dir}"
+     includes="org/apache/hadoop/**/*.java"
+     destdir="${build.classes}"
+     debug="${javac.debug}"
+     optimize="${javac.optimize}"
+     target="${javac.version}"
+     source="${javac.version}"
+     deprecation="${javac.deprecation}">
+      <classpath refid="classpath"/>
+    </javac>    
+  </target>
+
+  <!-- ================================================================== -->
+  <!-- Make hadoop.jar                                                     -->
+  <!-- ================================================================== -->
+  <!--                                                                    -->
+  <!-- ================================================================== -->
+  <target name="jar" depends="compile">
+    <copy file="${conf.dir}/hadoop-default.xml"
+          todir="${build.classes}"/>
+    <jar jarfile="${build.dir}/${final.name}.jar"
+         basedir="${build.classes}">
+      <manifest>
+      </manifest>
+    </jar>
+  </target>
+
+  <!-- ================================================================== -->
+  <!-- Compile test code                                                  --> 
+  <!-- ================================================================== -->
+  <target name="compile-test" depends="compile">
+    <javac 
+     encoding="${build.encoding}" 
+     srcdir="${test.src.dir}"
+     includes="org/apache/hadoop/**/*.java"
+     destdir="${test.build.classes}"
+     debug="${javac.debug}"
+     optimize="${javac.optimize}"
+     target="${javac.version}"
+     source="${javac.version}"
+     deprecation="${javac.deprecation}">
+      <classpath refid="test.classpath"/>
+    </javac>    
+  </target>
+
+  <!-- ================================================================== -->
+  <!-- Run unit tests                                                     --> 
+  <!-- ================================================================== -->
+  <target name="test" depends="compile, compile-test">
+
+    <delete dir="${test.build.data}"/>
+    <mkdir dir="${test.build.data}"/>
+
+    <copy file="${test.src.dir}/hadoop-site.xml"
+          todir="${test.build.classes}"/>
+
+    <junit printsummary="yes" haltonfailure="no" fork="yes" dir="${basedir}"
+      errorProperty="tests.failed" failureProperty="tests.failed">
+      <sysproperty key="test.build.data" value="${test.build.data}"/>
+      <sysproperty key="test.src.dir" value="${test.src.dir}"/>
+      <classpath refid="test.classpath"/>
+      <formatter type="plain" />
+      <batchtest todir="${test.build.dir}" unless="testcase">
+        <fileset dir="${test.src.dir}"
+                 includes="**/Test*.java" excludes="**/${test.exclude}.java" />
+      </batchtest>
+      <batchtest todir="${test.build.dir}" if="testcase">
+        <fileset dir="${test.src.dir}" includes="**/${testcase}.java"/>
+      </batchtest>
+    </junit>
+
+    <fail if="tests.failed">Tests failed!</fail>
+
+  </target>   
+
+  <target name="nightly" depends="test, tar">
+  </target>
+
+  <!-- ================================================================== -->
+  <!-- Documentation                                                      -->
+  <!-- ================================================================== -->
+  <target name="javadoc" depends="compile">
+    <mkdir dir="${build.javadoc}"/>
+    <javadoc
+      overview="${src.dir}/overview.html"
+      destdir="${build.javadoc}"
+      author="true"
+      version="true"
+      use="true"
+      windowtitle="${Name} ${version} API"
+      doctitle="${Name} ${version} API"
+      bottom="Copyright &amp;copy; ${year} The Apache Software Foundation"
+      >
+    	<packageset dir="${src.dir}"/>
+        <link href="${javadoc.link.java}"/>
+        <classpath refid="classpath"/>
+    </javadoc>
+  </target>	
+	
+  <target name="default-doc">
+    <style basedir="${conf.dir}" destdir="${docs.dir}"
+           includes="hadoop-default.xml" style="conf/hadoop-conf.xsl"/>
+  </target>
+
+  <!-- ================================================================== -->
+  <!-- D I S T R I B U T I O N                                            -->
+  <!-- ================================================================== -->
+  <!--                                                                    -->
+  <!-- ================================================================== -->
+  <target name="package" depends="jar, javadoc">
+    <mkdir dir="${dist.dir}"/>
+    <mkdir dir="${dist.dir}/lib"/>
+    <mkdir dir="${dist.dir}/bin"/>
+    <mkdir dir="${dist.dir}/docs"/>
+    <mkdir dir="${dist.dir}/docs/api"/>
+
+    <copy todir="${dist.dir}/lib" includeEmptyDirs="false">
+      <fileset dir="lib"/>
+    </copy>
+
+    <copy todir="${dist.dir}/webapps">
+      <fileset dir="${build.webapps}"/>
+    </copy>
+
+    <copy file="${build.dir}/${final.name}.jar" todir="${dist.dir}"/>
+
+    <copy todir="${dist.dir}/bin">
+      <fileset dir="bin"/>
+    </copy>
+
+    <copy todir="${dist.dir}/conf">
+      <fileset dir="${conf.dir}" excludes="**/*.template"/>
+    </copy>
+
+    <chmod perm="ugo+x" type="file">
+        <fileset dir="${dist.dir}/bin"/>
+    </chmod>
+
+    <copy todir="${dist.dir}/docs/api">
+      <fileset dir="${build.javadoc}"/>
+    </copy>
+
+    <copy todir="${dist.dir}">
+      <fileset dir=".">
+        <include name="*.txt" />
+      </fileset>
+    </copy>
+
+    <copy todir="${dist.dir}/src" includeEmptyDirs="true">
+      <fileset dir="src"/>
+    </copy>
+
+    <copy todir="${dist.dir}/" file="build.xml"/>
+    <copy todir="${dist.dir}/" file="default.properties"/>
+
+  </target>
+
+  <!-- ================================================================== -->
+  <!-- Make release tarball                                               -->
+  <!-- ================================================================== -->
+  <target name="tar" depends="package">
+    <tar compression="gzip" longfile="gnu"
+      destfile="${build.dir}/${final.name}.tar.gz">
+      <tarfileset dir="${build.dir}" mode="664">
+	<exclude name="${final.name}/bin/*" />
+        <include name="${final.name}/**" />
+      </tarfileset>
+      <tarfileset dir="${build.dir}" mode="755">
+        <include name="${final.name}/bin/*" />
+      </tarfileset>
+    </tar>
+  </target>
+	
+  <!-- ================================================================== -->
+  <!-- Clean.  Delete the build files, and their directories              -->
+  <!-- ================================================================== -->
+  <target name="clean">
+    <delete dir="${build.dir}"/>
+  </target>
+
+</project>

+ 237 - 0
conf/hadoop-default.xml

@@ -0,0 +1,237 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="nutch-conf.xsl"?>
+
+<!-- Do not modify this file directly.  Instead, copy entries that you -->
+<!-- wish to modify from this file into nutch-site.xml and change them -->
+<!-- there.  If nutch-site.xml does not already exist, create it.      -->
+
+<nutch-conf>
+
+<!-- file properties -->
+
+<property>
+  <name>file.content.limit</name>
+  <value>65536</value>
+  <description>The length limit for downloaded content, in bytes.
+  If this value is larger than zero, content longer than it will be
+  truncated; otherwise (zero or negative), no truncation at all.
+  </description>
+</property>
+
+<property>
+  <name>file.content.ignored</name>
+  <value>true</value>
+  <description>If true, no file content will be saved during fetch.
+  And it is probably what we want to set most of time, since file:// URLs
+  are meant to be local and we can always use them directly at parsing
+  and indexing stages. Otherwise file contents will be saved.
+  !! NO IMPLEMENTED YET !!
+  </description>
+</property>
+
+<!-- i/o properties -->
+
+<property>
+  <name>io.sort.factor</name>
+  <value>10</value>
+  <description>The number of streams to merge at once while sorting
+  files.  This determines the number of open file handles.</description>
+</property>
+
+<property>
+  <name>io.sort.mb</name>
+  <value>100</value>
+  <description>The total amount of buffer memory to use while sorting 
+  files, in megabytes.  By default, gives each merge stream 1MB, which
+  should minimize seeks.</description>
+</property>
+
+<property>
+  <name>io.file.buffer.size</name>
+  <value>4096</value>
+  <description>The size of buffer for use in sequence files.
+  The size of this buffer should probably be a multiple of hardware
+  page size (4096 on Intel x86), and it determines how much data is
+  buffered during read and write operations.</description>
+</property>
+  
+<property>
+  <name>io.bytes.per.checksum</name>
+  <value>512</value>
+  <description>The number of bytes per checksum.  Must not be larger than
+  io.file.buffer.size.</description>
+</property>
+
+<property>
+  <name>io.skip.checksum.errors</name>
+  <value>false</value>
+  <description>If true, when a checksum error is encountered while
+  reading a sequence file, entries are skipped, instead of throwing an
+  exception.</description>
+</property>
+  
+<property>
+  <name>io.map.index.skip</name>
+  <value>0</value>
+  <description>Number of index entries to skip between each entry.
+  Zero by default. Setting this to values larger than zero can
+  facilitate opening large map files using less memory.</description>
+</property>
+
+<!-- file system properties -->
+
+<property>
+  <name>fs.default.name</name>
+  <value>local</value>
+  <description>The name of the default file system.  Either the
+  literal string "local" or a host:port for DFS.</description>
+</property>
+
+<property>
+  <name>dfs.datanode.port</name>
+  <value>50010</value>
+  <description>The port number that the dfs datanode server uses as a starting 
+	       point to look for a free port to listen on.
+</description>
+</property>
+
+<property>
+  <name>dfs.name.dir</name>
+  <value>/tmp/nutch/dfs/name</value>
+  <description>Determines where on the local filesystem the DFS name node
+      should store the name table.</description>
+</property>
+
+<property>
+  <name>dfs.data.dir</name>
+  <value>/tmp/nutch/dfs/data</value>
+  <description>Determines where on the local filesystem an DFS data node
+  should store its blocks.  If this is a comma- or space-delimited
+  list of directories, then data will be stored in all named
+  directories, typically on different devices.</description>
+</property>
+
+<property>
+  <name>dfs.replication</name>
+  <value>3</value>
+  <description>How many copies we try to have at all times. The actual
+  number of replications is at max the number of datanodes in the
+  cluster.</description>
+</property>
+
+<!-- map/reduce properties -->
+
+<property>
+  <name>mapred.job.tracker</name>
+  <value>local</value>
+  <description>The host and port that the MapReduce job tracker runs
+  at.  If "local", then jobs are run in-process as a single map
+  and reduce task.
+  </description>
+</property>
+
+<property>
+  <name>mapred.job.tracker.info.port</name>
+  <value>50030</value>
+  <description>The port that the MapReduce job tracker info webserver runs at.
+  </description>
+</property>
+
+<property>
+  <name>mapred.task.tracker.output.port</name>
+  <value>50040</value>
+  <description>The port number that the MapReduce task tracker output server uses as a starting
+               point to look for a free port to listen on.
+  </description>
+</property>
+
+<property>
+  <name>mapred.task.tracker.report.port</name>
+  <value>50050</value>
+  <description>The port number that the MapReduce task tracker report server uses as a starting
+               point to look for a free port to listen on.
+  </description>
+</property>
+
+<property>
+  <name>mapred.local.dir</name>
+  <value>/tmp/nutch/mapred/local</value>
+  <description>The local directory where MapReduce stores intermediate
+  data files.  May be a space- or comma- separated list of
+  directories on different devices in order to spread disk i/o.
+  </description>
+</property>
+
+<property>
+  <name>mapred.system.dir</name>
+  <value>/tmp/nutch/mapred/system</value>
+  <description>The shared directory where MapReduce stores control files.
+  </description>
+</property>
+
+<property>
+  <name>mapred.temp.dir</name>
+  <value>/tmp/nutch/mapred/temp</value>
+  <description>A shared directory for temporary files.
+  </description>
+</property>
+
+<property>
+  <name>mapred.map.tasks</name>
+  <value>2</value>
+  <description>The default number of map tasks per job.  Typically set
+  to a prime several times greater than number of available hosts.
+  Ignored when mapred.job.tracker is "local".  
+  </description>
+</property>
+
+<property>
+  <name>mapred.reduce.tasks</name>
+  <value>1</value>
+  <description>The default number of reduce tasks per job.  Typically set
+  to a prime close to the number of available hosts.  Ignored when
+  mapred.job.tracker is "local".
+  </description>
+</property>
+
+<property>
+  <name>mapred.task.timeout</name>
+  <value>600000</value>
+  <description>The number of milliseconds before a task will be
+  terminated if it neither reads an input, writes an output, nor
+  updates its status string.
+  </description>
+</property>
+
+<property>
+  <name>mapred.tasktracker.tasks.maximum</name>
+  <value>2</value>
+  <description>The maximum number of tasks that will be run
+  simultaneously by a task tracker.
+  </description>
+</property>
+
+<property>
+  <name>mapred.child.heap.size</name>
+  <value>200m</value>
+  <description>The heap size (-Xmx) that will be used for task tracker
+  child processes.</description>
+</property>
+
+<property>
+  <name>mapred.combine.buffer.size</name>
+  <value>100000</value>
+  <description>The number of entries the combining collector caches before
+  combining them and writing to disk.</description>
+</property>
+
+
+<!-- ipc properties -->
+
+<property>
+  <name>ipc.client.timeout</name>
+  <value>60000</value>
+  <description>Defines the timeout for IPC calls in milliseconds.</description>
+</property>
+
+</nutch-conf>

+ 8 - 0
conf/hadoop-site.xml.template

@@ -0,0 +1,8 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="nutch-conf.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<nutch-conf>
+
+</nutch-conf>

+ 8 - 0
conf/mapred-default.xml.template

@@ -0,0 +1,8 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="nutch-conf.xsl"?>
+
+<!-- Put mapred-specific property overrides in this file. -->
+
+<nutch-conf>
+
+</nutch-conf>

+ 202 - 0
lib/jetty-5.1.4.LICENSE.txt

@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.

二進制
lib/jetty-5.1.4.jar


二進制
lib/jetty-ext/ant.jar


二進制
lib/jetty-ext/commons-el.jar


二進制
lib/jetty-ext/jasper-compiler.jar


二進制
lib/jetty-ext/jasper-runtime.jar


二進制
lib/jetty-ext/jsp-api.jar


+ 100 - 0
lib/junit-3.8.1.LICENSE.txt

@@ -0,0 +1,100 @@
+Common Public License Version 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS COMMON PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+1. DEFINITIONS
+
+"Contribution" means:
+
+    a) in the case of the initial Contributor, the initial code and documentation distributed under this Agreement, and
+
+    b) in the case of each subsequent Contributor:
+
+    i) changes to the Program, and
+
+    ii) additions to the Program;
+
+    where such changes and/or additions to the Program originate from and are distributed by that particular Contributor. A Contribution 'originates' from a Contributor if it was added to the Program by such Contributor itself or anyone acting on such Contributor's behalf. Contributions do not include additions to the Program which: (i) are separate modules of software distributed in conjunction with the Program under their own license agreement, and (ii) are not derivative works of the Program.
+
+"Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents " mean patent claims licensable by a Contributor which are necessarily infringed by the use or sale of its Contribution alone or when combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement, including all Contributors.
+
+2. GRANT OF RIGHTS
+
+    a) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce, prepare derivative works of, publicly display, publicly perform, distribute and sublicense the Contribution of such Contributor, if any, and such derivative works, in source code and object code form.
+
+    b) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed Patents to make, use, sell, offer to sell, import and otherwise transfer the Contribution of such Contributor, if any, in source code and object code form. This patent license shall apply to the combination of the Contribution and the Program if, at the time the Contribution is added by the Contributor, such addition of the Contribution causes such combination to be covered by the Licensed Patents. The patent license shall not apply to any other combinations which include the Contribution. No hardware per se is licensed hereunder.
+
+    c) Recipient understands that although each Contributor grants the licenses to its Contributions set forth herein, no assurances are provided by any Contributor that the Program does not infringe the patent or other intellectual property rights of any other entity. Each Contributor disclaims any liability to Recipient for claims brought by any other entity based on infringement of intellectual property rights or otherwise. As a condition to exercising the rights and licenses granted hereunder, each Recipient hereby assumes sole responsibility to secure any other intellectual property rights needed, if any. For example, if a third party patent license is required to allow Recipient to distribute the Program, it is Recipient's responsibility to acquire that license before distributing the Program.
+
+    d) Each Contributor represents that to its knowledge it has sufficient copyright rights in its Contribution, if any, to grant the copyright license set forth in this Agreement.
+
+3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under its own license agreement, provided that:
+
+    a) it complies with the terms and conditions of this Agreement; and
+
+    b) its license agreement:
+
+    i) effectively disclaims on behalf of all Contributors all warranties and conditions, express and implied, including warranties or conditions of title and non-infringement, and implied warranties or conditions of merchantability and fitness for a particular purpose;
+
+    ii) effectively excludes on behalf of all Cntributors all liability for damages, including direct, indirect, special, incidental and consequential damages, such as lost profits;
+
+    iii) states that any provisions which differ from this Agreement are offered by that Contributor alone and not by any other party; and
+
+    iv) states that source code for the Program is available from such Contributor, and informs licensees how to obtain it in a reasonable manner on or through a medium customarily used for software exchange. 
+
+When the Program is made available in source code form:
+
+    a) it must be made available under this Agreement; and
+
+    b) a copy of this Agreement must be included with each copy of the Program. 
+
+Contributors may not remove or alter any copyright notices contained within the Program.
+
+Each Contributor must identify itself as the originator of its Contribution, if any, in a manner that reasonably allows subsequent Recipients to identify the originator of the Contribution.
+
+4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with respect to end users, business partners and the like. While this license is intended to facilitate the commercial use of the Program, the Contributor who includes the Program in a commercial product offering should do so in a manner which does not create potential liability for other Contributors. Therefore, if a Contributor includes the Program in a commercial product offering, such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify every other Contributor ("Indemnified Contributor") against any losses, damages and costs (collectively "Losses") arising from claims, lawsuits and other legal actions brought by a third party against the Indemnified Contributor to the extent caused by the acts or omissions of such Commercial Contributor in connection with its distribution of the Program in a commercial product offering. The obligations in this section do not apply to any claims or Losses relating to any actual or alleged intellectual property infringement. In order to qualify, an Indemnified Contributor must: a) promptly notify the Commercial Contributor in writing of such claim, and b) allow the Commercial Contributor to control, and cooperate with the Commercial Contributor in, the defense and any related settlement negotiations. The Indemnified Contributor may participate in any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product offering, Product X. That Contributor is then a Commercial Contributor. If that Commercial Contributor then makes performance claims, or offers warranties related to Product X, those performance claims and warranties are such Commercial Contributor's responsibility alone. Under this section, the Commercial Contributor would have to defend claims against the other Contributors related to those performance claims and warranties, and if a court requires any other Contributor to pay any damages as a result, the Commercial Contributor must pay those damages.
+
+5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the appropriateness of using and distributing the Program and assumes all risks associated with its exercise of rights under this Agreement, including but not limited to the risks and costs of program errors, compliance with applicable laws, damage to or loss of data, programs or equipment, and unavailability or interruption of operations.
+
+6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST PR LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable.
+
+If Recipient institutes patent litigation against a Contributor with respect to a patent applicable to software (including a cross-claim or counterclaim in a lawsuit), then any patent licenses granted by that Contributor to such Recipient under this Agreement shall terminate as of the date such litigation is filed. In addition, if Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. IBM is the initial Agreement Steward. IBM may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation.
+OFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable.
+
+If Recipient institutes patent litigation against a Contributor with respect to a patent applicable to software (including a cross-claim or counterclaim in a lawsuit), then any patent licenses granted by that Contributor to such Recipient under this Agreement shall terminate as of the date such litigation is filed. In addition, if Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. IBM is the initial Agreement Steward. IBM may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation.

二進制
lib/junit-3.8.1.jar


二進制
lib/servlet-api.jar


+ 27 - 0
src/java/org/apache/hadoop/conf/Configurable.java

@@ -0,0 +1,27 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.conf;
+
+/** Something that may be configured with a {@link Configuration}. */
+public interface Configurable {
+
+  /** Set the configuration to be used by this object. */
+  void setConf(Configuration conf);
+
+  /** Return the configuration used by this object. */
+  Configuration getConf();
+}

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

@@ -0,0 +1,445 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.conf;
+
+import java.util.*;
+import java.net.URL;
+import java.io.*;
+import java.util.logging.Logger;
+
+import javax.xml.parsers.*;
+
+import org.w3c.dom.*;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+
+/** Provides access to configuration parameters.
+ * <p>An ordered list of configuration parameter files with
+ * default and always-overrides site parameters.
+ * <p>Default values for all parameters are specified in a file named
+ * <tt>nutch-default.xml</tt> located on the classpath.  Overrides for these
+ * defaults should be in an optional file named <tt>nutch-site.xml</tt>, also
+ * located on the classpath.  Typically these files reside in the
+ * <tt>conf/</tt> subdirectory at the top-level of a Nutch installation.
+ * <p>The resource files are read upon first access of values (set, get,
+ * or write) after {@link #addConfResource(String)} or
+ * {@link #addConfResource(File)}.
+ */
+public class Configuration {
+  private static final Logger LOG =
+    LogFormatter.getLogger("org.apache.hadoop.conf.Configuration");
+
+  private ArrayList resourceNames = new ArrayList();
+  private Properties properties;
+  private ClassLoader classLoader = 
+    Thread.currentThread().getContextClassLoader();
+
+  /** A new configuration. */
+  public Configuration() {
+    resourceNames.add("nutch-default.xml");
+    resourceNames.add("nutch-site.xml");
+  }
+
+  /** A new configuration with the same settings cloned from another. */
+  public Configuration(Configuration other) {
+    this.resourceNames = (ArrayList)other.resourceNames.clone();
+    if (other.properties != null)
+      this.properties = (Properties)other.properties.clone();
+  }
+
+  /** Adds a resource name to the chain of resources read.  Such resources are
+   * located on the CLASSPATH.  The first resource is always
+   * <tt>nutch-default.xml</tt>, and the last is always
+   * <tt>nutch-site.xml</tt>.  New resources are inserted between these, so
+   * they can override defaults, but not site-specifics. */
+  public synchronized void addConfResource(String name) {
+    addConfResourceInternal(name);
+  }
+
+  /** Adds a file to the chain of resources read.  The first resource is always
+   * <tt>nutch-default.xml</tt>, and the last is always
+   * <tt>nutch-site.xml</tt>.  New resources are inserted between these, so
+   * they can override defaults, but not site-specifics. */
+  public synchronized void addConfResource(File file) {
+    addConfResourceInternal(file);
+  }
+
+  private synchronized void addConfResourceInternal(Object name) {
+    resourceNames.add(resourceNames.size()-1, name); // add second to last
+    properties = null;                            // trigger reload
+  }
+  
+  /**
+   * Returns the value of the <code>name</code> property, or null if no such
+   * property exists.
+   */
+  public Object getObject(String name) { return getProps().get(name);}
+
+  /** Sets the value of the <code>name</code> property. */
+  public void setObject(String name, Object value) {
+    getProps().put(name, value);
+  }
+
+  /** Returns the value of the <code>name</code> property.  If no such property
+   * exists, then <code>defaultValue</code> is returned.
+   */
+  public Object get(String name, Object defaultValue) {
+    Object res = getObject(name);
+    if (res != null) return res;
+    else return defaultValue;
+  }
+  
+  /** Returns the value of the <code>name</code> property, or null if no
+   * such property exists. */
+  public String get(String name) { return getProps().getProperty(name);}
+
+  /** Sets the value of the <code>name</code> property. */
+  public void set(String name, Object value) {
+    getProps().setProperty(name, value.toString());
+  }
+  
+  /** Returns the value of the <code>name</code> property.  If no such property
+   * exists, then <code>defaultValue</code> is returned.
+   */
+  public String get(String name, String defaultValue) {
+     return getProps().getProperty(name, defaultValue);
+  }
+  
+  /** Returns the value of the <code>name</code> property as an integer.  If no
+   * such property is specified, or if the specified value is not a valid
+   * integer, then <code>defaultValue</code> is returned.
+   */
+  public int getInt(String name, int defaultValue) {
+    String valueString = get(name);
+    if (valueString == null)
+      return defaultValue;
+    try {
+      return Integer.parseInt(valueString);
+    } catch (NumberFormatException e) {
+      return defaultValue;
+    }
+  }
+
+  /** Sets the value of the <code>name</code> property to an integer. */
+  public void setInt(String name, int value) {
+    set(name, Integer.toString(value));
+  }
+
+
+  /** Returns the value of the <code>name</code> property as a long.  If no
+   * such property is specified, or if the specified value is not a valid
+   * long, then <code>defaultValue</code> is returned.
+   */
+  public long getLong(String name, long defaultValue) {
+    String valueString = get(name);
+    if (valueString == null)
+      return defaultValue;
+    try {
+      return Long.parseLong(valueString);
+    } catch (NumberFormatException e) {
+      return defaultValue;
+    }
+  }
+
+  /** Sets the value of the <code>name</code> property to a long. */
+  public void setLong(String name, long value) {
+    set(name, Long.toString(value));
+  }
+
+  /** Returns the value of the <code>name</code> property as a float.  If no
+   * such property is specified, or if the specified value is not a valid
+   * float, then <code>defaultValue</code> is returned.
+   */
+  public float getFloat(String name, float defaultValue) {
+    String valueString = get(name);
+    if (valueString == null)
+      return defaultValue;
+    try {
+      return Float.parseFloat(valueString);
+    } catch (NumberFormatException e) {
+      return defaultValue;
+    }
+  }
+
+  /** Returns the value of the <code>name</code> property as an boolean.  If no
+   * such property is specified, or if the specified value is not a valid
+   * boolean, then <code>defaultValue</code> is returned.  Valid boolean values
+   * are "true" and "false".
+   */
+  public boolean getBoolean(String name, boolean defaultValue) {
+    String valueString = get(name);
+    if ("true".equals(valueString))
+      return true;
+    else if ("false".equals(valueString))
+      return false;
+    else return defaultValue;
+  }
+
+  /** Sets the value of the <code>name</code> property to an integer. */
+  public void setBoolean(String name, boolean value) {
+    set(name, Boolean.toString(value));
+  }
+
+  /** Returns the value of the <code>name</code> property as an array of
+   * strings.  If no such property is specified, then <code>null</code>
+   * is returned.  Values are whitespace or comma delimted.
+   */
+  public String[] getStrings(String name) {
+    String valueString = get(name);
+    if (valueString == null)
+      return null;
+    StringTokenizer tokenizer = new StringTokenizer (valueString,", \t\n\r\f");
+    List values = new ArrayList();
+    while (tokenizer.hasMoreTokens()) {
+      values.add(tokenizer.nextToken());
+    }
+    return (String[])values.toArray(new String[values.size()]);
+  }
+
+  /** Returns the value of the <code>name</code> property as a Class.  If no
+   * such property is specified, then <code>defaultValue</code> is returned.
+   */
+  public Class getClass(String name, Class defaultValue) {
+    String valueString = get(name);
+    if (valueString == null)
+      return defaultValue;
+    try {
+      return Class.forName(valueString);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Returns the value of the <code>name</code> property as a Class.  If no
+   * such property is specified, then <code>defaultValue</code> is returned.
+   * An error is thrown if the returned class does not implement the named
+   * interface. 
+   */
+  public Class getClass(String propertyName, Class defaultValue,Class xface) {
+    try {
+      Class theClass = getClass(propertyName, defaultValue);
+      if (theClass != null && !xface.isAssignableFrom(theClass))
+        throw new RuntimeException(theClass+" not "+xface.getName());
+      return theClass;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Sets the value of the <code>name</code> property to the name of a class.
+   * First checks that the class implements the named interface. 
+   */
+  public void setClass(String propertyName, Class theClass, Class xface) {
+    if (!xface.isAssignableFrom(theClass))
+      throw new RuntimeException(theClass+" not "+xface.getName());
+    set(propertyName, theClass.getName());
+  }
+
+  /** Returns the URL for the named resource. */
+  public URL getResource(String name) {
+    return classLoader.getResource(name);
+  }
+  /** Returns an input stream attached to the configuration resource with the
+   * given <code>name</code>.
+   */
+  public InputStream getConfResourceAsInputStream(String name) {
+    try {
+      URL url= getResource(name);
+
+      if (url == null) {
+        LOG.info(name + " not found");
+        return null;
+      } else {
+        LOG.info("found resource " + name + " at " + url);
+      }
+
+      return url.openStream();
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  /** Returns a reader attached to the configuration resource with the
+   * given <code>name</code>.
+   */
+  public Reader getConfResourceAsReader(String name) {
+    try {
+      URL url= getResource(name);
+
+      if (url == null) {
+        LOG.info(name + " not found");
+        return null;
+      } else {
+        LOG.info("found resource " + name + " at " + url);
+      }
+
+      return new InputStreamReader(url.openStream());
+    } catch (Exception e) {
+      return null;
+    }
+  }
+
+  private synchronized Properties getProps() {
+    if (properties == null) {
+      Properties defaults = new Properties();
+      Properties newProps = new Properties(defaults);
+      ListIterator i = resourceNames.listIterator();
+      while (i.hasNext()) {
+        if (i.nextIndex() == 0) {                 // load defaults
+          loadResource(defaults, i.next(), false);
+        } else if (i.nextIndex()==resourceNames.size()-1) { // load site
+          loadResource(newProps, i.next(), true);
+        } else {                                  // load intermediate
+          loadResource(newProps, i.next(), false);
+        }
+      }
+      properties = newProps;
+    }
+    return properties;
+  }
+
+  private void loadResource(Properties properties,
+                            Object name, boolean quietFail) {
+    try {
+      DocumentBuilder builder =
+        DocumentBuilderFactory.newInstance().newDocumentBuilder();
+      Document doc = null;
+
+      if (name instanceof String) {               // a CLASSPATH resource
+        URL url = getResource((String)name);
+        if (url != null) {
+          LOG.info("parsing " + url);
+          doc = builder.parse(url.toString());
+        }
+      } else if (name instanceof File) {          // a file resource
+        File file = (File)name;
+        if (file.exists()) {
+          LOG.info("parsing " + file);
+          doc = builder.parse(file);
+        }
+      }
+
+      if (doc == null) {
+        if (quietFail)
+          return;
+        throw new RuntimeException(name + " not found");
+      }
+
+      Element root = doc.getDocumentElement();
+      if (!"nutch-conf".equals(root.getTagName()))
+        LOG.severe("bad conf file: top-level element not <nutch-conf>");
+      NodeList props = root.getChildNodes();
+      for (int i = 0; i < props.getLength(); i++) {
+        Node propNode = props.item(i);
+        if (!(propNode instanceof Element))
+          continue;
+        Element prop = (Element)propNode;
+        if (!"property".equals(prop.getTagName()))
+          LOG.warning("bad conf file: element not <property>");
+        NodeList fields = prop.getChildNodes();
+        String attr = null;
+        String value = null;
+        for (int j = 0; j < fields.getLength(); j++) {
+          Node fieldNode = fields.item(j);
+          if (!(fieldNode instanceof Element))
+            continue;
+          Element field = (Element)fieldNode;
+          if ("name".equals(field.getTagName()))
+            attr = ((Text)field.getFirstChild()).getData();
+          if ("value".equals(field.getTagName()) && field.hasChildNodes())
+            value = ((Text)field.getFirstChild()).getData();
+        }
+        if (attr != null && value != null)
+          properties.setProperty(attr, value);
+      }
+        
+    } catch (Exception e) {
+      LOG.severe("error parsing conf file: " + e);
+      throw new RuntimeException(e);
+    }
+    
+  }
+
+  /** Writes non-default properties in this configuration.*/
+  public void write(OutputStream out) throws IOException {
+    Properties properties = getProps();
+    try {
+      Document doc =
+        DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
+      Element conf = doc.createElement("nutch-conf");
+      doc.appendChild(conf);
+      conf.appendChild(doc.createTextNode("\n"));
+      for (Enumeration e = properties.keys(); e.hasMoreElements();) {
+        String name = (String)e.nextElement();
+        Object object = properties.get(name);
+        String value = null;
+        if(object instanceof String) {
+          value = (String) object;
+        }else {
+          continue;
+        }
+        Element propNode = doc.createElement("property");
+        conf.appendChild(propNode);
+      
+        Element nameNode = doc.createElement("name");
+        nameNode.appendChild(doc.createTextNode(name));
+        propNode.appendChild(nameNode);
+      
+        Element valueNode = doc.createElement("value");
+        valueNode.appendChild(doc.createTextNode(value));
+        propNode.appendChild(valueNode);
+
+        conf.appendChild(doc.createTextNode("\n"));
+      }
+    
+      DOMSource source = new DOMSource(doc);
+      StreamResult result = new StreamResult(out);
+      TransformerFactory transFactory = TransformerFactory.newInstance();
+      Transformer transformer = transFactory.newTransformer();
+      transformer.transform(source, result);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+
+  public String toString() {
+    StringBuffer sb = new StringBuffer(resourceNames.size()*30);
+    sb.append("Configuration: ");
+    ListIterator i = resourceNames.listIterator();
+    while (i.hasNext()) {
+      if (i.nextIndex() != 0) {
+        sb.append(" , ");
+      }
+      Object obj = i.next();
+      if (obj instanceof File) {
+        sb.append((File)obj);
+      } else {
+        sb.append((String)obj);
+      }
+    }
+    return sb.toString();
+  }
+
+  /** For debugging.  List non-default properties to the terminal and exit. */
+  public static void main(String[] args) throws Exception {
+    new Configuration().write(System.out);
+  }
+
+}

+ 39 - 0
src/java/org/apache/hadoop/conf/Configured.java

@@ -0,0 +1,39 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.conf;
+
+/** Base class for things that may be configured with a {@link Configuration}. */
+public class Configured implements Configurable {
+
+  private Configuration conf;
+
+  /** Construct a Configured. */
+  public Configured(Configuration conf) {
+    setConf(conf);
+  }
+
+  // inherit javadoc
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  // inherit javadoc
+  public Configuration getConf() {
+    return conf;
+  }
+
+}

+ 126 - 0
src/java/org/apache/hadoop/dfs/Block.java

@@ -0,0 +1,126 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import org.apache.hadoop.io.*;
+
+import java.io.*;
+import java.util.*;
+
+/**************************************************
+ * A Block is a Nutch FS primitive, identified by a 
+ * long.
+ *
+ * @author Mike Cafarella
+ **************************************************/
+public class Block implements Writable, Comparable {
+    static Random r = new Random();
+
+    /**
+     */
+    public static boolean isBlockFilename(File f) {
+        if (f.getName().startsWith("blk_")) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    long blkid;
+    long len;
+
+    /**
+     */
+    public Block() {
+        this.blkid = r.nextLong();
+        this.len = 0;
+    }
+
+    /**
+     */
+    public Block(long blkid, long len) {
+        this.blkid = blkid;
+        this.len = len;
+    }
+
+    /**
+     * Find the blockid from the given filename
+     */
+    public Block(File f, long len) {
+        String name = f.getName();
+        name = name.substring("blk_".length());
+        this.blkid = Long.parseLong(name);
+        this.len = len;
+    }
+
+    /**
+     */
+    public long getBlockId() {
+        return blkid;
+    }
+
+    /**
+     */
+    public String getBlockName() {
+        return "blk_" + String.valueOf(blkid);
+    }
+
+    /**
+     */
+    public long getNumBytes() {
+        return len;
+    }
+    public void setNumBytes(long len) {
+        this.len = len;
+    }
+
+    /**
+     */
+    public String toString() {
+        return getBlockName();
+    }
+
+    /////////////////////////////////////
+    // Writable
+    /////////////////////////////////////
+    public void write(DataOutput out) throws IOException {
+        out.writeLong(blkid);
+        out.writeLong(len);
+    }
+
+    public void readFields(DataInput in) throws IOException {
+        this.blkid = in.readLong();
+        this.len = in.readLong();
+    }
+
+    /////////////////////////////////////
+    // Comparable
+    /////////////////////////////////////
+    public int compareTo(Object o) {
+        Block b = (Block) o;
+        if (getBlockId() < b.getBlockId()) {
+            return -1;
+        } else if (getBlockId() == b.getBlockId()) {
+            return 0;
+        } else {
+            return 1;
+        }
+    }
+    public boolean equals(Object o) {
+        Block b = (Block) o;
+        return (this.compareTo(b) == 0);
+    }
+}

+ 108 - 0
src/java/org/apache/hadoop/dfs/BlockCommand.java

@@ -0,0 +1,108 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import org.apache.hadoop.io.*;
+
+import java.io.*;
+
+/****************************************************
+ * A BlockCommand is an instruction to a datanode regarding
+ * some blocks under its control
+ * 
+ * @author Michael Cafarella
+ ****************************************************/
+public class BlockCommand implements Writable {
+    boolean transferBlocks = false;
+    boolean invalidateBlocks = false;
+    Block blocks[];
+    DatanodeInfo targets[][];
+
+    public BlockCommand() {
+        this.transferBlocks = false;
+        this.invalidateBlocks = false;        
+        this.blocks = new Block[0];
+        this.targets = new DatanodeInfo[0][];
+    }
+
+    public BlockCommand(Block blocks[], DatanodeInfo targets[][]) {
+        this.transferBlocks = true;
+        this.invalidateBlocks = false;
+        this.blocks = blocks;
+        this.targets = targets;
+    }
+
+    public BlockCommand(Block blocks[]) {
+        this.transferBlocks = false;
+        this.invalidateBlocks = true;
+        this.blocks = blocks;
+        this.targets = new DatanodeInfo[0][];
+    }
+
+    public boolean transferBlocks() {
+        return transferBlocks;
+    }
+
+    public boolean invalidateBlocks() {
+        return invalidateBlocks;
+    }
+    
+    public Block[] getBlocks() {
+        return blocks;
+    }
+
+    public DatanodeInfo[][] getTargets() {
+        return targets;
+    }
+
+    ///////////////////////////////////////////
+    // Writable
+    ///////////////////////////////////////////
+    public void write(DataOutput out) throws IOException {
+        out.writeBoolean(transferBlocks);
+        out.writeBoolean(invalidateBlocks);        
+        out.writeInt(blocks.length);
+        for (int i = 0; i < blocks.length; i++) {
+            blocks[i].write(out);
+        }
+        out.writeInt(targets.length);
+        for (int i = 0; i < targets.length; i++) {
+            out.writeInt(targets[i].length);
+            for (int j = 0; j < targets[i].length; j++) {
+                targets[i][j].write(out);
+            }
+        }
+    }
+
+    public void readFields(DataInput in) throws IOException {
+        this.transferBlocks = in.readBoolean();
+        this.invalidateBlocks = in.readBoolean();
+        this.blocks = new Block[in.readInt()];
+        for (int i = 0; i < blocks.length; i++) {
+            blocks[i] = new Block();
+            blocks[i].readFields(in);
+        }
+
+        this.targets = new DatanodeInfo[in.readInt()][];
+        for (int i = 0; i < targets.length; i++) {
+            this.targets[i] = new DatanodeInfo[in.readInt()];
+            for (int j = 0; j < targets[i].length; j++) {
+                targets[i][j] = new DatanodeInfo();
+                targets[i][j].readFields(in);
+            }
+        }
+    }
+}

+ 140 - 0
src/java/org/apache/hadoop/dfs/ClientProtocol.java

@@ -0,0 +1,140 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import java.io.*;
+import org.apache.hadoop.io.*;
+
+/**********************************************************************
+ * Protocol that an DFS client uses to communicate with the NameNode.
+ * It's used to manipulate the namespace, and obtain datanode info.
+ *
+ * @author Mike Cafarella
+ **********************************************************************/
+public interface ClientProtocol {
+
+    /**
+     * Open an existing file.  Get back block and datanode info
+     */
+    public LocatedBlock[] open(String src) throws IOException;
+
+    /**
+     * Create a new file.  Get back block and datanode info
+     */
+    public LocatedBlock create(String src, String clientName, boolean overwrite) throws IOException;
+
+    /**
+     * The client wants to write an additional block to the indicated
+     * filename (which must currently be open for writing).  Return
+     * block and datanode info.  A null response means the caller
+     * should attempt the call again.
+     */
+    public LocatedBlock addBlock(String src) throws IOException;
+
+    /**
+     * The client wants to report a block it has just successfully
+     * written to one or more datanodes.  Client-written blocks are
+     * always reported by the client, not by the datanode.
+     */
+    public void reportWrittenBlock(LocatedBlock b) throws IOException;
+
+    /**
+     * The client wants to abandon writing to the indicated block,
+     * part of the indicated (currently-open) filename.
+     */
+    public void abandonBlock(Block b, String src) throws IOException;
+
+    /**
+     * The client wants to abandon writing to the current file, and
+     * let anyone else grab it.
+     */
+    public void abandonFileInProgress(String src) throws IOException;
+
+    /**
+     * The client is done writing data to the given filename, and would 
+     * like to complete it.  Returns whether the file has been closed
+     * correctly (true) or whether caller should try again (false).
+     * (Because the namenode is waiting for a block to complete).
+     */
+    public boolean complete(String src, String clientName) throws IOException;
+    
+    /**
+     * The client wants to read the indicated filename at a certain offset.
+     * Return a list of hostnames where the data can be found.  (Return
+     * a set of hostnames for every block.)
+     */
+    public String[][] getHints(String src, long start, long len) throws IOException;
+
+    /**
+     * Rename an item in the fs namespace
+     */
+    public boolean rename(String src, String dst) throws IOException;
+
+    /**
+     * Remove the given filename from the filesystem
+     */
+    public boolean delete(String src) throws IOException;
+
+    /**
+     * Check whether the given file exists
+     */
+    public boolean exists(String src) throws IOException;
+
+    /**
+     * Check whether the given filename is a directory or not.
+     */
+    public boolean isDir(String src) throws IOException;
+
+    /**
+     * Create a directory (or hierarchy of directories) with the given
+     * name.
+     */
+    public boolean mkdirs(String src) throws IOException;
+
+    /**
+     * The client is trying to obtain a lock.  Return whether the lock has
+     * been seized correctly (true), or whether the client should try again
+     * (false).
+     */
+    public boolean obtainLock(String src, String clientName, boolean exclusive) throws IOException;
+
+    /**
+     * The client wants to release a held lock.  Return whether the lock was
+     * correctly released (true), or whether the client should wait and try the 
+     * call again (false).
+     */
+    public boolean releaseLock(String src, String clientName) throws IOException;
+
+    /**
+     * The client machine wants to obtain a lease
+     */
+    public void renewLease(String clientName) throws IOException;
+
+    /**
+     * Get a listing of the indicated directory
+     */
+    public DFSFileInfo[] getListing(String src) throws IOException;
+
+    /**
+     * Get a set of statistics about the filesystem.
+     */
+    public long[] getStats() throws IOException;
+
+    /**
+     * Get a full report on the system's current datanodes.
+     */
+    public DatanodeInfo[] getDatanodeReport() throws IOException;
+}

+ 95 - 0
src/java/org/apache/hadoop/dfs/DF.java

@@ -0,0 +1,95 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.BufferedReader;
+
+import java.util.StringTokenizer;
+import java.util.Iterator;
+
+/** Filesystem disk space usage statistics.  Uses the unix 'df' program.
+ * Tested on Linux, FreeBSD and Cygwin. */
+public class DF {
+  private String filesystem;
+  private long capacity;
+  private long used;
+  private long available;
+  private int percentUsed;
+  private String mount;
+  
+  public DF(String path) throws IOException {
+
+    Process process = Runtime.getRuntime().exec(new String[] {"df","-k",path});
+
+    try {
+      if (process.waitFor() == 0) {
+        BufferedReader lines =
+          new BufferedReader(new InputStreamReader(process.getInputStream()));
+
+        lines.readLine();                         // skip headings
+
+        StringTokenizer tokens =
+          new StringTokenizer(lines.readLine(), " \t\n\r\f%");
+        
+        this.filesystem = tokens.nextToken();
+        if (!tokens.hasMoreTokens()) {            // for long filesystem name
+          tokens = new StringTokenizer(lines.readLine(), " \t\n\r\f%");
+        }
+        this.capacity = Long.parseLong(tokens.nextToken()) * 1024;
+        this.used = Long.parseLong(tokens.nextToken()) * 1024;
+        this.available = Long.parseLong(tokens.nextToken()) * 1024;
+        this.percentUsed = Integer.parseInt(tokens.nextToken());
+        this.mount = tokens.nextToken();
+
+      } else {
+        throw new IOException
+          (new BufferedReader(new InputStreamReader(process.getErrorStream()))
+           .readLine());
+      }
+    } catch (InterruptedException e) {
+      throw new IOException(e.toString());
+    } finally {
+      process.destroy();
+    }
+  }
+
+  /// ACCESSORS
+
+  public String getFilesystem() { return filesystem; }
+  public long getCapacity() { return capacity; }
+  public long getUsed() { return used; }
+  public long getAvailable() { return available; }
+  public int getPercentUsed() { return percentUsed; }
+  public String getMount() { return mount; }
+  
+  public String toString() {
+    return
+      "df -k " + mount +"\n" +
+      filesystem + "\t" +
+      capacity / 1024 + "\t" +
+      used / 1024 + "\t" +
+      available / 1024 + "\t" +
+      percentUsed + "%\t" +
+      mount;
+  }
+
+  public static void main(String[] args) throws Exception {
+    System.out.println(new DF(args[0]));
+  }
+}

+ 754 - 0
src/java/org/apache/hadoop/dfs/DataNode.java

@@ -0,0 +1,754 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+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 java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.logging.*;
+
+/**********************************************************
+ * DataNode controls just one critical table:
+ *   block-> BLOCK_SIZE stream of bytes
+ *
+ * This info is stored on disk (the NameNode is responsible for
+ * asking other machines to replicate the data).  The DataNode
+ * reports the table's contents to the NameNode upon startup
+ * and every so often afterwards.
+ *
+ * @author Mike Cafarella
+ **********************************************************/
+public class DataNode implements FSConstants, Runnable {
+    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.dfs.DataNode");
+  //
+    // REMIND - mjc - I might bring "maxgigs" back so user can place 
+    // artificial  limit on space
+    //private static final long GIGABYTE = 1024 * 1024 * 1024;
+    //private static long numGigs = Configuration.get().getLong("dfs.datanode.maxgigs", 100);
+    //
+
+    /**
+     * Util method to build socket addr from string
+     */
+    public static InetSocketAddress createSocketAddr(String s) throws IOException {
+        String target = s;
+        int colonIndex = target.indexOf(':');
+        if (colonIndex < 0) {
+            throw new RuntimeException("Not a host:port pair: " + s);
+        }
+        String host = target.substring(0, colonIndex);
+        int port = Integer.parseInt(target.substring(colonIndex + 1));
+
+        return new InetSocketAddress(host, port);
+    }
+
+
+    private static Vector subThreadList = null;
+    DatanodeProtocol namenode;
+    FSDataset data;
+    String localName;
+    boolean shouldRun = true;
+    Vector receivedBlockList = new Vector();
+    int xmitsInProgress = 0;
+    Daemon dataXceiveServer = null;
+    long blockReportInterval;
+    private long datanodeStartupPeriod;
+    private Configuration fConf;
+
+    /**
+     * Create given a configuration and a dataDir.
+     */
+    public DataNode(Configuration conf, String datadir) throws IOException {
+        this(InetAddress.getLocalHost().getHostName(), 
+             new File(datadir),
+             createSocketAddr(conf.get("fs.default.name", "local")), conf);
+    }
+
+    /**
+     * Needs a directory to find its data (and config info)
+     */
+    public DataNode(String machineName, File datadir, InetSocketAddress nameNodeAddr, Configuration conf) throws IOException {
+        this.namenode = (DatanodeProtocol) RPC.getProxy(DatanodeProtocol.class, nameNodeAddr, conf);
+        this.data = new FSDataset(datadir, conf);
+
+        ServerSocket ss = null;
+        int tmpPort = conf.getInt("dfs.datanode.port", 50010);
+        while (ss == null) {
+            try {
+                ss = new ServerSocket(tmpPort);
+                LOG.info("Opened server at " + tmpPort);
+            } catch (IOException ie) {
+                LOG.info("Could not open server at " + tmpPort + ", trying new port");
+                tmpPort++;
+            }
+        }
+        this.localName = machineName + ":" + tmpPort;
+        this.dataXceiveServer = new Daemon(new DataXceiveServer(ss));
+        this.dataXceiveServer.start();
+
+        long blockReportIntervalBasis =
+          conf.getLong("dfs.blockreport.intervalMsec", BLOCKREPORT_INTERVAL);
+        this.blockReportInterval =
+          blockReportIntervalBasis - new Random().nextInt((int)(blockReportIntervalBasis/10));
+        this.datanodeStartupPeriod =
+          conf.getLong("dfs.datanode.startupMsec", DATANODE_STARTUP_PERIOD);
+    }
+
+    /**
+     */
+    public String getNamenode() {
+        //return namenode.toString();
+	return "<namenode>";
+    }
+
+    /**
+     * Shut down this instance of the datanode.
+     * Returns only after shutdown is complete.
+     */
+    void shutdown() {
+        this.shouldRun = false;
+        ((DataXceiveServer) this.dataXceiveServer.getRunnable()).kill();
+        try {
+            this.dataXceiveServer.join();
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    /**
+     * Main loop for the DataNode.  Runs until shutdown.
+     */
+    public void offerService() throws Exception {
+        long wakeups = 0;
+        long lastHeartbeat = 0, lastBlockReport = 0;
+        long sendStart = System.currentTimeMillis();
+        int heartbeatsSent = 0;
+        LOG.info("using BLOCKREPORT_INTERVAL of " + blockReportInterval + "msec");
+
+        //
+        // Now loop for a long time....
+        //
+        while (shouldRun) {
+            long now = System.currentTimeMillis();
+
+            //
+            // Every so often, send heartbeat or block-report
+            //
+            synchronized (receivedBlockList) {
+                if (now - lastHeartbeat > HEARTBEAT_INTERVAL) {
+                    //
+                    // All heartbeat messages include following info:
+                    // -- Datanode name
+                    // -- data transfer port
+                    // -- Total capacity
+                    // -- Bytes remaining
+                    //
+                    namenode.sendHeartbeat(localName, data.getCapacity(), data.getRemaining());
+                    //LOG.info("Just sent heartbeat, with name " + localName);
+                    lastHeartbeat = now;
+		}
+		if (now - lastBlockReport > blockReportInterval) {
+                    //
+                    // Send latest blockinfo report if timer has expired.
+                    // Get back a list of local block(s) that are obsolete
+                    // and can be safely GC'ed.
+                    //
+                    Block toDelete[] = namenode.blockReport(localName, data.getBlockReport());
+                    data.invalidate(toDelete);
+                    lastBlockReport = now;
+                    continue;
+		}
+		if (receivedBlockList.size() > 0) {
+                    //
+                    // Send newly-received blockids to namenode
+                    //
+                    Block blockArray[] = (Block[]) receivedBlockList.toArray(new Block[receivedBlockList.size()]);
+                    receivedBlockList.removeAllElements();
+                    namenode.blockReceived(localName, blockArray);
+                }
+
+		//
+		// Only perform block operations (transfer, delete) after 
+		// a startup quiet period.  The assumption is that all the
+		// datanodes will be started together, but the namenode may
+		// have been started some time before.  (This is esp. true in
+		// the case of network interruptions.)  So, wait for some time
+		// to pass from the time of connection to the first block-transfer.
+		// Otherwise we transfer a lot of blocks unnecessarily.
+		//
+		if (now - sendStart > datanodeStartupPeriod) {
+		    //
+		    // Check to see if there are any block-instructions from the
+		    // namenode that this datanode should perform.
+		    //
+		    BlockCommand cmd = namenode.getBlockwork(localName, xmitsInProgress);
+		    if (cmd != null && cmd.transferBlocks()) {
+			//
+			// Send a copy of a block to another datanode
+			//
+			Block blocks[] = cmd.getBlocks();
+			DatanodeInfo xferTargets[][] = cmd.getTargets();
+			
+			for (int i = 0; i < blocks.length; i++) {
+			    if (!data.isValidBlock(blocks[i])) {
+				String errStr = "Can't send invalid block " + blocks[i];
+				LOG.info(errStr);
+				namenode.errorReport(localName, errStr);
+				break;
+			    } else {
+				if (xferTargets[i].length > 0) {
+				    LOG.info("Starting thread to transfer block " + blocks[i] + " to " + xferTargets[i]);
+				    new Daemon(new DataTransfer(xferTargets[i], blocks[i])).start();
+				}
+			    }
+			}
+                    } else if (cmd != null && cmd.invalidateBlocks()) {
+                        //
+                        // Some local block(s) are obsolete and can be 
+                        // safely garbage-collected.
+                        //
+                        data.invalidate(cmd.getBlocks());
+                    }
+                }
+
+                //
+                // There is no work to do;  sleep until hearbeat timer elapses, 
+                // or work arrives, and then iterate again.
+                //
+                long waitTime = HEARTBEAT_INTERVAL - (now - lastHeartbeat);
+                if (waitTime > 0 && receivedBlockList.size() == 0) {
+                    try {
+                        receivedBlockList.wait(waitTime);
+                    } catch (InterruptedException ie) {
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * Server used for receiving/sending a block of data
+     */
+    class DataXceiveServer implements Runnable {
+        boolean shouldListen = true;
+        ServerSocket ss;
+        public DataXceiveServer(ServerSocket ss) {
+            this.ss = ss;
+        }
+
+        /**
+         */
+        public void run() {
+            try {
+                while (shouldListen) {
+                    Socket s = ss.accept();
+                    s.setSoTimeout(READ_TIMEOUT);
+                    new Daemon(new DataXceiver(s)).start();
+                }
+                ss.close();
+            } catch (IOException ie) {
+                LOG.info("Exiting DataXceiveServer due to " + ie.toString());
+            }
+        }
+        public void kill() {
+            this.shouldListen = false;
+            try {
+                this.ss.close();
+            } catch (IOException iex) {
+            }
+        }
+    }
+
+    /**
+     * Thread for processing incoming/outgoing data stream
+     */
+    class DataXceiver implements Runnable {
+        Socket s;
+        public DataXceiver(Socket s) {
+            this.s = s;
+        }
+
+        /**
+         */
+        public void run() {
+            try {
+                DataInputStream in = new DataInputStream(new BufferedInputStream(s.getInputStream()));
+                try {
+                    byte op = (byte) in.read();
+                    if (op == OP_WRITE_BLOCK) {
+                        //
+                        // Read in the header
+                        //
+                        DataOutputStream reply = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
+                        try {
+                            boolean shouldReportBlock = in.readBoolean();
+                            Block b = new Block();
+                            b.readFields(in);
+                            int numTargets = in.readInt();
+                            if (numTargets <= 0) {
+                                throw new IOException("Mislabelled incoming datastream.");
+                            }
+                            DatanodeInfo targets[] = new DatanodeInfo[numTargets];
+                            for (int i = 0; i < targets.length; i++) {
+                                DatanodeInfo tmp = new DatanodeInfo();
+                                tmp.readFields(in);
+                                targets[i] = tmp;
+                            }
+                            byte encodingType = (byte) in.read();
+                            long len = in.readLong();
+
+                            //
+                            // Make sure curTarget is equal to this machine
+                            //
+                            DatanodeInfo curTarget = targets[0];
+
+                            //
+                            // Track all the places we've successfully written the block
+                            //
+                            Vector mirrors = new Vector();
+
+                            //
+                            // Open local disk out
+                            //
+                            DataOutputStream out = new DataOutputStream(new BufferedOutputStream(data.writeToBlock(b)));
+                            InetSocketAddress mirrorTarget = null;
+                            try {
+                                //
+                                // Open network conn to backup machine, if 
+                                // appropriate
+                                //
+                                DataInputStream in2 = null;
+                                DataOutputStream out2 = null;
+                                if (targets.length > 1) {
+                                    // Connect to backup machine
+                                    mirrorTarget = createSocketAddr(targets[1].getName().toString());
+                                    try {
+                                        Socket s = new Socket(mirrorTarget.getAddress(), mirrorTarget.getPort());
+                                        s.setSoTimeout(READ_TIMEOUT);
+                                        out2 = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
+                                        in2 = new DataInputStream(new BufferedInputStream(s.getInputStream()));
+
+                                        // Write connection header
+                                        out2.write(OP_WRITE_BLOCK);
+                                        out2.writeBoolean(shouldReportBlock);
+                                        b.write(out2);
+                                        out2.writeInt(targets.length - 1);
+                                        for (int i = 1; i < targets.length; i++) {
+                                            targets[i].write(out2);
+                                        }
+                                        out2.write(encodingType);
+                                        out2.writeLong(len);
+                                    } catch (IOException ie) {
+                                        if (out2 != null) {
+                                            try {
+                                                out2.close();
+                                                in2.close();
+                                            } catch (IOException out2close) {
+                                            } finally {
+                                                out2 = null;
+                                                in2 = null;
+                                            }
+                                        }
+                                    }
+                                }
+
+                                //
+                                // Process incoming data, copy to disk and
+                                // maybe to network.
+                                //
+                                try {
+                                    boolean anotherChunk = true;
+                                    byte buf[] = new byte[BUFFER_SIZE];
+
+                                    while (anotherChunk) {
+                                        while (len > 0) {
+                                            int bytesRead = in.read(buf, 0, (int)Math.min(buf.length, len));
+                                            if (bytesRead < 0) {
+                                              throw new EOFException("EOF reading from "+s.toString());
+                                            }
+                                            if (bytesRead > 0) {
+                                                try {
+                                                    out.write(buf, 0, bytesRead);
+                                                } catch (IOException iex) {
+                                                    shutdown();
+                                                    throw iex;
+                                                }
+                                                if (out2 != null) {
+                                                    try {
+                                                        out2.write(buf, 0, bytesRead);
+                                                    } catch (IOException out2e) {
+                                                        //
+                                                        // If stream-copy fails, continue 
+                                                        // writing to disk.  We shouldn't 
+                                                        // interrupt client write.
+                                                        //
+                                                        try {
+                                                            out2.close();
+                                                            in2.close();
+                                                        } catch (IOException out2close) {
+                                                        } finally {
+                                                            out2 = null;
+                                                            in2 = null;
+                                                        }
+                                                    }
+                                                }
+                                                len -= bytesRead;
+                                            }
+                                        }
+
+                                        if (encodingType == RUNLENGTH_ENCODING) {
+                                            anotherChunk = false;
+                                        } else if (encodingType == CHUNKED_ENCODING) {
+                                            len = in.readLong();
+                                            if (out2 != null) {
+                                                out2.writeLong(len);
+                                            }
+                                            if (len == 0) {
+                                                anotherChunk = false;
+                                            }
+                                        }
+                                    }
+
+                                    if (out2 == null) {
+                                        LOG.info("Received block " + b + " from " + s.getInetAddress());
+                                    } else {
+                                        out2.flush();
+                                        long complete = in2.readLong();
+                                        if (complete != WRITE_COMPLETE) {
+                                            LOG.info("Conflicting value for WRITE_COMPLETE: " + complete);
+                                        }
+                                        LocatedBlock newLB = new LocatedBlock();
+                                        newLB.readFields(in2);
+                                        DatanodeInfo mirrorsSoFar[] = newLB.getLocations();
+                                        for (int k = 0; k < mirrorsSoFar.length; k++) {
+                                            mirrors.add(mirrorsSoFar[k]);
+                                        }
+                                        LOG.info("Received block " + b + " from " + s.getInetAddress() + " and mirrored to " + mirrorTarget);
+                                    }
+                                } finally {
+                                    if (out2 != null) {
+                                        out2.close();
+                                        in2.close();
+                                    }
+                                }
+                            } finally {
+                                try {
+                                    out.close();
+                                } catch (IOException iex) {
+                                    shutdown();
+                                    throw iex;
+                                }
+                            }
+                            data.finalizeBlock(b);
+
+                            // 
+                            // Tell the namenode that we've received this block 
+                            // in full, if we've been asked to.  This is done
+                            // during NameNode-directed block transfers, but not
+                            // client writes.
+                            //
+                            if (shouldReportBlock) {
+                                synchronized (receivedBlockList) {
+                                    receivedBlockList.add(b);
+                                    receivedBlockList.notifyAll();
+                                }
+                            }
+
+                            //
+                            // Tell client job is done, and reply with
+                            // the new LocatedBlock.
+                            //
+                            reply.writeLong(WRITE_COMPLETE);
+                            mirrors.add(curTarget);
+                            LocatedBlock newLB = new LocatedBlock(b, (DatanodeInfo[]) mirrors.toArray(new DatanodeInfo[mirrors.size()]));
+                            newLB.write(reply);
+                        } finally {
+                            reply.close();
+                        }
+                    } else if (op == OP_READ_BLOCK || op == OP_READSKIP_BLOCK) {
+                        //
+                        // Read in the header
+                        //
+                        Block b = new Block();
+                        b.readFields(in);
+
+                        long toSkip = 0;
+                        if (op == OP_READSKIP_BLOCK) {
+                            toSkip = in.readLong();
+                        }
+
+                        //
+                        // Open reply stream
+                        //
+                        DataOutputStream out = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
+                        try {
+                            //
+                            // Write filelen of -1 if error
+                            //
+                            if (! data.isValidBlock(b)) {
+                                out.writeLong(-1);
+                            } else {
+                                //
+                                // Get blockdata from disk
+                                //
+                                long len = data.getLength(b);
+                                DataInputStream in2 = new DataInputStream(data.getBlockData(b));
+                                out.writeLong(len);
+
+                                if (op == OP_READSKIP_BLOCK) {
+                                    if (toSkip > len) {
+                                        toSkip = len;
+                                    }
+                                    long amtSkipped = 0;
+                                    try {
+                                        amtSkipped = in2.skip(toSkip);
+                                    } catch (IOException iex) {
+                                        shutdown();
+                                        throw iex;
+                                    }
+                                    out.writeLong(amtSkipped);
+                                }
+
+                                byte buf[] = new byte[BUFFER_SIZE];
+                                try {
+                                    int bytesRead = 0;
+                                    try {
+                                        bytesRead = in2.read(buf);
+                                    } catch (IOException iex) {
+                                        shutdown();
+                                        throw iex;
+                                    }
+                                    while (bytesRead >= 0) {
+                                        out.write(buf, 0, bytesRead);
+                                        len -= bytesRead;
+                                        try {
+                                            bytesRead = in2.read(buf);
+                                        } catch (IOException iex) {
+                                            shutdown();
+                                            throw iex;
+                                        }
+                                    }
+                                } catch (SocketException se) {
+                                    // This might be because the reader
+                                    // closed the stream early
+                                } finally {
+                                    try {
+                                        in2.close();
+                                    } catch (IOException iex) {
+                                        shutdown();
+                                        throw iex;
+                                    }
+                                }
+                            }
+                            LOG.info("Served block " + b + " to " + s.getInetAddress());
+                        } finally {
+                            out.close();
+                        }
+                    } else {
+                        while (op >= 0) {
+                            System.out.println("Faulty op: " + op);
+                            op = (byte) in.read();
+                        }
+                        throw new IOException("Unknown opcode for incoming data stream");
+                    }
+                } finally {
+                    in.close();
+                }
+            } catch (IOException ie) {
+              LOG.log(Level.WARNING, "DataXCeiver", ie);
+            } finally {
+                try {
+                    s.close();
+                } catch (IOException ie2) {
+                }
+            }
+        }
+    }
+
+    /**
+     * Used for transferring a block of data
+     */
+    class DataTransfer implements Runnable {
+        InetSocketAddress curTarget;
+        DatanodeInfo targets[];
+        Block b;
+        byte buf[];
+
+        /**
+         * Connect to the first item in the target list.  Pass along the 
+         * entire target list, the block, and the data.
+         */
+        public DataTransfer(DatanodeInfo targets[], Block b) throws IOException {
+            this.curTarget = createSocketAddr(targets[0].getName().toString());
+            this.targets = targets;
+            this.b = b;
+            this.buf = new byte[BUFFER_SIZE];
+        }
+
+        /**
+         * Do the deed, write the bytes
+         */
+        public void run() {
+	    xmitsInProgress++;
+            try {
+                Socket s = new Socket(curTarget.getAddress(), curTarget.getPort());
+                DataOutputStream out = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
+                try {
+                    long filelen = data.getLength(b);
+                    DataInputStream in = new DataInputStream(new BufferedInputStream(data.getBlockData(b)));
+                    try {
+                        //
+                        // Header info
+                        //
+                        out.write(OP_WRITE_BLOCK);
+                        out.writeBoolean(true);
+                        b.write(out);
+                        out.writeInt(targets.length);
+                        for (int i = 0; i < targets.length; i++) {
+                            targets[i].write(out);
+                        }
+                        out.write(RUNLENGTH_ENCODING);
+                        out.writeLong(filelen);
+
+                        //
+                        // Write the data
+                        //
+                        while (filelen > 0) {
+                            int bytesRead = in.read(buf, 0, (int) Math.min(filelen, buf.length));
+                            out.write(buf, 0, bytesRead);
+                            filelen -= bytesRead;
+                        }
+                    } finally {
+                        in.close();
+                    }
+                } finally {
+                    out.close();
+                }
+                LOG.info("Transmitted block " + b + " to " + curTarget);
+            } catch (IOException ie) {
+            } finally {
+		xmitsInProgress--;
+	    }
+        }
+    }
+
+    /**
+     * No matter what kind of exception we get, keep retrying to offerService().
+     * That's the loop that connects to the NameNode and provides basic DataNode
+     * functionality.
+     *
+     * Only stop when "shouldRun" is turned off (which can only happen at shutdown).
+     */
+    public void run() {
+        LOG.info("Starting DataNode in: "+data.data);
+        while (shouldRun) {
+            try {
+                offerService();
+            } catch (Exception ex) {
+                LOG.info("Exception: " + ex);
+              if (shouldRun) {
+                LOG.info("Lost connection to namenode.  Retrying...");
+                try {
+                  Thread.sleep(5000);
+                } catch (InterruptedException ie) {
+                }
+              }
+            }
+        }
+      LOG.info("Finishing DataNode in: "+data.data);
+    }
+
+    /** Start datanode daemons.
+     * Start a datanode daemon for each comma separated data directory
+     * specified in property dfs.data.dir
+     */
+    public static void run(Configuration conf) throws IOException {
+        String[] dataDirs = conf.getStrings("dfs.data.dir");
+        subThreadList = new Vector(dataDirs.length);
+        for (int i = 0; i < dataDirs.length; i++) {
+          DataNode dn = makeInstanceForDir(dataDirs[i], conf);
+          if (dn != null) {
+            Thread t = new Thread(dn, "DataNode: "+dataDirs[i]);
+            t.setDaemon(true); // needed for JUnit testing
+            t.start();
+            subThreadList.add(t);
+          }
+        }
+    }
+
+  /** Start datanode daemons.
+   * Start a datanode daemon for each comma separated data directory
+   * specified in property dfs.data.dir and wait for them to finish.
+   * If this thread is specifically interrupted, it will stop waiting.
+   */
+  private static void runAndWait(Configuration conf) throws IOException {
+    run(conf);
+
+    //  Wait for sub threads to exit
+    for (Iterator iterator = subThreadList.iterator(); iterator.hasNext();) {
+      Thread threadDataNode = (Thread) iterator.next();
+      try {
+        threadDataNode.join();
+      } catch (InterruptedException e) {
+        if (Thread.currentThread().isInterrupted()) {
+          // did someone knock?
+          return;
+        }
+      }
+    }
+  }
+
+  /**
+   * Make an instance of DataNode after ensuring that given data directory
+   * (and parent directories, if necessary) can be created.
+   * @param dataDir where the new DataNode instance should keep its files.
+   * @param conf Configuration instance to use.
+   * @return DataNode instance for given data dir and conf, or null if directory
+   * cannot be created.
+   * @throws IOException
+   */
+  static DataNode makeInstanceForDir(String dataDir, Configuration conf) throws IOException {
+    DataNode dn = null;
+    File data = new File(dataDir);
+    data.mkdirs();
+    if (!data.isDirectory()) {
+      LOG.warning("Can't start DataNode in non-directory: "+dataDir);
+      return null;
+    } else {
+      dn = new DataNode(conf, dataDir);
+    }
+    return dn;
+  }
+
+  public String toString() {
+    return "DataNode{" +
+        "data=" + data +
+        ", localName='" + localName + "'" +
+        ", xmitsInProgress=" + xmitsInProgress +
+        "}";
+  }
+
+  /**
+     */
+    public static void main(String args[]) throws IOException {
+        LogFormatter.setShowThreadIDs(true);
+        runAndWait(new Configuration());
+    }
+}

+ 167 - 0
src/java/org/apache/hadoop/dfs/DatanodeInfo.java

@@ -0,0 +1,167 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+
+import java.io.*;
+import java.util.*;
+
+/**************************************************
+ * DatanodeInfo tracks stats on a given node
+ *
+ * @author Mike Cafarella
+ **************************************************/
+public class DatanodeInfo implements Writable, Comparable {
+    private UTF8 name;
+    private long capacityBytes, remainingBytes, lastUpdate;
+    private volatile TreeSet blocks;
+
+    /** Create an empty DatanodeInfo.
+     */
+    public DatanodeInfo() {
+        this(new UTF8(), 0, 0);
+    }
+
+   /**
+    * @param name hostname:portNumber as UTF8 object.
+    */
+    public DatanodeInfo(UTF8 name) {
+        this.name = name;
+        this.blocks = new TreeSet();
+        updateHeartbeat(0, 0);        
+    }
+
+   /**
+    * @param name hostname:portNumber as UTF8 object.
+    */
+    public DatanodeInfo(UTF8 name, long capacity, long remaining) {
+        this.name = name;
+        this.blocks = new TreeSet();
+        updateHeartbeat(capacity, remaining);
+    }
+
+   /**
+    */
+    public void updateBlocks(Block newBlocks[]) {
+        blocks.clear();
+        for (int i = 0; i < newBlocks.length; i++) {
+            blocks.add(newBlocks[i]);
+        }
+    }
+
+   /**
+    */
+    public void addBlock(Block b) {
+        blocks.add(b);
+    }
+
+    /**
+     */
+    public void updateHeartbeat(long capacity, long remaining) {
+        this.capacityBytes = capacity;
+        this.remainingBytes = remaining;
+        this.lastUpdate = System.currentTimeMillis();
+    }
+
+    /**
+     * @return hostname:portNumber as UTF8 object.
+     */
+    public UTF8 getName() {
+        return name;
+    }
+
+    /**
+     * @return hostname and no :portNumber as UTF8 object.
+     */
+    public UTF8 getHost() {
+        String nameStr = name.toString();
+        int colon = nameStr.indexOf(":");
+        if (colon < 0) {
+            return name;
+        } else {
+            return new UTF8(nameStr.substring(0, colon));
+        }
+    }
+    public String toString() {
+        return name.toString();
+    }
+    public Block[] getBlocks() {
+        return (Block[]) blocks.toArray(new Block[blocks.size()]);
+    }
+    public Iterator getBlockIterator() {
+        return blocks.iterator();
+    }
+    public long getCapacity() {
+        return capacityBytes;
+    }
+    public long getRemaining() {
+        return remainingBytes;
+    }
+    public long lastUpdate() {
+        return lastUpdate;
+    }
+
+  /** Comparable.
+   * Basis of compare is the UTF8 name (host:portNumber) only.
+   * @param o
+   * @return as specified by Comparable.
+   */
+    public int compareTo(Object o) {
+        DatanodeInfo d = (DatanodeInfo) o;
+        return name.compareTo(d.getName());
+    }
+
+    /////////////////////////////////////////////////
+    // Writable
+    /////////////////////////////////////////////////
+    /**
+     */
+    public void write(DataOutput out) throws IOException {
+        name.write(out);
+        out.writeLong(capacityBytes);
+        out.writeLong(remainingBytes);
+        out.writeLong(lastUpdate);
+
+        /**
+        out.writeInt(blocks.length);
+        for (int i = 0; i < blocks.length; i++) {
+            blocks[i].write(out);
+        }
+        **/
+    }
+
+    /**
+     */
+    public void readFields(DataInput in) throws IOException {
+        this.name = new UTF8();
+        this.name.readFields(in);
+        this.capacityBytes = in.readLong();
+        this.remainingBytes = in.readLong();
+        this.lastUpdate = in.readLong();
+
+        /**
+        int numBlocks = in.readInt();
+        this.blocks = new Block[numBlocks];
+        for (int i = 0; i < blocks.length; i++) {
+            blocks[i] = new Block();
+            blocks[i].readFields(in);
+        }
+        **/
+    }
+}
+

+ 36 - 0
src/java/org/apache/hadoop/dfs/DatanodeProtocol.java

@@ -0,0 +1,36 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.dfs.
+
+import java.io.*;
+import org.apache.hadoop.io.*;
+
+/**********************************************************************
+ * Protocol that an DFS datanode uses to communicate with the NameNode.
+ * It's used to upload current load information and block records.
+ *
+ * @author Michael Cafarella
+ **********************************************************************/
+public interface DatanodeProtocol {
+
+    public void sendHeartbeat(String sender, long capacity, long remaining) throws IOException;
+    public Block[] blockReport(String sender, Block blocks[]) throws IOException;
+    public void blockReceived(String sender, Block blocks[]) throws IOException;
+    public void errorReport(String sender, String msg) throws IOException;
+
+    public BlockCommand getBlockwork(String sender, int xmitsInProgress) throws IOException;
+}

+ 114 - 0
src/java/org/apache/hadoop/dfs/FSConstants.java

@@ -0,0 +1,114 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import org.apache.hadoop.conf.Configuration;
+
+/************************************
+ * Some handy constants
+ *
+ * @author Mike Cafarella
+ ************************************/
+public interface FSConstants {
+    public static int BLOCK_SIZE = 32 * 1000 * 1000;
+    //public static int BLOCK_SIZE = 19;
+
+    public static final long WRITE_COMPLETE = 0xcafae11a;
+
+    //
+    // IPC Opcodes 
+    //
+    // Processed at namenode
+    public static final byte OP_ERROR = (byte) 0;
+    public static final byte OP_HEARTBEAT = (byte) 1;
+    public static final byte OP_BLOCKRECEIVED = (byte) 2;
+    public static final byte OP_BLOCKREPORT = (byte) 3;
+    public static final byte OP_TRANSFERDATA = (byte) 4;
+
+    // Processed at namenode, from client
+    public static final byte OP_CLIENT_OPEN = (byte) 20;
+    public static final byte OP_CLIENT_STARTFILE = (byte) 21;
+    public static final byte OP_CLIENT_ADDBLOCK = (byte) 22;
+    public static final byte OP_CLIENT_RENAMETO = (byte) 23;
+    public static final byte OP_CLIENT_DELETE = (byte) 24;  
+    public static final byte OP_CLIENT_COMPLETEFILE = (byte) 25;
+    public static final byte OP_CLIENT_LISTING = (byte) 26;
+    public static final byte OP_CLIENT_OBTAINLOCK = (byte) 27;
+    public static final byte OP_CLIENT_RELEASELOCK = (byte) 28;
+    public static final byte OP_CLIENT_EXISTS = (byte) 29;
+    public static final byte OP_CLIENT_ISDIR = (byte) 30;
+    public static final byte OP_CLIENT_MKDIRS = (byte) 31;
+    public static final byte OP_CLIENT_RENEW_LEASE = (byte) 32;
+    public static final byte OP_CLIENT_ABANDONBLOCK = (byte) 33;
+    public static final byte OP_CLIENT_RAWSTATS = (byte) 34;
+    public static final byte OP_CLIENT_DATANODEREPORT = (byte) 35;
+    public static final byte OP_CLIENT_DATANODE_HINTS = (byte) 36;
+    
+    // Processed at datanode, back from namenode
+    public static final byte OP_ACK = (byte) 40;
+    public static final byte OP_TRANSFERBLOCKS = (byte) 41;    
+    public static final byte OP_INVALIDATE_BLOCKS = (byte) 42;
+    public static final byte OP_FAILURE = (byte) 43;
+
+    // Processed at client, back from namenode
+    public static final byte OP_CLIENT_OPEN_ACK = (byte) 60;
+    public static final byte OP_CLIENT_STARTFILE_ACK = (byte) 61;
+    public static final byte OP_CLIENT_ADDBLOCK_ACK = (byte) 62;
+    public static final byte OP_CLIENT_RENAMETO_ACK = (byte) 63;
+    public static final byte OP_CLIENT_DELETE_ACK = (byte) 64;
+    public static final byte OP_CLIENT_COMPLETEFILE_ACK = (byte) 65;
+    public static final byte OP_CLIENT_TRYAGAIN = (byte) 66;
+    public static final byte OP_CLIENT_LISTING_ACK = (byte) 67;
+    public static final byte OP_CLIENT_OBTAINLOCK_ACK = (byte) 68;
+    public static final byte OP_CLIENT_RELEASELOCK_ACK = (byte) 69;
+    public static final byte OP_CLIENT_EXISTS_ACK = (byte) 70;  
+    public static final byte OP_CLIENT_ISDIR_ACK = (byte) 71;
+    public static final byte OP_CLIENT_MKDIRS_ACK = (byte) 72;
+    public static final byte OP_CLIENT_RENEW_LEASE_ACK = (byte) 73;    
+    public static final byte OP_CLIENT_ABANDONBLOCK_ACK = (byte) 74;
+    public static final byte OP_CLIENT_RAWSTATS_ACK = (byte) 75;
+    public static final byte OP_CLIENT_DATANODEREPORT_ACK = (byte) 76;
+    public static final byte OP_CLIENT_DATANODE_HINTS_ACK = (byte) 77;
+
+    // Processed at datanode stream-handler
+    public static final byte OP_WRITE_BLOCK = (byte) 80;
+    public static final byte OP_READ_BLOCK = (byte) 81;
+    public static final byte OP_READSKIP_BLOCK = (byte) 82;
+
+    // Encoding types
+    public static final byte RUNLENGTH_ENCODING = 0;
+    public static final byte CHUNKED_ENCODING = 1;
+
+    // Return codes for file create
+    public static final int OPERATION_FAILED = 0;
+    public static final int STILL_WAITING = 1;
+    public static final int COMPLETE_SUCCESS = 2;
+
+    //
+    // Timeouts, constants
+    //
+    public static long HEARTBEAT_INTERVAL = 3 * 1000;
+    public static long EXPIRE_INTERVAL = 10 * 60 * 1000;
+    public static long BLOCKREPORT_INTERVAL = 60 * 60 * 1000;
+    public static long DATANODE_STARTUP_PERIOD = 2 * 60 * 1000;
+    public static long LEASE_PERIOD = 60 * 1000;
+    public static int READ_TIMEOUT = 60 * 1000;
+
+    //TODO mb@media-style.com: should be conf injected?
+    public static final int BUFFER_SIZE = new Configuration().getInt("io.file.buffer.size", 4096);
+
+}
+

+ 429 - 0
src/java/org/apache/hadoop/dfs/FSDataset.java

@@ -0,0 +1,429 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import java.io.*;
+import java.util.*;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.*;
+
+/**************************************************
+ * FSDataset manages a set of data blocks.  Each block
+ * has a unique name and an extent on disk.
+ *
+ * @author Mike Cafarella
+ ***************************************************/
+public class FSDataset implements FSConstants {
+    static final double USABLE_DISK_PCT = 0.98;
+
+  /**
+     * A node type that can be built into a tree reflecting the
+     * hierarchy of blocks on the local disk.
+     */
+    class FSDir {
+        File dir;
+        FSDir children[];
+
+        /**
+         */
+        public FSDir(File dir) {
+            this.dir = dir;
+            this.children = null;
+        }
+
+        /**
+         */
+        public File getDirName() {
+            return dir;
+        }
+
+        /**
+         */
+        public FSDir[] getChildren() {
+            return children;
+        }
+
+        /**
+         */
+        public void addBlock(Block b, File src) {
+            addBlock(b, src, b.getBlockId(), 0);
+        }
+
+        /**
+         */
+        void addBlock(Block b, File src, long blkid, int depth) {
+            //
+            // Add to the local dir, if no child dirs
+            //
+            if (children == null) {
+                src.renameTo(new File(dir, b.getBlockName()));
+
+                //
+                // Test whether this dir's contents should be busted 
+                // up into subdirs.
+                //
+
+                // REMIND - mjc - sometime soon, we'll want this code
+                // working.  It prevents the datablocks from all going
+                // into a single huge directory.
+                /**
+                File localFiles[] = dir.listFiles();
+                if (localFiles.length == 16) {
+                    //
+                    // Create all the necessary subdirs
+                    //
+                    this.children = new FSDir[16];
+                    for (int i = 0; i < children.length; i++) {
+                        String str = Integer.toBinaryString(i);
+                        try {
+                            File subdir = new File(dir, "dir_" + str);
+                            subdir.mkdir();
+                            children[i] = new FSDir(subdir);
+                        } catch (StringIndexOutOfBoundsException excep) {
+                            excep.printStackTrace();
+                            System.out.println("Ran into problem when i == " + i + " an str = " + str);
+                        }
+                    }
+
+                    //
+                    // Move existing files into new dirs
+                    //
+                    for (int i = 0; i < localFiles.length; i++) {
+                        Block srcB = new Block(localFiles[i]);
+                        File dst = getBlockFilename(srcB, blkid, depth);
+                        if (!src.renameTo(dst)) {
+                            System.out.println("Unexpected problem in renaming " + src);
+                        }
+                    }
+                }
+                **/
+            } else {
+                // Find subdir
+                children[getHalfByte(blkid, depth)].addBlock(b, src, blkid, depth+1);
+            }
+        }
+
+        /**
+         * Fill in the given blockSet with any child blocks
+         * found at this node.
+         */
+        public void getBlockInfo(TreeSet blockSet) {
+            if (children != null) {
+                for (int i = 0; i < children.length; i++) {
+                    children[i].getBlockInfo(blockSet);
+                }
+            }
+
+            File blockFiles[] = dir.listFiles();
+            for (int i = 0; i < blockFiles.length; i++) {
+                if (Block.isBlockFilename(blockFiles[i])) {
+                    blockSet.add(new Block(blockFiles[i], blockFiles[i].length()));
+                }
+            }
+        }
+
+        /**
+         * Find the file that corresponds to the given Block
+         */
+        public File getBlockFilename(Block b) {
+            return getBlockFilename(b, b.getBlockId(), 0);
+        }
+
+        /**
+         * Helper method to find file for a Block
+         */         
+        private File getBlockFilename(Block b, long blkid, int depth) {
+            if (children == null) {
+                return new File(dir, b.getBlockName());
+            } else {
+                // 
+                // Lift the 4 bits starting at depth, going left->right.
+                // That means there are 2^4 possible children, or 16.
+                // The max depth is thus ((len(long) / 4) == 16).
+                //
+                return children[getHalfByte(blkid, depth)].getBlockFilename(b, blkid, depth+1);
+            }
+        }
+
+        /**
+         * Returns a number 0-15, inclusive.  Pulls out the right
+         * half-byte from the indicated long.
+         */
+        private int getHalfByte(long blkid, int halfByteIndex) {
+            blkid = blkid >> ((15 - halfByteIndex) * 4);
+            return (int) ((0x000000000000000F) & blkid);
+        }
+
+        public String toString() {
+          return "FSDir{" +
+              "dir=" + dir +
+              ", children=" + (children == null ? null : Arrays.asList(children)) +
+              "}";
+        }
+    }
+
+    //////////////////////////////////////////////////////
+    //
+    // FSDataSet
+    //
+    //////////////////////////////////////////////////////
+
+    String dirpath = null;
+    File data = null, tmp = null;
+    long reserved = 0;
+    FSDir dirTree;
+    TreeSet ongoingCreates = new TreeSet();
+
+    /**
+     * An FSDataset has a directory where it loads its data files.
+     */
+    public FSDataset(File dir, Configuration conf) throws IOException {
+        this.dirpath = dir.getCanonicalPath();
+        this.data = new File(dir, "data");
+        if (! data.exists()) {
+            data.mkdirs();
+        }
+        this.tmp = new File(dir, "tmp");
+        if (tmp.exists()) {
+            FileUtil.fullyDelete(tmp, conf);
+        }
+        this.tmp.mkdirs();
+        this.dirTree = new FSDir(data);
+    }
+
+    /**
+     * Return total capacity, used and unused
+     */
+    public long getCapacity() throws IOException {
+        return new DF(dirpath).getCapacity();
+    }
+
+    /**
+     * Return how many bytes can still be stored in the FSDataset
+     */
+    public long getRemaining() throws IOException {
+        return ((long) Math.round(USABLE_DISK_PCT * new DF(dirpath).getAvailable())) - reserved;
+    }
+
+    /**
+     * Find the block's on-disk length
+     */
+    public long getLength(Block b) throws IOException {
+        if (! isValidBlock(b)) {
+            throw new IOException("Block " + b + " is not valid.");
+        }
+        File f = getFile(b);
+        return f.length();
+    }
+
+    /**
+     * Get a stream of data from the indicated block.
+     */
+    public InputStream getBlockData(Block b) throws IOException {
+        if (! isValidBlock(b)) {
+            throw new IOException("Block " + b + " is not valid.");
+        }
+        return new FileInputStream(getFile(b));
+    }
+
+    /**
+     * A Block b will be coming soon!
+     */
+    public boolean startBlock(Block b) throws IOException {
+        //
+        // Make sure the block isn't 'valid'
+        //
+        if (isValidBlock(b)) {
+            throw new IOException("Block " + b + " is valid, and cannot be created.");
+        }
+        return true;
+    }
+
+    /**
+     * Start writing to a block file
+     */
+    public OutputStream writeToBlock(Block b) throws IOException {
+        //
+        // Make sure the block isn't a valid one - we're still creating it!
+        //
+        if (isValidBlock(b)) {
+            throw new IOException("Block " + b + " is valid, and cannot be written to.");
+        }
+
+        //
+        // Serialize access to /tmp, and check if file already there.
+        //
+        File f = null;
+        synchronized (ongoingCreates) {
+            //
+            // Is it already in the create process?
+            //
+            if (ongoingCreates.contains(b)) {
+                throw new IOException("Block " + b + " has already been started (though not completed), and thus cannot be created.");
+            }
+
+            //
+            // Check if we have too little space
+            //
+            if (getRemaining() < BLOCK_SIZE) {
+                throw new IOException("Insufficient space for an additional block");
+            }
+
+            //
+            // OK, all's well.  Register the create, adjust 
+            // 'reserved' size, & create file
+            //
+            ongoingCreates.add(b);
+            reserved += BLOCK_SIZE;
+            f = getTmpFile(b);
+	    try {
+		if (f.exists()) {
+		    throw new IOException("Unexpected problem in startBlock() for " + b + ".  File " + f + " should not be present, but is.");
+		}
+
+		//
+		// Create the zero-length temp file
+		//
+		if (!f.createNewFile()) {
+		    throw new IOException("Unexpected problem in startBlock() for " + b + ".  File " + f + " should be creatable, but is already present.");
+		}
+	    } catch (IOException ie) {
+                System.out.println("Exception!  " + ie);
+		ongoingCreates.remove(b);		
+		reserved -= BLOCK_SIZE;
+                throw ie;
+	    }
+        }
+
+        //
+        // Finally, allow a writer to the block file
+        // REMIND - mjc - make this a filter stream that enforces a max
+        // block size, so clients can't go crazy
+        //
+        return new FileOutputStream(f);
+    }
+
+    //
+    // REMIND - mjc - eventually we should have a timeout system
+    // in place to clean up block files left by abandoned clients.
+    // We should have some timer in place, so that if a blockfile
+    // is created but non-valid, and has been idle for >48 hours,
+    // we can GC it safely.
+    //
+
+    /**
+     * Complete the block write!
+     */
+    public void finalizeBlock(Block b) throws IOException {
+        File f = getTmpFile(b);
+        if (! f.exists()) {
+            throw new IOException("No temporary file " + f + " for block " + b);
+        }
+        
+        synchronized (ongoingCreates) {
+            //
+            // Make sure still registered as ongoing
+            //
+            if (! ongoingCreates.contains(b)) {
+                throw new IOException("Tried to finalize block " + b + ", but not in ongoingCreates table");
+            }
+
+            long finalLen = f.length();
+            b.setNumBytes(finalLen);
+
+            //
+            // Move the file
+            // (REMIND - mjc - shame to move the file within a synch
+            // section!  Maybe remove this?)
+            //
+            dirTree.addBlock(b, f);
+
+            //
+            // Done, so deregister from ongoingCreates
+            //
+            if (! ongoingCreates.remove(b)) {
+                throw new IOException("Tried to finalize block " + b + ", but could not find it in ongoingCreates after file-move!");
+            } 
+            reserved -= BLOCK_SIZE;
+        }
+    }
+
+    /**
+     * Return a table of block data
+     */
+    public Block[] getBlockReport() {
+        TreeSet blockSet = new TreeSet();
+        dirTree.getBlockInfo(blockSet);
+        Block blockTable[] = new Block[blockSet.size()];
+        int i = 0;
+        for (Iterator it = blockSet.iterator(); it.hasNext(); i++) {
+            blockTable[i] = (Block) it.next();
+        }
+        return blockTable;
+    }
+
+    /**
+     * Check whether the given block is a valid one.
+     */
+    public boolean isValidBlock(Block b) {
+        File f = getFile(b);
+        if (f.exists()) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * We're informed that a block is no longer valid.  We
+     * could lazily garbage-collect the block, but why bother?
+     * just get rid of it.
+     */
+    public void invalidate(Block invalidBlks[]) throws IOException {
+        for (int i = 0; i < invalidBlks.length; i++) {
+            File f = getFile(invalidBlks[i]);
+
+            long len = f.length();
+            if (!f.delete()) {
+                throw new IOException("Unexpected error trying to delete block " + invalidBlks[i] + " at file " + f);
+            }
+        }
+    }
+
+    /**
+     * Turn the block identifier into a filename.
+     */
+    File getFile(Block b) {
+        // REMIND - mjc - should cache this result for performance
+        return dirTree.getBlockFilename(b);
+    }
+
+    /**
+     * Get the temp file, if this block is still being created.
+     */
+    File getTmpFile(Block b) {
+        // REMIND - mjc - should cache this result for performance
+        return new File(tmp, b.getBlockName());
+    }
+
+    public String toString() {
+      return "FSDataset{" +
+        "dirpath='" + dirpath + "'" +
+        "}";
+    }
+
+}

+ 744 - 0
src/java/org/apache/hadoop/dfs/FSDirectory.java

@@ -0,0 +1,744 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import org.apache.hadoop.io.*;
+
+import java.io.*;
+import java.util.*;
+
+/*************************************************
+ * FSDirectory stores the filesystem directory state.
+ * It handles writing/loading values to disk, and logging
+ * changes as we go.
+ *
+ * It keeps the filename->blockset mapping always-current
+ * and logged to disk.
+ * 
+ * @author Mike Cafarella
+ *************************************************/
+public class FSDirectory implements FSConstants {
+    static String FS_IMAGE = "fsimage";
+    static String NEW_FS_IMAGE = "fsimage.new";
+    static String OLD_FS_IMAGE = "fsimage.old";
+
+    private static final byte OP_ADD = 0;
+    private static final byte OP_RENAME = 1;
+    private static final byte OP_DELETE = 2;
+    private static final byte OP_MKDIR = 3;
+
+    /******************************************************
+     * We keep an in-memory representation of the file/block
+     * hierarchy.
+     ******************************************************/
+    class INode {
+        public String name;
+        public INode parent;
+        public TreeMap children = new TreeMap();
+        public Block blocks[];
+
+        /**
+         */
+        INode(String name, INode parent, Block blocks[]) {
+            this.name = name;
+            this.parent = parent;
+            this.blocks = blocks;
+        }
+
+        /**
+         * This is the external interface
+         */
+        INode getNode(String target) {
+            if (! target.startsWith("/") || target.length() == 0) {
+                return null;
+            } else if (parent == null && "/".equals(target)) {
+                return this;
+            } else {
+                Vector components = new Vector();
+                int start = 0;
+                int slashid = 0;
+                while (start < target.length() && (slashid = target.indexOf('/', start)) >= 0) {
+                    components.add(target.substring(start, slashid));
+                    start = slashid + 1;
+                }
+                if (start < target.length()) {
+                    components.add(target.substring(start));
+                }
+                return getNode(components, 0);
+            }
+        }
+
+        /**
+         */
+        INode getNode(Vector components, int index) {
+            if (! name.equals((String) components.elementAt(index))) {
+                return null;
+            }
+            if (index == components.size()-1) {
+                return this;
+            }
+
+            // Check with children
+            INode child = (INode) children.get(components.elementAt(index+1));
+            if (child == null) {
+                return null;
+            } else {
+                return child.getNode(components, index+1);
+            }
+        }
+
+        /**
+         */
+        INode addNode(String target, Block blks[]) {
+            if (getNode(target) != null) {
+                return null;
+            } else {
+                String parentName = DFSFile.getDFSParent(target);
+                if (parentName == null) {
+                    return null;
+                }
+
+                INode parentNode = getNode(parentName);
+                if (parentNode == null) {
+                    return null;
+                } else {
+                    String targetName = new File(target).getName();
+                    INode newItem = new INode(targetName, parentNode, blks);
+                    parentNode.children.put(targetName, newItem);
+                    return newItem;
+                }
+            }
+        }
+
+        /**
+         */
+        boolean removeNode() {
+            if (parent == null) {
+                return false;
+            } else {
+                parent.children.remove(name);
+                return true;
+            }
+        }
+
+        /**
+         * Collect all the blocks at this INode and all its children.
+         * This operation is performed after a node is removed from the tree,
+         * and we want to GC all the blocks at this node and below.
+         */
+        void collectSubtreeBlocks(Vector v) {
+            if (blocks != null) {
+                for (int i = 0; i < blocks.length; i++) {
+                    v.add(blocks[i]);
+                }
+            }
+            for (Iterator it = children.values().iterator(); it.hasNext(); ) {
+                INode child = (INode) it.next();
+                child.collectSubtreeBlocks(v);
+            }
+        }
+
+        /**
+         */
+        int numItemsInTree() {
+            int total = 0;
+            for (Iterator it = children.values().iterator(); it.hasNext(); ) {
+                INode child = (INode) it.next();
+                total += child.numItemsInTree();
+            }
+            return total + 1;
+        }
+
+        /**
+         */
+        String computeName() {
+            if (parent != null) {
+                return parent.computeName() + "/" + name;
+            } else {
+                return name;
+            }
+        }
+
+        /**
+         */
+        long computeFileLength() {
+            long total = 0;
+            if (blocks != null) {
+                for (int i = 0; i < blocks.length; i++) {
+                    total += blocks[i].getNumBytes();
+                }
+            }
+            return total;
+        }
+
+        /**
+         */
+        long computeContentsLength() {
+            long total = computeFileLength();
+            for (Iterator it = children.values().iterator(); it.hasNext(); ) {
+                INode child = (INode) it.next();
+                total += child.computeContentsLength();
+            }
+            return total;
+        }
+
+        /**
+         */
+        void listContents(Vector v) {
+            if (parent != null && blocks != null) {
+                v.add(this);
+            }
+
+            for (Iterator it = children.values().iterator(); it.hasNext(); ) {
+                INode child = (INode) it.next();
+                v.add(child);
+            }
+        }
+
+        /**
+         */
+        void saveImage(String parentPrefix, DataOutputStream out) throws IOException {
+            String fullName = "";
+            if (parent != null) {
+                fullName = parentPrefix + "/" + name;
+                new UTF8(fullName).write(out);
+                if (blocks == null) {
+                    out.writeInt(0);
+                } else {
+                    out.writeInt(blocks.length);
+                    for (int i = 0; i < blocks.length; i++) {
+                        blocks[i].write(out);
+                    }
+                }
+            }
+            for (Iterator it = children.values().iterator(); it.hasNext(); ) {
+                INode child = (INode) it.next();
+                child.saveImage(fullName, out);
+            }
+        }
+    }
+
+    INode rootDir = new INode("", null, null);
+    TreeSet activeBlocks = new TreeSet();
+    TreeMap activeLocks = new TreeMap();
+    DataOutputStream editlog = null;
+    boolean ready = false;
+
+    /**
+     * Create a FileSystem directory, and load its info
+     * from the indicated place.
+     */
+    public FSDirectory(File dir) throws IOException {
+        File fullimage = new File(dir, "image");
+        if (! fullimage.exists()) {
+            fullimage.mkdirs();
+        }
+        File edits = new File(dir, "edits");
+        if (loadFSImage(fullimage, edits)) {
+            saveFSImage(fullimage, edits);
+        }
+
+        synchronized (this) {
+            this.ready = true;
+            this.notifyAll();
+            this.editlog = new DataOutputStream(new FileOutputStream(edits));
+        }
+    }
+
+    /**
+     * Shutdown the filestore
+     */
+    public void close() throws IOException {
+        editlog.close();
+    }
+
+    /**
+     * Block until the object is ready to be used.
+     */
+    void waitForReady() {
+        if (! ready) {
+            synchronized (this) {
+                while (!ready) {
+                    try {
+                        this.wait(5000);
+                    } catch (InterruptedException ie) {
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * Load in the filesystem image.  It's a big list of
+     * filenames and blocks.  Return whether we should
+     * "re-save" and consolidate the edit-logs
+     */
+    boolean loadFSImage(File fsdir, File edits) throws IOException {
+        //
+        // Atomic move sequence, to recover from interrupted save
+        //
+        File curFile = new File(fsdir, FS_IMAGE);
+        File newFile = new File(fsdir, NEW_FS_IMAGE);
+        File oldFile = new File(fsdir, OLD_FS_IMAGE);
+
+        // Maybe we were interrupted between 2 and 4
+        if (oldFile.exists() && curFile.exists()) {
+            oldFile.delete();
+            if (edits.exists()) {
+                edits.delete();
+            }
+        } else if (oldFile.exists() && newFile.exists()) {
+            // Or maybe between 1 and 2
+            newFile.renameTo(curFile);
+            oldFile.delete();
+        } else if (curFile.exists() && newFile.exists()) {
+            // Or else before stage 1, in which case we lose the edits
+            newFile.delete();
+        }
+
+        //
+        // Load in bits
+        //
+        if (curFile.exists()) {
+            DataInputStream in = new DataInputStream(new BufferedInputStream(new FileInputStream(curFile)));
+            try {
+                int numFiles = in.readInt();
+                for (int i = 0; i < numFiles; i++) {
+                    UTF8 name = new UTF8();
+                    name.readFields(in);
+                    int numBlocks = in.readInt();
+                    if (numBlocks == 0) {
+                        unprotectedAddFile(name, null);
+                    } else {
+                        Block blocks[] = new Block[numBlocks];
+                        for (int j = 0; j < numBlocks; j++) {
+                            blocks[j] = new Block();
+                            blocks[j].readFields(in);
+                        }
+                        unprotectedAddFile(name, blocks);
+                    }
+                }
+            } finally {
+                in.close();
+            }
+        }
+
+        if (edits.exists() && loadFSEdits(edits) > 0) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Load an edit log, and apply the changes to the in-memory structure
+     *
+     * This is where we apply edits that we've been writing to disk all
+     * along.
+     */
+    int loadFSEdits(File edits) throws IOException {
+        int numEdits = 0;
+
+        if (edits.exists()) {
+            DataInputStream in = new DataInputStream(new BufferedInputStream(new FileInputStream(edits)));
+            try {
+                while (in.available() > 0) {
+                    byte opcode = in.readByte();
+                    numEdits++;
+                    switch (opcode) {
+                    case OP_ADD: {
+                        UTF8 name = new UTF8();
+                        name.readFields(in);
+                        ArrayWritable aw = new ArrayWritable(Block.class);
+                        aw.readFields(in);
+                        Writable writables[] = (Writable[]) aw.get();
+                        Block blocks[] = new Block[writables.length];
+                        System.arraycopy(writables, 0, blocks, 0, blocks.length);
+                        unprotectedAddFile(name, blocks);
+                        break;
+                    } 
+                    case OP_RENAME: {
+                        UTF8 src = new UTF8();
+                        UTF8 dst = new UTF8();
+                        src.readFields(in);
+                        dst.readFields(in);
+                        unprotectedRenameTo(src, dst);
+                        break;
+                    }
+                    case OP_DELETE: {
+                        UTF8 src = new UTF8();
+                        src.readFields(in);
+                        unprotectedDelete(src);
+                        break;
+                    }
+                    case OP_MKDIR: {
+                        UTF8 src = new UTF8();
+                        src.readFields(in);
+                        unprotectedMkdir(src.toString());
+                        break;
+                    }
+                    default: {
+                        throw new IOException("Never seen opcode " + opcode);
+                    }
+                    }
+                }
+            } finally {
+                in.close();
+            }
+        }
+        return numEdits;
+    }
+
+    /**
+     * Save the contents of the FS image
+     */
+    void saveFSImage(File fullimage, File edits) throws IOException {
+        File curFile = new File(fullimage, FS_IMAGE);
+        File newFile = new File(fullimage, NEW_FS_IMAGE);
+        File oldFile = new File(fullimage, OLD_FS_IMAGE);
+
+        //
+        // Write out data
+        //
+        DataOutputStream out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(newFile)));
+        try {
+            out.writeInt(rootDir.numItemsInTree() - 1);
+            rootDir.saveImage("", out);
+        } finally {
+            out.close();
+        }
+
+        //
+        // Atomic move sequence
+        //
+        // 1.  Move cur to old
+        curFile.renameTo(oldFile);
+        
+        // 2.  Move new to cur
+        newFile.renameTo(curFile);
+
+        // 3.  Remove pending-edits file (it's been integrated with newFile)
+        edits.delete();
+        
+        // 4.  Delete old
+        oldFile.delete();
+    }
+
+    /**
+     * Write an operation to the edit log
+     */
+    void logEdit(byte op, Writable w1, Writable w2) {
+        synchronized (editlog) {
+            try {
+                editlog.write(op);
+                if (w1 != null) {
+                    w1.write(editlog);
+                }
+                if (w2 != null) {
+                    w2.write(editlog);
+                }
+            } catch (IOException ie) {
+            }
+        }
+    }
+
+    /**
+     * Add the given filename to the fs.
+     */
+    public boolean addFile(UTF8 src, Block blocks[]) {
+        waitForReady();
+
+        // Always do an implicit mkdirs for parent directory tree
+        mkdirs(DFSFile.getDFSParent(src.toString()));
+        if (unprotectedAddFile(src, blocks)) {
+            logEdit(OP_ADD, src, new ArrayWritable(Block.class, blocks));
+            return true;
+        } else {
+            return false;
+        }
+    }
+    
+    /**
+     */
+    boolean unprotectedAddFile(UTF8 name, Block blocks[]) {
+        synchronized (rootDir) {
+            if (blocks != null) {
+                // Add file->block mapping
+                for (int i = 0; i < blocks.length; i++) {
+                    activeBlocks.add(blocks[i]);
+                }
+            }
+            return (rootDir.addNode(name.toString(), blocks) != null);
+        }
+    }
+
+    /**
+     * Change the filename
+     */
+    public boolean renameTo(UTF8 src, UTF8 dst) {
+        waitForReady();
+        if (unprotectedRenameTo(src, dst)) {
+            logEdit(OP_RENAME, src, dst);
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     */
+    boolean unprotectedRenameTo(UTF8 src, UTF8 dst) {
+        synchronized(rootDir) {
+            INode removedNode = rootDir.getNode(src.toString());
+            if (removedNode == null) {
+                return false;
+            }
+            removedNode.removeNode();
+            if (isDir(dst)) {
+                dst = new UTF8(dst.toString() + "/" + new File(src.toString()).getName());
+            }
+            INode newNode = rootDir.addNode(dst.toString(), removedNode.blocks);
+            if (newNode != null) {
+                newNode.children = removedNode.children;
+                for (Iterator it = newNode.children.values().iterator(); it.hasNext(); ) {
+                    INode child = (INode) it.next();
+                    child.parent = newNode;
+                }
+                return true;
+            } else {
+                rootDir.addNode(src.toString(), removedNode.blocks);
+                return false;
+            }
+        }
+    }
+
+    /**
+     * Remove the file from management, return blocks
+     */
+    public Block[] delete(UTF8 src) {
+        waitForReady();
+        logEdit(OP_DELETE, src, null);
+        return unprotectedDelete(src);
+    }
+
+    /**
+     */
+    Block[] unprotectedDelete(UTF8 src) {
+        synchronized (rootDir) {
+            INode targetNode = rootDir.getNode(src.toString());
+            if (targetNode == null) {
+                return null;
+            } else {
+                //
+                // Remove the node from the namespace and GC all
+                // the blocks underneath the node.
+                //
+                if (! targetNode.removeNode()) {
+                    return null;
+                } else {
+                    Vector v = new Vector();
+                    targetNode.collectSubtreeBlocks(v);
+                    for (Iterator it = v.iterator(); it.hasNext(); ) {
+                        Block b = (Block) it.next();
+                        activeBlocks.remove(b);
+                    }
+                    return (Block[]) v.toArray(new Block[v.size()]);
+                }
+            }
+        }
+    }
+
+    /**
+     */
+    public int obtainLock(UTF8 src, UTF8 holder, boolean exclusive) {
+        TreeSet holders = (TreeSet) activeLocks.get(src);
+        if (holders == null) {
+            holders = new TreeSet();
+            activeLocks.put(src, holders);
+        }
+        if (exclusive && holders.size() > 0) {
+            return STILL_WAITING;
+        } else {
+            holders.add(holder);
+            return COMPLETE_SUCCESS;
+        }
+    }
+
+    /**
+     */
+    public int releaseLock(UTF8 src, UTF8 holder) {
+        TreeSet holders = (TreeSet) activeLocks.get(src);
+        if (holders != null && holders.contains(holder)) {
+            holders.remove(holder);
+            if (holders.size() == 0) {
+                activeLocks.remove(src);
+            }
+            return COMPLETE_SUCCESS;
+        } else {
+            return OPERATION_FAILED;
+        }
+    }
+
+    /**
+     * Get a listing of files given path 'src'
+     *
+     * This function is admittedly very inefficient right now.  We'll
+     * make it better later.
+     */
+    public DFSFileInfo[] getListing(UTF8 src) {
+        String srcs = normalizePath(src);
+
+        synchronized (rootDir) {
+            INode targetNode = rootDir.getNode(srcs);
+            if (targetNode == null) {
+                return null;
+            } else {
+                Vector contents = new Vector();
+                targetNode.listContents(contents);
+
+                DFSFileInfo listing[] = new DFSFileInfo[contents.size()];
+                int i = 0;
+                for (Iterator it = contents.iterator(); it.hasNext(); i++) {
+                    INode cur = (INode) it.next();
+                    UTF8 curName = new UTF8(cur.computeName());
+                    listing[i] = new DFSFileInfo(curName, cur.computeFileLength(), cur.computeContentsLength(), isDir(curName));
+                    //listing[i] = new DFSFileInfo(curName, cur.computeFileLength(), 0, isDir(curName));
+                    //listing[i] = new DFSFileInfo(curName, cur.computeFileLength(), 0, false);
+                }
+                return listing;
+            }
+        }
+    }
+
+    /**
+     * Get the blocks associated with the file
+     */
+    public Block[] getFile(UTF8 src) {
+        waitForReady();
+        synchronized (rootDir) {
+            INode targetNode = rootDir.getNode(src.toString());
+            if (targetNode == null) {
+                return null;
+            } else {
+                return targetNode.blocks;
+            }
+        }
+    }
+
+    /** 
+     * Check whether the filepath could be created
+     */
+    public boolean isValidToCreate(UTF8 src) {
+        String srcs = normalizePath(src);
+        synchronized (rootDir) {
+            if (srcs.startsWith("/") && 
+                ! srcs.endsWith("/") && 
+                rootDir.getNode(srcs) == null) {
+                return true;
+            } else {
+                return false;
+            }
+        }
+    }
+
+    /**
+     * Check whether it's a directory
+     */
+    public boolean isDir(UTF8 src) {
+        synchronized (rootDir) {
+            INode node = rootDir.getNode(normalizePath(src));
+            if (node != null && node.blocks == null) {
+                return true;
+            } else {
+                return false;
+            }
+        }
+    }
+
+    /**
+     * Create the given directory and all its parent dirs.
+     */
+    public boolean mkdirs(UTF8 src) {
+        return mkdirs(src.toString());
+    }
+
+    /**
+     * Create directory entries for every item
+     */
+    boolean mkdirs(String src) {
+        src = normalizePath(new UTF8(src));
+
+        // Use this to collect all the dirs we need to construct
+        Vector v = new Vector();
+
+        // The dir itself
+        v.add(src);
+
+        // All its parents
+        String parent = DFSFile.getDFSParent(src);
+        while (parent != null) {
+            v.add(parent);
+            parent = DFSFile.getDFSParent(parent);
+        }
+
+        // Now go backwards through list of dirs, creating along
+        // the way
+        boolean lastSuccess = false;
+        int numElts = v.size();
+        for (int i = numElts - 1; i >= 0; i--) {
+            String cur = (String) v.elementAt(i);
+            INode inserted = unprotectedMkdir(cur);
+            if (inserted != null) {
+                logEdit(OP_MKDIR, new UTF8(inserted.computeName()), null);
+                lastSuccess = true;
+            } else {
+                lastSuccess = false;
+            }
+        }
+        return lastSuccess;
+    }
+
+    /**
+     */
+    INode unprotectedMkdir(String src) {
+        synchronized (rootDir) {
+            return rootDir.addNode(src, null);
+        }
+    }
+
+    /**
+     */
+    String normalizePath(UTF8 src) {
+        String srcs = src.toString();
+        if (srcs.length() > 1 && srcs.endsWith("/")) {
+            srcs = srcs.substring(0, srcs.length() - 1);
+        }
+        return srcs;
+    }
+
+    /**
+     * Returns whether the given block is one pointed-to by a file.
+     */
+    public boolean isValidBlock(Block b) {
+        synchronized (rootDir) {
+            if (activeBlocks.contains(b)) {
+                return true;
+            } else {
+                return false;
+            }
+        }
+    }
+}

+ 1342 - 0
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -0,0 +1,1342 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.util.LogFormatter;
+
+import java.io.*;
+import java.util.*;
+import java.util.logging.*;
+
+/***************************************************
+ * The FSNamesystem tracks several important tables.
+ *
+ * 1)  valid fsname --> blocklist  (kept on disk, logged)
+ * 2)  Set of all valid blocks (inverted #1)
+ * 3)  block --> machinelist (kept in memory, rebuilt dynamically from reports)
+ * 4)  machine --> blocklist (inverted #2)
+ * 5)  LRU cache of updated-heartbeat machines
+ ***************************************************/
+public class FSNamesystem implements FSConstants {
+    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.fs.FSNamesystem");
+
+   
+
+    //
+    // Stores the correct file name hierarchy
+    //
+    FSDirectory dir;
+
+    //
+    // Stores the block-->datanode(s) map.  Updated only in response
+    // to client-sent information.
+    //
+    TreeMap blocksMap = new TreeMap();
+
+    //
+    // Stores the datanode-->block map.  Done by storing a 
+    // set of datanode info objects, sorted by name.  Updated only in
+    // response to client-sent information.
+    //
+    TreeMap datanodeMap = new TreeMap();
+
+    //
+    // Keeps a Vector for every named machine.  The Vector contains
+    // blocks that have recently been invalidated and are thought to live
+    // on the machine in question.
+    //
+    TreeMap recentInvalidateSets = new TreeMap();
+
+    //
+    // Keeps a TreeSet for every named node.  Each treeset contains
+    // a list of the blocks that are "extra" at that location.  We'll
+    // eventually remove these extras.
+    //
+    TreeMap excessReplicateMap = new TreeMap();
+
+    //
+    // Keeps track of files that are being created, plus the
+    // blocks that make them up.
+    //
+    TreeMap pendingCreates = new TreeMap();
+
+    //
+    // Keeps track of the blocks that are part of those pending creates
+    //
+    TreeSet pendingCreateBlocks = new TreeSet();
+
+    //
+    // Stats on overall usage
+    //
+    long totalCapacity = 0, totalRemaining = 0;
+
+    //
+    Random r = new Random();
+
+    //
+    // Stores a set of datanode info objects, sorted by heartbeat
+    //
+    TreeSet heartbeats = new TreeSet(new Comparator() {
+        public int compare(Object o1, Object o2) {
+            DatanodeInfo d1 = (DatanodeInfo) o1;
+            DatanodeInfo d2 = (DatanodeInfo) o2;            
+            long lu1 = d1.lastUpdate();
+            long lu2 = d2.lastUpdate();
+            if (lu1 < lu2) {
+                return -1;
+            } else if (lu1 > lu2) {
+                return 1;
+            } else {
+                return d1.getName().compareTo(d2.getName());
+            }
+        }
+    });
+
+    //
+    // Store set of Blocks that need to be replicated 1 or more times.
+    // We also store pending replication-orders.
+    //
+    private TreeSet neededReplications = new TreeSet();
+    private TreeSet pendingReplications = new TreeSet();
+
+    //
+    // Used for handling lock-leases
+    //
+    private TreeMap leases = new TreeMap();
+    private TreeSet sortedLeases = new TreeSet();
+
+    //
+    // Threaded object that checks to see if we have been
+    // getting heartbeats from all clients. 
+    //
+    HeartbeatMonitor hbmon = null;
+    LeaseMonitor lmon = null;
+    Daemon hbthread = null, lmthread = null;
+    boolean fsRunning = true;
+    long systemStart = 0;
+    private Configuration conf;
+
+    //  DESIRED_REPLICATION is how many copies we try to have at all times
+    private int desiredReplication;
+    //  The maximum number of replicates we should allow for a single block
+    private int maxReplication;
+    //  How many outgoing replication streams a given node should have at one time
+    private int maxReplicationStreams;
+    // MIN_REPLICATION is how many copies we need in place or else we disallow the write
+    private int minReplication;
+    // HEARTBEAT_RECHECK is how often a datanode sends its hearbeat
+    private int heartBeatRecheck;
+   //  Whether we should use disk-availability info when determining target
+    private boolean useAvailability;
+
+    private boolean allowSameHostTargets;
+    
+    /**
+     * dir is where the filesystem directory state 
+     * is stored
+     */
+    public FSNamesystem(File dir, Configuration conf) throws IOException {
+        this.dir = new FSDirectory(dir);
+        this.hbthread = new Daemon(new HeartbeatMonitor());
+        this.lmthread = new Daemon(new LeaseMonitor());
+        hbthread.start();
+        lmthread.start();
+        this.systemStart = System.currentTimeMillis();
+        this.conf = conf;
+        
+        this.desiredReplication = conf.getInt("dfs.replication", 3);
+        this.maxReplication = desiredReplication;
+        this.maxReplicationStreams = conf.getInt("dfs.max-repl-streams", 2);
+        this.minReplication = 1;
+        this.heartBeatRecheck= 1000;
+        this.useAvailability = conf.getBoolean("dfs.availability.allocation", false);
+        this.allowSameHostTargets =
+           conf.getBoolean("test.dfs.same.host.targets.allowed", false);
+    }
+
+    /** Close down this filesystem manager.
+     * Causes heartbeat and lease daemons to stop; waits briefly for
+     * them to finish, but a short timeout returns control back to caller.
+     */
+    public void close() {
+      synchronized (this) {
+        fsRunning = false;
+      }
+        try {
+            hbthread.join(3000);
+        } catch (InterruptedException ie) {
+        } finally {
+          // using finally to ensure we also wait for lease daemon
+          try {
+            lmthread.join(3000);
+          } catch (InterruptedException ie) {
+          }
+        }
+    }
+
+    /////////////////////////////////////////////////////////
+    //
+    // These methods are called by NutchFS clients
+    //
+    /////////////////////////////////////////////////////////
+    /**
+     * The client wants to open the given filename.  Return a
+     * list of (block,machineArray) pairs.  The sequence of unique blocks
+     * in the list indicates all the blocks that make up the filename.
+     *
+     * The client should choose one of the machines from the machineArray
+     * at random.
+     */
+    public Object[] open(UTF8 src) {
+        Object results[] = null;
+        Block blocks[] = dir.getFile(src);
+        if (blocks != null) {
+            results = new Object[2];
+            DatanodeInfo machineSets[][] = new DatanodeInfo[blocks.length][];
+
+            for (int i = 0; i < blocks.length; i++) {
+                TreeSet containingNodes = (TreeSet) blocksMap.get(blocks[i]);
+                if (containingNodes == null) {
+                    machineSets[i] = new DatanodeInfo[0];
+                } else {
+                    machineSets[i] = new DatanodeInfo[containingNodes.size()];
+                    int j = 0;
+                    for (Iterator it = containingNodes.iterator(); it.hasNext(); j++) {
+                        machineSets[i][j] = (DatanodeInfo) it.next();
+                    }
+                }
+            }
+
+            results[0] = blocks;
+            results[1] = machineSets;
+        }
+        return results;
+    }
+
+    /**
+     * The client would like to create a new block for the indicated
+     * filename.  Return an array that consists of the block, plus a set 
+     * of machines.  The first on this list should be where the client 
+     * writes data.  Subsequent items in the list must be provided in
+     * the connection to the first datanode.
+     * @return Return an array that consists of the block, plus a set
+     * of machines, or null if src is invalid for creation (based on
+     * {@link FSDirectory#isValidToCreate(UTF8)}.
+     */
+    public synchronized Object[] startFile(UTF8 src, UTF8 holder, boolean overwrite) {
+        Object results[] = null;
+        if (pendingCreates.get(src) == null) {
+            boolean fileValid = dir.isValidToCreate(src);
+            if (overwrite && ! fileValid) {
+                delete(src);
+                fileValid = true;
+            }
+
+            if (fileValid) {
+                results = new Object[2];
+
+                // Get the array of replication targets 
+                DatanodeInfo targets[] = chooseTargets(this.desiredReplication, null);
+                if (targets.length < this.minReplication) {
+                    LOG.warning("Target-length is " + targets.length +
+                        ", below MIN_REPLICATION (" + this.minReplication+ ")");
+                    return null;
+                }
+
+                // Reserve space for this pending file
+                pendingCreates.put(src, new Vector());
+                synchronized (leases) {
+                    Lease lease = (Lease) leases.get(holder);
+                    if (lease == null) {
+                        lease = new Lease(holder);
+                        leases.put(holder, lease);
+                        sortedLeases.add(lease);
+                    } else {
+                        sortedLeases.remove(lease);
+                        lease.renew();
+                        sortedLeases.add(lease);
+                    }
+                    lease.startedCreate(src);
+                }
+
+                // Create next block
+                results[0] = allocateBlock(src);
+                results[1] = targets;
+            } else { // ! fileValid
+              LOG.warning("Cannot start file because it is invalid. src=" + src);
+            }
+        } else {
+            LOG.warning("Cannot start file because pendingCreates is non-null. src=" + src);
+        }
+        return results;
+    }
+
+    /**
+     * The client would like to obtain an additional block for the indicated
+     * filename (which is being written-to).  Return an array that consists
+     * of the block, plus a set of machines.  The first on this list should
+     * be where the client writes data.  Subsequent items in the list must
+     * be provided in the connection to the first datanode.
+     *
+     * Make sure the previous blocks have been reported by datanodes and
+     * are replicated.  Will return an empty 2-elt array if we want the
+     * client to "try again later".
+     */
+    public synchronized Object[] getAdditionalBlock(UTF8 src) {
+        Object results[] = null;
+        if (dir.getFile(src) == null && pendingCreates.get(src) != null) {
+            results = new Object[2];
+
+            //
+            // If we fail this, bad things happen!
+            //
+            if (checkFileProgress(src)) {
+                // Get the array of replication targets 
+                DatanodeInfo targets[] = chooseTargets(this.desiredReplication, null);
+                if (targets.length < this.minReplication) {
+                    return null;
+                }
+
+                // Create next block
+                results[0] = allocateBlock(src);
+                results[1] = targets;
+            }
+        }
+        return results;
+    }
+
+    /**
+     * The client would like to let go of the given block
+     */
+    public synchronized boolean abandonBlock(Block b, UTF8 src) {
+        //
+        // Remove the block from the pending creates list
+        //
+        Vector pendingVector = (Vector) pendingCreates.get(src);
+        if (pendingVector != null) {
+            for (Iterator it = pendingVector.iterator(); it.hasNext(); ) {
+                Block cur = (Block) it.next();
+                if (cur.compareTo(b) == 0) {
+                    pendingCreateBlocks.remove(cur);
+                    it.remove();
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Abandon the entire file in progress
+     */
+    public synchronized void abandonFileInProgress(UTF8 src) throws IOException {
+        internalReleaseCreate(src);
+    }
+
+    /**
+     * Finalize the created file and make it world-accessible.  The
+     * FSNamesystem will already know the blocks that make up the file.
+     * Before we return, we make sure that all the file's blocks have 
+     * been reported by datanodes and are replicated correctly.
+     */
+    public synchronized int completeFile(UTF8 src, UTF8 holder) {
+        if (dir.getFile(src) != null || pendingCreates.get(src) == null) {
+	    LOG.info("Failed to complete " + src + "  because dir.getFile()==" + dir.getFile(src) + " and " + pendingCreates.get(src));
+            return OPERATION_FAILED;
+        } else if (! checkFileProgress(src)) {
+            return STILL_WAITING;
+        } else {
+            Vector pendingVector = (Vector) pendingCreates.get(src);
+            Block pendingBlocks[] = (Block[]) pendingVector.toArray(new Block[pendingVector.size()]);
+
+            //
+            // We have the pending blocks, but they won't have
+            // length info in them (as they were allocated before
+            // data-write took place).  So we need to add the correct
+            // length info to each
+            //
+            // REMIND - mjc - this is very inefficient!  We should
+            // improve this!
+            //
+            for (int i = 0; i < pendingBlocks.length; i++) {
+                Block b = pendingBlocks[i];
+                TreeSet containingNodes = (TreeSet) blocksMap.get(b);
+                DatanodeInfo node = (DatanodeInfo) containingNodes.first();
+                for (Iterator it = node.getBlockIterator(); it.hasNext(); ) {
+                    Block cur = (Block) it.next();
+                    if (b.getBlockId() == cur.getBlockId()) {
+                        b.setNumBytes(cur.getNumBytes());
+                        break;
+                    }
+                }
+            }
+            
+            //
+            // Now we can add the (name,blocks) tuple to the filesystem
+            //
+            if (dir.addFile(src, pendingBlocks)) {
+                // The file is no longer pending
+                pendingCreates.remove(src);
+                for (int i = 0; i < pendingBlocks.length; i++) {
+                    pendingCreateBlocks.remove(pendingBlocks[i]);
+                }
+
+                synchronized (leases) {
+                    Lease lease = (Lease) leases.get(holder);
+                    if (lease != null) {
+                        lease.completedCreate(src);
+                        if (! lease.hasLocks()) {
+                            leases.remove(holder);
+                            sortedLeases.remove(lease);
+                        }
+                    }
+                }
+
+                //
+                // REMIND - mjc - this should be done only after we wait a few secs.
+                // The namenode isn't giving datanodes enough time to report the
+                // replicated blocks that are automatically done as part of a client
+                // write.
+                //
+
+                // Now that the file is real, we need to be sure to replicate
+                // the blocks.
+                for (int i = 0; i < pendingBlocks.length; i++) {
+                    TreeSet containingNodes = (TreeSet) blocksMap.get(pendingBlocks[i]);
+                    if (containingNodes.size() < this.desiredReplication) {
+                        synchronized (neededReplications) {
+                            LOG.info("Completed file " + src + ", at holder " + holder + ".  There is/are only " + containingNodes.size() + " copies of block " + pendingBlocks[i] + ", so replicating up to " + this.desiredReplication);
+                            neededReplications.add(pendingBlocks[i]);
+                        }
+                    }
+                }
+                return COMPLETE_SUCCESS;
+            } else {
+                System.out.println("AddFile() for " + src + " failed");
+            }
+	    LOG.info("Dropped through on file add....");
+        }
+
+        return OPERATION_FAILED;
+    }
+
+    /**
+     * Allocate a block at the given pending filename
+     */
+    synchronized Block allocateBlock(UTF8 src) {
+        Block b = new Block();
+        Vector v = (Vector) pendingCreates.get(src);
+        v.add(b);
+        pendingCreateBlocks.add(b);
+        return b;
+    }
+
+    /**
+     * Check that the indicated file's blocks are present and
+     * replicated.  If not, return false.
+     */
+    synchronized boolean checkFileProgress(UTF8 src) {
+        Vector v = (Vector) pendingCreates.get(src);
+
+        for (Iterator it = v.iterator(); it.hasNext(); ) {
+            Block b = (Block) it.next();
+            TreeSet containingNodes = (TreeSet) blocksMap.get(b);
+            if (containingNodes == null || containingNodes.size() < this.minReplication) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    ////////////////////////////////////////////////////////////////
+    // Here's how to handle block-copy failure during client write:
+    // -- As usual, the client's write should result in a streaming
+    // backup write to a k-machine sequence.
+    // -- If one of the backup machines fails, no worries.  Fail silently.
+    // -- Before client is allowed to close and finalize file, make sure
+    // that the blocks are backed up.  Namenode may have to issue specific backup
+    // commands to make up for earlier datanode failures.  Once all copies
+    // are made, edit namespace and return to client.
+    ////////////////////////////////////////////////////////////////
+
+    /**
+     * Change the indicated filename.
+     */
+    public boolean renameTo(UTF8 src, UTF8 dst) {
+        return dir.renameTo(src, dst);
+    }
+
+    /**
+     * Remove the indicated filename from the namespace.  This may
+     * invalidate some blocks that make up the file.
+     */
+    public synchronized boolean delete(UTF8 src) {
+        Block deletedBlocks[] = (Block[]) dir.delete(src);
+        if (deletedBlocks != null) {
+            for (int i = 0; i < deletedBlocks.length; i++) {
+                Block b = deletedBlocks[i];
+
+                TreeSet containingNodes = (TreeSet) blocksMap.get(b);
+                if (containingNodes != null) {
+                    for (Iterator it = containingNodes.iterator(); it.hasNext(); ) {
+                        DatanodeInfo node = (DatanodeInfo) it.next();
+                        Vector invalidateSet = (Vector) recentInvalidateSets.get(node.getName());
+                        if (invalidateSet == null) {
+                            invalidateSet = new Vector();
+                            recentInvalidateSets.put(node.getName(), invalidateSet);
+                        }
+                        invalidateSet.add(b);
+                    }
+                }
+            }
+        }
+
+        return (deletedBlocks != null);
+    }
+
+    /**
+     * Return whether the given filename exists
+     */
+    public boolean exists(UTF8 src) {
+        if (dir.getFile(src) != null || dir.isDir(src)) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Whether the given name is a directory
+     */
+    public boolean isDir(UTF8 src) {
+        return dir.isDir(src);
+    }
+
+    /**
+     * Create all the necessary directories
+     */
+    public boolean mkdirs(UTF8 src) {
+        return dir.mkdirs(src);
+    }
+
+    /**
+     * Figure out a few hosts that are likely to contain the
+     * block(s) referred to by the given (filename, start, len) tuple.
+     */
+    public UTF8[][] getDatanodeHints(UTF8 src, long start, long len) {
+        if (start < 0 || len < 0) {
+            return new UTF8[0][];
+        }
+
+        int startBlock = -1;
+        int endBlock = -1;
+        Block blocks[] = dir.getFile(src);
+
+        //
+        // First, figure out where the range falls in
+        // the blocklist.
+        //
+        long startpos = start;
+        long endpos = start + len;
+        for (int i = 0; i < blocks.length; i++) {
+            if (startpos >= 0) {
+                startpos -= blocks[i].getNumBytes();
+                if (startpos <= 0) {
+                    startBlock = i;
+                }
+            }
+            if (endpos >= 0) {
+                endpos -= blocks[i].getNumBytes();
+                if (endpos <= 0) {
+                    endBlock = i;
+                    break;
+                }
+            }
+        }
+
+        //
+        // Next, create an array of hosts where each block can
+        // be found
+        //
+        if (startBlock < 0 || endBlock < 0) {
+            return new UTF8[0][];
+        } else {
+            UTF8 hosts[][] = new UTF8[endBlock - startBlock + 1][];
+            for (int i = startBlock; i <= endBlock; i++) {
+                TreeSet containingNodes = (TreeSet) blocksMap.get(blocks[i]);
+                Vector v = new Vector();
+                for (Iterator it = containingNodes.iterator(); it.hasNext(); ) {
+                    DatanodeInfo cur = (DatanodeInfo) it.next();
+                    v.add(cur.getHost());
+                }
+                hosts[i] = (UTF8[]) v.toArray(new UTF8[v.size()]);
+            }
+            return hosts;
+        }
+    }
+
+    /************************************************************
+     * A Lease governs all the locks held by a single client.
+     * For each client there's a corresponding lease, whose
+     * timestamp is updated when the client periodically
+     * checks in.  If the client dies and allows its lease to
+     * expire, all the corresponding locks can be released.
+     *************************************************************/
+    class Lease implements Comparable {
+        public UTF8 holder;
+        public long lastUpdate;
+        TreeSet locks = new TreeSet();
+        TreeSet creates = new TreeSet();
+
+        public Lease(UTF8 holder) {
+            this.holder = holder;
+            renew();
+        }
+        public void renew() {
+            this.lastUpdate = System.currentTimeMillis();
+        }
+        public boolean expired() {
+            if (System.currentTimeMillis() - lastUpdate > LEASE_PERIOD) {
+                return true;
+            } else {
+                return false;
+            }
+        }
+        public void obtained(UTF8 src) {
+            locks.add(src);
+        }
+        public void released(UTF8 src) {
+            locks.remove(src);
+        }
+        public void startedCreate(UTF8 src) {
+            creates.add(src);
+        }
+        public void completedCreate(UTF8 src) {
+            creates.remove(src);
+        }
+        public boolean hasLocks() {
+            return (locks.size() + creates.size()) > 0;
+        }
+        public void releaseLocks() {
+            for (Iterator it = locks.iterator(); it.hasNext(); ) {
+                UTF8 src = (UTF8) it.next();
+                internalReleaseLock(src, holder);
+            }
+            locks.clear();
+            for (Iterator it = creates.iterator(); it.hasNext(); ) {
+                UTF8 src = (UTF8) it.next();
+                internalReleaseCreate(src);
+            }
+            creates.clear();
+        }
+
+        /**
+         */
+        public String toString() {
+            return "[Lease.  Holder: " + holder.toString() + ", heldlocks: " + locks.size() + ", pendingcreates: " + creates.size() + "]";
+        }
+
+        /**
+         */
+        public int compareTo(Object o) {
+            Lease l1 = (Lease) this;
+            Lease l2 = (Lease) o;
+            long lu1 = l1.lastUpdate;
+            long lu2 = l2.lastUpdate;
+            if (lu1 < lu2) {
+                return -1;
+            } else if (lu1 > lu2) {
+                return 1;
+            } else {
+                return l1.holder.compareTo(l2.holder);
+            }
+        }
+    }
+    /******************************************************
+     * LeaseMonitor checks for leases that have expired,
+     * and disposes of them.
+     ******************************************************/
+    class LeaseMonitor implements Runnable {
+        public void run() {
+            while (fsRunning) {
+                synchronized (FSNamesystem.this) {
+                    synchronized (leases) {
+                        Lease top;
+                        while ((sortedLeases.size() > 0) &&
+                               ((top = (Lease) sortedLeases.first()) != null)) {
+                            if (top.expired()) {
+                                top.releaseLocks();
+                                leases.remove(top.holder);
+                                LOG.info("Removing lease " + top + ", leases remaining: " + sortedLeases.size());
+                                if (!sortedLeases.remove(top)) {
+                                    LOG.info("Unknown failure trying to remove " + top + " from lease set.");
+                                }
+                            } else {
+                                break;
+                            }
+                        }
+                    }
+                }
+                try {
+                    Thread.sleep(2000);
+                } catch (InterruptedException ie) {
+                }
+            }
+        }
+    }
+
+    /**
+     * Get a lock (perhaps exclusive) on the given file
+     */
+    public synchronized int obtainLock(UTF8 src, UTF8 holder, boolean exclusive) {
+        int result = dir.obtainLock(src, holder, exclusive);
+        if (result == COMPLETE_SUCCESS) {
+            synchronized (leases) {
+                Lease lease = (Lease) leases.get(holder);
+                if (lease == null) {
+                    lease = new Lease(holder);
+                    leases.put(holder, lease);
+                    sortedLeases.add(lease);
+                } else {
+                    sortedLeases.remove(lease);
+                    lease.renew();
+                    sortedLeases.add(lease);
+                }
+                lease.obtained(src);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Release the lock on the given file
+     */
+    public synchronized int releaseLock(UTF8 src, UTF8 holder) {
+        int result = internalReleaseLock(src, holder);
+        if (result == COMPLETE_SUCCESS) {
+            synchronized (leases) {
+                Lease lease = (Lease) leases.get(holder);
+                if (lease != null) {
+                    lease.released(src);
+                    if (! lease.hasLocks()) {
+                        leases.remove(holder);
+                        sortedLeases.remove(lease);
+                    }
+                }
+            }
+        }
+        return result;
+    }
+    private int internalReleaseLock(UTF8 src, UTF8 holder) {
+        return dir.releaseLock(src, holder);
+    }
+    private void internalReleaseCreate(UTF8 src) {
+        Vector v = (Vector) pendingCreates.remove(src);
+        for (Iterator it2 = v.iterator(); it2.hasNext(); ) {
+            Block b = (Block) it2.next();
+            pendingCreateBlocks.remove(b);
+        }
+    }
+
+    /**
+     * Renew the lease(s) held by the given client
+     */
+    public void renewLease(UTF8 holder) {
+        synchronized (leases) {
+            Lease lease = (Lease) leases.get(holder);
+            if (lease != null) {
+                sortedLeases.remove(lease);
+                lease.renew();
+                sortedLeases.add(lease);
+            }
+        }
+    }
+
+    /**
+     * Get a listing of all files at 'src'.  The Object[] array
+     * exists so we can return file attributes (soon to be implemented)
+     */
+    public DFSFileInfo[] getListing(UTF8 src) {
+        return dir.getListing(src);
+    }
+
+    /////////////////////////////////////////////////////////
+    //
+    // These methods are called by datanodes
+    //
+    /////////////////////////////////////////////////////////
+    /**
+     * The given node has reported in.  This method should:
+     * 1) Record the heartbeat, so the datanode isn't timed out
+     * 2) Adjust usage stats for future block allocation
+     */
+    public void gotHeartbeat(UTF8 name, long capacity, long remaining) {
+        synchronized (heartbeats) {
+            synchronized (datanodeMap) {
+                long capacityDiff = 0;
+                long remainingDiff = 0;
+                DatanodeInfo nodeinfo = (DatanodeInfo) datanodeMap.get(name);
+
+                if (nodeinfo == null) {
+                    LOG.info("Got brand-new heartbeat from " + name);
+                    nodeinfo = new DatanodeInfo(name, capacity, remaining);
+                    datanodeMap.put(name, nodeinfo);
+                    capacityDiff = capacity;
+                    remainingDiff = remaining;
+                } else {
+                    capacityDiff = capacity - nodeinfo.getCapacity();
+                    remainingDiff = remaining - nodeinfo.getRemaining();
+                    heartbeats.remove(nodeinfo);
+                    nodeinfo.updateHeartbeat(capacity, remaining);
+                }
+                heartbeats.add(nodeinfo);
+                totalCapacity += capacityDiff;
+                totalRemaining += remainingDiff;
+            }
+        }
+    }
+
+    /**
+     * Periodically calls heartbeatCheck().
+     */
+    class HeartbeatMonitor implements Runnable {
+        /**
+         */
+        public void run() {
+            while (fsRunning) {
+                heartbeatCheck();
+                try {
+                    Thread.sleep(heartBeatRecheck);
+                } catch (InterruptedException ie) {
+                }
+            }
+        }
+    }
+
+    /**
+     * Check if there are any expired heartbeats, and if so,
+     * whether any blocks have to be re-replicated.
+     */
+    synchronized void heartbeatCheck() {
+        synchronized (heartbeats) {
+            DatanodeInfo nodeInfo = null;
+
+            while ((heartbeats.size() > 0) &&
+                   ((nodeInfo = (DatanodeInfo) heartbeats.first()) != null) &&
+                   (nodeInfo.lastUpdate() < System.currentTimeMillis() - EXPIRE_INTERVAL)) {
+                LOG.info("Lost heartbeat for " + nodeInfo.getName());
+
+                heartbeats.remove(nodeInfo);
+                synchronized (datanodeMap) {
+                    datanodeMap.remove(nodeInfo.getName());
+                }
+                totalCapacity -= nodeInfo.getCapacity();
+                totalRemaining -= nodeInfo.getRemaining();
+
+                Block deadblocks[] = nodeInfo.getBlocks();
+                if (deadblocks != null) {
+                    for (int i = 0; i < deadblocks.length; i++) {
+                        removeStoredBlock(deadblocks[i], nodeInfo);
+                    }
+                }
+
+                if (heartbeats.size() > 0) {
+                    nodeInfo = (DatanodeInfo) heartbeats.first();
+                }
+            }
+        }
+    }
+    
+    /**
+     * The given node is reporting all its blocks.  Use this info to 
+     * update the (machine-->blocklist) and (block-->machinelist) tables.
+     */
+    public synchronized Block[] processReport(Block newReport[], UTF8 name) {
+        DatanodeInfo node = (DatanodeInfo) datanodeMap.get(name);
+        if (node == null) {
+            throw new IllegalArgumentException("Unexpected exception.  Received block report from node " + name + ", but there is no info for " + name);
+        }
+
+        //
+        // Modify the (block-->datanode) map, according to the difference
+        // between the old and new block report.
+        //
+        int oldPos = 0, newPos = 0;
+        Block oldReport[] = node.getBlocks();
+        while (oldReport != null && newReport != null && oldPos < oldReport.length && newPos < newReport.length) {
+            int cmp = oldReport[oldPos].compareTo(newReport[newPos]);
+            
+            if (cmp == 0) {
+                // Do nothing, blocks are the same
+                oldPos++;
+                newPos++;
+            } else if (cmp < 0) {
+                // The old report has a block the new one does not
+                removeStoredBlock(oldReport[oldPos], node);
+                oldPos++;
+            } else {
+                // The new report has a block the old one does not
+                addStoredBlock(newReport[newPos], node);
+                newPos++;
+            }
+        }
+        while (oldReport != null && oldPos < oldReport.length) {
+            // The old report has a block the new one does not
+            removeStoredBlock(oldReport[oldPos], node);
+            oldPos++;
+        }
+        while (newReport != null && newPos < newReport.length) {
+            // The new report has a block the old one does not
+            addStoredBlock(newReport[newPos], node);
+            newPos++;
+        }
+
+        //
+        // Modify node so it has the new blockreport
+        //
+        node.updateBlocks(newReport);
+
+        //
+        // We've now completely updated the node's block report profile.
+        // We now go through all its blocks and find which ones are invalid,
+        // no longer pending, or over-replicated.
+        //
+        // (Note it's not enough to just invalidate blocks at lease expiry 
+        // time; datanodes can go down before the client's lease on 
+        // the failed file expires and miss the "expire" event.)
+        //
+        // This function considers every block on a datanode, and thus
+        // should only be invoked infrequently.
+        //
+        Vector obsolete = new Vector();
+        for (Iterator it = node.getBlockIterator(); it.hasNext(); ) {
+            Block b = (Block) it.next();
+
+            if (! dir.isValidBlock(b) && ! pendingCreateBlocks.contains(b)) {
+                LOG.info("Obsoleting block " + b);
+                obsolete.add(b);
+            }
+        }
+        return (Block[]) obsolete.toArray(new Block[obsolete.size()]);
+    }
+
+    /**
+     * Modify (block-->datanode) map.  Remove block from set of 
+     * needed replications if this takes care of the problem.
+     */
+    synchronized void addStoredBlock(Block block, DatanodeInfo node) {
+        TreeSet containingNodes = (TreeSet) blocksMap.get(block);
+        if (containingNodes == null) {
+            containingNodes = new TreeSet();
+            blocksMap.put(block, containingNodes);
+        }
+        if (! containingNodes.contains(node)) {
+            containingNodes.add(node);
+        } else {
+            LOG.info("Redundant addStoredBlock request received for block " + block + " on node " + node);
+        }
+
+        synchronized (neededReplications) {
+            if (dir.isValidBlock(block)) {
+                if (containingNodes.size() >= this.desiredReplication) {
+                    neededReplications.remove(block);
+                    pendingReplications.remove(block);
+                } else if (containingNodes.size() < this.desiredReplication) {
+                    if (! neededReplications.contains(block)) {
+                        neededReplications.add(block);
+                    }
+                }
+
+                //
+                // Find how many of the containing nodes are "extra", if any.
+                // If there are any extras, call chooseExcessReplicates() to
+                // mark them in the excessReplicateMap.
+                //
+                Vector nonExcess = new Vector();
+                for (Iterator it = containingNodes.iterator(); it.hasNext(); ) {
+                    DatanodeInfo cur = (DatanodeInfo) it.next();
+                    TreeSet excessBlocks = (TreeSet) excessReplicateMap.get(cur.getName());
+                    if (excessBlocks == null || ! excessBlocks.contains(block)) {
+                        nonExcess.add(cur);
+                    }
+                }
+                if (nonExcess.size() > this.maxReplication) {
+                    chooseExcessReplicates(nonExcess, block, this.maxReplication);    
+                }
+            }
+        }
+    }
+
+    /**
+     * We want a max of "maxReps" replicates for any block, but we now have too many.  
+     * In this method, copy enough nodes from 'srcNodes' into 'dstNodes' such that:
+     *
+     * srcNodes.size() - dstNodes.size() == maxReps
+     *
+     * For now, we choose nodes randomly.  In the future, we might enforce some
+     * kind of policy (like making sure replicates are spread across racks).
+     */
+    void chooseExcessReplicates(Vector nonExcess, Block b, int maxReps) {
+        while (nonExcess.size() - maxReps > 0) {
+            int chosenNode = r.nextInt(nonExcess.size());
+            DatanodeInfo cur = (DatanodeInfo) nonExcess.elementAt(chosenNode);
+            nonExcess.removeElementAt(chosenNode);
+
+            TreeSet excessBlocks = (TreeSet) excessReplicateMap.get(cur.getName());
+            if (excessBlocks == null) {
+                excessBlocks = new TreeSet();
+                excessReplicateMap.put(cur.getName(), excessBlocks);
+            }
+            excessBlocks.add(b);
+
+            //
+            // The 'excessblocks' tracks blocks until we get confirmation
+            // that the datanode has deleted them; the only way we remove them
+            // is when we get a "removeBlock" message.  
+            //
+            // The 'invalidate' list is used to inform the datanode the block 
+            // should be deleted.  Items are removed from the invalidate list
+            // upon giving instructions to the namenode.
+            //
+            Vector invalidateSet = (Vector) recentInvalidateSets.get(cur.getName());
+            if (invalidateSet == null) {
+                invalidateSet = new Vector();
+                recentInvalidateSets.put(cur.getName(), invalidateSet);
+            }
+            invalidateSet.add(b);
+        }
+    }
+
+    /**
+     * Modify (block-->datanode) map.  Possibly generate 
+     * replication tasks, if the removed block is still valid.
+     */
+    synchronized void removeStoredBlock(Block block, DatanodeInfo node) {
+        TreeSet containingNodes = (TreeSet) blocksMap.get(block);
+        if (containingNodes == null || ! containingNodes.contains(node)) {
+            throw new IllegalArgumentException("No machine mapping found for block " + block + ", which should be at node " + node);
+        }
+        containingNodes.remove(node);
+
+        //
+        // It's possible that the block was removed because of a datanode
+        // failure.  If the block is still valid, check if replication is
+        // necessary.  In that case, put block on a possibly-will-
+        // be-replicated list.
+        //
+        if (dir.isValidBlock(block) && (containingNodes.size() < this.desiredReplication)) {
+            synchronized (neededReplications) {
+                neededReplications.add(block);
+            }
+        }
+
+        //
+        // We've removed a block from a node, so it's definitely no longer
+        // in "excess" there.
+        //
+        TreeSet excessBlocks = (TreeSet) excessReplicateMap.get(node.getName());
+        if (excessBlocks != null) {
+            excessBlocks.remove(block);
+            if (excessBlocks.size() == 0) {
+                excessReplicateMap.remove(node.getName());
+            }
+        }
+    }
+
+    /**
+     * The given node is reporting that it received a certain block.
+     */
+    public synchronized void blockReceived(Block block, UTF8 name) {
+        DatanodeInfo node = (DatanodeInfo) datanodeMap.get(name);
+        if (node == null) {
+            throw new IllegalArgumentException("Unexpected exception.  Got blockReceived message from node " + name + ", but there is no info for " + name);
+        }
+        //
+        // Modify the blocks->datanode map
+        // 
+        addStoredBlock(block, node);
+
+        //
+        // Supplement node's blockreport
+        //
+        node.addBlock(block);
+    }
+
+    /**
+     * Total raw bytes
+     */
+    public long totalCapacity() {
+        return totalCapacity;
+    }
+
+    /**
+     * Total non-used raw bytes
+     */
+    public long totalRemaining() {
+        return totalRemaining;
+    }
+
+    /**
+     */
+    public DatanodeInfo[] datanodeReport() {
+        DatanodeInfo results[] = null;
+        synchronized (heartbeats) {
+            synchronized (datanodeMap) {
+                results = new DatanodeInfo[datanodeMap.size()];
+                int i = 0;
+                for (Iterator it = datanodeMap.values().iterator(); it.hasNext(); ) {
+                    DatanodeInfo cur = (DatanodeInfo) it.next();
+                    results[i++] = cur;
+                }
+            }
+        }
+        return results;
+    }
+
+    /////////////////////////////////////////////////////////
+    //
+    // These methods are called by the Namenode system, to see
+    // if there is any work for a given datanode.
+    //
+    /////////////////////////////////////////////////////////
+
+    /**
+     * Check if there are any recently-deleted blocks a datanode should remove.
+     */
+    public synchronized Block[] blocksToInvalidate(UTF8 sender) {
+        Vector invalidateSet = (Vector) recentInvalidateSets.remove(sender);
+        if (invalidateSet != null) {
+            return (Block[]) invalidateSet.toArray(new Block[invalidateSet.size()]);
+        } else {
+            return null;
+        }
+    }
+
+    /**
+     * Return with a list of Block/DataNodeInfo sets, indicating
+     * where various Blocks should be copied, ASAP.
+     *
+     * The Array that we return consists of two objects:
+     * The 1st elt is an array of Blocks.
+     * The 2nd elt is a 2D array of DatanodeInfo objs, identifying the
+     *     target sequence for the Block at the appropriate index.
+     *
+     */
+    public synchronized Object[] pendingTransfers(DatanodeInfo srcNode, int xmitsInProgress) {
+        synchronized (neededReplications) {
+            Object results[] = null;
+	    int scheduledXfers = 0;
+
+            if (neededReplications.size() > 0) {
+                //
+                // Go through all blocks that need replications.  See if any
+                // are present at the current node.  If so, ask the node to
+                // replicate them.
+                //
+                Vector replicateBlocks = new Vector();
+                Vector replicateTargetSets = new Vector();
+                for (Iterator it = neededReplications.iterator(); it.hasNext(); ) {
+                    //
+                    // We can only reply with 'maxXfers' or fewer blocks
+                    //
+                    if (scheduledXfers >= this.maxReplicationStreams - xmitsInProgress) {
+                        break;
+                    }
+
+                    Block block = (Block) it.next();
+                    if (! dir.isValidBlock(block)) {
+                        it.remove();
+                    } else {
+                        TreeSet containingNodes = (TreeSet) blocksMap.get(block);
+                        if (containingNodes.contains(srcNode)) {
+                            DatanodeInfo targets[] = chooseTargets(Math.min(this.desiredReplication - containingNodes.size(), this.maxReplicationStreams - xmitsInProgress), containingNodes);
+                            if (targets.length > 0) {
+                                // Build items to return
+                                replicateBlocks.add(block);
+                                replicateTargetSets.add(targets);
+				scheduledXfers += targets.length;
+                            }
+                        }
+                    }
+                }
+
+                //
+                // Move the block-replication into a "pending" state.
+                // The reason we use 'pending' is so we can retry
+                // replications that fail after an appropriate amount of time.  
+                // (REMIND - mjc - this timer is not yet implemented.)
+                //
+                if (replicateBlocks.size() > 0) {
+                    int i = 0;
+                    for (Iterator it = replicateBlocks.iterator(); it.hasNext(); i++) {
+                        Block block = (Block) it.next();
+                        DatanodeInfo targets[] = (DatanodeInfo[]) replicateTargetSets.elementAt(i);
+                        TreeSet containingNodes = (TreeSet) blocksMap.get(block);
+
+                        if (containingNodes.size() + targets.length >= this.desiredReplication) {
+                            neededReplications.remove(block);
+                            pendingReplications.add(block);
+                        }
+
+			LOG.info("Pending transfer (block " + block.getBlockName() + ") from " + srcNode.getName() + " to " + targets.length + " destinations");
+                    }
+
+                    //
+                    // Build returned objects from above lists
+                    //
+                    DatanodeInfo targetMatrix[][] = new DatanodeInfo[replicateTargetSets.size()][];
+                    for (i = 0; i < targetMatrix.length; i++) {
+                        targetMatrix[i] = (DatanodeInfo[]) replicateTargetSets.elementAt(i);
+                    }
+
+                    results = new Object[2];
+                    results[0] = replicateBlocks.toArray(new Block[replicateBlocks.size()]);
+                    results[1]  = targetMatrix;
+                }
+            }
+            return results;
+        }
+    }
+
+    /**
+     * Get a certain number of targets, if possible.
+     * If not, return as many as we can.
+     * @param desiredReplicates number of duplicates wanted.
+     * @param forbiddenNodes of DatanodeInfo instances that should not be
+     * considered targets.
+     * @return array of DatanodeInfo instances uses as targets.
+     */
+    DatanodeInfo[] chooseTargets(int desiredReplicates, TreeSet forbiddenNodes) {
+        TreeSet alreadyChosen = new TreeSet();
+        Vector targets = new Vector();
+
+        for (int i = 0; i < desiredReplicates; i++) {
+            DatanodeInfo target = chooseTarget(forbiddenNodes, alreadyChosen);
+            if (target != null) {
+                targets.add(target);
+                alreadyChosen.add(target);
+            } else {
+                break; // calling chooseTarget again won't help
+            }
+        }
+        return (DatanodeInfo[]) targets.toArray(new DatanodeInfo[targets.size()]);
+    }
+
+    /**
+     * Choose a target from available machines, excepting the
+     * given ones.
+     *
+     * Right now it chooses randomly from available boxes.  In future could 
+     * choose according to capacity and load-balancing needs (or even 
+     * network-topology, to avoid inter-switch traffic).
+     * @param forbidden1 DatanodeInfo targets not allowed, null allowed.
+     * @param forbidden2 DatanodeInfo targets not allowed, null allowed.
+     * @return DatanodeInfo instance to use or null if something went wrong
+     * (a log message is emitted if null is returned).
+     */
+    DatanodeInfo chooseTarget(TreeSet forbidden1, TreeSet forbidden2) {
+        //
+        // Check if there are any available targets at all
+        //
+        int totalMachines = datanodeMap.size();
+        if (totalMachines == 0) {
+            LOG.warning("While choosing target, totalMachines is " + totalMachines);
+            return null;
+        }
+
+        TreeSet forbiddenMachines = new TreeSet();
+        //
+        // In addition to already-chosen datanode/port pairs, we want to avoid
+        // already-chosen machinenames.  (There can be multiple datanodes per
+        // machine.)  We might relax this requirement in the future, though. (Maybe
+        // so that at least one replicate is off the machine.)
+        //
+        UTF8 hostOrHostAndPort = null;
+        if (forbidden1 != null) {
+          // add name [and host] of all elements in forbidden1 to forbiddenMachines
+            for (Iterator it = forbidden1.iterator(); it.hasNext(); ) {
+                DatanodeInfo cur = (DatanodeInfo) it.next();
+                if (allowSameHostTargets) {
+                  hostOrHostAndPort = cur.getName(); // forbid same host:port
+                } else {
+                  hostOrHostAndPort = cur.getHost(); // forbid same host
+                }
+                forbiddenMachines.add(hostOrHostAndPort);
+            }
+        }
+        if (forbidden2 != null) {
+          // add name [and host] of all elements in forbidden2 to forbiddenMachines
+            for (Iterator it = forbidden2.iterator(); it.hasNext(); ) {
+                DatanodeInfo cur = (DatanodeInfo) it.next();
+              if (allowSameHostTargets) {
+                hostOrHostAndPort = cur.getName(); // forbid same host:port
+              } else {
+                hostOrHostAndPort = cur.getHost(); // forbid same host
+              }
+              forbiddenMachines.add(hostOrHostAndPort);
+            }
+        }
+
+        //
+        // Now build list of machines we can actually choose from
+        //
+        long totalRemaining = 0;
+        Vector targetList = new Vector();
+        for (Iterator it = datanodeMap.values().iterator(); it.hasNext(); ) {
+            DatanodeInfo node = (DatanodeInfo) it.next();
+            if (allowSameHostTargets) {
+                hostOrHostAndPort = node.getName(); // match host:port
+            } else {
+                hostOrHostAndPort = node.getHost(); // match host
+            }
+            if (! forbiddenMachines.contains(hostOrHostAndPort)) {
+                targetList.add(node);
+                totalRemaining += node.getRemaining();
+            }
+        }
+
+        //
+        // Now pick one
+        //
+        if (targetList.size() == 0) {
+            LOG.warning("Zero targets found, forbidden1.size=" +
+                ( forbidden1 != null ? forbidden1.size() : 0 ) +
+                " allowSameHostTargets=" + allowSameHostTargets +
+                " forbidden2.size()=" +
+                ( forbidden2 != null ? forbidden2.size() : 0 ));
+            return null;
+        } else if (! this.useAvailability) {
+            int target = r.nextInt(targetList.size());
+            return (DatanodeInfo) targetList.elementAt(target);
+        } else {
+            // Choose node according to target capacity
+            double target = r.nextDouble() * totalRemaining;
+
+            for (Iterator it = targetList.iterator(); it.hasNext(); ) {
+                DatanodeInfo node = (DatanodeInfo) it.next();
+                target -= node.getRemaining();
+                if (target <= 0) {
+                    return node;
+                }
+            }
+
+            LOG.warning("Impossible state.  When trying to choose target node, could not find any.  This may indicate that datanode capacities are being updated during datanode selection.  Anyway, now returning an arbitrary target to recover...");
+            return (DatanodeInfo) targetList.elementAt(r.nextInt(targetList.size()));
+        }
+    }
+}

+ 79 - 0
src/java/org/apache/hadoop/dfs/LocatedBlock.java

@@ -0,0 +1,79 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import org.apache.hadoop.io.*;
+
+import java.io.*;
+
+/****************************************************
+ * A LocatedBlock is a pair of Block, DatanodeInfo[]
+ * objects.  It tells where to find a Block.
+ * 
+ * @author Michael Cafarella
+ ****************************************************/
+public class LocatedBlock implements Writable {
+    Block b;
+    DatanodeInfo locs[];
+
+    /**
+     */
+    public LocatedBlock() {
+        this.b = new Block();
+        this.locs = new DatanodeInfo[0];
+    }
+
+    /**
+     */
+    public LocatedBlock(Block b, DatanodeInfo[] locs) {
+        this.b = b;
+        this.locs = locs;
+    }
+
+    /**
+     */
+    public Block getBlock() {
+        return b;
+    }
+
+    /**
+     */
+    DatanodeInfo[] getLocations() {
+        return locs;
+    }
+
+    ///////////////////////////////////////////
+    // Writable
+    ///////////////////////////////////////////
+    public void write(DataOutput out) throws IOException {
+        b.write(out);
+        out.writeInt(locs.length);
+        for (int i = 0; i < locs.length; i++) {
+            locs[i].write(out);
+        }
+    }
+
+    public void readFields(DataInput in) throws IOException {
+        this.b = new Block();
+        b.readFields(in);
+        int count = in.readInt();
+        this.locs = new DatanodeInfo[count];
+        for (int i = 0; i < locs.length; i++) {
+            locs[i] = new DatanodeInfo();
+            locs[i].readFields(in);
+        }
+    }
+}

+ 856 - 0
src/java/org/apache/hadoop/dfs/NDFSClient.java

@@ -0,0 +1,856 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+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 java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.logging.*;
+
+/********************************************************
+ * DFSClient can connect to a Nutch Filesystem and perform basic file tasks.
+ * Connects to a namenode daemon.
+ * @author Mike Cafarella, Tessa MacDuff
+ ********************************************************/
+public class DFSClient implements FSConstants {
+    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.fs.DFSClient");
+    static int MAX_BLOCK_ACQUIRE_FAILURES = 10;
+    ClientProtocol namenode;
+    boolean running = true;
+    Random r = new Random();
+    String clientName;
+    Daemon leaseChecker;
+
+
+    /** Create a new DFSClient connected to the given namenode server.
+     */
+    public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf) {
+        this.namenode = (ClientProtocol) RPC.getProxy(ClientProtocol.class, nameNodeAddr, conf);
+        this.clientName = "DFSClient_" + r.nextInt();
+        this.leaseChecker = new Daemon(new LeaseChecker());
+        this.leaseChecker.start();
+    }
+
+    /**
+     */
+    public void close() throws IOException {
+        this.running = false;
+        try {
+            leaseChecker.join();
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    /**
+     * Get hints about the location of the indicated block(s).  The
+     * array returned is as long as there are blocks in the indicated
+     * range.  Each block may have one or more locations.
+     */
+    public String[][] getHints(UTF8 src, long start, long len) throws IOException {
+        return namenode.getHints(src.toString(), start, len);
+    }
+
+    /**
+     * Create an input stream that obtains a nodelist from the
+     * namenode, and then reads from all the right places.  Creates
+     * inner subclass of InputStream that does the right out-of-band
+     * work.
+     */
+    public NFSInputStream open(UTF8 src) throws IOException {
+        // Get block info from namenode
+        return new DFSInputStream(src.toString());
+    }
+
+    public NFSOutputStream create(UTF8 src, boolean overwrite) throws IOException {
+        return new DFSOutputStream(src, overwrite);
+    }
+
+    /**
+     * Make a direct connection to namenode and manipulate structures
+     * there.
+     */
+    public boolean rename(UTF8 src, UTF8 dst) throws IOException {
+        return namenode.rename(src.toString(), dst.toString());
+    }
+
+    /**
+     * Make a direct connection to namenode and manipulate structures
+     * there.
+     */
+    public boolean delete(UTF8 src) throws IOException {
+        return namenode.delete(src.toString());
+    }
+
+    /**
+     */
+    public boolean exists(UTF8 src) throws IOException {
+        return namenode.exists(src.toString());
+    }
+
+    /**
+     */
+    public boolean isDirectory(UTF8 src) throws IOException {
+        return namenode.isDir(src.toString());
+    }
+
+    /**
+     */
+    public DFSFileInfo[] listFiles(UTF8 src) throws IOException {
+        return namenode.getListing(src.toString());
+    }
+
+    /**
+     */
+    public long totalRawCapacity() throws IOException {
+        long rawNums[] = namenode.getStats();
+        return rawNums[0];
+    }
+
+    /**
+     */
+    public long totalRawUsed() throws IOException {
+        long rawNums[] = namenode.getStats();
+        return rawNums[1];
+    }
+
+    public DatanodeInfo[] datanodeReport() throws IOException {
+        return namenode.getDatanodeReport();
+    }
+
+    /**
+     */
+    public boolean mkdirs(UTF8 src) throws IOException {
+        return namenode.mkdirs(src.toString());
+    }
+
+    /**
+     */
+    public void lock(UTF8 src, boolean exclusive) throws IOException {
+        long start = System.currentTimeMillis();
+        boolean hasLock = false;
+        while (! hasLock) {
+            hasLock = namenode.obtainLock(src.toString(), clientName, exclusive);
+            if (! hasLock) {
+                try {
+                    Thread.sleep(400);
+                    if (System.currentTimeMillis() - start > 5000) {
+                        LOG.info("Waiting to retry lock for " + (System.currentTimeMillis() - start) + " ms.");
+                        Thread.sleep(2000);
+                    }
+                } catch (InterruptedException ie) {
+                }
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    public void release(UTF8 src) throws IOException {
+        boolean hasReleased = false;
+        while (! hasReleased) {
+            hasReleased = namenode.releaseLock(src.toString(), clientName);
+            if (! hasReleased) {
+                LOG.info("Could not release.  Retrying...");
+                try {
+                    Thread.sleep(2000);
+                } catch (InterruptedException ie) {
+                }
+            }
+        }
+    }
+
+    /**
+     * Pick the best/closest node  which to stream the data.
+     * For now, just pick the first on the list.
+     */
+    private DatanodeInfo bestNode(DatanodeInfo nodes[], TreeSet deadNodes) throws IOException {
+        if ((nodes == null) || 
+            (nodes.length - deadNodes.size() < 1)) {
+            throw new IOException("No live nodes contain current block");
+        }
+        DatanodeInfo chosenNode = null;
+        do {
+            chosenNode = nodes[Math.abs(r.nextInt()) % nodes.length];
+        } while (deadNodes.contains(chosenNode));
+        return chosenNode;
+    }
+
+    /***************************************************************
+     * Periodically check in with the namenode and renew all the leases
+     * when the lease period is half over.
+     ***************************************************************/
+    class LeaseChecker implements Runnable {
+        /**
+         */
+        public void run() {
+            long lastRenewed = 0;
+            while (running) {
+                if (System.currentTimeMillis() - lastRenewed > (LEASE_PERIOD / 2)) {
+                    try {
+                        namenode.renewLease(clientName);
+                        lastRenewed = System.currentTimeMillis();
+                    } catch (IOException ie) {
+                    }
+                }
+                try {
+                    Thread.sleep(1000);
+                } catch (InterruptedException ie) {
+                }
+            }
+        }
+    }
+
+    /****************************************************************
+     * DFSInputStream provides bytes from a named file.  It handles 
+     * negotiation of the namenode and various datanodes as necessary.
+     ****************************************************************/
+    class DFSInputStream extends NFSInputStream {
+        boolean closed = false;
+
+        private String src;
+        private DataInputStream blockStream;
+        private DataOutputStream partnerStream;
+        private Block blocks[] = null;
+        private DatanodeInfo nodes[][] = null;
+        private long pos = 0;
+        private long filelen = 0;
+        private long blockEnd = -1;
+
+        /**
+         */
+        public DFSInputStream(String src) throws IOException {
+            this.src = src;
+            openInfo();
+            this.blockStream = null;
+            this.partnerStream = null;
+            for (int i = 0; i < blocks.length; i++) {
+                this.filelen += blocks[i].getNumBytes();
+            }
+        }
+
+        /**
+         * Grab the open-file info from namenode
+         */
+        void openInfo() throws IOException {
+            Block oldBlocks[] = this.blocks;
+
+            LocatedBlock results[] = namenode.open(src);            
+            Vector blockV = new Vector();
+            Vector nodeV = new Vector();
+            for (int i = 0; i < results.length; i++) {
+                blockV.add(results[i].getBlock());
+                nodeV.add(results[i].getLocations());
+            }
+            Block newBlocks[] = (Block[]) blockV.toArray(new Block[blockV.size()]);
+
+            if (oldBlocks != null) {
+                for (int i = 0; i < oldBlocks.length; i++) {
+                    if (! oldBlocks[i].equals(newBlocks[i])) {
+                        throw new IOException("Blocklist for " + src + " has changed!");
+                    }
+                }
+                if (oldBlocks.length != newBlocks.length) {
+                    throw new IOException("Blocklist for " + src + " now has different length");
+                }
+            }
+            this.blocks = newBlocks;
+            this.nodes = (DatanodeInfo[][]) nodeV.toArray(new DatanodeInfo[nodeV.size()][]);
+        }
+
+        /**
+         * Open a DataInputStream to a DataNode so that it can be read from.
+         * We get block ID and the IDs of the destinations at startup, from the namenode.
+         */
+        private synchronized void blockSeekTo(long target) throws IOException {
+            if (target >= filelen) {
+                throw new IOException("Attempted to read past end of file");
+            }
+
+            if (blockStream != null) {
+                blockStream.close();
+                partnerStream.close();
+            }
+
+            //
+            // Compute desired block
+            //
+            int targetBlock = -1;
+            long targetBlockStart = 0;
+            long targetBlockEnd = 0;
+            for (int i = 0; i < blocks.length; i++) {
+                long blocklen = blocks[i].getNumBytes();
+                targetBlockEnd = targetBlockStart + blocklen - 1;
+
+                if (target >= targetBlockStart && target <= targetBlockEnd) {
+                    targetBlock = i;
+                    break;
+                } else {
+                    targetBlockStart = targetBlockEnd + 1;                    
+                }
+            }
+            if (targetBlock < 0) {
+                throw new IOException("Impossible situation: could not find target position " + target);
+            }
+            long offsetIntoBlock = target - targetBlockStart;
+
+            //
+            // Connect to best DataNode for desired Block, with potential offset
+            //
+            int failures = 0;
+            InetSocketAddress targetAddr = null;
+            Socket s = null;
+            TreeSet deadNodes = new TreeSet();
+            while (s == null) {
+                DatanodeInfo chosenNode;
+
+                try {
+                    chosenNode = bestNode(nodes[targetBlock], deadNodes);
+                    targetAddr = DataNode.createSocketAddr(chosenNode.getName().toString());
+                } catch (IOException ie) {
+                    /**
+                    if (failures >= MAX_BLOCK_ACQUIRE_FAILURES) {
+                        throw new IOException("Could not obtain block " + blocks[targetBlock]);
+                    }
+                    **/
+                    if (nodes[targetBlock] == null || nodes[targetBlock].length == 0) {
+                        LOG.info("No node available for block " + blocks[targetBlock]);
+                    }
+                    LOG.info("Could not obtain block from any node:  " + ie);
+                    try {
+                        Thread.sleep(10000);
+                    } catch (InterruptedException iex) {
+                    }
+                    deadNodes.clear();
+                    openInfo();
+                    failures++;
+                    continue;
+                }
+                try {
+                    s = new Socket(targetAddr.getAddress(), targetAddr.getPort());
+                    s.setSoTimeout(READ_TIMEOUT);
+
+                    //
+                    // Xmit header info to datanode
+                    //
+                    DataOutputStream out = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
+                    out.write(OP_READSKIP_BLOCK);
+                    blocks[targetBlock].write(out);
+                    out.writeLong(offsetIntoBlock);
+                    out.flush();
+
+                    //
+                    // Get bytes in block, set streams
+                    //
+                    DataInputStream in = new DataInputStream(new BufferedInputStream(s.getInputStream()));
+                    long curBlockSize = in.readLong();
+                    long amtSkipped = in.readLong();
+                    if (curBlockSize != blocks[targetBlock].len) {
+                        throw new IOException("Recorded block size is " + blocks[targetBlock].len + ", but datanode reports size of " + curBlockSize);
+                    }
+                    if (amtSkipped != offsetIntoBlock) {
+                        throw new IOException("Asked for offset of " + offsetIntoBlock + ", but only received offset of " + amtSkipped);
+                    }
+
+                    this.pos = target;
+                    this.blockEnd = targetBlockEnd;
+                    this.blockStream = in;
+                    this.partnerStream = out;
+                } catch (IOException ex) {
+                    // Put chosen node into dead list, continue
+                    LOG.info("Failed to connect to " + targetAddr + ":" + ex);
+                    deadNodes.add(chosenNode);
+                    if (s != null) {
+                        try {
+                            s.close();
+                        } catch (IOException iex) {
+                        }                        
+                    }
+                    s = null;
+                }
+            }
+        }
+
+        /**
+         * Close it down!
+         */
+        public synchronized void close() throws IOException {
+            if (closed) {
+                throw new IOException("Stream closed");
+            }
+
+            if (blockStream != null) {
+                blockStream.close();
+                blockStream = null;
+                partnerStream.close();
+            }
+            super.close();
+            closed = true;
+        }
+
+        /**
+         * Basic read()
+         */
+        public synchronized int read() throws IOException {
+            if (closed) {
+                throw new IOException("Stream closed");
+            }
+            int result = -1;
+            if (pos < filelen) {
+                if (pos > blockEnd) {
+                    blockSeekTo(pos);
+                }
+                result = blockStream.read();
+                if (result >= 0) {
+                    pos++;
+                }
+            }
+            return result;
+        }
+
+        /**
+         * Read the entire buffer.
+         */
+        public synchronized int read(byte buf[], int off, int len) throws IOException {
+            if (closed) {
+                throw new IOException("Stream closed");
+            }
+            if (pos < filelen) {
+                if (pos > blockEnd) {
+                    blockSeekTo(pos);
+                }
+                int result = blockStream.read(buf, off, Math.min(len, (int) (blockEnd - pos + 1)));
+                if (result >= 0) {
+                    pos += result;
+                }
+                return result;
+            }
+            return -1;
+        }
+
+        /**
+         * Seek to a new arbitrary location
+         */
+        public synchronized void seek(long targetPos) throws IOException {
+            if (targetPos >= filelen) {
+                throw new IOException("Cannot seek after EOF");
+            }
+            pos = targetPos;
+            blockEnd = -1;
+        }
+
+        /**
+         */
+        public synchronized long getPos() throws IOException {
+            return pos;
+        }
+
+        /**
+         */
+        public synchronized int available() throws IOException {
+            if (closed) {
+                throw new IOException("Stream closed");
+            }
+            return (int) (filelen - pos);
+        }
+
+        /**
+         * We definitely don't support marks
+         */
+        public boolean markSupported() {
+            return false;
+        }
+        public void mark(int readLimit) {
+        }
+        public void reset() throws IOException {
+            throw new IOException("Mark not supported");
+        }
+    }
+
+    /****************************************************************
+     * DFSOutputStream creates files from a stream of bytes.
+     ****************************************************************/
+    class DFSOutputStream extends NFSOutputStream {
+        boolean closed = false;
+
+        private byte outBuf[] = new byte[BUFFER_SIZE];
+        private int pos = 0;
+
+        private UTF8 src;
+        boolean closingDown = false;
+        private boolean overwrite;
+        private boolean blockStreamWorking;
+        private DataOutputStream blockStream;
+        private DataInputStream blockReplyStream;
+        private File backupFile;
+        private OutputStream backupStream;
+        private Block block;
+        private DatanodeInfo targets[]; 
+        private long filePos = 0;
+        private int bytesWrittenToBlock = 0;
+
+        /**
+         * Create a new output stream to the given DataNode.
+         */
+        public DFSOutputStream(UTF8 src, boolean overwrite) throws IOException {
+            this.src = src;
+            this.overwrite = overwrite;
+            this.blockStream = null;
+            this.blockReplyStream = null;
+            this.blockStreamWorking = false;
+            this.backupFile = File.createTempFile("dfsout", "bak");
+            this.backupStream = new BufferedOutputStream(new FileOutputStream(backupFile));
+            nextBlockOutputStream(true);
+        }
+
+        /**
+         * Open a DataOutputStream to a DataNode so that it can be written to.
+         * This happens when a file is created and each time a new block is allocated.
+         * Must get block ID and the IDs of the destinations from the namenode.
+         */
+        private synchronized void nextBlockOutputStream(boolean firstTime) throws IOException {
+            if (! firstTime && blockStreamWorking) {
+                blockStream.close();
+                blockReplyStream.close();
+                blockStreamWorking = false;
+            }
+
+            boolean retry = false;
+            long start = System.currentTimeMillis();
+            do {
+                retry = false;
+                
+                long localstart = System.currentTimeMillis();
+                boolean blockComplete = false;
+                LocatedBlock lb = null;                
+                while (! blockComplete) {
+                    if (firstTime) {
+                        lb = namenode.create(src.toString(), clientName.toString(), overwrite);
+                    } else {
+                        lb = namenode.addBlock(src.toString());
+                    }
+
+                    if (lb == null) {
+                        try {
+                            Thread.sleep(400);
+                            if (System.currentTimeMillis() - localstart > 5000) {
+                                LOG.info("Waiting to find new output block node for " + (System.currentTimeMillis() - start) + "ms");
+                            }
+                        } catch (InterruptedException ie) {
+                        }
+                    } else {
+                        blockComplete = true;
+                    }
+                }
+
+                block = lb.getBlock();
+                DatanodeInfo nodes[] = lb.getLocations();
+
+                //
+                // Connect to first DataNode in the list.  Abort if this fails.
+                //
+                InetSocketAddress target = DataNode.createSocketAddr(nodes[0].getName().toString());
+                Socket s = null;
+                try {
+                    s = new Socket(target.getAddress(), target.getPort());
+                    s.setSoTimeout(READ_TIMEOUT);
+                } catch (IOException ie) {
+                    // Connection failed.  Let's wait a little bit and retry
+                    try {
+                        if (System.currentTimeMillis() - start > 5000) {
+                            LOG.info("Waiting to find target node");
+                        }
+                        Thread.sleep(6000);
+                    } catch (InterruptedException iex) {
+                    }
+                    if (firstTime) {
+                        namenode.abandonFileInProgress(src.toString());
+                    } else {
+                        namenode.abandonBlock(block, src.toString());
+                    }
+                    retry = true;
+                    continue;
+                }
+
+                //
+                // Xmit header info to datanode
+                //
+                DataOutputStream out = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
+                out.write(OP_WRITE_BLOCK);
+                out.writeBoolean(false);
+                block.write(out);
+                out.writeInt(nodes.length);
+                for (int i = 0; i < nodes.length; i++) {
+                    nodes[i].write(out);
+                }
+                out.write(CHUNKED_ENCODING);
+                bytesWrittenToBlock = 0;
+                blockStream = out;
+                blockReplyStream = new DataInputStream(new BufferedInputStream(s.getInputStream()));
+                blockStreamWorking = true;
+            } while (retry);
+        }
+
+        /**
+         * We're referring to the file pos here
+         */
+        public synchronized long getPos() throws IOException {
+            return filePos;
+        }
+			
+        /**
+         * Writes the specified byte to this output stream.
+         */
+        public synchronized void write(int b) throws IOException {
+            if (closed) {
+                throw new IOException("Stream closed");
+            }
+
+            if ((bytesWrittenToBlock + pos == BLOCK_SIZE) ||
+                (pos >= BUFFER_SIZE)) {
+                flush();
+            }
+            outBuf[pos++] = (byte) b;
+            filePos++;
+        }
+
+        /**
+         * Writes the specified bytes to this output stream.
+         */
+      public synchronized void write(byte b[], int off, int len)
+        throws IOException {
+            if (closed) {
+                throw new IOException("Stream closed");
+            }
+            while (len > 0) {
+              int remaining = BUFFER_SIZE - pos;
+              int toWrite = Math.min(remaining, len);
+              System.arraycopy(b, off, outBuf, pos, toWrite);
+              pos += toWrite;
+              off += toWrite;
+              len -= toWrite;
+              filePos += toWrite;
+
+              if ((bytesWrittenToBlock + pos >= BLOCK_SIZE) ||
+                  (pos == BUFFER_SIZE)) {
+                flush();
+              }
+            }
+        }
+
+        /**
+         * Flush the buffer, getting a stream to a new block if necessary.
+         */
+        public synchronized void flush() throws IOException {
+            if (closed) {
+                throw new IOException("Stream closed");
+            }
+
+            if (bytesWrittenToBlock + pos >= BLOCK_SIZE) {
+                flushData(BLOCK_SIZE - bytesWrittenToBlock);
+            }
+            if (bytesWrittenToBlock == BLOCK_SIZE) {
+                endBlock();
+                nextBlockOutputStream(false);
+            }
+            flushData(pos);
+        }
+
+        /**
+         * Actually flush the accumulated bytes to the remote node,
+         * but no more bytes than the indicated number.
+         */
+        private synchronized void flushData(int maxPos) throws IOException {
+            int workingPos = Math.min(pos, maxPos);
+            
+            if (workingPos > 0 || 
+                (workingPos == 0 && closingDown)) {
+                //
+                // To the blockStream, write length, then bytes
+                //
+                if (blockStreamWorking) {
+                    try {
+                        blockStream.writeLong(workingPos);
+                        blockStream.write(outBuf, 0, workingPos);
+                    } catch (IOException ie) {
+                        try {
+                            blockStream.close();
+                        } catch (IOException ie2) {
+                        }
+                        try {
+                            blockReplyStream.close();
+                        } catch (IOException ie2) {
+                        }
+                        namenode.abandonBlock(block, src.toString());
+                        blockStreamWorking = false;
+                    }
+                }
+                //
+                // To the local block backup, write just the bytes
+                //
+                backupStream.write(outBuf, 0, workingPos);
+
+                //
+                // Track position
+                //
+                bytesWrittenToBlock += workingPos;
+                System.arraycopy(outBuf, workingPos, outBuf, 0, pos - workingPos);
+                pos -= workingPos;
+            }
+        }
+
+        /**
+         * We're done writing to the current block.
+         */
+        private synchronized void endBlock() throws IOException {
+            boolean mustRecover = ! blockStreamWorking;
+
+            //
+            // A zero-length set of data indicates the end of the block
+            //
+            if (blockStreamWorking) {
+                try {
+                    internalClose();
+                } catch (IOException ie) {
+                    try {
+                        blockStream.close();
+                    } catch (IOException ie2) {
+                    }
+                    try {
+                        blockReplyStream.close();
+                    } catch (IOException ie2) {
+                    }
+                    namenode.abandonBlock(block, src.toString());
+                    mustRecover = true;
+                } finally {
+                    blockStreamWorking = false;
+                }
+            }
+
+            //
+            // Done with local copy
+            //
+            backupStream.close();
+
+            //
+            // If necessary, recover from a failed datanode connection.
+            //
+            while (mustRecover) {
+                nextBlockOutputStream(false);
+                InputStream in = new FileInputStream(backupFile);
+                try {
+                    byte buf[] = new byte[BUFFER_SIZE];
+                    int bytesRead = in.read(buf);
+                    while (bytesRead >= 0) {
+                        blockStream.writeLong((long) bytesRead);
+                        blockStream.write(buf, 0, bytesRead);
+                        bytesRead = in.read(buf);
+                    }
+                    internalClose();
+                    LOG.info("Recovered from failed datanode connection");
+                    mustRecover = false;
+                } catch (IOException ie) {
+                    try {
+                        blockStream.close();
+                    } catch (IOException ie2) {
+                    }
+                    try {
+                        blockReplyStream.close();
+                    } catch (IOException ie2) {
+                    }
+                    namenode.abandonBlock(block, src.toString());
+                    blockStreamWorking = false;
+                }
+            }
+
+            //
+            // Delete local backup, start new one
+            //
+            backupFile.delete();
+            backupFile = File.createTempFile("dfsout", "bak");
+            backupStream = new BufferedOutputStream(new FileOutputStream(backupFile));
+        }
+
+        /**
+         * Close down stream to remote datanode.  Called from two places
+         * in endBlock();
+         */
+        private synchronized void internalClose() throws IOException {
+            blockStream.writeLong(0);
+            blockStream.flush();
+
+            long complete = blockReplyStream.readLong();
+            if (complete != WRITE_COMPLETE) {
+                LOG.info("Did not receive WRITE_COMPLETE flag: " + complete);
+                throw new IOException("Did not receive WRITE_COMPLETE_FLAG: " + complete);
+            }
+                    
+            LocatedBlock lb = new LocatedBlock();
+            lb.readFields(blockReplyStream);
+            namenode.reportWrittenBlock(lb);
+
+            blockStream.close();
+            blockReplyStream.close();
+        }
+
+        /**
+         * Closes this output stream and releases any system 
+         * resources associated with this stream.
+         */
+        public synchronized void close() throws IOException {
+            if (closed) {
+                throw new IOException("Stream closed");
+            }
+
+            closingDown = true;
+            flush();
+            endBlock();
+
+            backupStream.close();
+            backupFile.delete();
+
+            if (blockStreamWorking) {
+                blockStream.close();                
+                blockReplyStream.close();
+                blockStreamWorking = false;
+            }
+            super.close();
+
+            long localstart = System.currentTimeMillis();
+            boolean fileComplete = false;
+            while (! fileComplete) {
+                fileComplete = namenode.complete(src.toString(), clientName.toString());
+                if (!fileComplete) {
+                    try {
+                        Thread.sleep(400);
+                        if (System.currentTimeMillis() - localstart > 5000) {
+                            LOG.info("Could not complete file, retrying...");
+                        }
+                    } catch (InterruptedException ie) {
+                    }
+                }
+            }
+            closed = true;
+            closingDown = false;
+        }
+    }
+}

+ 98 - 0
src/java/org/apache/hadoop/dfs/NDFSFile.java

@@ -0,0 +1,98 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import java.io.*;
+
+
+/*****************************************************************
+ * DFSFile is a traditional java File that's been annotated with
+ * some extra information.
+ *
+ * @author Mike Cafarella
+ *****************************************************************/
+public class DFSFile extends File {
+    DFSFileInfo info;
+
+    /** Separator used in DFS filenames. */
+    public static final String DFS_FILE_SEPARATOR = "/";
+    
+    /**
+     */
+    public DFSFile(DFSFileInfo info) {
+        super(info.getPath());
+        this.info = info;
+    }
+
+    /**
+     * A number of File methods are unsupported in this subclass
+     */
+    public boolean canRead() {
+        return false;
+    }
+    public boolean canWrite() {
+        return false;
+    }
+    public boolean createNewFile() {
+        return false;
+    }
+    public boolean delete() {
+        return false;
+    }
+    public void deleteOnExit() {
+    }
+    public boolean isHidden() {
+        return false;
+    }
+
+    /**
+     * We need to reimplement some of them
+     */
+    public boolean isDirectory() {
+        return info.isDir();
+    }
+    public boolean isFile() {
+        return ! isDirectory();
+    }
+    public long length() {
+        return info.getLen();
+    }
+
+    /**
+     * And add a few extras
+     */
+    public long getContentsLength() {
+        return info.getContentsLen();
+    }
+    
+    /**
+     * Retrieving parent path from DFS path string
+     * @param path - DFS path 
+     * @return - parent path of DFS path, or null if no parent exist.
+     */
+    public static String getDFSParent(String path) {
+        if (path == null)
+            return null;
+        if (DFS_FILE_SEPARATOR.equals(path))
+            return null;
+        int index = path.lastIndexOf(DFS_FILE_SEPARATOR); 
+        if (index == -1)
+            return null;
+        if (index == 0)
+            return DFS_FILE_SEPARATOR;
+        return path.substring(0, index);
+    }
+}

+ 102 - 0
src/java/org/apache/hadoop/dfs/NDFSFileInfo.java

@@ -0,0 +1,102 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.dfs.
+
+import org.apache.hadoop.io.*;
+
+import java.io.*;
+
+/******************************************************
+ * DFSFileInfo tracks info about remote files, including
+ * name, size, etc.  
+ * 
+ * @author Mike Cafarella
+ ******************************************************/
+public class DFSFileInfo implements Writable {
+    UTF8 path;
+    long len;
+    long contentsLen;
+    boolean isDir;
+
+    /**
+     */
+    public DFSFileInfo() {
+    }
+
+    /**
+     */
+    public DFSFileInfo(UTF8 path, long len, long contentsLen, boolean isDir) {
+        this.path = path;
+        this.len = len;
+        this.contentsLen = contentsLen;
+        this.isDir = isDir;
+    }
+
+    /**
+     */
+    public String getPath() {
+        return path.toString();
+    }
+
+    /**
+     */
+    public String getName() {
+        return new File(path.toString()).getName();
+    }
+
+    /**
+     */
+    public String getParent() {
+        return DFSFile.getDFSParent(path.toString());
+    }
+
+    /**
+     */
+    public long getLen() {
+        return len;
+    }
+
+    /**
+     */
+    public long getContentsLen() {
+        return contentsLen;
+    }
+
+    /**
+     */
+    public boolean isDir() {
+        return isDir;
+    }
+
+    //////////////////////////////////////////////////
+    // Writable
+    //////////////////////////////////////////////////
+    public void write(DataOutput out) throws IOException {
+        path.write(out);
+        out.writeLong(len);
+        out.writeLong(contentsLen);
+        out.writeBoolean(isDir);
+    }
+
+    public void readFields(DataInput in) throws IOException {
+        this.path = new UTF8();
+        this.path.readFields(in);
+        this.len = in.readLong();
+        this.contentsLen = in.readLong();
+        this.isDir = in.readBoolean();
+    }
+}
+

+ 352 - 0
src/java/org/apache/hadoop/dfs/NameNode.java

@@ -0,0 +1,352 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+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 java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.logging.*;
+
+/**********************************************************
+ * NameNode controls two critical tables:
+ *   1)  filename->blocksequence,version
+ *   2)  block->machinelist
+ *
+ * The first table is stored on disk and is very precious.
+ * The second table is rebuilt every time the NameNode comes
+ * up.
+ *
+ * @author Mike Cafarella
+ **********************************************************/
+public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
+    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.dfs.NameNode");
+
+    private FSNamesystem namesystem;
+    private Server server;
+    private int handlerCount = 2;
+
+    /** only used for testing purposes  */
+    private boolean stopRequested = false;
+
+    /**
+     * Create a NameNode at the default location
+     */
+    public NameNode(Configuration conf) throws IOException {
+        this(new File(conf.get("dfs.name.dir",
+                                          "/tmp/nutch/dfs/name")),
+             DataNode.createSocketAddr
+             (conf.get("fs.default.name", "local")).getPort(), conf);
+    }
+
+    /**
+     * Create a NameNode at the specified location and start it.
+     */
+    public NameNode(File dir, int port, Configuration conf) throws IOException {
+        this.namesystem = new FSNamesystem(dir, conf);
+        this.handlerCount = conf.getInt("dfs.namenode.handler.count", 10);
+        this.server = RPC.getServer(this, port, handlerCount, false, conf);
+        this.server.start();
+    }
+
+    /**
+     * Wait for service to finish.
+     * (Normally, it runs forever.)
+     */
+    public void join() {
+        try {
+            this.server.join();
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    /**
+     * Stop all NameNode threads and wait for all to finish.
+     * Package-only access since this is intended for JUnit testing.
+    */
+    void stop() {
+      if (! stopRequested) {
+        stopRequested = true;
+        namesystem.close();
+        server.stop();
+        //this.join();
+      }
+    }
+
+    /////////////////////////////////////////////////////
+    // ClientProtocol
+    /////////////////////////////////////////////////////
+    /**
+     */
+    public LocatedBlock[] open(String src) throws IOException {
+        Object openResults[] = namesystem.open(new UTF8(src));
+        if (openResults == null) {
+            throw new IOException("Cannot open filename " + src);
+        } else {
+            Block blocks[] = (Block[]) openResults[0];
+            DatanodeInfo sets[][] = (DatanodeInfo[][]) openResults[1];
+            LocatedBlock results[] = new LocatedBlock[blocks.length];
+            for (int i = 0; i < blocks.length; i++) {
+                results[i] = new LocatedBlock(blocks[i], sets[i]);
+            }
+            return results;
+        }
+    }
+
+    /**
+     */
+    public LocatedBlock create(String src, String clientName, boolean overwrite) throws IOException {
+        Object results[] = namesystem.startFile(new UTF8(src), new UTF8(clientName), overwrite);
+        if (results == null) {
+            throw new IOException("Cannot create file " + src + " on client " + clientName);
+        } else {
+            Block b = (Block) results[0];
+            DatanodeInfo targets[] = (DatanodeInfo[]) results[1];
+            return new LocatedBlock(b, targets);
+        }
+    }
+
+    /**
+     */
+    public LocatedBlock addBlock(String src) throws IOException {
+        int retries = 5;
+        Object results[] = namesystem.getAdditionalBlock(new UTF8(src));
+        while (results != null && results[0] == null && retries > 0) {
+            try {
+                Thread.sleep(100);
+            } catch (InterruptedException ie) {
+            }
+            results = namesystem.getAdditionalBlock(new UTF8(src));
+            retries--;
+        }
+
+        if (results == null) {
+            throw new IOException("Cannot obtain additional block for file " + src);
+        } else if (results[0] == null) {
+            return null;
+        } else {
+            Block b = (Block) results[0];
+            DatanodeInfo targets[] = (DatanodeInfo[]) results[1];
+            return new LocatedBlock(b, targets);
+        }
+    }
+
+    /**
+     * The client can report in a set written blocks that it wrote.
+     * These blocks are reported via the client instead of the datanode
+     * to prevent weird heartbeat race conditions.
+     */
+    public void reportWrittenBlock(LocatedBlock lb) throws IOException {
+        Block b = lb.getBlock();
+        DatanodeInfo targets[] = lb.getLocations();
+        for (int i = 0; i < targets.length; i++) {
+            namesystem.blockReceived(b, targets[i].getName());
+        }
+    }
+
+    /**
+     * The client needs to give up on the block.
+     */
+    public void abandonBlock(Block b, String src) throws IOException {
+        if (! namesystem.abandonBlock(b, new UTF8(src))) {
+            throw new IOException("Cannot abandon block during write to " + src);
+        }
+    }
+    /**
+     */
+    public void abandonFileInProgress(String src) throws IOException {
+        namesystem.abandonFileInProgress(new UTF8(src));
+    }
+    /**
+     */
+    public boolean complete(String src, String clientName) throws IOException {
+        int returnCode = namesystem.completeFile(new UTF8(src), new UTF8(clientName));
+        if (returnCode == STILL_WAITING) {
+            return false;
+        } else if (returnCode == COMPLETE_SUCCESS) {
+            return true;
+        } else {
+            throw new IOException("Could not complete write to file " + src + " by " + clientName);
+        }
+    }
+    /**
+     */
+    public String[][] getHints(String src, long start, long len) throws IOException {
+        UTF8 hosts[][] = namesystem.getDatanodeHints(new UTF8(src), start, len);
+        if (hosts == null) {
+            return new String[0][];
+        } else {
+            String results[][] = new String[hosts.length][];
+            for (int i = 0; i < hosts.length; i++) {
+                results[i] = new String[hosts[i].length];
+                for (int j = 0; j < results[i].length; j++) {
+                    results[i][j] = hosts[i][j].toString();
+                }
+            }
+            return results;
+        }
+    }
+    /**
+     */
+    public boolean rename(String src, String dst) throws IOException {
+        return namesystem.renameTo(new UTF8(src), new UTF8(dst));
+    }
+
+    /**
+     */
+    public boolean delete(String src) throws IOException {
+        return namesystem.delete(new UTF8(src));
+    }
+
+    /**
+     */
+    public boolean exists(String src) throws IOException {
+        return namesystem.exists(new UTF8(src));
+    }
+
+    /**
+     */
+    public boolean isDir(String src) throws IOException {
+        return namesystem.isDir(new UTF8(src));
+    }
+
+    /**
+     */
+    public boolean mkdirs(String src) throws IOException {
+        return namesystem.mkdirs(new UTF8(src));
+    }
+
+    /**
+     */
+    public boolean obtainLock(String src, String clientName, boolean exclusive) throws IOException {
+        int returnCode = namesystem.obtainLock(new UTF8(src), new UTF8(clientName), exclusive);
+        if (returnCode == COMPLETE_SUCCESS) {
+            return true;
+        } else if (returnCode == STILL_WAITING) {
+            return false;
+        } else {
+            throw new IOException("Failure when trying to obtain lock on " + src);
+        }
+    }
+
+    /**
+     */
+    public boolean releaseLock(String src, String clientName) throws IOException {
+        int returnCode = namesystem.releaseLock(new UTF8(src), new UTF8(clientName));
+        if (returnCode == COMPLETE_SUCCESS) {
+            return true;
+        } else if (returnCode == STILL_WAITING) {
+            return false;
+        } else {
+            throw new IOException("Failure when trying to release lock on " + src);
+        }
+    }
+
+    /**
+     */
+    public void renewLease(String clientName) throws IOException {
+        namesystem.renewLease(new UTF8(clientName));        
+    }
+
+    /**
+     */
+    public DFSFileInfo[] getListing(String src) throws IOException {
+        return namesystem.getListing(new UTF8(src));
+    }
+
+    /**
+     */
+    public long[] getStats() throws IOException {
+        long results[] = new long[2];
+        results[0] = namesystem.totalCapacity();
+        results[1] = namesystem.totalCapacity() - namesystem.totalRemaining();
+        return results;
+    }
+
+    /**
+     */
+    public DatanodeInfo[] getDatanodeReport() throws IOException {
+        DatanodeInfo results[] = namesystem.datanodeReport();
+        if (results == null || results.length == 0) {
+            throw new IOException("Cannot find datanode report");
+        }
+        return results;
+    }
+
+    ////////////////////////////////////////////////////////////////
+    // DatanodeProtocol
+    ////////////////////////////////////////////////////////////////
+    /**
+     */
+    public void sendHeartbeat(String sender, long capacity, long remaining) {
+        namesystem.gotHeartbeat(new UTF8(sender), capacity, remaining);
+    }
+
+    public Block[] blockReport(String sender, Block blocks[]) {
+        LOG.info("Block report from "+sender+": "+blocks.length+" blocks.");
+        return namesystem.processReport(blocks, new UTF8(sender));
+    }
+
+    public void blockReceived(String sender, Block blocks[]) {
+        for (int i = 0; i < blocks.length; i++) {
+            namesystem.blockReceived(blocks[i], new UTF8(sender));
+        }
+    }
+
+    /**
+     */
+    public void errorReport(String sender, String msg) {
+        // Log error message from datanode
+        //LOG.info("Report from " + sender + ": " + msg);
+    }
+
+    /**
+     * Return a block-oriented command for the datanode to execute.
+     * This will be either a transfer or a delete operation.
+     */
+    public BlockCommand getBlockwork(String sender, int xmitsInProgress) {
+        //
+        // Ask to perform pending transfers, if any
+        //
+        Object xferResults[] = namesystem.pendingTransfers(new DatanodeInfo(new UTF8(sender)), xmitsInProgress);
+        if (xferResults != null) {
+            return new BlockCommand((Block[]) xferResults[0], (DatanodeInfo[][]) xferResults[1]);
+        }
+
+        //
+        // If there are no transfers, check for recently-deleted blocks that
+        // should be removed.  This is not a full-datanode sweep, as is done during
+        // a block report.  This is just a small fast removal of blocks that have
+        // just been removed.
+        //
+        Block blocks[] = namesystem.blocksToInvalidate(new UTF8(sender));
+        if (blocks != null) {
+            return new BlockCommand(blocks);
+        }
+        return null;
+    }
+
+    /**
+     */
+    public static void main(String argv[]) throws IOException, InterruptedException {
+        NameNode namenode = new NameNode(new Configuration());
+        namenode.join();
+    }
+}

+ 26 - 0
src/java/org/apache/hadoop/fs/ChecksumException.java

@@ -0,0 +1,26 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.
+
+import java.io.IOException;
+
+/** Thrown for checksum errors. */
+public class ChecksumException extends IOException {
+  public ChecksumException(String description) {
+    super(description);
+  }
+}

+ 25 - 0
src/java/org/apache/hadoop/fs/FSError.java

@@ -0,0 +1,25 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.
+
+/** Thrown for unexpected filesystem errors, presumed to reflect disk errors
+ * in the native filesystem. */
+public class FSError extends Error {
+  FSError(Throwable cause) {
+    super(cause);
+  }
+}

+ 122 - 0
src/java/org/apache/hadoop/fs/FileUtil.java

@@ -0,0 +1,122 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.
+
+import java.io.*;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A collection of file-processing util methods
+ */
+public class FileUtil {
+    /**
+     * Delete a directory and all its contents.  If
+     * we return false, the directory may be partially-deleted.
+     */
+    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 {
+        // 20041022, xing.
+        // Currently nfs.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);
+    }
+
+    /**
+     * 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) {
+            return false;
+        }
+
+        File dstParent = dst.getParentFile();
+        if ((dstParent != null) && (!nfs.exists(dstParent))) {
+            nfs.mkdirs(dstParent);
+        }
+
+        if (nfs.isFile(src)) {
+            NFSInputStream in = nfs.openRaw(src);
+            try {
+                NFSOutputStream out = nfs.createRaw(dst, true);
+                byte buf[] = new byte[conf.getInt("io.file.buffer.size", 4096)];
+                try {
+                    int readBytes = in.read(buf);
+
+                    while (readBytes >= 0) {
+                        out.write(buf, 0, readBytes);
+                        readBytes = in.read(buf);
+                    }
+                } finally {
+                    out.close();
+                }
+            } finally {
+                in.close();
+            }
+        } else {
+            nfs.mkdirs(dst);
+            File contents[] = nfs.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)) {
+                        return false;
+                    }
+                }
+            }
+        }
+        return true;
+    }
+
+    /**
+     * 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 {
+        //
+        // Resolve the real target.
+        //
+        if (nfs.exists(dst) && nfs.isDirectory(dst)) {
+            dst = new File(dst, src.getName());
+        } else if (nfs.exists(dst)) {
+            throw new IOException("Destination " + dst + " already exists");
+        }
+
+        //
+        // Copy the items
+        //
+        if (! nfs.isDirectory(src)) {
+            //
+            // If the source is a file, then just copy the contents
+            //
+            copyContents(nfs, 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);
+            for (int i = 0; i < contents.length; i++) {
+                recursiveCopy(nfs, contents[i], new File(dst, contents[i].getName()), conf);
+            }
+        }
+    }
+}

+ 415 - 0
src/java/org/apache/hadoop/fs/LocalFileSystem.java

@@ -0,0 +1,415 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.
+
+import java.io.*;
+import java.util.*;
+import java.nio.channels.*;
+
+import org.apache.hadoop.dfs.DFSFile;
+import org.apache.hadoop.dfs.DF;
+import org.apache.hadoop.dfs.DFSFileInfo;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.UTF8;
+
+/****************************************************************
+ * Implement the NutchFileSystem 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 {
+    TreeMap sharedLockDataSet = new TreeMap();
+    TreeMap nonsharedLockDataSet = new TreeMap();
+    TreeMap lockObjSet = new TreeMap();
+    // by default use copy/delete instead of rename
+    boolean useCopyForRename = true;
+    
+    /**
+     */
+    public LocalFileSystem(Configuration conf) throws IOException {
+        super(conf);
+        // if you find an OS which reliably supports non-POSIX
+        // rename(2) across filesystems / volumes, you can
+        // uncomment this.
+        // String os = System.getProperty("os.name");
+        // if (os.toLowerCase().indexOf("os-with-super-rename") != -1)
+        //     useCopyForRename = false;
+    }
+
+    /**
+     * Return 1x1 'localhost' cell if the file exists.
+     * Return null if otherwise.
+     */
+    public String[][] getFileCacheHints(File f, long start, long len) throws IOException {
+        if (! f.exists()) {
+            return null;
+        } else {
+            String result[][] = new String[1][];
+            result[0] = new String[1];
+            result[0][0] = "localhost";
+            return result;
+        }
+    }
+
+    public String getName() { return "local"; }
+
+    /*******************************************************
+     * For open()'s NFSInputStream
+     *******************************************************/
+    class LocalNFSFileInputStream extends NFSInputStream {
+        FileInputStream fis;
+
+        public LocalNFSFileInputStream(File f) throws IOException {
+          this.fis = new FileInputStream(f);
+        }
+
+        public void seek(long pos) throws IOException {
+          fis.getChannel().position(pos);
+        }
+
+        public long getPos() throws IOException {
+          return fis.getChannel().position();
+        }
+
+        /*
+         * Just forward to the fis
+         */
+        public int available() throws IOException { return fis.available(); }
+        public void close() throws IOException { fis.close(); }
+        public boolean markSupport() { return false; }
+
+        public int read() throws IOException {
+          try {
+            return fis.read();
+          } catch (IOException e) {               // unexpected exception
+            throw new FSError(e);                 // assume native fs error
+          }
+        }
+
+        public int read(byte[] b, int off, int len) throws IOException {
+          try {
+            return fis.read(b, off, len);
+          } catch (IOException e) {               // unexpected exception
+            throw new FSError(e);                 // assume native fs error
+          }
+        }
+
+        public long skip(long n) throws IOException { return fis.skip(n); }
+    }
+    
+    /**
+     * Open the file at f
+     */
+    public NFSInputStream openRaw(File f) throws IOException {
+        if (! f.exists()) {
+            throw new FileNotFoundException(f.toString());
+        }
+        return new LocalNFSFileInputStream(f);
+    }
+
+    /*********************************************************
+     * For create()'s NFSOutputStream.
+     *********************************************************/
+    class LocalNFSFileOutputStream extends NFSOutputStream {
+      FileOutputStream fos;
+
+      public LocalNFSFileOutputStream(File f) throws IOException {
+        this.fos = new FileOutputStream(f);
+      }
+
+      public long getPos() throws IOException {
+        return fos.getChannel().position();
+      }
+
+      /*
+       * Just forward to the fos
+       */
+      public void close() throws IOException { fos.close(); }
+      public void flush() throws IOException { fos.flush(); }
+
+      public void write(byte[] b, int off, int len) throws IOException {
+        try {
+          fos.write(b, off, len);
+        } catch (IOException e) {               // unexpected exception
+          throw new FSError(e);                 // assume native fs error
+        }
+      }
+      public void write(int b) throws IOException {
+        try {
+          fos.write(b);
+        } catch (IOException e) {               // unexpected exception
+          throw new FSError(e);                 // assume native fs error
+        }
+      }
+    }
+
+    public NFSOutputStream createRaw(File f, boolean overwrite)
+      throws IOException {
+        if (f.exists() && ! overwrite) {
+            throw new IOException("File already exists:"+f);
+        }
+        File parent = f.getParentFile();
+        if (parent != null)
+          parent.mkdirs();
+
+        return new LocalNFSFileOutputStream(f);
+    }
+
+    /**
+     * Rename files/dirs
+     */
+    public boolean renameRaw(File src, File dst) throws IOException {
+        if (useCopyForRename) {
+            FileUtil.copyContents(this, src, dst, true, conf);
+            return fullyDelete(src);
+        } else return src.renameTo(dst);
+    }
+
+    /**
+     * Get rid of File f, whether a true file or dir.
+     */
+    public boolean deleteRaw(File f) throws IOException {
+        if (f.isFile()) {
+            return f.delete();
+        } else return fullyDelete(f);
+    }
+
+    /**
+     */
+    public boolean exists(File f) throws IOException {
+        return f.exists();
+    }
+
+    /**
+     */
+    public boolean isDirectory(File f) throws IOException {
+        return f.isDirectory();
+    }
+
+    /**
+     */
+    public long getLength(File f) throws IOException {
+        return f.length();
+    }
+
+    /**
+     */
+    public File[] listFilesRaw(File f) throws IOException {
+        File[] files = f.listFiles();
+        if (files == null) return null;
+        // 20041022, xing, Watch out here:
+        // currently DFSFile.java does not support those methods
+        //    public boolean canRead()
+        //    public boolean canWrite()
+        //    public boolean createNewFile()
+        //    public boolean delete()
+        //    public void deleteOnExit()
+        //    public boolean isHidden()
+        // so you can not rely on returned list for these operations.
+        DFSFile[] nfiles = new DFSFile[files.length];
+        for (int i = 0; i < files.length; i++) {
+            long len = files[i].length();
+            UTF8 name = new UTF8(files[i].toString());
+            DFSFileInfo info = new DFSFileInfo(name, len, len, files[i].isDirectory());
+            nfiles[i] = new DFSFile(info);
+        }
+        return nfiles;
+    }
+
+    /**
+     */
+    public void mkdirs(File f) throws IOException {
+        f.mkdirs();
+    }
+
+    /**
+     * Obtain a filesystem lock at File f.
+     */
+    public synchronized void lock(File f, boolean shared) throws IOException {
+        f.createNewFile();
+
+        FileLock lockObj = null;
+        if (shared) {
+            FileInputStream lockData = new FileInputStream(f);
+            lockObj = lockData.getChannel().lock(0L, Long.MAX_VALUE, shared);
+            sharedLockDataSet.put(f, lockData);
+        } else {
+            FileOutputStream lockData = new FileOutputStream(f);
+            lockObj = lockData.getChannel().lock(0L, Long.MAX_VALUE, shared);
+            nonsharedLockDataSet.put(f, lockData);
+        }
+        lockObjSet.put(f, lockObj);
+    }
+
+    /**
+     * Release a held lock
+     */
+    public synchronized void release(File f) throws IOException {
+        FileLock lockObj = (FileLock) lockObjSet.get(f);
+        FileInputStream sharedLockData = (FileInputStream) sharedLockDataSet.get(f);
+        FileOutputStream nonsharedLockData = (FileOutputStream) nonsharedLockDataSet.get(f);
+
+        if (lockObj == null) {
+            throw new IOException("Given target not held as lock");
+        }
+        if (sharedLockData == null && nonsharedLockData == null) {
+            throw new IOException("Given target not held as lock");
+        }
+
+        lockObj.release();
+        lockObjSet.remove(f);
+        if (sharedLockData != null) {
+            sharedLockData.close();
+            sharedLockDataSet.remove(f);
+        } else {
+            nonsharedLockData.close();
+            nonsharedLockDataSet.remove(f);
+        }
+    }
+
+    /**
+     * In the case of the local filesystem, we can just rename the file.
+     */
+    public void moveFromLocalFile(File src, File dst) throws IOException {
+        if (! src.equals(dst)) {
+            if (useCopyForRename) {
+                FileUtil.copyContents(this, src, dst, true, this.conf);
+                fullyDelete(src);
+            } else src.renameTo(dst);
+        }
+    }
+
+    /**
+     * Similar to moveFromLocalFile(), except the source is kept intact.
+     */
+    public void copyFromLocalFile(File src, File dst) throws IOException {
+        if (! src.equals(dst)) {
+            FileUtil.copyContents(this, src, dst, true, this.conf);
+        }
+    }
+
+    /**
+     * We can't delete the src file in this case.  Too bad.
+     */
+    public void copyToLocalFile(File src, File dst) throws IOException {
+        if (! src.equals(dst)) {
+            FileUtil.copyContents(this, src, dst, true, this.conf);
+        }
+    }
+
+    /**
+     * We can write output directly to the final location
+     */
+    public File startLocalOutput(File nfsOutputFile, File tmpLocalFile) throws IOException {
+        return nfsOutputFile;
+    }
+
+    /**
+     * It's in the right place - nothing to do.
+     */
+    public void completeLocalOutput(File nfsWorkingFile, File tmpLocalFile) throws IOException {
+    }
+
+    /**
+     * We can read directly from the real local fs.
+     */
+    public File startLocalInput(File nfsInputFile, File tmpLocalFile) throws IOException {
+        return nfsInputFile;
+    }
+
+    /**
+     * We're done reading.  Nothing to clean up.
+     */
+    public void completeLocalInput(File localFile) throws IOException {
+        // Ignore the file, it's at the right destination!
+    }
+
+    /**
+     * Shut down the FS.  Not necessary for regular filesystem.
+     */
+    public void close() throws IOException {
+    }
+
+    /**
+     */
+    public String toString() {
+        return "LocalFS";
+    }
+    
+    /**
+     * Implement our own version instead of using the one in FileUtil,
+     * to avoid infinite recursion.
+     * @param dir
+     * @return
+     * @throws IOException
+     */
+    private boolean fullyDelete(File dir) throws IOException {
+        File contents[] = dir.listFiles();
+        if (contents != null) {
+            for (int i = 0; i < contents.length; i++) {
+                if (contents[i].isFile()) {
+                    if (! contents[i].delete()) {
+                        return false;
+                    }
+                } else {
+                    if (! fullyDelete(contents[i])) {
+                        return false;
+                    }
+                }
+            }
+        }
+        return dir.delete();
+    }
+
+    /** 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,
+                                      long start, long length, int crc) {
+      try {
+        // canonicalize f   
+        f = f.getCanonicalFile();
+      
+        // find highest writable parent dir of f on the same device
+        String device = new DF(f.toString()).getMount();
+        File parent = f.getParentFile();
+        File dir;
+        do {
+          dir = parent;
+          parent = parent.getParentFile();
+        } while (parent.canWrite() && parent.toString().startsWith(device));
+
+        // move the file there
+        File badDir = new File(dir, "bad_files");
+        badDir.mkdirs();
+        String suffix = "." + new Random().nextInt();
+        File badFile = new File(badDir,f.getName()+suffix);
+        LOG.warning("Moving bad file " + f + " to " + badFile);
+        in.close();                               // close it first
+        f.renameTo(badFile);                      // rename it
+
+        // move checksum file too
+        File checkFile = getChecksumFile(f);
+        checkFile.renameTo(new File(badDir, checkFile.getName()+suffix));
+
+      } catch (IOException e) {
+        LOG.warning("Error moving bad file " + f + ": " + e);
+      }
+    }
+
+}

+ 320 - 0
src/java/org/apache/hadoop/fs/NDFSFileSystem.java

@@ -0,0 +1,320 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.dfs.*;
+import org.apache.hadoop.conf.Configuration;
+
+/****************************************************************
+ * Implementation of the abstract NutchFileSystem 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 {
+    private static final String HOME_DIR =
+      "/user/" + System.getProperty("user.name") + "/";
+
+    private Random r = new Random();
+    private String name;
+
+    DFSClient dfs;
+
+    /**
+     * Create the ShareSet automatically, and then go on to
+     * the regular constructor.
+     */
+    public DistributedFileSystem(InetSocketAddress namenode, Configuration conf) throws IOException {
+      super(conf);
+      this.dfs = new DFSClient(namenode, conf);
+      this.name = namenode.getHostName() + ":" + namenode.getPort();
+    }
+
+    public String getName() { return name; }
+
+    private UTF8 getPath(File file) {
+      String path = getDFSPath(file);
+      if (!path.startsWith(DFSFile.DFS_FILE_SEPARATOR)) {
+        path = getDFSPath(new File(HOME_DIR, path)); // make absolute
+      }
+      return new UTF8(path);
+    }
+
+    public String[][] getFileCacheHints(File f, long start, long len) throws IOException {
+      return dfs.getHints(getPath(f), start, len);
+    }
+
+    public NFSInputStream openRaw(File f) throws IOException {
+      return dfs.open(getPath(f));
+    }
+
+    public NFSOutputStream createRaw(File f, boolean overwrite)
+      throws IOException {
+      return dfs.create(getPath(f), overwrite);
+    }
+
+    /**
+     * Rename files/dirs
+     */
+    public boolean renameRaw(File src, File dst) throws IOException {
+      return dfs.rename(getPath(src), getPath(dst));
+    }
+
+    /**
+     * Get rid of File f, whether a true file or dir.
+     */
+    public boolean deleteRaw(File f) throws IOException {
+        return dfs.delete(getPath(f));
+    }
+
+    /**
+     */
+    public boolean exists(File f) throws IOException {
+        return dfs.exists(getPath(f));
+    }
+
+    /**
+     */
+    public boolean isDirectory(File f) throws IOException {
+        return dfs.isDirectory(getPath(f));
+    }
+
+    /**
+     */
+    public long getLength(File f) throws IOException {
+        DFSFileInfo info[] = dfs.listFiles(getPath(f));
+        return info[0].getLen();
+    }
+
+    /**
+     */
+    public File[] listFilesRaw(File f) throws IOException {
+        DFSFileInfo info[] = dfs.listFiles(getPath(f));
+        if (info == null) {
+            return new File[0];
+        } else {
+            File results[] = new DFSFile[info.length];
+            for (int i = 0; i < info.length; i++) {
+                results[i] = new DFSFile(info[i]);
+            }
+            return results;
+        }
+    }
+
+    /**
+     */
+    public void mkdirs(File f) throws IOException {
+        dfs.mkdirs(getPath(f));
+    }
+
+    /**
+     * Obtain a filesystem lock at File f.
+     */
+    public void lock(File f, boolean shared) throws IOException {
+        dfs.lock(getPath(f), ! shared);
+    }
+
+    /**
+     * Release a held lock
+     */
+    public void release(File f) throws IOException {
+        dfs.release(getPath(f));
+    }
+
+    /**
+     * Remove the src when finished.
+     */
+    public void moveFromLocalFile(File src, File dst) throws IOException {
+        doFromLocalFile(src, dst, true);
+    }
+
+    /**
+     * keep the src when finished.
+     */
+    public void copyFromLocalFile(File src, File dst) throws IOException {
+        doFromLocalFile(src, dst, false);
+    }
+
+    private void doFromLocalFile(File src, File dst, boolean deleteSource) throws IOException {
+        if (exists(dst)) {
+            if (! isDirectory(dst)) {
+                throw new IOException("Target " + dst + " already exists");
+            } else {
+                dst = new File(dst, src.getName());
+                if (exists(dst)) {
+                    throw new IOException("Target " + dst + " already exists");
+                }
+            }
+        }
+
+        if (src.isDirectory()) {
+            mkdirs(dst);
+            File contents[] = src.listFiles();
+            for (int i = 0; i < contents.length; i++) {
+                doFromLocalFile(contents[i], new File(dst, contents[i].getName()), deleteSource);
+            }
+        } else {
+            byte buf[] = new byte[this.conf.getInt("io.file.buffer.size", 4096)];
+            InputStream in = new BufferedInputStream(new FileInputStream(src));
+            try {
+                OutputStream out = create(dst);
+                try {
+                    int bytesRead = in.read(buf);
+                    while (bytesRead >= 0) {
+                        out.write(buf, 0, bytesRead);
+                        bytesRead = in.read(buf);
+                    }
+                } finally {
+                    out.close();
+                }
+            } finally {
+                in.close();
+            } 
+        }
+        if (deleteSource)
+            src.delete();
+    }
+
+    /**
+     * Takes a hierarchy of files from the NFS system and writes to
+     * the given local target.
+     */
+    public void copyToLocalFile(File src, File dst) throws IOException {
+        if (dst.exists()) {
+            if (! dst.isDirectory()) {
+                throw new IOException("Target " + dst + " already exists");
+            } else {
+                dst = new File(dst, src.getName());
+                if (dst.exists()) {
+                    throw new IOException("Target " + dst + " already exists");
+                }
+            }
+        }
+        dst = dst.getCanonicalFile();
+
+        if (isDirectory(src)) {
+            dst.mkdirs();
+            File contents[] = listFiles(src);
+            for (int i = 0; i < contents.length; i++) {
+                copyToLocalFile(contents[i], new File(dst, contents[i].getName()));
+            }
+        } else {
+            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);
+                try {
+                    int bytesRead = in.read(buf);
+                    while (bytesRead >= 0) {
+                        out.write(buf, 0, bytesRead);
+                        bytesRead = in.read(buf);
+                    }
+                } finally {
+                    out.close();
+                }
+            } finally {
+                in.close();
+            } 
+        }
+    }
+
+    /**
+     * 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);
+        }
+        return tmpLocalFile;
+    }
+
+    /**
+     * Move completed local data to DFS destination
+     */
+    public void completeLocalOutput(File nfsOutputFile, File tmpLocalFile) throws IOException {
+        moveFromLocalFile(tmpLocalFile, nfsOutputFile);
+    }
+
+    /**
+     * Fetch remote DFS file, place at tmpLocalFile
+     */
+    public File startLocalInput(File nfsInputFile, File tmpLocalFile) throws IOException {
+        copyToLocalFile(nfsInputFile, tmpLocalFile);
+        return tmpLocalFile;
+    }
+
+    /**
+     * We're done with the local stuff, so delete it
+     */
+    public void completeLocalInput(File localFile) throws IOException {
+        // Get rid of the local copy - we don't need it anymore.
+        FileUtil.fullyDelete(localFile, this.conf);
+    }
+
+    /**
+     * Shut down the FS.  Not necessary for regular filesystem.
+     */
+    public void close() throws IOException {
+        dfs.close();
+    }
+
+    /**
+     */
+    public String toString() {
+        return "DFS[" + dfs + "]";
+    }
+
+    /**
+     */
+    public DFSClient getClient() {
+        return dfs;
+    }
+    
+    private String getDFSPath(File f) {
+      List l = new ArrayList();
+      l.add(f.getName());
+      File parent = f.getParentFile();
+      while (parent != null) {
+        l.add(parent.getName());
+        parent = parent.getParentFile();
+      }
+      StringBuffer path = new StringBuffer();
+      path.append(l.get(l.size() - 1));
+      for (int i = l.size() - 2; i >= 0; i--) {
+        path.append(DFSFile.DFS_FILE_SEPARATOR);
+        path.append(l.get(i));
+      }
+      return path.toString();
+    }
+
+    public void reportChecksumFailure(File f, NFSInputStream in,
+                                      long start, long length, int crc) {
+      
+      // ignore for now, causing task to fail, and hope that when task is
+      // retried it gets a different copy of the block that is not corrupt.
+
+      // FIXME: we should move the bad block(s) involved to a bad block
+      // directory on their datanode, and then re-replicate the blocks, so that
+      // no data is lost. a task may fail, but on retry it should succeed.
+    }
+}

+ 263 - 0
src/java/org/apache/hadoop/fs/NDFSShell.java

@@ -0,0 +1,263 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * 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.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+
+/**************************************************
+ * This class provides some DFS administrative access.
+ *
+ * @author Mike Cafarella
+ **************************************************/
+public class DFSShell {
+    NutchFileSystem nfs;
+
+    /**
+     */
+    public DFSShell(NutchFileSystem nfs) {
+        this.nfs = nfs;
+    }
+
+    /**
+     * 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));
+    }
+
+    /**
+     * 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));
+    }
+
+    /**
+     * Obtain the indicated DFS file and copy to the local name.
+     * srcf is kept.
+     */
+    void copyToLocal(String srcf, File dst) throws IOException {
+        nfs.copyToLocalFile(new File(srcf), dst);
+    }
+
+    /**
+     * Obtain the indicated DFS file and copy to the local name.
+     * srcf is removed.
+     */
+    void moveToLocal(String srcf, File dst) throws IOException {
+        System.err.println("Option '-moveToLocal' is not implemented yet.");
+    }
+
+    /**
+     * 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));
+        if (items == null) {
+            System.out.println("Could not get listing for " + src);
+        } else {
+            System.out.println("Found " + items.length + " items");
+            for (int i = 0; i < items.length; i++) {
+                File cur = items[i];
+                System.out.println(cur.getPath() + "\t" + (cur.isDirectory() ? "<dir>" : ("" + cur.length())));
+            }
+        }
+    }
+
+    /**
+     */
+    public void du(String src) throws IOException {
+        File items[] = nfs.listFiles(new File(src));
+        if (items == null) {
+            System.out.println("Could not get listing for " + src);
+        } else {
+            System.out.println("Found " + items.length + " items");
+            for (int i = 0; i < items.length; i++) {
+                DFSFile cur = (DFSFile) items[i];
+                System.out.println(cur.getPath() + "\t" + cur.getContentsLength());
+            }
+        }
+    }
+
+    /**
+     * Create the given dir
+     */
+    public void mkdir(String src) throws IOException {
+        File f = new File(src);
+        nfs.mkdirs(f);
+    }
+    
+    /**
+     * Rename an DFS file
+     */
+    public void rename(String srcf, String dstf) throws IOException {
+        if (nfs.rename(new File(srcf), new File(dstf))) {
+            System.out.println("Renamed " + srcf + " to " + dstf);
+        } else {
+            System.out.println("Rename failed");
+        }
+    }
+
+    /**
+     * 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)) {
+            System.out.println("Copied " + srcf + " to " + dstf);
+        } else {
+            System.out.println("Copy failed");
+        }
+    }
+
+    /**
+     * Delete an DFS file
+     */
+    public void delete(String srcf) throws IOException {
+        if (nfs.delete(new File(srcf))) {
+            System.out.println("Deleted " + srcf);
+        } else {
+            System.out.println("Delete failed");
+        }
+    }
+
+    /**
+     * Return an abbreviated English-language desc of the byte length
+     */
+    String byteDesc(long len) {
+        double val = 0.0;
+        String ending = "";
+        if (len < 1024 * 1024) {
+            val = (1.0 * len) / 1024;
+            ending = " k";
+        } else if (len < 1024 * 1024 * 1024) {
+            val = (1.0 * len) / (1024 * 1024);
+            ending = " Mb";
+        } else {
+            val = (1.0 * len) / (1024 * 1024 * 1024);
+            ending = " Gb";
+        }
+        return limitDecimal(val, 2) + ending;
+    }
+
+    String limitDecimal(double d, int placesAfterDecimal) {
+        String strVal = Double.toString(d);
+        int decpt = strVal.indexOf(".");
+        if (decpt >= 0) {
+            strVal = strVal.substring(0, Math.min(strVal.length(), decpt + 1 + placesAfterDecimal));
+        }
+        return strVal;
+    }
+
+    /**
+     * Gives a report on how the NutchFileSystem is doing
+     */
+    public void report() throws IOException {
+        if (nfs instanceof DistributedFileSystem) {
+            DistributedFileSystem dfsfs = (DistributedFileSystem) nfs;
+            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("/"));
+            for (int i = 0; i < topItems.length; i++) {
+                DFSFile cur = (DFSFile) topItems[i];
+                totalEffectiveBytes += cur.getContentsLength();
+            }
+
+            System.out.println("Total raw bytes: " + total + " (" + byteDesc(total) + ")");
+            System.out.println("Used raw bytes: " + used + " (" + byteDesc(used) + ")");
+            System.out.println("% used: " + limitDecimal(((1.0 * used) / total) * 100, 2) + "%");
+            System.out.println();
+            System.out.println("Total effective bytes: " + totalEffectiveBytes + " (" + byteDesc(totalEffectiveBytes) + ")");
+            System.out.println("Effective replication multiplier: " + (1.0 * used / totalEffectiveBytes));
+
+            System.out.println("-------------------------------------------------");
+            System.out.println("Datanodes available: " + info.length);
+            System.out.println();
+            for (int i = 0; i < info.length; i++) {
+                System.out.println("Name: " + info[i].getName().toString());
+                long c = info[i].getCapacity();
+                long r = info[i].getRemaining();
+                long u = c - r;
+                System.out.println("Total raw bytes: " + c + " (" + byteDesc(c) + ")");
+                System.out.println("Used raw bytes: " + u + " (" + byteDesc(u) + ")");
+                System.out.println("% used: " + limitDecimal(((1.0 * u) / c) * 100, 2) + "%");
+                System.out.println("Last contact with namenode: " + new Date(info[i].lastUpdate()));
+                System.out.println();
+            }
+        }
+    }
+
+    /**
+     * main() has some simple utility methods
+     */
+    public static void main(String argv[]) throws IOException {
+        if (argv.length < 1) {
+            System.out.println("Usage: java DFSShell [-local | -dfs <namenode:port>]" +
+                    " [-ls <path>] [-du <path>] [-mv <src> <dst>] [-cp <src> <dst>] [-rm <src>]" +
+                    " [-put <localsrc> <dst>] [-copyFromLocal <localsrc> <dst>] [-moveFromLocal <localsrc> <dst>]" + 
+                    " [-get <src> <localdst>] [-copyToLocal <src> <localdst>] [-moveToLocal <src> <localdst>]" +
+                    " [-mkdir <path>] [-report]");
+            return;
+        }
+
+        Configuration conf = new Configuration();
+        int i = 0;
+        NutchFileSystem nfs = NutchFileSystem.parseArgs(argv, i, conf);
+        try {
+            DFSShell tc = new DFSShell(nfs);
+
+            String cmd = argv[i++];
+            if ("-put".equals(cmd) || "-copyFromLocal".equals(cmd)) {
+                tc.copyFromLocal(new File(argv[i++]), argv[i++]);
+            } else if ("-moveFromLocal".equals(cmd)) {
+                tc.moveFromLocal(new File(argv[i++]), argv[i++]);
+            } else if ("-get".equals(cmd) || "-copyToLocal".equals(cmd)) {
+                tc.copyToLocal(argv[i++], new File(argv[i++]));
+            } else if ("-moveToLocal".equals(cmd)) {
+                tc.moveToLocal(argv[i++], new File(argv[i++]));
+            } else if ("-ls".equals(cmd)) {
+                String arg = i < argv.length ? argv[i++] : "";
+                tc.ls(arg);
+            } else if ("-mv".equals(cmd)) {
+                tc.rename(argv[i++], argv[i++]);
+            } else if ("-cp".equals(cmd)) {
+                tc.copy(argv[i++], argv[i++], conf);
+            } else if ("-rm".equals(cmd)) {
+                tc.delete(argv[i++]);
+            } else if ("-du".equals(cmd)) {
+                String arg = i < argv.length ? argv[i++] : "";
+                tc.du(arg);
+            } else if ("-mkdir".equals(cmd)) {
+                tc.mkdir(argv[i++]);
+            } else if ("-report".equals(cmd)) {
+                tc.report();
+            }
+            System.exit(0);
+        } finally {
+            nfs.close();
+        }
+    }
+}

+ 249 - 0
src/java/org/apache/hadoop/fs/NFSDataInputStream.java

@@ -0,0 +1,249 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.
+
+import java.io.*;
+import java.util.Arrays;
+import java.util.logging.*;
+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}
+ * and buffers input through a {@link BufferedInputStream}. */
+public class NFSDataInputStream 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 int HEADER_LENGTH = 8;
+  
+  private int bytesPerSum = 1;
+  
+  /** Verify that data matches checksums. */
+  private class Checker extends FilterInputStream implements Seekable {
+    private NutchFileSystem fs;
+    private File file;
+    private NFSDataInputStream sums;
+    private Checksum sum = new CRC32();
+    private int inSum;
+
+    public Checker(NutchFileSystem fs, File file, Configuration conf)
+      throws IOException {
+      super(fs.openRaw(file));
+      
+      this.fs = fs;
+      this.file = file;
+      File sumFile = fs.getChecksumFile(file);
+      try {
+        this.sums = new NFSDataInputStream(fs.openRaw(sumFile), conf);
+        byte[] version = new byte[VERSION.length];
+        sums.readFully(version);
+        if (!Arrays.equals(version, VERSION))
+          throw new IOException("Not a checksum file: "+sumFile);
+        bytesPerSum = sums.readInt();
+      } catch (FileNotFoundException e) {         // quietly ignore
+        stopSumming();
+      } catch (IOException e) {                   // loudly ignore
+        LOG.warning("Problem opening checksum file: "+ file + ".  Ignoring with exception " + e + ".");
+        stopSumming();
+      }
+    }
+
+    public void seek(long desired) throws IOException {
+      ((Seekable)in).seek(desired);
+      if (sums != null) {
+        if (desired % bytesPerSum != 0)
+          throw new IOException("Seek to non-checksummed position.");
+        try {
+          sums.seek(HEADER_LENGTH + 4*(desired/bytesPerSum));
+        } catch (IOException e) {
+          LOG.warning("Problem seeking checksum file: "+e+". Ignoring.");
+          stopSumming();
+        }
+        sum.reset();
+        inSum = 0;
+      }
+    }
+    
+    public int read(byte b[], int off, int len) throws IOException {
+      int read = in.read(b, off, len);
+
+      if (sums != null) {
+        int summed = 0;
+        while (summed < read) {
+          
+          int goal = bytesPerSum - inSum;
+          int inBuf = read - summed;
+          int toSum = inBuf <= goal ? inBuf : goal;
+          
+          sum.update(b, off+summed, toSum);
+          summed += toSum;
+          
+          inSum += toSum;
+          if (inSum == bytesPerSum) {
+            verifySum(read-(summed-bytesPerSum));
+          }
+        }
+      }
+        
+      return read;
+    }
+
+    private void verifySum(int delta) throws IOException {
+      int crc;
+      try {
+        crc = sums.readInt();
+      } catch (IOException e) {
+        LOG.warning("Problem reading checksum file: "+e+". Ignoring.");
+        stopSumming();
+        return;
+      }
+      int sumValue = (int)sum.getValue();
+      sum.reset();
+      inSum = 0;
+      if (crc != sumValue) {
+        long pos = getPos() - delta;
+        fs.reportChecksumFailure(file, (NFSInputStream)in,
+                                 pos, bytesPerSum, crc);
+        throw new ChecksumException("Checksum error: "+file+" at "+pos);
+      }
+    }
+
+    public long getPos() throws IOException {
+      return ((NFSInputStream)in).getPos();
+    }
+
+    public void close() throws IOException {
+      super.close();
+      stopSumming();
+    }
+
+    private void stopSumming() {
+      if (sums != null) {
+        try {
+          sums.close();
+        } catch (IOException f) {}
+        sums = null;
+        bytesPerSum = 1;
+      }
+    }
+  }
+
+  /** Cache the file position.  This improves performance significantly.*/
+  private static class PositionCache extends FilterInputStream {
+    long position;
+
+    public PositionCache(InputStream in) throws IOException {
+      super(in);
+    }
+
+    // This is the only read() method called by BufferedInputStream, so we trap
+    // calls to it in order to cache the position.
+    public int read(byte b[], int off, int len) throws IOException {
+      int result = in.read(b, off, len);
+      position += result;
+      return result;
+    }
+
+    public void seek(long desired) throws IOException {
+      ((Seekable)in).seek(desired);               // seek underlying stream
+      position = desired;                         // update position
+    }
+      
+    public long getPos() throws IOException {
+      return position;                            // return cached position
+    }
+    
+  }
+
+  /** Buffer input.  This improves performance significantly.*/
+  private class Buffer extends BufferedInputStream {
+    public Buffer(PositionCache in, int bufferSize)
+      throws IOException {
+      super(in, bufferSize);
+    }
+
+    public void seek(long desired) throws IOException {
+      long end = ((PositionCache)in).getPos();
+      long start = end - this.count;
+      if (desired >= start && desired < end) {
+        this.pos = (int)(desired - start);        // can position within buffer
+      } else {
+        this.count = 0;                           // invalidate buffer
+        this.pos = 0;
+
+        long delta = desired % bytesPerSum;
+        
+        // seek to last checksummed point, if any
+        ((PositionCache)in).seek(desired - delta);
+
+        // scan to desired position
+        for (int i = 0; i < delta; i++) {
+          read();
+        }
+      }
+
+    }
+      
+    public long getPos() throws IOException {     // adjust for buffer
+      return ((PositionCache)in).getPos() - (this.count - this.pos);
+    }
+
+    // optimized version of read()
+    public int read() throws IOException {
+      if (pos >= count)
+        return super.read();
+      return buf[pos++] & 0xff;
+    }
+
+}
+  
+  
+  public NFSDataInputStream(NutchFileSystem 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)
+    throws IOException {
+    super(null);
+    int bufferSize = conf.getInt("io.file.buffer.size", 4096);
+    this.in = new Buffer(new PositionCache(new Checker(fs, file, conf)), bufferSize);
+  }
+    
+  /** Construct without checksums. */
+  public NFSDataInputStream(NFSInputStream in, Configuration conf) throws IOException {
+    this(in, conf.getInt("io.file.buffer.size", 4096));
+  }
+  /** Construct without checksums. */
+  public NFSDataInputStream(NFSInputStream in, int bufferSize)
+    throws IOException {
+    super(null);
+    this.in = new Buffer(new PositionCache(in), bufferSize);
+  }
+  
+  public void seek(long desired) throws IOException {
+    ((Buffer)in).seek(desired);
+  }
+
+  public long getPos() throws IOException {
+    return ((Buffer)in).getPos();
+  }
+
+}

+ 146 - 0
src/java/org/apache/hadoop/fs/NFSDataOutputStream.java

@@ -0,0 +1,146 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+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},
+ * buffers output through a {@link BufferedOutputStream} and creates a checksum
+ * file. */
+public class NFSDataOutputStream 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 Checksum sum = new CRC32();
+    private int inSum;
+    private int bytesPerSum;
+
+    public Summer(NutchFileSystem 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);
+
+      sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
+      sums.writeInt(this.bytesPerSum);
+    }
+
+    public void write(byte b[], int off, int len) throws IOException {
+      int summed = 0;
+      while (summed < len) {
+
+        int goal = this.bytesPerSum - inSum;
+        int inBuf = len - summed;
+        int toSum = inBuf <= goal ? inBuf : goal;
+
+        sum.update(b, off+summed, toSum);
+        summed += toSum;
+
+        inSum += toSum;
+        if (inSum == this.bytesPerSum) {
+          writeSum();
+        }
+      }
+
+      out.write(b, off, len);
+    }
+
+    private void writeSum() throws IOException {
+      if (inSum != 0) {
+        sums.writeInt((int)sum.getValue());
+        sum.reset();
+        inSum = 0;
+      }
+    }
+
+    public void close() throws IOException {
+      writeSum();
+      sums.close();
+      super.close();
+    }
+
+  }
+
+  private static class PositionCache extends FilterOutputStream {
+    long position;
+
+    public PositionCache(OutputStream out) throws IOException {
+      super(out);
+    }
+
+    // This is the only write() method called by BufferedOutputStream, so we
+    // trap calls to it in order to cache the position.
+    public void write(byte b[], int off, int len) throws IOException {
+      out.write(b, off, len);
+      position += len;                            // update position
+    }
+      
+    public long getPos() throws IOException {
+      return position;                            // return cached position
+    }
+    
+  }
+
+  private static class Buffer extends BufferedOutputStream {
+    public Buffer(OutputStream out, int bufferSize) throws IOException {
+      super(out, bufferSize);
+    }
+
+    public long getPos() throws IOException {
+      return ((PositionCache)out).getPos() + this.count;
+    }
+
+    // optimized version of write(int)
+    public void write(int b) throws IOException {
+      if (count >= buf.length) {
+        super.write(b);
+      } else {
+        buf[count++] = (byte)b;
+      }
+    }
+
+  }
+
+  public NFSDataOutputStream(NutchFileSystem fs, File file,
+                             boolean overwrite, Configuration conf)
+    throws IOException {
+    super(new Buffer(new PositionCache(new Summer(fs, file, overwrite, conf)),
+            conf.getInt("io.file.buffer.size", 4096)));
+  }
+
+  /** Construct without checksums. */
+  public NFSDataOutputStream(NFSOutputStream out, Configuration conf) throws IOException {
+    this(out, conf.getInt("io.file.buffer.size", 4096));
+  }
+
+  /** Construct without checksums. */
+  public NFSDataOutputStream(NFSOutputStream out, int bufferSize)
+    throws IOException {
+    super(new Buffer(new PositionCache(out), bufferSize));
+  }
+
+  public long getPos() throws IOException {
+    return ((Buffer)out).getPos();
+  }
+
+}

+ 38 - 0
src/java/org/apache/hadoop/fs/NFSInputStream.java

@@ -0,0 +1,38 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.
+
+import java.io.*;
+
+/****************************************************************
+ * NFSInputStream 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 {
+    /**
+     * Seek to the given offset from the start of the file.
+     * The next read() will be from that location.  Can't
+     * seek past the end of the file.
+     */
+    public abstract void seek(long pos) throws IOException;
+
+    /**
+     * Return the current offset from the start of the file
+     */
+    public abstract long getPos() throws IOException;
+}

+ 30 - 0
src/java/org/apache/hadoop/fs/NFSOutputStream.java

@@ -0,0 +1,30 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.
+
+import java.io.*;
+
+/****************************************************************
+ * NFSOutputStream is an OutputStream that can track its position.
+ *
+ * @author Mike Cafarella
+ *****************************************************************/
+public abstract class NFSOutputStream extends OutputStream {
+    /**
+     * Return the current offset from the start of the file
+     */
+    public abstract long getPos() throws IOException;
+}

+ 390 - 0
src/java/org/apache/hadoop/fs/NutchFileSystem.java

@@ -0,0 +1,390 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.logging.*;
+
+import org.apache.hadoop.dfs.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.util.LogFormatter;
+
+/****************************************************************
+ * An abstract base class for a fairly simple
+ * distributed file system.
+ * A Nutch installation might consist
+ * of multiple machines, which should swap files transparently.
+ * This interface allows other Nutch systems to find and place
+ * files into the distributed Nutch-controlled file world.
+ * <p>
+ * A local implementation exists for testing and for small Nutch instances.
+ * <p>
+ * The standard job of NutchFileSystem is to take the location-
+ * independent NutchFile objects, and resolve them using local
+ * knowledge and local instances of ShareGroup.
+ * <p>
+ * The local implementation is {@link LocalFileSystem} and distributed
+ * implementation is {@link DistributedFileSystem}.
+ * @author Mike Cafarella
+ *****************************************************************/
+public abstract class NutchFileSystem {
+    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.dfs.DistributedFileSystem");
+
+    private static final HashMap NAME_TO_FS = new HashMap();
+    /**
+     * Parse the cmd-line args, starting at i.  Remove consumed args
+     * from array.  We expect param in the form:
+     * '-local | -dfs <namenode:port>'
+     *
+     * @deprecated use fs.default.name config option instead
+     */
+    public static NutchFileSystem 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;
+        String cmd = argv[i];
+        if ("-dfs".equals(cmd)) {
+            i++;
+            InetSocketAddress addr = DataNode.createSocketAddr(argv[i++]);
+            nfs = new DistributedFileSystem(addr, conf);
+        } else if ("-local".equals(cmd)) {
+            i++;
+            nfs = new LocalFileSystem(conf);
+        } else {
+            nfs = get(conf);                          // using default
+            LOG.info("No FS indicated, using default:"+nfs.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;
+    }
+
+    /** Returns the configured filesystem implementation.*/
+    public static NutchFileSystem 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).*/
+    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);
+      int ioFileBufferSize = conf.getInt("io.file.buffer.size", 4096);
+      if (fs == null) {
+        if ("local".equals(name)) {
+          fs = new LocalFileSystem(conf);
+        } else {
+          fs = new DistributedFileSystem(DataNode.createSocketAddr(name), conf);
+        }
+        NAME_TO_FS.put(name, fs);
+      }
+      return fs;
+    }
+
+    /** Return the name of the checksum file associated with a file.*/
+    public static File getChecksumFile(File file) {
+      return new File(file.getParentFile(), "."+file.getName()+".crc");
+    }
+
+    /** Return true iff file is a checksum file name.*/
+    public static boolean isChecksumFile(File file) {
+      String name = file.getName();
+      return name.startsWith(".") && name.endsWith(".crc");
+    }
+
+    ///////////////////////////////////////////////////////////////
+    // NutchFileSystem
+    ///////////////////////////////////////////////////////////////
+    /**
+     */
+    public NutchFileSystem(Configuration conf) {
+        this.conf = conf;
+    }
+
+    /**
+     * Return a 2D array of size 1x1 or greater, containing hostnames 
+     * where portions of the given file can be found.  For a nonexistent 
+     * file or regions, null will be returned.
+     *
+     * 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'.
+     */
+    public abstract String[][] getFileCacheHints(File f, long start, long len) throws IOException;
+
+    /**
+     * Opens an NFSDataInputStream 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);
+    }
+    
+    /**
+     * Opens an NFSDataInputStream 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);
+    }
+
+    /**
+     * Opens an InputStream for the indicated File, whether local
+     * or via DFS.
+     */
+    public abstract NFSInputStream openRaw(File f) throws IOException;
+
+    /**
+     * Opens an NFSDataOutputStream at the indicated File.
+     * Files are overwritten by default.
+     */
+    public NFSDataOutputStream create(File f) throws IOException {
+      return create(f, true,this.conf.getInt("io.file.buffer.size", 4096));
+    }
+
+    /**
+     * Opens an NFSDataOutputStream 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,
+                                      int bufferSize) throws IOException {
+      return new NFSDataOutputStream(this, f, overwrite, this.conf);
+    }
+
+    /** Opens an OutputStream 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.
+     */
+    public abstract NFSOutputStream createRaw(File f, boolean overwrite)
+      throws IOException;
+
+    /**
+     * Creates the given File as a brand-new zero-length file.  If
+     * create fails, or if it already existed, return false.
+     */
+    public boolean createNewFile(File f) throws IOException {
+        if (exists(f)) {
+            return false;
+        } else {
+            OutputStream out = createRaw(f, false);
+            try {
+            } finally {
+              out.close();
+            }
+            return true;
+        }
+    }
+
+    /**
+     * Renames File src to File dst.  Can take place on local fs
+     * or remote DFS.
+     */
+    public boolean rename(File src, File dst) throws IOException {
+      if (isDirectory(src)) {
+        return renameRaw(src, dst);
+      } else {
+
+        boolean value = renameRaw(src, dst);
+
+        File checkFile = getChecksumFile(src);
+        if (exists(checkFile))
+          renameRaw(checkFile, getChecksumFile(dst)); // try to rename checksum
+
+        return value;
+      }
+      
+    }
+
+    /**
+     * Renames File src to File dst.  Can take place on local fs
+     * or remote DFS.
+     */
+    public abstract boolean renameRaw(File src, File dst) throws IOException;
+
+    /**
+     * Deletes File
+     */
+    public boolean delete(File f) throws IOException {
+      if (isDirectory(f)) {
+        return deleteRaw(f);
+      } else {
+        deleteRaw(getChecksumFile(f));            // try to delete checksum
+        return deleteRaw(f);
+      }
+    }
+
+    /**
+     * Deletes File
+     */
+    public abstract boolean deleteRaw(File f) throws IOException;
+
+    /**
+     * Check if exists
+     */
+    public abstract boolean exists(File f) throws IOException;
+
+    public abstract boolean isDirectory(File f) throws IOException;
+
+    public boolean isFile(File f) throws IOException {
+        if (exists(f) && ! isDirectory(f)) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+    
+    public abstract long getLength(File f) throws IOException;
+
+    public File[] listFiles(File f) throws IOException {
+      return listFiles(f, new FileFilter() {
+          public boolean accept(File file) {
+            return !isChecksumFile(file);
+          }
+        });
+    }
+
+    public abstract File[] listFilesRaw(File f) throws IOException;
+
+    public File[] listFiles(File f, FileFilter filter) throws IOException {
+        Vector results = new Vector();
+        File listing[] = listFilesRaw(f);
+        if (listing != null) {
+          for (int i = 0; i < listing.length; i++) {
+            if (filter.accept(listing[i])) {
+              results.add(listing[i]);
+            }
+          }
+        }
+        return (File[]) results.toArray(new File[results.size()]);
+    }
+
+    /**
+     * Make the given file and all non-existent parents into
+     * directories.
+     */
+    public abstract void mkdirs(File f) throws IOException;
+
+    /**
+     * Obtain a lock on the given File
+     */
+    public abstract void lock(File f, boolean shared) throws IOException;
+
+    /**
+     * Release the lock
+     */
+    public abstract void release(File f) throws IOException;
+
+    /**
+     * The src file is on the local disk.  Add it to NFS 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 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.
+     */
+    public abstract void copyToLocalFile(File src, File dst) throws IOException;
+
+    /**
+     * the same as copyToLocalFile(File src, File dst), except that
+     * the source is removed afterward.
+     */
+    // not implemented yet
+    //public abstract void moveToLocalFile(File src, File dst) throws IOException;
+
+    /**
+     * 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.
+     */
+    public abstract File startLocalOutput(File nfsOutputFile, File tmpLocalFile) throws IOException;
+
+    /**
+     * Called when we're all done writing to the target.  A local NFS 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.
+     */
+    public abstract void completeLocalOutput(File nfsOutputFile, 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.
+     */
+    public abstract File startLocalInput(File nfsInputFile, File tmpLocalFile) throws IOException;
+
+    /**
+     * Called when we're all done writing to the target.  A local NFS 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.
+     */
+    public abstract void completeLocalInput(File localFile) throws IOException;
+
+    /**
+     * No more filesystem operations are needed.  Will
+     * release any held locks.
+     */
+    public abstract void close() throws IOException;
+
+    /**
+     * Report a checksum error to the file system.
+     * @param f the file name containing the error
+     * @param in the stream open on the file
+     * @param start the position of the beginning of the bad data in the file
+     * @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,
+                                               long start, long length,
+                                               int crc);
+
+}

+ 28 - 0
src/java/org/apache/hadoop/fs/Seekable.java

@@ -0,0 +1,28 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.
+
+import java.io.*;
+
+/* Stream which permits seeking. */
+public interface Seekable {
+  /**
+   * Seek to the given offset from the start of the file.
+   * The next read() will be from that location.  Can't
+   * seek past the end of the file.
+   */
+  void seek(long pos) throws IOException;
+}

+ 79 - 0
src/java/org/apache/hadoop/io/ArrayFile.java

@@ -0,0 +1,79 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.*;
+
+/** A dense file-based mapping from integers to values. */
+public class ArrayFile extends MapFile {
+
+  protected ArrayFile() {}                            // no public ctor
+
+  /** Write a new array file. */
+  public static class Writer extends MapFile.Writer {
+    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);
+    }
+
+    /** Append a value to the file. */
+    public synchronized void append(Writable value) throws IOException {
+      super.append(count, value);                 // add to map
+      count.set(count.get()+1);                   // increment count
+    }
+  }
+
+  /** Provide access to an existing array file. */
+  public static class Reader extends MapFile.Reader {
+    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);
+    }
+
+    /** Positions the reader before its <code>n</code>th value. */
+    public synchronized void seek(long n) throws IOException {
+      key.set(n);
+      seek(key);
+    }
+
+    /** Read and return the next value in the file. */
+    public synchronized Writable next(Writable value) throws IOException {
+      return next(key, value) ? value : null;
+    }
+
+    /** Returns the key associated with the most recent call to {@link
+     * #seek(long)}, {@link #next(Writable)}, or {@link
+     * #get(long,Writable)}. */
+    public synchronized long key() throws IOException {
+      return key.get();
+    }
+
+    /** Return the <code>n</code>th value in the file. */
+    public synchronized Writable get(long n, Writable value)
+      throws IOException {
+      key.set(n);
+      return get(key, value);
+    }
+  }
+
+}

+ 102 - 0
src/java/org/apache/hadoop/io/ArrayWritable.java

@@ -0,0 +1,102 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+import java.lang.reflect.Array;
+
+/** A Writable for arrays containing instances of a class. */
+public class ArrayWritable implements Writable {
+  private Class valueClass;
+  private Writable[] values;
+
+  public ArrayWritable() {
+    this.valueClass = null;
+  }
+
+  public ArrayWritable(Class valueClass) {
+    this.valueClass = valueClass;
+  }
+
+  public ArrayWritable(Class valueClass, Writable[] values) {
+    this(valueClass);
+    this.values = values;
+  }
+
+  public ArrayWritable(String[] strings) {
+    this(UTF8.class, new Writable[strings.length]);
+    for (int i = 0; i < strings.length; i++) {
+      values[i] = new UTF8(strings[i]);
+    }
+  }
+
+  public void setValueClass(Class valueClass) {
+    if (valueClass != this.valueClass) {
+        this.valueClass = valueClass;
+        this.values = null;
+    }
+  }
+  
+  public Class getValueClass() {
+    return valueClass;
+  }
+
+  public String[] toStrings() {
+    String[] strings = new String[values.length];
+    for (int i = 0; i < values.length; i++) {
+      strings[i] = values[i].toString();
+    }
+    return strings;
+  }
+
+  public Object toArray() {
+    Object result = Array.newInstance(valueClass, values.length);
+    for (int i = 0; i < values.length; i++) {
+      Array.set(result, i, values[i]);
+    }
+    return result;
+  }
+
+  public void set(Writable[] values) { this.values = values; }
+
+  public Writable[] get() { return values; }
+
+  public void readFields(DataInput in) throws IOException {
+    values = new Writable[in.readInt()];          // construct values
+    for (int i = 0; i < values.length; i++) {
+      Writable value;                             // construct value
+      try {
+        value = (Writable)valueClass.newInstance();
+      } catch (InstantiationException e) {
+        throw new RuntimeException(e.toString());
+      } catch (IllegalAccessException e) {
+        throw new RuntimeException(e.toString());
+      }
+      value.readFields(in);                       // read a value
+      values[i] = value;                          // store it in values
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(values.length);                 // write values
+    for (int i = 0; i < values.length; i++) {
+      values[i].write(out);
+    }
+  }
+
+}
+

+ 107 - 0
src/java/org/apache/hadoop/io/BooleanWritable.java

@@ -0,0 +1,107 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+
+/** 
+ * A WritableComparable for booleans. 
+ */
+public class BooleanWritable implements WritableComparable {
+    private boolean value;
+
+    /** 
+     */
+    public BooleanWritable() {};
+
+    /** 
+     */
+    public BooleanWritable(boolean value) {
+        set(value);
+    }
+
+    /** 
+     * Set the value of the BooleanWritable
+     */    
+    public void set(boolean value) {
+        this.value = value;
+    }
+
+    /**
+     * Returns the value of the BooleanWritable
+     */
+    public boolean get() {
+        return value;
+    }
+
+    /**
+     */
+    public void readFields(DataInput in) throws IOException {
+        value = in.readBoolean();
+    }
+
+    /**
+     */
+    public void write(DataOutput out) throws IOException {
+        out.writeBoolean(value);
+    }
+
+    /**
+     */
+    public boolean equals(Object o) {
+        if (!(o instanceof BooleanWritable)) {
+            return false;
+        }
+        BooleanWritable other = (BooleanWritable) o;
+        return this.value == other.value;
+    }
+
+    public int hashCode() {
+      return value ? 0 : 1;
+    }
+
+
+
+    /**
+     */
+    public int compareTo(Object o) {
+        boolean a = this.value;
+        boolean b = ((BooleanWritable) o).value;
+        return ((a == b) ? 0 : (a == false) ? -1 : 1);
+    }
+
+    /** 
+     * A Comparator optimized for BooleanWritable. 
+     */ 
+    public static class Comparator extends WritableComparator {
+        public Comparator() {
+            super(BooleanWritable.class);
+        }
+
+        public int compare(byte[] b1, int s1, int l1,
+                           byte[] b2, int s2, int l2) {
+            boolean a = (readInt(b1, s1) == 1) ? true : false;
+            boolean b = (readInt(b2, s2) == 1) ? true : false;
+            return ((a == b) ? 0 : (a == false) ? -1 : 1);
+        }
+    }
+
+
+    static {
+      WritableComparator.define(BooleanWritable.class, new Comparator());
+    }
+}

+ 50 - 0
src/java/org/apache/hadoop/io/BytesWritable.java

@@ -0,0 +1,50 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+/** A Writable for byte arrays.
+ * 
+ * @author Doug Cutting
+ */
+public class BytesWritable implements Writable {
+  private byte[] bytes;
+
+  public BytesWritable() {}
+
+  public BytesWritable(byte[] bytes) {
+    this.bytes = bytes;
+  }
+
+  public byte[] get() {
+    return bytes;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    bytes = new byte[in.readInt()];
+    in.readFully(bytes, 0, bytes.length);
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(bytes.length);
+    out.write(bytes);
+  }
+
+}

+ 83 - 0
src/java/org/apache/hadoop/io/CompressedWritable.java

@@ -0,0 +1,83 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.DataInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ByteArrayInputStream;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+/** A base-class for Writables which store themselves compressed and lazily
+ * inflate on field access.  This is useful for large objects whose fields are
+ * not be altered during a map or reduce operation: leaving the field data
+ * compressed makes copying the instance from one file to another much
+ * faster. */
+public abstract class CompressedWritable implements Writable {
+  // if non-null, the compressed field data of this instance.
+  private byte[] compressed;
+
+  public CompressedWritable() {}
+
+  public final void readFields(DataInput in) throws IOException {
+    compressed = new byte[in.readInt()];
+    in.readFully(compressed, 0, compressed.length);
+  }
+
+  /** Must be called by all methods which access fields to ensure that the data
+   * has been uncompressed. */
+  protected void ensureInflated() {
+    if (compressed != null) {
+      try {
+        ByteArrayInputStream deflated = new ByteArrayInputStream(compressed);
+        DataInput inflater =
+          new DataInputStream(new InflaterInputStream(deflated));
+        readFieldsCompressed(inflater);
+        compressed = null;
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  /** Subclasses implement this instead of {@link #readFields(DataInput)}. */
+  protected abstract void readFieldsCompressed(DataInput in)
+    throws IOException;
+
+  public final void write(DataOutput out) throws IOException {
+    if (compressed == null) {
+      ByteArrayOutputStream deflated = new ByteArrayOutputStream();
+      Deflater deflater = new Deflater(Deflater.BEST_SPEED);
+      DataOutputStream dout =
+        new DataOutputStream(new DeflaterOutputStream(deflated, deflater));
+      writeCompressed(dout);
+      dout.close();
+      compressed = deflated.toByteArray();
+    }
+    out.writeInt(compressed.length);
+    out.write(compressed);
+  }
+
+  /** Subclasses implement this instead of {@link #write(DataOutput)}. */
+  protected abstract void writeCompressed(DataOutput out) throws IOException;
+
+}

+ 87 - 0
src/java/org/apache/hadoop/io/DataInputBuffer.java

@@ -0,0 +1,87 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+
+
+/** A reusable {@link DataInput} implementation that reads from an in-memory
+ * buffer.
+ *
+ * <p>This saves memory over creating a new DataInputStream and
+ * ByteArrayInputStream each time data is read.
+ *
+ * <p>Typical usage is something like the following:<pre>
+ *
+ * DataInputBuffer buffer = new DataInputBuffer();
+ * while (... loop condition ...) {
+ *   byte[] data = ... get data ...;
+ *   int dataLength = ... get data length ...;
+ *   buffer.reset(data, dataLength);
+ *   ... read buffer using DataInput methods ...
+ * }
+ * </pre>
+ *  
+ * @author Doug Cutting
+ */
+public class DataInputBuffer extends DataInputStream {
+
+  private static class Buffer extends ByteArrayInputStream {
+    public Buffer() {
+      super(new byte[] {});
+    }
+
+    public void reset(byte[] input, int start, int length) {
+      this.buf = input;
+      this.count = start+length;
+      this.mark = start;
+      this.pos = start;
+    }
+
+    public int getPosition() { return pos; }
+    public int getLength() { return count; }
+  }
+
+  private Buffer buffer;
+  
+  /** Constructs a new empty buffer. */
+  public DataInputBuffer() {
+    this(new Buffer());
+  }
+
+  private DataInputBuffer(Buffer buffer) {
+    super(buffer);
+    this.buffer = buffer;
+  }
+
+  /** Resets the data that the buffer reads. */
+  public void reset(byte[] input, int length) {
+    buffer.reset(input, 0, length);
+  }
+
+  /** Resets the data that the buffer reads. */
+  public void reset(byte[] input, int start, int length) {
+    buffer.reset(input, start, length);
+  }
+
+  /** Returns the current position in the input. */
+  public int getPosition() { return buffer.getPosition(); }
+
+  /** Returns the length of the input. */
+  public int getLength() { return buffer.getLength(); }
+
+}

+ 91 - 0
src/java/org/apache/hadoop/io/DataOutputBuffer.java

@@ -0,0 +1,91 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+
+/** A reusable {@link DataOutput} implementation that writes to an in-memory
+ * buffer.
+ *
+ * <p>This saves memory over creating a new DataOutputStream and
+ * ByteArrayOutputStream each time data is written.
+ *
+ * <p>Typical usage is something like the following:<pre>
+ *
+ * DataOutputBuffer buffer = new DataOutputBuffer();
+ * while (... loop condition ...) {
+ *   buffer.reset();
+ *   ... write buffer using DataOutput methods ...
+ *   byte[] data = buffer.getData();
+ *   int dataLength = buffer.getLength();
+ *   ... write data to its ultimate destination ...
+ * }
+ * </pre>
+ *  
+ * @author Doug Cutting
+ */
+public class DataOutputBuffer extends DataOutputStream {
+
+  private static class Buffer extends ByteArrayOutputStream {
+    public byte[] getData() { return buf; }
+    public int getLength() { return count; }
+    public void reset() { count = 0; }
+
+    public void write(DataInput in, int len) throws IOException {
+      int newcount = count + len;
+      if (newcount > buf.length) {
+        byte newbuf[] = new byte[Math.max(buf.length << 1, newcount)];
+        System.arraycopy(buf, 0, newbuf, 0, count);
+        buf = newbuf;
+      }
+      in.readFully(buf, count, len);
+      count = newcount;
+    }
+  }
+
+  private Buffer buffer;
+  
+  /** Constructs a new empty buffer. */
+  public DataOutputBuffer() {
+    this(new Buffer());
+  }
+  
+  private DataOutputBuffer(Buffer buffer) {
+    super(buffer);
+    this.buffer = buffer;
+  }
+
+  /** Returns the current contents of the buffer.
+   *  Data is only valid to {@link #getLength()}.
+   */
+  public byte[] getData() { return buffer.getData(); }
+
+  /** Returns the length of the valid data currently in the buffer. */
+  public int getLength() { return buffer.getLength(); }
+
+  /** Resets the buffer to empty. */
+  public DataOutputBuffer reset() {
+    this.written = 0;
+    buffer.reset();
+    return this;
+  }
+
+  /** Writes bytes from a DataInput directly into the buffer. */
+  public void write(DataInput in, int length) throws IOException {
+    buffer.write(in, length);
+  }
+}

+ 85 - 0
src/java/org/apache/hadoop/io/FloatWritable.java

@@ -0,0 +1,85 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+
+/** A WritableComparable for floats. */
+public class FloatWritable implements WritableComparable {
+  private float value;
+
+  public FloatWritable() {}
+
+  public FloatWritable(float value) { set(value); }
+
+  /** Set the value of this FloatWritable. */
+  public void set(float value) { this.value = value; }
+
+  /** Return the value of this FloatWritable. */
+  public float get() { return value; }
+
+  public void readFields(DataInput in) throws IOException {
+    value = in.readFloat();
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeFloat(value);
+  }
+
+  /** Returns true iff <code>o</code> is a FloatWritable with the same value. */
+  public boolean equals(Object o) {
+    if (!(o instanceof FloatWritable))
+      return false;
+    FloatWritable other = (FloatWritable)o;
+    return this.value == other.value;
+  }
+
+  public int hashCode() {
+    return Float.floatToIntBits(value);
+  }
+
+  /** Compares two FloatWritables. */
+  public int compareTo(Object o) {
+    float thisValue = this.value;
+    float thatValue = ((FloatWritable)o).value;
+    return (thisValue<thatValue ? -1 : (thisValue==thatValue ? 0 : 1));
+  }
+
+  public String toString() {
+    return Float.toString(value);
+  }
+
+  /** A Comparator optimized for FloatWritable. */ 
+  public static class Comparator extends WritableComparator {
+    public Comparator() {
+      super(FloatWritable.class);
+    }
+
+    public int compare(byte[] b1, int s1, int l1,
+                       byte[] b2, int s2, int l2) {
+      float thisValue = readFloat(b1, s1);
+      float thatValue = readFloat(b2, s2);
+      return (thisValue<thatValue ? -1 : (thisValue==thatValue ? 0 : 1));
+    }
+  }
+
+  static {                                        // register this comparator
+    WritableComparator.define(FloatWritable.class, new Comparator());
+  }
+
+}
+

+ 84 - 0
src/java/org/apache/hadoop/io/IntWritable.java

@@ -0,0 +1,84 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+
+/** A WritableComparable for ints. */
+public class IntWritable implements WritableComparable {
+  private int value;
+
+  public IntWritable() {}
+
+  public IntWritable(int value) { set(value); }
+
+  /** Set the value of this IntWritable. */
+  public void set(int value) { this.value = value; }
+
+  /** Return the value of this IntWritable. */
+  public int get() { return value; }
+
+  public void readFields(DataInput in) throws IOException {
+    value = in.readInt();
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(value);
+  }
+
+  /** Returns true iff <code>o</code> is a IntWritable with the same value. */
+  public boolean equals(Object o) {
+    if (!(o instanceof IntWritable))
+      return false;
+    IntWritable other = (IntWritable)o;
+    return this.value == other.value;
+  }
+
+  public int hashCode() {
+    return (int)value;
+  }
+
+  /** Compares two IntWritables. */
+  public int compareTo(Object o) {
+    int thisValue = this.value;
+    int thatValue = ((IntWritable)o).value;
+    return thisValue - thatValue;
+  }
+
+  public String toString() {
+    return Integer.toString(value);
+  }
+
+  /** A Comparator optimized for IntWritable. */ 
+  public static class Comparator extends WritableComparator {
+    public Comparator() {
+      super(IntWritable.class);
+    }
+
+    public int compare(byte[] b1, int s1, int l1,
+                       byte[] b2, int s2, int l2) {
+      int thisValue = readInt(b1, s1);
+      int thatValue = readInt(b2, s2);
+      return thisValue - thatValue;
+    }
+  }
+
+  static {                                        // register this comparator
+    WritableComparator.define(IntWritable.class, new Comparator());
+  }
+}
+

+ 95 - 0
src/java/org/apache/hadoop/io/LongWritable.java

@@ -0,0 +1,95 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+
+/** A WritableComparable for longs. */
+public class LongWritable implements WritableComparable {
+  private long value;
+
+  public LongWritable() {}
+
+  public LongWritable(long value) { set(value); }
+
+  /** Set the value of this LongWritable. */
+  public void set(long value) { this.value = value; }
+
+  /** Return the value of this LongWritable. */
+  public long get() { return value; }
+
+  public void readFields(DataInput in) throws IOException {
+    value = in.readLong();
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(value);
+  }
+
+  /** Returns true iff <code>o</code> is a LongWritable with the same value. */
+  public boolean equals(Object o) {
+    if (!(o instanceof LongWritable))
+      return false;
+    LongWritable other = (LongWritable)o;
+    return this.value == other.value;
+  }
+
+  public int hashCode() {
+    return (int)value;
+  }
+
+  /** Compares two LongWritables. */
+  public int compareTo(Object o) {
+    long thisValue = this.value;
+    long thatValue = ((LongWritable)o).value;
+    return (thisValue<thatValue ? -1 : (thisValue==thatValue ? 0 : 1));
+  }
+
+  public String toString() {
+    return Long.toString(value);
+  }
+
+  /** A Comparator optimized for LongWritable. */ 
+  public static class Comparator extends WritableComparator {
+    public Comparator() {
+      super(LongWritable.class);
+    }
+
+    public int compare(byte[] b1, int s1, int l1,
+                       byte[] b2, int s2, int l2) {
+      long thisValue = readLong(b1, s1);
+      long thatValue = readLong(b2, s2);
+      return (thisValue<thatValue ? -1 : (thisValue==thatValue ? 0 : 1));
+    }
+  }
+
+  /** A decreasing Comparator optimized for LongWritable. */ 
+  public static class DecreasingComparator extends Comparator {
+    public int compare(WritableComparable a, WritableComparable b) {
+      return -super.compare(a, b);
+    }
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+      return -super.compare(b1, s1, l1, b2, s2, l2);
+    }
+  }
+
+  static {                                       // register default comparator
+    WritableComparator.define(LongWritable.class, new Comparator());
+  }
+
+}
+

+ 199 - 0
src/java/org/apache/hadoop/io/MD5Hash.java

@@ -0,0 +1,199 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.util.Arrays;
+import java.security.*;
+
+/** A Writable for MD5 hash values.
+ *
+ * @author Doug Cutting
+ */
+public class MD5Hash implements WritableComparable {
+  public static final int MD5_LEN = 16;
+  private static final MessageDigest DIGESTER;
+  static {
+    try {
+      DIGESTER = MessageDigest.getInstance("MD5");
+    } catch (NoSuchAlgorithmException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private byte[] digest;
+
+  /** Constructs an MD5Hash. */
+  public MD5Hash() {
+    this.digest = new byte[MD5_LEN];
+  }
+
+  /** Constructs an MD5Hash from a hex string. */
+  public MD5Hash(String hex) {
+    setDigest(hex);
+  }
+  
+  /** Constructs an MD5Hash with a specified value. */
+  public MD5Hash(byte[] digest) {
+    if (digest.length != MD5_LEN)
+      throw new IllegalArgumentException("Wrong length: " + digest.length);
+    this.digest = digest;
+  }
+  
+  // javadoc from Writable
+  public void readFields(DataInput in) throws IOException {
+    in.readFully(digest);
+  }
+
+  /** Constructs, reads and returns an instance. */
+  public static MD5Hash read(DataInput in) throws IOException {
+    MD5Hash result = new MD5Hash();
+    result.readFields(in);
+    return result;
+  }
+
+  // javadoc from Writable
+  public void write(DataOutput out) throws IOException {
+    out.write(digest);
+  }
+
+  /** Copy the contents of another instance into this instance. */
+  public void set(MD5Hash that) {
+    System.arraycopy(that.digest, 0, this.digest, 0, MD5_LEN);
+  }
+
+  /** Returns the digest bytes. */
+  public byte[] getDigest() { return digest; }
+
+  /** Construct a hash value for a byte array. */
+  public static MD5Hash digest(byte[] data) {
+    return digest(data, 0, data.length);
+  }
+
+  /** Construct a hash value for a byte array. */
+  public static MD5Hash digest(byte[] data, int start, int len) {
+    byte[] digest;
+    synchronized (DIGESTER) {
+      DIGESTER.update(data, start, len);
+      digest = DIGESTER.digest();
+    }
+    return new MD5Hash(digest);
+  }
+
+  /** Construct a hash value for a String. */
+  public static MD5Hash digest(String string) {
+    return digest(UTF8.getBytes(string));
+  }
+
+  /** Construct a hash value for a String. */
+  public static MD5Hash digest(UTF8 utf8) {
+    return digest(utf8.getBytes(), 0, utf8.getLength());
+  }
+
+  /** Construct a half-sized version of this MD5.  Fits in a long **/
+  public long halfDigest() {
+    long value = 0;
+    for (int i = 0; i < 8; i++)
+      value |= ((digest[i] & 0xffL) << (8*(7-i)));
+    return value;
+  }
+
+  /** Returns true iff <code>o</code> is an MD5Hash whose digest contains the
+   * same values.  */
+  public boolean equals(Object o) {
+    if (!(o instanceof MD5Hash))
+      return false;
+    MD5Hash other = (MD5Hash)o;
+    return Arrays.equals(this.digest, other.digest);
+  }
+
+  /** Returns a hash code value for this object.*/
+  public int hashCode() {
+    return                                        // xor four ints
+      (digest[ 0] | (digest[ 1]<<8) | (digest[ 2]<<16) | (digest[ 3]<<24)) ^
+      (digest[ 4] | (digest[ 5]<<8) | (digest[ 6]<<16) | (digest[ 7]<<24)) ^
+      (digest[ 8] | (digest[ 9]<<8) | (digest[10]<<16) | (digest[11]<<24)) ^
+      (digest[12] | (digest[13]<<8) | (digest[14]<<16) | (digest[15]<<24));
+  }
+
+
+  /** Compares this object with the specified object for order.*/
+  public int compareTo(Object o) {
+    MD5Hash that = (MD5Hash)o;
+    return WritableComparator.compareBytes(this.digest, 0, MD5_LEN,
+                                           that.digest, 0, MD5_LEN);
+  }
+
+  /** A WritableComparator optimized for MD5Hash keys. */
+  public static class Comparator extends WritableComparator {
+    public Comparator() {
+      super(MD5Hash.class);
+    }
+
+    public int compare(byte[] b1, int s1, int l1,
+                       byte[] b2, int s2, int l2) {
+      return compareBytes(b1, s1, MD5_LEN, b2, s2, MD5_LEN);
+    }
+  }
+
+  static {                                        // register this comparator
+    WritableComparator.define(MD5Hash.class, new Comparator());
+  }
+
+  private static final char[] HEX_DIGITS =
+  {'0','1','2','3','4','5','6','7','8','9','a','b','c','d','e','f'};
+
+  /** Returns a string representation of this object. */
+  public String toString() {
+    StringBuffer buf = new StringBuffer(MD5_LEN*2);
+    for (int i = 0; i < MD5_LEN; i++) {
+      int b = digest[i];
+      buf.append(HEX_DIGITS[(b >> 4) & 0xf]);
+      buf.append(HEX_DIGITS[b & 0xf]);
+    }
+    return buf.toString();
+  }
+
+  /** Sets the digest value from a hex string. */
+  public void setDigest(String hex) {
+    if (hex.length() != MD5_LEN*2)
+      throw new IllegalArgumentException("Wrong length: " + hex.length());
+    byte[] digest = new byte[MD5_LEN];
+    for (int i = 0; i < MD5_LEN; i++) {
+      int j = i << 1;
+      digest[i] = (byte)(charToNibble(hex.charAt(j)) << 4 |
+                         charToNibble(hex.charAt(j+1)));
+    }
+    this.digest = digest;
+  }
+
+  private static final int charToNibble(char c) {
+    if (c >= '0' && c <= '9') {
+      return c - '0';
+    } else if (c >= 'a' && c <= 'f') {
+      return 0xa + (c - 'a');
+    } else if (c >= 'A' && c <= 'F') {
+      return 0xA + (c - 'A');
+    } else {
+      throw new RuntimeException("Not a hex character: " + c);
+    }
+  }
+
+
+}

+ 495 - 0
src/java/org/apache/hadoop/io/MapFile.java

@@ -0,0 +1,495 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+import java.util.Arrays;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.*;
+
+/** A file-based map from keys to values.
+ * 
+ * <p>A map is a directory containing two files, the <code>data</code> file,
+ * containing all keys and values in the map, and a smaller <code>index</code>
+ * file, containing a fraction of the keys.  The fraction is determined by
+ * {@link Writer#getIndexInterval()}.
+ *
+ * <p>The index file is read entirely into memory.  Thus key implementations
+ * should try to keep themselves small.
+ *
+ * <p>Map files are created by adding entries in-order.  To maintain a large
+ * database, perform updates by copying the previous version of a database and
+ * merging in a sorted change list, to create a new version of the database in
+ * a new file.  Sorting large change lists can be done with {@link
+ * SequenceFile.Sorter}.
+ */
+public class MapFile {
+  /** The name of the index file. */
+  public static final String INDEX_FILE_NAME = "index";
+
+  /** The name of the data file. */
+  public static final String DATA_FILE_NAME = "data";
+
+  protected MapFile() {}                          // no public ctor
+
+  /** Writes a new map. */
+  public static class Writer {
+    private SequenceFile.Writer data;
+    private SequenceFile.Writer index;
+
+    private int indexInterval = 128;
+
+    private long size;
+    private LongWritable position = new LongWritable();
+
+    // the following fields are used only for checking key order
+    private WritableComparator comparator;
+    private DataInputBuffer inBuf = new DataInputBuffer();
+    private DataOutputBuffer outBuf = new DataOutputBuffer();
+    private WritableComparable lastKey;
+
+
+    /** Create the named map for keys of the named class. */
+    public Writer(NutchFileSystem nfs, String dirName,
+                  Class keyClass, Class valClass)
+      throws IOException {
+      this(nfs, dirName, WritableComparator.get(keyClass), valClass, false);
+    }
+
+    /** Create the named map for keys of the named class. */
+    public Writer(NutchFileSystem nfs, String dirName,
+                  Class keyClass, Class valClass, boolean compress)
+      throws IOException {
+      this(nfs, dirName, WritableComparator.get(keyClass), valClass, compress);
+    }
+
+    /** Create the named map using the named key comparator. */
+    public Writer(NutchFileSystem nfs, String dirName,
+                  WritableComparator comparator, Class valClass)
+      throws IOException {
+      this(nfs, dirName, comparator, valClass, false);
+    }
+    /** Create the named map using the named key comparator. */
+    public Writer(NutchFileSystem nfs, String dirName,
+                  WritableComparator comparator, Class valClass,
+                  boolean compress)
+      throws IOException {
+
+      this.comparator = comparator;
+      this.lastKey = comparator.newKey();
+
+      File dir = new File(dirName);
+      nfs.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,
+                                compress);
+      this.index =
+        new SequenceFile.Writer(nfs, indexFile.getPath(),
+                                keyClass, LongWritable.class);
+    }
+    
+    /** The number of entries that are added before an index entry is added.*/
+    public int getIndexInterval() { return indexInterval; }
+
+    /** Sets the index interval.
+     * @see #getIndexInterval()
+     */
+    public void setIndexInterval(int interval) { indexInterval = interval; }
+
+    /** Close the map. */
+    public synchronized void close() throws IOException {
+      data.close();
+      index.close();
+    }
+
+    /** Append a key/value pair to the map.  The key must be greater or equal
+     * to the previous key added to the map. */
+    public synchronized void append(WritableComparable key, Writable val)
+      throws IOException {
+
+      checkKey(key);
+      
+      if (size % indexInterval == 0) {            // add an index entry
+        position.set(data.getLength());           // point to current eof
+        index.append(key, position);
+      }
+
+      data.append(key, val);                      // append key/value to data
+      size++;
+    }
+
+    private void checkKey(WritableComparable key) throws IOException {
+      // check that keys are well-ordered
+      if (size != 0 && comparator.compare(lastKey, key) > 0)
+        throw new IOException("key out of order: "+key+" after "+lastKey);
+          
+      // update lastKey with a copy of key by writing and reading
+      outBuf.reset();
+      key.write(outBuf);                          // write new key
+
+      inBuf.reset(outBuf.getData(), outBuf.getLength());
+      lastKey.readFields(inBuf);                  // read into lastKey
+    }
+
+  }
+  
+  /** Provide access to an existing map. */
+  public static class Reader {
+      
+    /** Number of index entries to skip between each entry.  Zero by default.
+    * Setting this to values larger than zero can facilitate opening large map
+    * files using less memory. */
+    private int INDEX_SKIP = 0;
+      
+    private WritableComparator comparator;
+
+    private DataOutputBuffer keyBuf = new DataOutputBuffer();
+    private DataOutputBuffer nextBuf = new DataOutputBuffer();
+    private int nextKeyLen = -1;
+    private long seekPosition = -1;
+    private int seekIndex = -1;
+    private long firstPosition;
+
+    private WritableComparable getKey;
+
+    // the data, on disk
+    private SequenceFile.Reader data;
+    private SequenceFile.Reader index;
+
+    // whether the index Reader was closed
+    private boolean indexClosed = false;
+
+    // the index, in memory
+    private int count = -1;
+    private WritableComparable[] keys;
+    private long[] positions;
+
+    /** Returns the class of keys in this file. */
+    public Class getKeyClass() { return data.getKeyClass(); }
+
+    /** Returns the class of values in this file. */
+    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);
+      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)
+      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.firstPosition = data.getPosition();
+
+      if (comparator == null)
+        this.comparator = WritableComparator.get(data.getKeyClass());
+      else
+        this.comparator = comparator;
+
+      this.getKey = this.comparator.newKey();
+
+      // open the index
+      this.index = new SequenceFile.Reader(nfs, indexFile.getPath(), conf);
+    }
+
+    private void readIndex() throws IOException {
+      // read the index entirely into memory
+      if (this.keys != null)
+        return;
+      this.count = 0;
+      this.keys = new WritableComparable[1024];
+      this.positions = new long[1024];
+      try {
+        int skip = INDEX_SKIP;
+        LongWritable position = new LongWritable();
+        WritableComparable lastKey = null;
+        while (true) {
+          WritableComparable k = comparator.newKey();
+
+          if (!index.next(k, position))
+            break;
+
+          // check order to make sure comparator is compatible
+          if (lastKey != null && comparator.compare(lastKey, k) > 0)
+            throw new IOException("key out of order: "+k+" after "+lastKey);
+          lastKey = k;
+          
+          if (skip > 0) {
+            skip--;
+            continue;                             // skip this entry
+          } else {
+            skip = INDEX_SKIP;                    // reset skip
+          }
+
+          if (count == keys.length) {                // time to grow arrays
+            int newLength = (keys.length*3)/2;
+            WritableComparable[] newKeys = new WritableComparable[newLength];
+            long[] newPositions = new long[newLength];
+            System.arraycopy(keys, 0, newKeys, 0, count);
+            System.arraycopy(positions, 0, newPositions, 0, count);
+            keys = newKeys;
+            positions = newPositions;
+          }
+
+          keys[count] = k;
+          positions[count] = position.get();
+          count++;
+        }
+      } catch (EOFException e) {
+        SequenceFile.LOG.warning("Unexpected EOF reading " + index +
+                                 " at entry #" + count + ".  Ignoring.");
+      } finally {
+	indexClosed = true;
+        index.close();
+      }
+    }
+
+    /** Re-positions the reader before its first key. */
+    public synchronized void reset() throws IOException {
+      data.seek(firstPosition);
+    }
+
+    /** Reads the final key from the file.
+     *
+     * @param key key to read into
+     */
+    public synchronized void finalKey(WritableComparable key)
+      throws IOException {
+
+      long originalPosition = data.getPosition(); // save position
+      try {
+        readIndex();                              // make sure index is valid
+        if (count > 0) {
+          data.seek(positions[count-1]);          // skip to last indexed entry
+        } else {
+          reset();                                // start at the beginning
+        }
+        while (data.next(key)) {}                 // scan to eof
+
+      } finally {
+        data.seek(originalPosition);              // restore position
+      }
+    }
+
+    /** Positions the reader at the named key, or if none such exists, at the
+     * first entry after the named key.  Returns true iff the named key exists
+     * in this map.
+     */
+    public synchronized boolean seek(WritableComparable key)
+      throws IOException {
+      readIndex();                                // make sure index is read
+      keyBuf.reset();                             // write key to keyBuf
+      key.write(keyBuf);
+
+      if (seekIndex != -1                         // seeked before
+          && seekIndex+1 < count           
+          && comparator.compare(key,keys[seekIndex+1])<0 // before next indexed
+          && comparator.compare(keyBuf.getData(), 0, keyBuf.getLength(),
+                                nextBuf.getData(), 0, nextKeyLen)
+          >= 0) {                                 // but after last seeked
+        // do nothing
+      } else {
+        seekIndex = binarySearch(key);
+        if (seekIndex < 0)                        // decode insertion point
+          seekIndex = -seekIndex-2;
+
+        if (seekIndex == -1)                      // belongs before first entry
+          seekPosition = firstPosition;           // use beginning of file
+        else
+          seekPosition = positions[seekIndex];    // else use index
+      }
+      data.seek(seekPosition);
+      
+      while ((nextKeyLen = data.next(nextBuf.reset())) != -1) {
+        int c = comparator.compare(keyBuf.getData(), 0, keyBuf.getLength(),
+                                   nextBuf.getData(), 0, nextKeyLen);
+        if (c <= 0) {                             // at or beyond desired
+          data.seek(seekPosition);                // back off to previous
+          return c == 0;
+        }
+        seekPosition = data.getPosition();
+      }
+
+      return false;
+    }
+
+    private int binarySearch(WritableComparable key) {
+      int low = 0;
+      int high = count-1;
+
+      while (low <= high) {
+        int mid = (low + high) >> 1;
+        WritableComparable midVal = keys[mid];
+        int cmp = comparator.compare(midVal, key);
+
+        if (cmp < 0)
+          low = mid + 1;
+        else if (cmp > 0)
+          high = mid - 1;
+        else
+          return mid;                             // key found
+      }
+      return -(low + 1);                          // key not found.
+    }
+
+    /** Read the next key/value pair in the map into <code>key</code> and
+     * <code>val</code>.  Returns true if such a pair exists and false when at
+     * the end of the map */
+    public synchronized boolean next(WritableComparable key, Writable val)
+      throws IOException {
+      return data.next(key, val);
+    }
+
+    /** Return the value for the named key, or null if none exists. */
+    public synchronized Writable get(WritableComparable key, Writable val)
+      throws IOException {
+      if (seek(key)) {
+        next(getKey, val);                        // don't smash key
+        return val;
+      } else
+        return null;
+    }
+
+    /** Close the map. */
+    public synchronized void close() throws IOException {
+      if (! indexClosed) {
+	index.close();
+      }
+      data.close();
+    }
+
+  }
+
+  /** Renames an existing map directory. */
+  public static void rename(NutchFileSystem nfs, String oldName, String newName)
+    throws IOException {
+    File oldDir = new File(oldName);
+    File newDir = new File(newName);
+    if (!nfs.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 {
+    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);
+  }
+
+  /**
+   * This method attempts to fix a corrupt MapFile by re-creating its index.
+   * @param nfs 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)
+   * @param dryrun do not perform any changes, just report what needs to be done
+   * @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,
+          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)) {
+      // 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)) {
+      // no fixing needed
+      return -1;
+    }
+    SequenceFile.Reader dataReader = new SequenceFile.Reader(nfs, data.toString(), conf);
+    if (!dataReader.getKeyClass().equals(keyClass)) {
+      throw new Exception(dr + "Wrong key class in " + dir + ", expected" + keyClass.getName() +
+              ", got " + dataReader.getKeyClass().getName());
+    }
+    if (!dataReader.getValueClass().equals(valueClass)) {
+      throw new Exception(dr + "Wrong value class in " + dir + ", expected" + valueClass.getName() +
+              ", got " + dataReader.getValueClass().getName());
+    }
+    long cnt = 0L;
+    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);
+    try {
+      long pos = 0L;
+      LongWritable position = new LongWritable();
+      while(dataReader.next(key, value)) {
+        cnt++;
+        if (cnt % indexInterval == 0) {
+          position.set(pos);
+          if (!dryrun) indexWriter.append(key, position);
+        }
+        pos = dataReader.getPosition();
+      }
+    } catch(Throwable t) {
+      // truncated data file. swallow it.
+    }
+    dataReader.close();
+    if (!dryrun) indexWriter.close();
+    return cnt;
+  }
+
+
+  public static void main(String[] args) throws Exception {
+    String usage = "Usage: MapFile inFile outFile";
+      
+    if (args.length != 2) {
+      System.err.println(usage);
+      System.exit(-1);
+    }
+      
+    String in = args[0];
+    String out = args[1];
+
+    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);
+    MapFile.Writer writer =
+      new MapFile.Writer(nfs, out, reader.getKeyClass(), reader.getValueClass());
+
+    WritableComparable key =
+      (WritableComparable)reader.getKeyClass().newInstance();
+    Writable value = (Writable)reader.getValueClass().newInstance();
+
+    while (reader.next(key, value))               // copy all entries
+      writer.append(key, value);
+
+    writer.close();
+  }
+
+}

+ 34 - 0
src/java/org/apache/hadoop/io/NullWritable.java

@@ -0,0 +1,34 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+
+/** Singleton Writable with no data. */
+public class NullWritable implements Writable {
+
+  private static final NullWritable THIS = new NullWritable();
+
+  private NullWritable() {}                       // no public ctor
+
+  /** Returns the single instance of this class. */
+  public static NullWritable get() { return THIS; }
+
+  public void readFields(DataInput in) throws IOException {}
+  public void write(DataOutput out) throws IOException {}
+}
+

+ 265 - 0
src/java/org/apache/hadoop/io/ObjectWritable.java

@@ -0,0 +1,265 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.lang.reflect.Proxy;
+import java.lang.reflect.Method;
+import java.lang.reflect.Array;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+
+import java.io.*;
+import java.util.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configurable;
+
+/** A polymorphic Writable that writes an instance with it's class name.
+ * Handles arrays, strings and primitive types without a Writable wrapper.
+ */
+public class ObjectWritable implements Writable, Configurable {
+
+  private Class declaredClass;
+  private Object instance;
+  private Configuration conf;
+
+  public ObjectWritable() {}
+  
+  public ObjectWritable(Object instance) {
+    set(instance);
+  }
+
+  public ObjectWritable(Class declaredClass, Object instance) {
+    this.declaredClass = declaredClass;
+    this.instance = instance;
+  }
+
+  /** Return the instance, or null if none. */
+  public Object get() { return instance; }
+  
+  /** Return the class this is meant to be. */
+  public Class getDeclaredClass() { return declaredClass; }
+  
+  /** Reset the instance. */
+  public void set(Object instance) {
+    this.declaredClass = instance.getClass();
+    this.instance = instance;
+  }
+  
+  public void readFields(DataInput in) throws IOException {
+    readObject(in, this, this.conf);
+  }
+  
+  public void write(DataOutput out) throws IOException {
+    writeObject(out, instance, declaredClass);
+  }
+
+  private static final Map PRIMITIVE_NAMES = new HashMap();
+  static {
+    PRIMITIVE_NAMES.put("boolean", Boolean.TYPE);
+    PRIMITIVE_NAMES.put("byte", Byte.TYPE);
+    PRIMITIVE_NAMES.put("char", Character.TYPE);
+    PRIMITIVE_NAMES.put("short", Short.TYPE);
+    PRIMITIVE_NAMES.put("int", Integer.TYPE);
+    PRIMITIVE_NAMES.put("long", Long.TYPE);
+    PRIMITIVE_NAMES.put("float", Float.TYPE);
+    PRIMITIVE_NAMES.put("double", Double.TYPE);
+    PRIMITIVE_NAMES.put("void", Void.TYPE);
+  }
+
+  private static class NullInstance implements Writable {
+    private Class declaredClass;
+    public NullInstance() {}
+    public NullInstance(Class declaredClass) {
+      this.declaredClass = declaredClass;
+    }
+    public void readFields(DataInput in) throws IOException {
+      String className = UTF8.readString(in);
+      declaredClass = (Class)PRIMITIVE_NAMES.get(className);
+      if (declaredClass == null) {
+        try {
+          declaredClass = Class.forName(className);
+        } catch (ClassNotFoundException e) {
+          throw new RuntimeException(e.toString());
+        }
+      }
+    }
+    public void write(DataOutput out) throws IOException {
+      UTF8.writeString(out, declaredClass.getName());
+    }
+  }
+
+  /** Write a {@link Writable}, {@link String}, primitive type, or an array of
+   * the preceding. */
+  public static void writeObject(DataOutput out, Object instance,
+                                 Class declaredClass) throws IOException {
+
+    if (instance == null) {                       // null
+      instance = new NullInstance(declaredClass);
+      declaredClass = NullInstance.class;
+    }
+
+    if (instance instanceof Writable) {           // Writable
+
+      // write instance's class, to support subclasses of the declared class
+      UTF8.writeString(out, instance.getClass().getName());
+      
+      ((Writable)instance).write(out);
+
+      return;
+    }
+
+    // write declared class for primitives, as they can't be subclassed, and
+    // the class of the instance may be a wrapper
+    UTF8.writeString(out, declaredClass.getName());
+
+    if (declaredClass.isArray()) {                // array
+      int length = Array.getLength(instance);
+      out.writeInt(length);
+      for (int i = 0; i < length; i++) {
+        writeObject(out, Array.get(instance, i),
+                    declaredClass.getComponentType());
+      }
+      
+    } else if (declaredClass == String.class) {   // String
+      UTF8.writeString(out, (String)instance);
+      
+    } else if (declaredClass.isPrimitive()) {     // primitive type
+
+      if (declaredClass == Boolean.TYPE) {        // boolean
+        out.writeBoolean(((Boolean)instance).booleanValue());
+      } else if (declaredClass == Character.TYPE) { // char
+        out.writeChar(((Character)instance).charValue());
+      } else if (declaredClass == Byte.TYPE) {    // byte
+        out.writeByte(((Byte)instance).byteValue());
+      } else if (declaredClass == Short.TYPE) {   // short
+        out.writeShort(((Short)instance).shortValue());
+      } else if (declaredClass == Integer.TYPE) { // int
+        out.writeInt(((Integer)instance).intValue());
+      } else if (declaredClass == Long.TYPE) {    // long
+        out.writeLong(((Long)instance).longValue());
+      } else if (declaredClass == Float.TYPE) {   // float
+        out.writeFloat(((Float)instance).floatValue());
+      } else if (declaredClass == Double.TYPE) {  // double
+        out.writeDouble(((Double)instance).doubleValue());
+      } else if (declaredClass == Void.TYPE) {    // void
+      } else {
+        throw new IllegalArgumentException("Not a primitive: "+declaredClass);
+      }
+      
+    } else {
+      throw new IOException("Can't write: "+instance+" as "+declaredClass);
+    }
+  }
+  
+  
+  /** Read a {@link Writable}, {@link String}, primitive type, or an array of
+   * the preceding. */
+  public static Object readObject(DataInput in, Configuration conf)
+    throws IOException {
+    return readObject(in, null, conf);
+  }
+    
+  /** Read a {@link Writable}, {@link String}, primitive type, or an array of
+   * the preceding. */
+  public static Object readObject(DataInput in, ObjectWritable objectWritable, Configuration conf)
+    throws IOException {
+    String className = UTF8.readString(in);
+    Class declaredClass = (Class)PRIMITIVE_NAMES.get(className);
+    if (declaredClass == null) {
+      try {
+        declaredClass = Class.forName(className);
+      } catch (ClassNotFoundException e) {
+        throw new RuntimeException(e.toString());
+      }
+    }    
+
+    Object instance;
+    
+    if (declaredClass == NullInstance.class) {         // null
+      NullInstance wrapper = new NullInstance();
+      wrapper.readFields(in);
+      declaredClass = wrapper.declaredClass;
+      instance = null;
+
+    } else if (declaredClass.isPrimitive()) {          // primitive types
+
+      if (declaredClass == Boolean.TYPE) {             // boolean
+        instance = Boolean.valueOf(in.readBoolean());
+      } else if (declaredClass == Character.TYPE) {    // char
+        instance = new Character(in.readChar());
+      } else if (declaredClass == Byte.TYPE) {         // byte
+        instance = new Byte(in.readByte());
+      } else if (declaredClass == Short.TYPE) {        // short
+        instance = new Short(in.readShort());
+      } else if (declaredClass == Integer.TYPE) {      // int
+        instance = new Integer(in.readInt());
+      } else if (declaredClass == Long.TYPE) {         // long
+        instance = new Long(in.readLong());
+      } else if (declaredClass == Float.TYPE) {        // float
+        instance = new Float(in.readFloat());
+      } else if (declaredClass == Double.TYPE) {       // double
+        instance = new Double(in.readDouble());
+      } else if (declaredClass == Void.TYPE) {         // void
+        instance = null;
+      } else {
+        throw new IllegalArgumentException("Not a primitive: "+declaredClass);
+      }
+
+    } else if (declaredClass.isArray()) {              // array
+      int length = in.readInt();
+      instance = Array.newInstance(declaredClass.getComponentType(), length);
+      for (int i = 0; i < length; i++) {
+        Array.set(instance, i, readObject(in, conf));
+      }
+      
+    } else if (declaredClass == String.class) {        // String
+      instance = UTF8.readString(in);
+      
+    } else {                                      // Writable
+      try {
+        Writable writable = (Writable)declaredClass.newInstance();
+        if(writable instanceof Configurable) {
+          ((Configurable) writable).setConf(conf);
+        }
+        writable.readFields(in);
+        instance = writable;
+      } catch (InstantiationException e) {
+        throw new RuntimeException(e);
+      } catch (IllegalAccessException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    if (objectWritable != null) {                 // store values
+      objectWritable.declaredClass = declaredClass;
+      objectWritable.instance = instance;
+    }
+
+    return instance;
+      
+  }
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  public Configuration getConf() {
+    return this.conf;
+  }
+  
+}

+ 887 - 0
src/java/org/apache/hadoop/io/SequenceFile.java

@@ -0,0 +1,887 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+import java.util.*;
+import java.util.zip.*;
+import java.util.logging.*;
+import java.nio.channels.*;
+import java.net.InetAddress;
+import java.rmi.server.UID;
+import java.security.MessageDigest;
+import org.apache.lucene.util.PriorityQueue;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.util.LogFormatter;
+
+/** Support for flat files of binary key/value pairs. */
+public class SequenceFile {
+  public static final Logger LOG =
+    LogFormatter.getLogger("org.apache.hadoop.io.SequenceFile");
+
+  private SequenceFile() {}                         // no public ctor
+
+  private static byte[] VERSION = new byte[] {
+    (byte)'S', (byte)'E', (byte)'Q', 3
+  };
+
+  private static final int SYNC_ESCAPE = -1;      // "length" of sync entries
+  private static final int SYNC_HASH_SIZE = 16;   // number of bytes in hash 
+  private static final int SYNC_SIZE = 4+SYNC_HASH_SIZE; // escape + hash
+
+  /** The number of bytes between sync points.*/
+  public static final int SYNC_INTERVAL = 100*SYNC_SIZE; 
+
+  /** Write key/value pairs to a sequence-format file. */
+  public static class Writer {
+    private NFSDataOutputStream out;
+    private DataOutputBuffer buffer = new DataOutputBuffer();
+    private NutchFileSystem nfs = null;
+    private File target = null;
+
+    private Class keyClass;
+    private Class valClass;
+
+    private boolean deflateValues;
+    private DataOutputBuffer deflateIn = new DataOutputBuffer();
+    private byte[] deflateOut = new byte[1024];
+    private Deflater deflater = new Deflater(Deflater.BEST_SPEED);
+
+    // Insert a globally unique 16-byte value every few entries, so that one
+    // can seek into the middle of a file and then synchronize with record
+    // starts and ends by scanning for this value.
+    private long lastSyncPos;                     // position of last sync
+    private byte[] sync;                          // 16 random bytes
+    {
+      try {                                       // use hash of uid + host
+        MessageDigest digester = MessageDigest.getInstance("MD5");
+        digester.update((new UID()+"@"+InetAddress.getLocalHost()).getBytes());
+        sync = digester.digest();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    /** Create the named file. */
+    public Writer(NutchFileSystem nfs, String name,
+                  Class keyClass, Class valClass)
+      throws IOException {
+      this(nfs, name, keyClass, valClass, false);
+    }
+    
+    /** Create the named file.
+     * @param compress if true, values are compressed.
+     */
+    public Writer(NutchFileSystem nfs, String name,
+                  Class keyClass, Class valClass, boolean compress)
+      throws IOException {
+      this.nfs = nfs;
+      this.target = new File(name);
+      init(nfs.create(target), keyClass, valClass, compress);
+    }
+    
+    /** Write to an arbitrary stream using a specified buffer size. */
+    private Writer(NFSDataOutputStream 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,
+                      Class keyClass, Class valClass,
+                      boolean compress) throws IOException {
+      this.out = out;
+      this.out.write(VERSION);
+
+      this.keyClass = keyClass;
+      this.valClass = valClass;
+
+      this.deflateValues = compress;
+
+      new UTF8(WritableName.getName(keyClass)).write(this.out);
+      new UTF8(WritableName.getName(valClass)).write(this.out);
+
+      this.out.writeBoolean(deflateValues);
+
+      out.write(sync);                            // write the sync bytes
+
+      this.out.flush();                           // flush header
+    }
+    
+
+    /** Returns the class of keys in this file. */
+    public Class getKeyClass() { return keyClass; }
+
+    /** Returns the class of values in this file. */
+    public Class getValueClass() { return valClass; }
+
+
+    /** Close the file. */
+    public synchronized void close() throws IOException {
+      if (out != null) {
+        out.close();
+        out = null;
+      }
+    }
+
+    /** Append a key/value pair. */
+    public synchronized void append(Writable key, Writable val)
+      throws IOException {
+      if (key.getClass() != keyClass)
+        throw new IOException("wrong key class: "+key+" is not "+keyClass);
+      if (val.getClass() != valClass)
+        throw new IOException("wrong value class: "+val+" is not "+valClass);
+
+      buffer.reset();
+
+      key.write(buffer);
+      int keyLength = buffer.getLength();
+      if (keyLength == 0)
+        throw new IOException("zero length keys not allowed: " + key);
+
+      if (deflateValues) {
+        deflateIn.reset();
+        val.write(deflateIn);
+        deflater.reset();
+        deflater.setInput(deflateIn.getData(), 0, deflateIn.getLength());
+        deflater.finish();
+        while (!deflater.finished()) {
+          int count = deflater.deflate(deflateOut);
+          buffer.write(deflateOut, 0, count);
+        }
+      } else {
+        val.write(buffer);
+      }
+
+      append(buffer.getData(), 0, buffer.getLength(), keyLength);
+    }
+
+    /** Append a key/value pair. */
+    public synchronized void append(byte[] data, int start, int length,
+                                    int keyLength) throws IOException {
+      if (keyLength == 0)
+        throw new IOException("zero length keys not allowed");
+
+      if (sync != null &&
+          out.getPos() >= lastSyncPos+SYNC_INTERVAL) { // time to emit sync
+        lastSyncPos = out.getPos();               // update lastSyncPos
+        //LOG.info("sync@"+lastSyncPos);
+        out.writeInt(SYNC_ESCAPE);                // escape it
+        out.write(sync);                          // write sync
+      }
+
+      out.writeInt(length);                       // total record length
+      out.writeInt(keyLength);                    // key portion length
+      out.write(data, start, length);             // data
+
+    }
+
+    /** Returns the current length of the output file. */
+    public synchronized long getLength() throws IOException {
+      return out.getPos();
+    }
+
+  }
+
+  /** Writes key/value pairs from a sequence-format file. */
+  public static class Reader {
+    private String file;
+    private NFSDataInputStream in;
+    private DataOutputBuffer outBuf = new DataOutputBuffer();
+    private DataInputBuffer inBuf = new DataInputBuffer();
+    private NutchFileSystem nfs = null;
+
+    private byte[] version = new byte[VERSION.length];
+
+    private Class keyClass;
+    private Class valClass;
+
+    private byte[] sync = new byte[SYNC_HASH_SIZE];
+    private byte[] syncCheck = new byte[SYNC_HASH_SIZE];
+    private boolean syncSeen;
+
+    private long end;
+    private int keyLength;
+
+    private boolean inflateValues;
+    private byte[] inflateIn = new byte[1024];
+    private DataOutputBuffer inflateOut = new DataOutputBuffer();
+    private Inflater inflater = new Inflater();
+    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));
+      this.conf = conf;
+    }
+
+    private Reader(NutchFileSystem nfs, String name, int bufferSize) throws IOException {
+      this.nfs = nfs;
+      this.file = name;
+      File file = new File(name);
+      this.in = nfs.open(file, bufferSize);
+      this.end = nfs.getLength(file);
+      init();
+    }
+    
+    private Reader(NutchFileSystem nfs, String file, int bufferSize, long start, long length)
+      throws IOException {
+      this.nfs = nfs;
+      this.file = file;
+      this.in = nfs.open(new File(file), bufferSize);
+      seek(start);
+      init();
+
+      this.end = in.getPos() + length;
+    }
+    
+    private void init() throws IOException {
+      in.readFully(version);
+
+      if ((version[0] != VERSION[0]) ||
+          (version[1] != VERSION[1]) ||
+          (version[2] != VERSION[2]))
+        throw new IOException(file + " not a SequenceFile");
+
+      if (version[3] > VERSION[3])
+        throw new VersionMismatchException(VERSION[3], version[3]);
+
+      UTF8 className = new UTF8();
+      
+      className.readFields(in);                   // read key class name
+      this.keyClass = WritableName.getClass(className.toString());
+      
+      className.readFields(in);                   // read val class name
+      this.valClass = WritableName.getClass(className.toString());
+
+      if (version[3] > 2) {                       // if version > 2
+        this.inflateValues = in.readBoolean();    // is compressed?
+      }
+
+      if (version[3] > 1) {                       // if version > 1
+        in.readFully(sync);                       // read sync bytes
+      }
+    }
+    
+    /** Close the file. */
+    public synchronized void close() throws IOException {
+      in.close();
+    }
+
+    /** Returns the class of keys in this file. */
+    public Class getKeyClass() { return keyClass; }
+
+    /** Returns the class of values in this file. */
+    public Class getValueClass() { return valClass; }
+
+    /** Returns true if values are compressed. */
+    public boolean isCompressed() { return inflateValues; }
+
+    /** Read the next key in the file into <code>key</code>, skipping its
+     * value.  True if another entry exists, and false at end of file. */
+    public synchronized boolean next(Writable key) throws IOException {
+      if (key.getClass() != keyClass)
+        throw new IOException("wrong key class: "+key+" is not "+keyClass);
+
+      outBuf.reset();
+
+      keyLength = next(outBuf);
+      if (keyLength < 0)
+        return false;
+
+      inBuf.reset(outBuf.getData(), outBuf.getLength());
+
+      key.readFields(inBuf);
+      if (inBuf.getPosition() != keyLength)
+        throw new IOException(key + " read " + inBuf.getPosition()
+                              + " bytes, should read " + keyLength);
+
+      return true;
+    }
+
+    /** Read the next key/value pair in the file into <code>key</code> and
+     * <code>val</code>.  Returns true if such a pair exists and false when at
+     * end of file */
+    public synchronized boolean next(Writable key, Writable val)
+      throws IOException {
+      if (val.getClass() != valClass)
+        throw new IOException("wrong value class: "+val+" is not "+valClass);
+
+      boolean more = next(key);
+
+      if (more) {
+
+        if (inflateValues) {
+          inflater.reset();
+          inflater.setInput(outBuf.getData(), keyLength,
+                            outBuf.getLength()-keyLength);
+          inflateOut.reset();
+          while (!inflater.finished()) {
+            try {
+              int count = inflater.inflate(inflateIn);
+              inflateOut.write(inflateIn, 0, count);
+            } catch (DataFormatException e) {
+              throw new IOException (e.toString());
+            }
+          }
+          inBuf.reset(inflateOut.getData(), inflateOut.getLength());
+        }
+        if(val instanceof Configurable) {
+          ((Configurable) val).setConf(this.conf);
+        }
+        val.readFields(inBuf);
+
+        if (inBuf.getPosition() != inBuf.getLength())
+          throw new IOException(val+" read "+(inBuf.getPosition()-keyLength)
+                                + " bytes, should read " +
+                                (inBuf.getLength()-keyLength));
+      }
+
+      return more;
+    }
+
+    /** Read the next key/value pair in the file into <code>buffer</code>.
+     * Returns the length of the key read, or -1 if at end of file.  The length
+     * of the value may be computed by calling buffer.getLength() before and
+     * after calls to this method. */
+    public synchronized int next(DataOutputBuffer buffer) throws IOException {
+      if (in.getPos() >= end)
+        return -1;
+
+      try {
+        int length = in.readInt();
+
+        if (version[3] > 1 && sync != null &&
+            length == SYNC_ESCAPE) {              // process a sync entry
+          //LOG.info("sync@"+in.getPos());
+          in.readFully(syncCheck);                // read syncCheck
+          if (!Arrays.equals(sync, syncCheck))    // check it
+            throw new IOException("File is corrupt!");
+          syncSeen = true;
+          length = in.readInt();                  // re-read length
+        } else {
+          syncSeen = false;
+        }
+        
+        int keyLength = in.readInt();
+        buffer.write(in, length);
+        return keyLength;
+
+      } catch (ChecksumException e) {             // checksum failure
+        handleChecksumException(e);
+        return next(buffer);
+      }
+    }
+
+    private void handleChecksumException(ChecksumException e)
+      throws IOException {
+      if (this.conf.getBoolean("io.skip.checksum.errors", false)) {
+        LOG.warning("Bad checksum at "+getPosition()+". Skipping entries.");
+        sync(getPosition()+this.conf.getInt("io.bytes.per.checksum", 512));
+      } else {
+        throw e;
+      }
+    }
+
+    /** Set the current byte position in the input file. */
+    public synchronized void seek(long position) throws IOException {
+      in.seek(position);
+    }
+
+    /** Seek to the next sync mark past a given position.*/
+    public synchronized void sync(long position) throws IOException {
+      if (position+SYNC_SIZE >= end) {
+        seek(end);
+        return;
+      }
+
+      try {
+        seek(position+4);                         // skip escape
+        in.readFully(syncCheck);
+        int syncLen = sync.length;
+        for (int i = 0; in.getPos() < end; i++) {
+          int j = 0;
+          for (; j < syncLen; j++) {
+            if (sync[j] != syncCheck[(i+j)%syncLen])
+              break;
+          }
+          if (j == syncLen) {
+            in.seek(in.getPos() - SYNC_SIZE);     // position before sync
+            return;
+          }
+          syncCheck[i%syncLen] = in.readByte();
+        }
+      } catch (ChecksumException e) {             // checksum failure
+        handleChecksumException(e);
+      }
+    }
+
+    /** Returns true iff the previous call to next passed a sync mark.*/
+    public boolean syncSeen() { return syncSeen; }
+
+    /** Return the current byte position in the input file. */
+    public synchronized long getPosition() throws IOException {
+      return in.getPos();
+    }
+
+    /** Returns the name of the file. */
+    public String toString() {
+      return file;
+    }
+
+  }
+
+  /** Sorts key/value pairs in a sequence-format file.
+   *
+   * <p>For best performance, applications should make sure that the {@link
+   * Writable#readFields(DataInput)} implementation of their keys is
+   * very efficient.  In particular, it should avoid allocating memory.
+   */
+  public static class Sorter {
+
+    private WritableComparator comparator;
+
+    private String inFile;                        // when sorting
+    private String[] inFiles;                     // when merging
+
+    private String outFile;
+
+    private int memory; // bytes
+    private int factor; // merged per pass
+
+    private NutchFileSystem nfs = null;
+
+    private Class keyClass;
+    private Class valClass;
+
+    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);
+    }
+
+    /** Sort and merge using an arbitrary {@link WritableComparator}. */
+    public Sorter(NutchFileSystem nfs, WritableComparator comparator, Class valClass, Configuration conf) {
+      this.nfs = nfs;
+      this.comparator = comparator;
+      this.keyClass = comparator.getKeyClass();
+      this.valClass = valClass;
+      this.memory = conf.getInt("io.sort.mb", 100) * 1024 * 1024;
+      this.factor = conf.getInt("io.sort.factor", 100);
+      this.conf = conf;
+    }
+
+    /** Set the number of streams to merge at once.*/
+    public void setFactor(int factor) { this.factor = factor; }
+
+    /** Get the number of streams to merge at once.*/
+    public int getFactor() { return factor; }
+
+    /** Set the total amount of buffer memory, in bytes.*/
+    public void setMemory(int memory) { this.memory = memory; }
+
+    /** Get the total amount of buffer memory, in bytes.*/
+    public int getMemory() { return memory; }
+
+    /** Perform a file sort.*/
+    public void sort(String inFile, String outFile) throws IOException {
+      if (nfs.exists(new File(outFile))) {
+        throw new IOException("already exists: " + outFile);
+      }
+
+      this.inFile = inFile;
+      this.outFile = outFile;
+
+      int segments = sortPass();
+      int pass = 1;
+      while (segments > 1) {
+        segments = mergePass(pass, segments <= factor);
+        pass++;
+      }
+    }
+
+    private int sortPass() throws IOException {
+      LOG.fine("running sort pass");
+      SortPass sortPass = new SortPass(this.conf);         // make the SortPass
+      try {
+        return sortPass.run();                    // run it
+      } finally {
+        sortPass.close();                         // close it
+      }
+    }
+
+    private class SortPass {
+      private int limit = memory/4;
+      private DataOutputBuffer buffer = new DataOutputBuffer();
+      private byte[] rawBuffer;
+
+      private int[] starts = new int[1024];
+      private int[] pointers = new int[starts.length];
+      private int[] pointersCopy = new int[starts.length];
+      private int[] keyLengths = new int[starts.length];
+      private int[] lengths = new int[starts.length];
+      
+      private Reader in;
+      private NFSDataOutputStream out;
+        private String outName;
+
+      public SortPass(Configuration conf) throws IOException {
+        in = new Reader(nfs, inFile, conf);
+      }
+      
+      public int run() throws IOException {
+        int segments = 0;
+        boolean atEof = false;
+        while (!atEof) {
+          int count = 0;
+          buffer.reset();
+          while (!atEof && buffer.getLength() < limit) {
+
+            int start = buffer.getLength();       // read an entry into buffer
+            int keyLength = in.next(buffer);
+            int length = buffer.getLength() - start;
+
+            if (keyLength == -1) {
+              atEof = true;
+              break;
+            }
+
+            if (count == starts.length)
+              grow();
+
+            starts[count] = start;                // update pointers
+            pointers[count] = count;
+            lengths[count] = length;
+            keyLengths[count] = keyLength;
+
+            count++;
+          }
+
+          // buffer is full -- sort & flush it
+          LOG.finer("flushing segment " + segments);
+          rawBuffer = buffer.getData();
+          sort(count);
+          flush(count, segments==0 && atEof);
+          segments++;
+        }
+        return segments;
+      }
+
+      public void close() throws IOException {
+        in.close();
+
+        if (out != null) {
+          out.close();
+        }
+      }
+
+      private void grow() {
+        int newLength = starts.length * 3 / 2;
+        starts = grow(starts, newLength);
+        pointers = grow(pointers, newLength);
+        pointersCopy = new int[newLength];
+        keyLengths = grow(keyLengths, newLength);
+        lengths = grow(lengths, newLength);
+      }
+
+      private int[] grow(int[] old, int newLength) {
+        int[] result = new int[newLength];
+        System.arraycopy(old, 0, result, 0, old.length);
+        return result;
+      }
+
+      private void flush(int count, boolean done) throws IOException {
+        if (out == null) {
+          outName = done ? outFile : outFile+".0";
+          out = nfs.create(new File(outName));
+        }
+
+        if (!done) {                              // an intermediate file
+
+          long length = buffer.getLength();       // compute its size
+          length += count*8;                      // allow for length/keyLength
+
+          out.writeLong(length);                  // write size
+          out.writeLong(count);                   // write count
+        }
+
+        Writer writer = new Writer(out, keyClass, valClass, in.isCompressed());
+        if (!done) {
+          writer.sync = null;                     // disable sync on temp files
+        }
+
+        for (int i = 0; i < count; i++) {         // write in sorted order
+          int p = pointers[i];
+          writer.append(rawBuffer, starts[p], lengths[p], keyLengths[p]);
+        }
+      }
+
+      private void sort(int count) {
+        System.arraycopy(pointers, 0, pointersCopy, 0, count);
+        mergeSort(pointersCopy, pointers, 0, count);
+      }
+
+      private int compare(int i, int j) {
+        return comparator.compare(rawBuffer, starts[i], keyLengths[i],
+                                  rawBuffer, starts[j], keyLengths[j]);
+      }
+
+      private void mergeSort(int src[], int dest[], int low, int high) {
+        int length = high - low;
+
+        // Insertion sort on smallest arrays
+        if (length < 7) {
+          for (int i=low; i<high; i++)
+            for (int j=i; j>low && compare(dest[j-1], dest[j])>0; j--)
+              swap(dest, j, j-1);
+          return;
+        }
+
+        // Recursively sort halves of dest into src
+        int mid = (low + high) >> 1;
+        mergeSort(dest, src, low, mid);
+        mergeSort(dest, src, mid, high);
+
+        // If list is already sorted, just copy from src to dest.  This is an
+        // optimization that results in faster sorts for nearly ordered lists.
+        if (compare(src[mid-1], src[mid]) <= 0) {
+          System.arraycopy(src, low, dest, low, length);
+          return;
+        }
+
+        // Merge sorted halves (now in src) into dest
+        for(int i = low, p = low, q = mid; i < high; i++) {
+          if (q>=high || p<mid && compare(src[p], src[q]) <= 0)
+            dest[i] = src[p++];
+          else
+            dest[i] = src[q++];
+        }
+      }
+
+      private void swap(int x[], int a, int b) {
+	int t = x[a];
+	x[a] = x[b];
+	x[b] = t;
+      }
+    }
+
+    private int mergePass(int pass, boolean last) throws IOException {
+      LOG.fine("running merge pass=" + pass);
+      MergePass mergePass = new MergePass(pass, last);
+      try {                                       // make a merge pass
+        return mergePass.run();                  // run it
+      } finally {
+        mergePass.close();                       // close it
+      }
+    }
+
+    private class MergePass {
+      private int pass;
+      private boolean last;
+
+      private MergeQueue queue;
+      private NFSDataInputStream in;
+      private String inName;
+
+      public MergePass(int pass, boolean last) throws IOException {
+        this.pass = pass;
+        this.last = last;
+
+        this.queue =
+          new MergeQueue(factor, last ? outFile : outFile+"."+pass, last);
+
+        this.inName = outFile+"."+(pass-1);
+        this.in = nfs.open(new File(inName));
+      }
+
+      public void close() throws IOException {
+        in.close();                               // close and delete input
+        nfs.delete(new File(inName));
+
+        queue.close();                            // close queue
+      }
+
+      public int run() throws IOException {
+        int segments = 0;
+        long end = nfs.getLength(new File(inName));
+
+        while (in.getPos() < end) {
+          LOG.finer("merging segment " + segments);
+          long totalLength = 0;
+          long totalCount = 0;
+          while (in.getPos() < end && queue.size() < factor) {
+            long length = in.readLong();
+            long count = in.readLong();
+
+            totalLength += length;
+
+            totalCount+= count;
+
+            Reader reader = new Reader(nfs, inName, memory/(factor+1),
+                                       in.getPos(), length);
+            reader.sync = null;                   // disable sync on temp files
+
+            MergeStream ms = new MergeStream(reader); // add segment to queue
+            if (ms.next()) {
+              queue.add(ms);
+            }
+            in.seek(reader.end);
+          }
+
+          if (!last) {                             // intermediate file
+            queue.out.writeLong(totalLength);     // write size
+            queue.out.writeLong(totalCount);      // write count
+          }
+
+          queue.merge();                          // do a merge
+
+          segments++;
+        }
+
+        return segments;
+      }
+    }
+
+    /** Merge the provided files.*/
+    public void merge(String[] inFiles, String outFile) throws IOException {
+      this.inFiles = inFiles;
+      this.outFile = outFile;
+      this.factor = inFiles.length;
+
+      if (new File(outFile).exists()) {
+        throw new IOException("already exists: " + outFile);
+      }
+
+      MergeFiles mergeFiles = new MergeFiles();
+      try {                                       // make a merge pass
+        mergeFiles.run();                         // run it
+      } finally {
+        mergeFiles.close();                       // close it
+      }
+
+    }
+
+    private class MergeFiles {
+      private MergeQueue queue;
+
+      public MergeFiles() throws IOException {
+        this.queue = new MergeQueue(factor, outFile, true);
+      }
+
+      public void close() throws IOException {
+        queue.close();
+      }
+
+      public void run() throws IOException {
+        LOG.finer("merging files=" + inFiles.length);
+        for (int i = 0; i < inFiles.length; i++) {
+          String inFile = inFiles[i];
+          MergeStream ms =
+            new MergeStream(new Reader(nfs, inFile, memory/(factor+1)));
+          if (ms.next())
+            queue.put(ms);
+        }
+
+        queue.merge();
+      }
+    }
+
+    private class MergeStream {
+      private Reader in;
+
+      private DataOutputBuffer buffer = new DataOutputBuffer();
+      private int keyLength;
+      
+      public MergeStream(Reader reader) throws IOException {
+        if (reader.keyClass != keyClass)
+          throw new IOException("wrong key class: " + reader.getKeyClass() +
+                                " is not " + keyClass);
+        if (reader.valClass != valClass)
+          throw new IOException("wrong value class: "+reader.getValueClass()+
+                                " is not " + valClass);
+        this.in = reader;
+      }
+
+      public boolean next() throws IOException {
+        buffer.reset();
+        keyLength = in.next(buffer);
+        return keyLength >= 0;
+      }
+    }
+
+    private class MergeQueue extends PriorityQueue {
+      private NFSDataOutputStream out;
+      private boolean done;
+      private boolean compress;
+
+      public void add(MergeStream stream) throws IOException {
+        if (size() == 0) {
+          compress = stream.in.isCompressed();
+        } else if (compress != stream.in.isCompressed()) {
+          throw new IOException("All merged files must be compressed or not.");
+        }
+        put(stream);
+      }
+
+      public MergeQueue(int size, String outName, boolean done)
+        throws IOException {
+        initialize(size);
+        this.out = nfs.create(new File(outName), true, memory/(factor+1));
+        this.done = done;
+      }
+
+      protected boolean lessThan(Object a, Object b) {
+        MergeStream msa = (MergeStream)a;
+        MergeStream msb = (MergeStream)b;
+        return comparator.compare(msa.buffer.getData(), 0, msa.keyLength,
+                                  msb.buffer.getData(), 0, msb.keyLength) < 0;
+      }
+
+      public void merge() throws IOException {
+        Writer writer = new Writer(out, keyClass, valClass, compress);
+        if (!done) {
+          writer.sync = null;                     // disable sync on temp files
+        }
+
+        while (size() != 0) {
+          MergeStream ms = (MergeStream)top();
+          DataOutputBuffer buffer = ms.buffer;    // write top entry
+          writer.append(buffer.getData(), 0, buffer.getLength(), ms.keyLength);
+          
+          if (ms.next()) {                        // has another entry
+            adjustTop();
+          } else {
+            pop();                                // done with this file
+            ms.in.close();
+          }
+        }
+      }
+
+      public void close() throws IOException {
+        MergeStream ms;                           // close inputs
+        while ((ms = (MergeStream)pop()) != null) {
+          ms.in.close();
+        }
+        out.close();                              // close output
+      }
+    }
+  }
+
+}

+ 89 - 0
src/java/org/apache/hadoop/io/SetFile.java

@@ -0,0 +1,89 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.*;
+
+/** A file-based set of keys. */
+public class SetFile extends MapFile {
+
+  protected SetFile() {}                            // no public ctor
+
+  /** Write a new set file. */
+  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);
+    }
+
+    /** Create the named set using the named key comparator. */
+    public Writer(NutchFileSystem nfs, String dirName, WritableComparator comparator)
+      throws IOException {
+      super(nfs, dirName, comparator, NullWritable.class);
+    }
+
+    /** Append a key to a set.  The key must be strictly greater than the
+     * previous key added to the set. */
+    public void append(WritableComparable key) throws IOException{
+      append(key, NullWritable.get());
+    }
+  }
+
+  /** Provide access to an existing set file. */
+  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);
+    }
+
+    /** Construct a set reader for the named set using the named comparator.*/
+    public Reader(NutchFileSystem nfs, String dirName, WritableComparator comparator, Configuration conf)
+      throws IOException {
+      super(nfs, dirName, comparator, conf);
+    }
+
+    // javadoc inherited
+    public boolean seek(WritableComparable key)
+      throws IOException {
+      return super.seek(key);
+    }
+
+    /** Read the next key in a set into <code>key</code>.  Returns
+     * true if such a key exists and false when at the end of the set. */
+    public boolean next(WritableComparable key)
+      throws IOException {
+      return next(key, NullWritable.get());
+    }
+
+    /** Read the matching key from a set into <code>key</code>.
+     * Returns <code>key</code>, or null if no match exists. */
+    public WritableComparable get(WritableComparable key)
+      throws IOException {
+      if (seek(key)) {
+        next(key);
+        return key;
+      } else
+        return null;
+    }
+  }
+
+}

+ 89 - 0
src/java/org/apache/hadoop/io/TwoDArrayWritable.java

@@ -0,0 +1,89 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+import java.lang.reflect.Array;
+
+/** A Writable for 2D arrays containing a matrix of instances of a class. */
+public class TwoDArrayWritable implements Writable {
+  private Class valueClass;
+  private Writable[][] values;
+
+  public TwoDArrayWritable(Class valueClass) {
+    this.valueClass = valueClass;
+  }
+
+  public TwoDArrayWritable(Class valueClass, Writable[][] values) {
+    this(valueClass);
+    this.values = values;
+  }
+
+  public Object toArray() {
+      int dimensions[] = {values.length, 0};
+      Object result = Array.newInstance(valueClass, dimensions);
+      for (int i = 0; i < values.length; i++) {
+          Object resultRow = Array.newInstance(valueClass, values[i].length);
+          Array.set(result, i, resultRow);
+          for (int j = 0; j < values[i].length; j++) {
+              Array.set(resultRow, j, values[i][j]);
+          }
+      }
+      return result;
+  }
+
+  public void set(Writable[][] values) { this.values = values; }
+
+  public Writable[][] get() { return values; }
+
+  public void readFields(DataInput in) throws IOException {
+    // construct matrix
+    values = new Writable[in.readInt()][];          
+    for (int i = 0; i < values.length; i++) {
+        values[i] = new Writable[in.readInt()];
+    }
+
+    // construct values
+    for (int i = 0; i < values.length; i++) {
+        for (int j = 0; j < values[i].length; j++) {
+            Writable value;                             // construct value
+            try {
+                value = (Writable)valueClass.newInstance();
+            } catch (InstantiationException e) {
+                throw new RuntimeException(e.toString());
+            } catch (IllegalAccessException e) {
+                throw new RuntimeException(e.toString());
+            }
+            value.readFields(in);                       // read a value
+            values[i][j] = value;                       // store it in values
+        }
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(values.length);                 // write values
+    for (int i = 0; i < values.length; i++) {
+        out.writeInt(values[i].length);
+    }
+    for (int i = 0; i < values.length; i++) {
+        for (int j = 0; j < values[i].length; j++) {
+            values[i][j].write(out);
+        }
+    }
+  }
+}
+

+ 287 - 0
src/java/org/apache/hadoop/io/UTF8.java

@@ -0,0 +1,287 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.IOException;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+import java.util.logging.Logger;
+import org.apache.hadoop.util.LogFormatter;
+
+/** A WritableComparable for strings that uses the UTF8 encoding.
+ * 
+ * <p>Also includes utilities for efficiently reading and writing UTF-8.
+ *
+ * @author Doug Cutting
+ */
+public class UTF8 implements WritableComparable {
+  private static final Logger LOG= LogFormatter.getLogger("org.apache.hadoop.io.UTF8");
+  private static final DataOutputBuffer OBUF = new DataOutputBuffer();
+  private static final DataInputBuffer IBUF = new DataInputBuffer();
+
+  private static final byte[] EMPTY_BYTES = new byte[0];
+
+  private byte[] bytes = EMPTY_BYTES;
+  private int length;
+
+  public UTF8() {
+    //set("");
+  }
+
+  /** Construct from a given string. */
+  public UTF8(String string) {
+    set(string);
+  }
+
+  /** Construct from a given string. */
+  public UTF8(UTF8 utf8) {
+    set(utf8);
+  }
+
+  /** The raw bytes. */
+  public byte[] getBytes() {
+    return bytes;
+  }
+
+  /** The number of bytes in the encoded string. */
+  public int getLength() {
+    return length;
+  }
+
+  /** Set to contain the contents of a string. */
+  public void set(String string) {
+    if (string.length() > 0xffff/3) {             // maybe too long
+      LOG.warning("truncating long string: " + string.length()
+                  + " chars, starting with " + string.substring(0, 20));
+      string = string.substring(0, 0xffff/3);
+    }
+
+    length = utf8Length(string);                  // compute length
+    if (length > 0xffff)                          // double-check length
+      throw new RuntimeException("string too long!");
+
+    if (bytes == null || length > bytes.length)   // grow buffer
+      bytes = new byte[length];
+
+    try {                                         // avoid sync'd allocations
+      synchronized (OBUF) {
+        OBUF.reset();
+        writeChars(OBUF, string, 0, string.length());
+        System.arraycopy(OBUF.getData(), 0, bytes, 0, length);
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Set to contain the contents of a string. */
+  public void set(UTF8 other) {
+    length = other.length;
+    if (bytes == null || length > bytes.length)   // grow buffer
+      bytes = new byte[length];
+    System.arraycopy(other.bytes, 0, bytes, 0, length);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    length = in.readUnsignedShort();
+    if (bytes == null || bytes.length < length)
+      bytes = new byte[length];
+    in.readFully(bytes, 0, length);
+  }
+
+  /** Skips over one UTF8 in the input. */
+  public static void skip(DataInput in) throws IOException {
+    int length = in.readUnsignedShort();
+    in.skipBytes(length);
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeShort(length);
+    out.write(bytes, 0, length);
+  }
+
+  /** Compare two UTF8s. */
+  public int compareTo(Object o) {
+    UTF8 that = (UTF8)o;
+    return WritableComparator.compareBytes(bytes, 0, length,
+                                           that.bytes, 0, that.length);
+  }
+
+  /** Convert to a String. */
+  public String toString() {
+    StringBuffer buffer = new StringBuffer(length);
+    try {
+      synchronized (IBUF) {
+        IBUF.reset(bytes, length);
+        readChars(IBUF, buffer, length);
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return buffer.toString();
+  }
+
+  /** Returns true iff <code>o</code> is a UTF8 with the same contents.  */
+  public boolean equals(Object o) {
+    if (!(o instanceof UTF8))
+      return false;
+    UTF8 that = (UTF8)o;
+    if (this.length != that.length)
+      return false;
+    else
+      return WritableComparator.compareBytes(bytes, 0, length,
+                                             that.bytes, 0, that.length) == 0;
+  }
+
+  public int hashCode() {
+    int hash = 1;
+    for (int i = 0; i < length; i++)
+      hash = (31 * hash) + (int)bytes[i];
+    return hash;
+  }
+
+  /** A WritableComparator optimized for UTF8 keys. */
+  public static class Comparator extends WritableComparator {
+    public Comparator() {
+      super(UTF8.class);
+    }
+
+    public int compare(byte[] b1, int s1, int l1,
+                       byte[] b2, int s2, int l2) {
+      int n1 = readUnsignedShort(b1, s1);
+      int n2 = readUnsignedShort(b2, s2);
+      return compareBytes(b1, s1+2, n1, b2, s2+2, n2);
+    }
+  }
+
+  static {                                        // register this comparator
+    WritableComparator.define(UTF8.class, new Comparator());
+  }
+
+  /// STATIC UTILITIES FROM HERE DOWN
+
+  /// These are probably not used much anymore, and might be removed...
+
+  /** Convert a string to a UTF-8 encoded byte array.
+   * @see String#getBytes(String)
+   */
+  public static byte[] getBytes(String string) {
+    byte[] result = new byte[utf8Length(string)];
+    try {                                         // avoid sync'd allocations
+      synchronized (OBUF) {
+        OBUF.reset();
+        writeChars(OBUF, string, 0, string.length());
+        System.arraycopy(OBUF.getData(), 0, result, 0, OBUF.getLength());
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return result;
+  }
+
+  /** Read a UTF-8 encoded string.
+   *
+   * @see DataInput#readUTF()
+   */
+  public static String readString(DataInput in) throws IOException {
+    int bytes = in.readUnsignedShort();
+    StringBuffer buffer = new StringBuffer(bytes);
+    readChars(in, buffer, bytes);
+    return buffer.toString();
+  }
+
+  private static void readChars(DataInput in, StringBuffer buffer, int nBytes)
+    throws IOException {
+    synchronized (OBUF) {
+      OBUF.reset();
+      OBUF.write(in, nBytes);
+      byte[] bytes = OBUF.getData();
+      int i = 0;
+      while (i < nBytes) {
+        byte b = bytes[i++];
+        if ((b & 0x80) == 0) {
+          buffer.append((char)(b & 0x7F));
+        } else if ((b & 0xE0) != 0xE0) {
+          buffer.append((char)(((b & 0x1F) << 6)
+                               | (bytes[i++] & 0x3F)));
+        } else {
+          buffer.append((char)(((b & 0x0F) << 12)
+                               | ((bytes[i++] & 0x3F) << 6)
+                               |  (bytes[i++] & 0x3F)));
+        }
+      }
+    }
+  }
+
+  /** Write a UTF-8 encoded string.
+   *
+   * @see DataOutput#writeUTF(String)
+   */
+  public static int writeString(DataOutput out, String s) throws IOException {
+    if (s.length() > 0xffff/3) {         // maybe too long
+      LOG.warning("truncating long string: " + s.length()
+                  + " chars, starting with " + s.substring(0, 20));
+      s = s.substring(0, 0xffff/3);
+    }
+
+    int len = utf8Length(s);
+    if (len > 0xffff)                             // double-check length
+      throw new IOException("string too long!");
+      
+    out.writeShort(len);
+    writeChars(out, s, 0, s.length());
+    return len;
+  }
+
+  /** Returns the number of bytes required to write this. */
+  private static int utf8Length(String string) {
+    int stringLength = string.length();
+    int utf8Length = 0;
+    for (int i = 0; i < stringLength; i++) {
+      int c = string.charAt(i);
+      if ((c >= 0x0001) && (c <= 0x007F)) {
+        utf8Length++;
+      } else if (c > 0x07FF) {
+        utf8Length += 3;
+      } else {
+        utf8Length += 2;
+      }
+    }
+    return utf8Length;
+  }
+
+  private static void writeChars(DataOutput out,
+                                 String s, int start, int length)
+    throws IOException {
+    final int end = start + length;
+    for (int i = start; i < end; i++) {
+      int code = s.charAt(i);
+      if (code >= 0x01 && code <= 0x7F) {
+        out.writeByte((byte)code);
+      } else if (code <= 0x07FF) {
+        out.writeByte((byte)(0xC0 | ((code >> 6) & 0x1F)));
+        out.writeByte((byte)(0x80 |   code       & 0x3F));
+      } else {
+        out.writeByte((byte)(0xE0 | ((code >> 12) & 0X0F)));
+        out.writeByte((byte)(0x80 | ((code >>  6) & 0x3F)));
+        out.writeByte((byte)(0x80 |  (code        & 0x3F)));
+      }
+    }
+  }
+
+}

+ 39 - 0
src/java/org/apache/hadoop/io/VersionMismatchException.java

@@ -0,0 +1,39 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.IOException;
+
+/** Thrown by {@link VersionedWritable#readFields(DataInput)} when the
+ * version of an object being read does not match the current implementation
+ * version as returned by {@link VersionedWritable#getVersion()}. */
+public class VersionMismatchException extends IOException {
+
+  private byte expectedVersion;
+  private byte foundVersion;
+
+  public VersionMismatchException(byte expectedVersionIn, byte foundVersionIn){
+    expectedVersion = expectedVersionIn;
+    foundVersion = foundVersionIn;
+  }
+
+  /** Returns a string representation of this object. */
+  public String toString(){
+    return "A record version mismatch occured. Expecting v"
+      + expectedVersion + ", found v" + foundVersion; 
+  }
+}

+ 50 - 0
src/java/org/apache/hadoop/io/VersionedWritable.java

@@ -0,0 +1,50 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.DataOutput;
+import java.io.DataInput;
+import java.io.IOException;
+
+/** A base class for Writables that provides version checking.
+ *
+ * <p>This is useful when a class may evolve, so that instances written by the
+ * old version of the class may still be processed by the new version.  To
+ * handle this situation, {@link #readFields(DataInput)}
+ * implementations should catch {@link VersionMismatchException}.
+ *
+ * @author Doug Cutting
+ */
+public abstract class VersionedWritable implements Writable {
+
+  /** Return the version number of the current implementation. */
+  public abstract byte getVersion();
+    
+  // javadoc from Writable
+  public void write(DataOutput out) throws IOException {
+    out.writeByte(getVersion());                  // store version
+  }
+
+  // javadoc from Writable
+  public void readFields(DataInput in) throws IOException {
+    byte version = in.readByte();                 // read version
+    if (version != getVersion())
+      throw new VersionMismatchException(getVersion(),version);
+  }
+
+    
+}

+ 41 - 0
src/java/org/apache/hadoop/io/Writable.java

@@ -0,0 +1,41 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.DataOutput;
+import java.io.DataInput;
+import java.io.IOException;
+
+/** A simple, efficient, serialization protocol, based on {@link DataInput} and
+ * {@link DataOutput}.
+ *
+ * <p>Implementations typically implement a static <code>read(DataInput)</code>
+ * method which constructs a new instance, calls {@link
+ * #readFields(DataInput)}, and returns the instance.
+ *
+ * @author Doug Cutting
+ */
+public interface Writable {
+  /** Writes the fields of this object to <code>out</code>. */
+  void write(DataOutput out) throws IOException;
+
+  /** Reads the fields of this object from <code>in</code>.  For efficiency,
+   * implementations should attempt to re-use storage in the existing object
+   * where possible.
+   */
+  void readFields(DataInput in) throws IOException;
+}

+ 24 - 0
src/java/org/apache/hadoop/io/WritableComparable.java

@@ -0,0 +1,24 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+/** An interface which extends both {@link Writable} and {@link Comparable}.
+ *
+ * @author Doug Cutting
+ */
+public interface WritableComparable extends Writable, Comparable {
+}

+ 153 - 0
src/java/org/apache/hadoop/io/WritableComparator.java

@@ -0,0 +1,153 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+import java.util.*;
+
+/** A Comparator for {@link WritableComparable}s.
+ *
+ * <p>This base implemenation uses the natural ordering.  To define alternate
+ * orderings, override {@link #compare(WritableComparable,WritableComparable)}.
+ *
+ * <p>One may optimize compare-intensive operations by overriding
+ * {@link #compare(byte[],int,int,byte[],int,int)}.  Static utility methods are
+ * provided to assist in optimized implementations of this method.
+ */
+public class WritableComparator implements Comparator {
+
+  private static HashMap comparators = new HashMap(); // registry
+
+  /** Get a comparator for a {@link WritableComparable} implementation. */
+  public static synchronized WritableComparator get(Class c) {
+    WritableComparator comparator = (WritableComparator)comparators.get(c);
+    if (comparator == null)
+      comparator = new WritableComparator(c);
+    return comparator;
+  }
+
+  /** Register an optimized comparator for a {@link WritableComparable}
+   * implementation. */
+  public static synchronized void define(Class c,
+                                         WritableComparator comparator) {
+    comparators.put(c, comparator);
+  }
+
+
+  private DataInputBuffer buffer = new DataInputBuffer();
+
+  private Class keyClass;
+  private WritableComparable key1;
+  private WritableComparable key2;
+
+  /** Construct for a {@link WritableComparable} implementation. */
+  protected WritableComparator(Class keyClass) {
+    this.keyClass = keyClass;
+    this.key1 = newKey();
+    this.key2 = newKey();
+  }
+
+  /** Returns the WritableComparable implementation class. */
+  public Class getKeyClass() { return keyClass; }
+
+  /** Construct a new {@link WritableComparable} instance. */
+  public WritableComparable newKey() {
+    try {
+      return (WritableComparable)keyClass.newInstance();
+    } catch (InstantiationException e) {
+      throw new RuntimeException(e);
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Optimization hook.  Override this to make SequenceFile.Sorter's scream.
+   *
+   * <p>The default implementation reads the data into two {@link
+   * WritableComparable}s (using {@link
+   * Writable#readFields(DataInput)}, then calls {@link
+   * #compare(WritableComparable,WritableComparable)}.
+   */
+  public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+    try {
+      buffer.reset(b1, s1, l1);                   // parse key1
+      key1.readFields(buffer);
+      
+      buffer.reset(b2, s2, l2);                   // parse key2
+      key2.readFields(buffer);
+      
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    
+    return compare(key1, key2);                   // compare them
+  }
+
+  /** Compare two WritableComparables.
+   *
+   * <p> The default implementation uses the natural ordering, calling {@link
+   * Comparable#compareTo(Object)}. */
+  public int compare(WritableComparable a, WritableComparable b) {
+    return a.compareTo(b);
+  }
+
+  public int compare(Object a, Object b) {
+    return compare((WritableComparable)a, (WritableComparable)b);
+  }
+
+  /** Lexicographic order of binary data. */
+  public static int compareBytes(byte[] b1, int s1, int l1,
+                                 byte[] b2, int s2, int l2) {
+    int end1 = s1 + l1;
+    int end2 = s2 + l2;
+    for (int i = s1, j = s2; i < end1 && j < end2; i++, j++) {
+      int a = (b1[i] & 0xff);
+      int b = (b2[j] & 0xff);
+      if (a != b) {
+        return a - b;
+      }
+    }
+    return l1 - l2;
+  }
+
+  /** Parse an unsigned short from a byte array. */
+  public static int readUnsignedShort(byte[] bytes, int start) {
+    return (((bytes[start]   & 0xff) <<  8) +
+            ((bytes[start+1] & 0xff)));
+  }
+
+  /** Parse an integer from a byte array. */
+  public static int readInt(byte[] bytes, int start) {
+    return (((bytes[start  ] & 0xff) << 24) +
+            ((bytes[start+1] & 0xff) << 16) +
+            ((bytes[start+2] & 0xff) <<  8) +
+            ((bytes[start+3] & 0xff)));
+
+  }
+
+  /** Parse a float from a byte array. */
+  public static float readFloat(byte[] bytes, int start) {
+    return Float.intBitsToFloat(readInt(bytes, start));
+  }
+
+  /** Parse a long from a byte array. */
+  public static long readLong(byte[] bytes, int start) {
+    return ((long)(readInt(bytes, start)) << 32) +
+      (readInt(bytes, start+4) & 0xFFFFFFFFL);
+  }
+
+}

+ 71 - 0
src/java/org/apache/hadoop/io/WritableName.java

@@ -0,0 +1,71 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.util.HashMap;
+import java.io.IOException;
+
+/** Utility to permit renaming of Writable implementation classes without
+ * invalidiating files that contain their class name.
+ * @author Doug Cutting
+ */
+public class WritableName {
+  private static HashMap NAME_TO_CLASS = new HashMap();
+  private static HashMap CLASS_TO_NAME = new HashMap();
+
+  static {                                        // define important types
+    WritableName.setName(NullWritable.class, "null");
+    WritableName.setName(LongWritable.class, "long");
+    WritableName.setName(UTF8.class, "UTF8");
+    WritableName.setName(MD5Hash.class, "MD5Hash");
+  }
+
+  private WritableName() {}                      // no public ctor
+
+  /** Set the name that a class should be known as to something other than the
+   * class name. */
+  public static synchronized void setName(Class writableClass, String name) {
+    CLASS_TO_NAME.put(writableClass, name);
+    NAME_TO_CLASS.put(name, writableClass);
+  }
+
+  /** Add an alternate name for a class. */
+  public static synchronized void addName(Class writableClass, String name) {
+    NAME_TO_CLASS.put(name, writableClass);
+  }
+
+  /** Return the name for a class.  Default is {@link Class#getName()}. */
+  public static synchronized String getName(Class writableClass) {
+    String name = (String)CLASS_TO_NAME.get(writableClass);
+    if (name != null)
+      return name;
+    return writableClass.getName();
+  }
+
+  /** Return the class for a name.  Default is {@link Class#forName(String)}.*/
+  public static synchronized Class getClass(String name) throws IOException {
+    Class writableClass = (Class)NAME_TO_CLASS.get(name);
+    if (writableClass != null)
+      return writableClass;
+    try {
+      return Class.forName(name);
+    } catch (ClassNotFoundException e) {
+      throw new IOException(e.toString());
+    }
+  }
+
+}

+ 193 - 0
src/java/org/apache/hadoop/io/WritableUtils.java

@@ -0,0 +1,193 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.io.
+
+import java.io.*;
+
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
+public final class WritableUtils  {
+
+  public static byte[] readCompressedByteArray(DataInput in) throws IOException {
+    int length = in.readInt();
+    if (length == -1) return null;
+    byte[] buffer = new byte[length];
+    in.readFully(buffer);      // could/should use readFully(buffer,0,length)?
+    GZIPInputStream gzi = new GZIPInputStream(new ByteArrayInputStream(buffer, 0, buffer.length));
+    byte[] outbuf = new byte[length];
+    ByteArrayOutputStream bos =  new ByteArrayOutputStream();
+     int len;
+     while((len=gzi.read(outbuf,0,outbuf.length)) != -1){
+       bos.write(outbuf,0,len);
+     }
+     byte[] decompressed =  bos.toByteArray();
+     bos.close();
+     gzi.close();
+     return decompressed;
+  }
+
+  public static void skipCompressedByteArray(DataInput in) throws IOException {
+    int length = in.readInt();
+    if (length != -1) in.skipBytes(length);
+  }
+
+  public static int  writeCompressedByteArray(DataOutput out, byte[] bytes) throws IOException {
+    if (bytes != null) {
+      ByteArrayOutputStream bos =  new ByteArrayOutputStream();
+      GZIPOutputStream gzout = new GZIPOutputStream(bos);
+      gzout.write(bytes,0,bytes.length);
+      gzout.close();
+      byte[] buffer = bos.toByteArray();
+      int len = buffer.length;
+      out.writeInt(len);
+      out.write(buffer,0,len);
+    /* debug only! Once we have confidence, can lose this. */
+      return ((bytes.length != 0) ? (100*buffer.length)/bytes.length : 0);
+    } else {
+      out.writeInt(-1);
+      return -1;
+    }
+  }
+
+
+  /* Ugly utility, maybe someone else can do this better  */
+  public static String readCompressedString(DataInput in) throws IOException {
+    byte[] bytes = readCompressedByteArray(in);
+    if (bytes == null) return null;
+    return new String(bytes, "UTF-8");
+  }
+
+
+  public static int  writeCompressedString(DataOutput out, String s) throws IOException {
+    return writeCompressedByteArray(out, (s != null) ? s.getBytes("UTF-8") : null);
+  }
+
+  /*
+   *
+   * Write a String as a Network Int n, followed by n Bytes
+   * Alternative to 16 bit read/writeUTF.
+   * Encoding standard is... ?
+   * 
+   */
+  public static void writeString(DataOutput out, String s) throws IOException {
+    if (s != null) {
+      byte[] buffer = s.getBytes("UTF-8");
+      int len = buffer.length;
+      out.writeInt(len);
+      out.write(buffer,0,len);
+    } else {
+      out.writeInt(-1);
+    }
+  }
+
+  /*
+   * Read a String as a Network Int n, followed by n Bytes
+   * Alternative to 16 bit read/writeUTF.
+   * Encoding standard is... ?
+   *
+   */
+  public static String readString(DataInput in) throws IOException{
+    int length = in.readInt();
+    if (length == -1) return null;
+    byte[] buffer = new byte[length];
+    in.readFully(buffer);      // could/should use readFully(buffer,0,length)?
+    return new String(buffer,"UTF-8");  
+  }
+
+
+  /*
+   * Write a String array as a Nework Int N, followed by Int N Byte Array Strings.
+   * Could be generalised using introspection.
+   *
+   */
+  public static void writeStringArray(DataOutput out, String[] s) throws IOException{
+    out.writeInt(s.length);
+    for(int i = 0; i < s.length; i++) {
+      writeString(out, s[i]);
+    }
+  }
+
+  /*
+   * Write a String array as a Nework Int N, followed by Int N Byte Array of
+   * compressed Strings. Handles also null arrays and null values.
+   * Could be generalised using introspection.
+   *
+   */
+  public static void writeCompressedStringArray(DataOutput out, String[] s) throws IOException{
+    if (s == null) {
+      out.writeInt(-1);
+      return;
+    }
+    out.writeInt(s.length);
+    for(int i = 0; i < s.length; i++) {
+      writeCompressedString(out, s[i]);
+    }
+  }
+
+  /*
+   * Write a String array as a Nework Int N, followed by Int N Byte Array Strings.
+   * Could be generalised using introspection. Actually this bit couldn't...
+   *
+   */
+  public static String[] readStringArray(DataInput in) throws IOException {
+    int len = in.readInt();
+    if (len == -1) return null;
+    String[] s = new String[len];
+    for(int i = 0; i < len; i++) {
+      s[i] = readString(in);
+    }
+    return s;
+  }
+
+
+  /*
+   * Write a String array as a Nework Int N, followed by Int N Byte Array Strings.
+   * Could be generalised using introspection. Handles null arrays and null values.
+   *
+   */
+  public static  String[] readCompressedStringArray(DataInput in) throws IOException {
+    int len = in.readInt();
+    if (len == -1) return null;
+    String[] s = new String[len];
+    for(int i = 0; i < len; i++) {
+      s[i] = readCompressedString(in);
+    }
+    return s;
+  }
+
+
+  /*
+   *
+   * Test Utility Method Display Byte Array. 
+   *
+   */
+  public static void displayByteArray(byte[] record){
+    int i;
+    for(i=0;i < record.length -1 ; i++){
+      if (i % 16 == 0) { System.out.println(); }
+      System.out.print(Integer.toHexString(record[i]  >> 4 & 0x0F));
+      System.out.print(Integer.toHexString(record[i] & 0x0F));
+      System.out.print(",");
+    }
+    System.out.print(Integer.toHexString(record[i]  >> 4 & 0x0F));
+    System.out.print(Integer.toHexString(record[i] & 0x0F));
+    System.out.println();
+  }
+
+
+}

+ 6 - 0
src/java/org/apache/hadoop/io/package.html

@@ -0,0 +1,6 @@
+<html>
+<body>
+Generic i/o code for use when reading and writing data to the network,
+to databases, and to files.
+</body>
+</html>

+ 370 - 0
src/java/org/apache/hadoop/ipc/Client.java

@@ -0,0 +1,370 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ipc.
+
+import java.net.Socket;
+import java.net.InetSocketAddress;
+import java.net.SocketTimeoutException;
+
+import java.io.IOException;
+import java.io.EOFException;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.FilterInputStream;
+import java.io.FilterOutputStream;
+
+import java.util.Hashtable;
+import java.util.logging.Logger;
+import java.util.logging.Level;
+
+import org.apache.hadoop.util.LogFormatter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.UTF8;
+
+/** A client for an IPC service.  IPC calls take a single {@link Writable} as a
+ * parameter, and return a {@link Writable} as their value.  A service runs on
+ * a port and is defined by a parameter class and a value class.
+ * 
+ * @author Doug Cutting
+ * @see Server
+ */
+public class Client {
+  public static final Logger LOG =
+    LogFormatter.getLogger("org.apache.hadoop.ipc.Client");
+
+  private Hashtable connections = new Hashtable();
+
+  private Class valueClass;                       // class of call values
+  private int timeout ;// timeout for calls
+  private int counter;                            // counter for call ids
+  private boolean running = true;                 // true while client runs
+  private Configuration conf;
+
+  /** A call waiting for a value. */
+  private class Call {
+    int id;                                       // call id
+    Writable param;                               // parameter
+    Writable value;                               // value, null if error
+    String error;                                 // error, null if value
+    long lastActivity;                            // time of last i/o
+    boolean done;                                 // true when call is done
+
+    protected Call(Writable param) {
+      this.param = param;
+      synchronized (Client.this) {
+        this.id = counter++;
+      }
+      touch();
+    }
+
+    /** Called by the connection thread when the call is complete and the
+     * value or error string are available.  Notifies by default.  */
+    public synchronized void callComplete() {
+        notify();                                 // notify caller
+    }
+
+    /** Update lastActivity with the current time. */
+    public synchronized void touch() {
+      lastActivity = System.currentTimeMillis();
+    }
+
+    /** Update lastActivity with the current time. */
+    public synchronized void setResult(Writable value, String error) {
+      this.value = value;
+      this.error = error;
+      this.done = true;
+    }
+    
+  }
+
+  /** Thread that reads responses and notifies callers.  Each connection owns a
+   * socket connected to a remote address.  Calls are multiplexed through this
+   * socket: responses may be delivered out of order. */
+  private class Connection extends Thread {
+    private InetSocketAddress address;            // address of server
+    private Socket socket;                        // connected socket
+    private DataInputStream in;                   
+    private DataOutputStream out;
+    private Hashtable calls = new Hashtable();    // currently active calls
+    private Call readingCall;
+    private Call writingCall;
+
+    public Connection(InetSocketAddress address) throws IOException {
+      this.address = address;
+      this.socket = new Socket(address.getAddress(), address.getPort());
+      socket.setSoTimeout(timeout);
+      this.in = new DataInputStream
+        (new BufferedInputStream
+         (new FilterInputStream(socket.getInputStream()) {
+             public int read(byte[] buf, int off, int len) throws IOException {
+               int value = super.read(buf, off, len);
+               if (readingCall != null) {
+                 readingCall.touch();
+               }
+               return value;
+             }
+          }));
+      this.out = new DataOutputStream
+        (new BufferedOutputStream
+         (new FilterOutputStream(socket.getOutputStream()) {
+             public void write(byte[] buf, int o, int len) throws IOException {
+               super.write(buf, o, len);
+               if (writingCall != null) {
+                 writingCall.touch();
+               }
+             }
+           }));
+      this.setDaemon(true);
+      this.setName("Client connection to "
+                   + address.getAddress().getHostAddress()
+                   + ":" + address.getPort());
+    }
+
+    public void run() {
+      LOG.info(getName() + ": starting");
+      try {
+        while (running) {
+          int id;
+          try {
+            id = in.readInt();                    // try to read an id
+          } catch (SocketTimeoutException e) {
+            continue;
+          }
+
+          if (LOG.isLoggable(Level.FINE))
+            LOG.fine(getName() + " got value #" + id);
+
+          Call call = (Call)calls.remove(new Integer(id));
+          boolean isError = in.readBoolean();     // read if error
+          if (isError) {
+            UTF8 utf8 = new UTF8();
+            utf8.readFields(in);                  // read error string
+            call.setResult(null, utf8.toString());
+          } else {
+            Writable value = makeValue();
+            try {
+              readingCall = call;
+              if(value instanceof Configurable) {
+                ((Configurable) value).setConf(conf);
+              }
+              value.readFields(in);                 // read value
+            } finally {
+              readingCall = null;
+            }
+            call.setResult(value, null);
+          }
+          call.callComplete();                   // deliver result to caller
+        }
+      } catch (EOFException eof) {
+          // This is what happens when the remote side goes down
+      } catch (Exception e) {
+        LOG.log(Level.INFO, getName() + " caught: " + e, e);
+      } finally {
+        close();
+      }
+    }
+
+    /** Initiates a call by sending the parameter to the remote server.
+     * Note: this is not called from the Connection thread, but by other
+     * threads.
+     */
+    public void sendParam(Call call) throws IOException {
+      boolean error = true;
+      try {
+        calls.put(new Integer(call.id), call);
+        synchronized (out) {
+          if (LOG.isLoggable(Level.FINE))
+            LOG.fine(getName() + " sending #" + call.id);
+          try {
+            writingCall = call;
+            out.writeInt(call.id);
+            call.param.write(out);
+            out.flush();
+          } finally {
+            writingCall = null;
+          }
+        }
+        error = false;
+      } finally {
+        if (error)
+          close();                                // close on error
+      }
+    }
+
+    /** Close the connection and remove it from the pool. */
+    public void close() {
+      LOG.info(getName() + ": closing");
+      synchronized (connections) {
+        connections.remove(address);              // remove connection
+      }
+      try {
+        socket.close();                           // close socket
+      } catch (IOException e) {}
+    }
+
+  }
+
+  /** Call implementation used for parallel calls. */
+  private class ParallelCall extends Call {
+    private ParallelResults results;
+    private int index;
+    
+    public ParallelCall(Writable param, ParallelResults results, int index) {
+      super(param);
+      this.results = results;
+      this.index = index;
+    }
+
+    /** Deliver result to result collector. */
+    public void callComplete() {
+      results.callComplete(this);
+    }
+  }
+
+  /** Result collector for parallel calls. */
+  private static class ParallelResults {
+    private Writable[] values;
+    private int size;
+    private int count;
+
+    public ParallelResults(int size) {
+      this.values = new Writable[size];
+      this.size = size;
+    }
+
+    /** Collect a result. */
+    public synchronized void callComplete(ParallelCall call) {
+      values[call.index] = call.value;            // store the value
+      count++;                                    // count it
+      if (count == size)                          // if all values are in
+        notify();                                 // then notify waiting caller
+    }
+  }
+
+  /** Construct an IPC client whose values are of the given {@link Writable}
+   * class. */
+  public Client(Class valueClass, Configuration conf) {
+    this.valueClass = valueClass;
+    this.timeout = conf.getInt("ipc.client.timeout",10000);
+    this.conf = conf;
+  }
+
+  /** Stop all threads related to this client.  No further calls may be made
+   * using this client. */
+  public void stop() {
+    LOG.info("Stopping client");
+    try {
+      Thread.sleep(timeout);                        // let all calls complete
+    } catch (InterruptedException e) {}
+    running = false;
+  }
+
+  /** Sets the timeout used for network i/o. */
+  public void setTimeout(int timeout) { this.timeout = timeout; }
+
+  /** Make a call, passing <code>param</code>, to the IPC server running at
+   * <code>address</code>, returning the value.  Throws exceptions if there are
+   * network problems or if the remote code threw an exception. */
+  public Writable call(Writable param, InetSocketAddress address)
+    throws IOException {
+    Connection connection = getConnection(address);
+    Call call = new Call(param);
+    synchronized (call) {
+      connection.sendParam(call);                 // send the parameter
+      long wait = timeout;
+      do {
+        try {
+          call.wait(wait);                       // wait for the result
+        } catch (InterruptedException e) {}
+        wait = timeout - (System.currentTimeMillis() - call.lastActivity);
+      } while (!call.done && wait > 0);
+
+      if (call.error != null) {
+        throw new IOException(call.error);
+      } else if (!call.done) {
+        throw new IOException("timed out waiting for response");
+      } else {
+        return call.value;
+      }
+    }
+  }
+
+  /** Makes a set of calls in parallel.  Each parameter is sent to the
+   * corresponding address.  When all values are available, or have timed out
+   * or errored, the collected results are returned in an array.  The array
+   * contains nulls for calls that timed out or errored.  */
+  public Writable[] call(Writable[] params, InetSocketAddress[] addresses)
+    throws IOException {
+    if (addresses.length == 0) return new Writable[0];
+
+    ParallelResults results = new ParallelResults(params.length);
+    synchronized (results) {
+      for (int i = 0; i < params.length; i++) {
+        ParallelCall call = new ParallelCall(params[i], results, i);
+        try {
+          Connection connection = getConnection(addresses[i]);
+          connection.sendParam(call);             // send each parameter
+        } catch (IOException e) {
+          LOG.info("Calling "+addresses[i]+" caught: " + e); // log errors
+          results.size--;                         //  wait for one fewer result
+        }
+      }
+      try {
+        results.wait(timeout);                    // wait for all results
+      } catch (InterruptedException e) {}
+
+      if (results.count == 0) {
+        throw new IOException("no responses");
+      } else {
+        return results.values;
+      }
+    }
+  }
+
+  /** Get a connection from the pool, or create a new one and add it to the
+   * pool.  Connections to a given host/port are reused. */
+  private Connection getConnection(InetSocketAddress address)
+    throws IOException {
+    Connection connection;
+    synchronized (connections) {
+      connection = (Connection)connections.get(address);
+      if (connection == null) {
+        connection = new Connection(address);
+        connections.put(address, connection);
+        connection.start();
+      }
+    }
+    return connection;
+  }
+
+  private Writable makeValue() {
+    Writable value;                             // construct value
+    try {
+      value = (Writable)valueClass.newInstance();
+    } catch (InstantiationException e) {
+      throw new RuntimeException(e.toString());
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException(e.toString());
+    }
+    return value;
+  }
+
+}

+ 238 - 0
src/java/org/apache/hadoop/ipc/RPC.java

@@ -0,0 +1,238 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ipc.
+
+import java.lang.reflect.Proxy;
+import java.lang.reflect.Method;
+import java.lang.reflect.Array;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+
+import java.net.InetSocketAddress;
+import java.util.logging.Logger;
+import java.io.*;
+import java.util.*;
+
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.util.LogFormatter;
+
+/** A simple RPC mechanism.
+ *
+ * A <i>protocol</i> is a Java interface.  All parameters and return types must
+ * be one of:
+ *
+ * <ul> <li>a primitive type, <code>boolean</code>, <code>byte</code>,
+ * <code>char</code>, <code>short</code>, <code>int</code>, <code>long</code>,
+ * <code>float</code>, <code>double</code>, or <code>void</code>; or</li>
+ *
+ * <li>a {@link String}; or</li>
+ *
+ * <li>a {@link Writable}; or</li>
+ *
+ * <li>an array of the above types</li> </ul>
+ *
+ * All methods in the protocol should throw only IOException.  No field data of
+ * the protocol instance is transmitted.
+ */
+public class RPC {
+  private static final Logger LOG =
+    LogFormatter.getLogger("org.apache.hadoop.ipc.RPC");
+
+  private RPC() {}                                  // no public ctor
+
+
+  /** A method invocation, including the method name and its parameters.*/
+  private static class Invocation implements Writable, Configurable {
+    private String methodName;
+    private Class[] parameterClasses;
+    private Object[] parameters;
+    private Configuration conf;
+
+    public Invocation() {}
+
+    public Invocation(Method method, Object[] parameters) {
+      this.methodName = method.getName();
+      this.parameterClasses = method.getParameterTypes();
+      this.parameters = parameters;
+    }
+
+    /** The name of the method invoked. */
+    public String getMethodName() { return methodName; }
+
+    /** The parameter classes. */
+    public Class[] getParameterClasses() { return parameterClasses; }
+
+    /** The parameter instances. */
+    public Object[] getParameters() { return parameters; }
+
+    public void readFields(DataInput in) throws IOException {
+      methodName = UTF8.readString(in);
+      parameters = new Object[in.readInt()];
+      parameterClasses = new Class[parameters.length];
+      ObjectWritable objectWritable = new ObjectWritable();
+      for (int i = 0; i < parameters.length; i++) {
+        parameters[i] = ObjectWritable.readObject(in, objectWritable, this.conf);
+        parameterClasses[i] = objectWritable.getDeclaredClass();
+      }
+    }
+
+    public void write(DataOutput out) throws IOException {
+      UTF8.writeString(out, methodName);
+      out.writeInt(parameterClasses.length);
+      for (int i = 0; i < parameterClasses.length; i++) {
+        ObjectWritable.writeObject(out, parameters[i], parameterClasses[i]);
+      }
+    }
+
+    public String toString() {
+      StringBuffer buffer = new StringBuffer();
+      buffer.append(methodName);
+      buffer.append("(");
+      for (int i = 0; i < parameters.length; i++) {
+        if (i != 0)
+          buffer.append(", ");
+        buffer.append(parameters[i]);
+      }
+      buffer.append(")");
+      return buffer.toString();
+    }
+
+    public void setConf(Configuration conf) {
+      this.conf = conf;
+    }
+
+    public Configuration getConf() {
+      return this.conf;
+    }
+
+  }
+
+  //TODO mb@media-style.com: static client or non-static client?
+  private static Client CLIENT;
+
+  private static class Invoker implements InvocationHandler {
+    private InetSocketAddress address;
+
+    public Invoker(InetSocketAddress address, Configuration conf) {
+      this.address = address;
+      CLIENT = (Client) conf.getObject(Client.class.getName());
+      if(CLIENT == null) {
+          CLIENT = new Client(ObjectWritable.class, conf);
+          conf.setObject(Client.class.getName(), CLIENT);
+      }
+    }
+
+    public Object invoke(Object proxy, Method method, Object[] args)
+      throws Throwable {
+      ObjectWritable value = (ObjectWritable)
+        CLIENT.call(new Invocation(method, args), address);
+      return value.get();
+    }
+  }
+
+  /** Construct a client-side proxy object that implements the named protocol,
+   * talking to a server at the named address. */
+  public static Object getProxy(Class protocol, InetSocketAddress addr, Configuration conf) {
+    return Proxy.newProxyInstance(protocol.getClassLoader(),
+                                  new Class[] { protocol },
+                                  new Invoker(addr, conf));
+  }
+
+  /** Expert: Make multiple, parallel calls to a set of servers. */
+  public static Object[] call(Method method, Object[][] params,
+                              InetSocketAddress[] addrs, Configuration conf)
+    throws IOException {
+
+    Invocation[] invocations = new Invocation[params.length];
+    for (int i = 0; i < params.length; i++)
+      invocations[i] = new Invocation(method, params[i]);
+    CLIENT = (Client) conf.getObject(Client.class.getName());
+    if(CLIENT == null) {
+        CLIENT = new Client(ObjectWritable.class, conf);
+        conf.setObject(Client.class.getName(), CLIENT);
+    }
+    Writable[] wrappedValues = CLIENT.call(invocations, addrs);
+    
+    if (method.getReturnType() == Void.TYPE) {
+      return null;
+    }
+
+    Object[] values =
+      (Object[])Array.newInstance(method.getReturnType(),wrappedValues.length);
+    for (int i = 0; i < values.length; i++)
+      if (wrappedValues[i] != null)
+        values[i] = ((ObjectWritable)wrappedValues[i]).get();
+    
+    return values;
+  }
+  
+
+  /** Construct a server for a protocol implementation instance listening on a
+   * port. */
+  public static Server getServer(final Object instance, final int port, Configuration conf) {
+    return getServer(instance, port, 1, false, conf);
+  }
+
+  /** Construct a server for a protocol implementation instance listening on a
+   * port. */
+  public static Server getServer(final Object instance, final int port,
+                                 final int numHandlers,
+                                 final boolean verbose, Configuration conf) {
+    return new Server(port, Invocation.class, numHandlers, conf) {
+        
+        Class implementation = instance.getClass();
+
+        public Writable call(Writable param) throws IOException {
+          try {
+            Invocation call = (Invocation)param;
+            if (verbose) log("Call: " + call);
+
+            Method method =
+              implementation.getMethod(call.getMethodName(),
+                                       call.getParameterClasses());
+
+            Object value = method.invoke(instance, call.getParameters());
+            if (verbose) log("Return: "+value);
+
+            return new ObjectWritable(method.getReturnType(), value);
+
+          } catch (InvocationTargetException e) {
+            Throwable target = e.getTargetException();
+            if (target instanceof IOException) {
+              throw (IOException)target;
+            } else {
+              IOException ioe = new IOException(target.toString());
+              ioe.setStackTrace(target.getStackTrace());
+              throw ioe;
+            }
+          } catch (Throwable e) {
+            IOException ioe = new IOException(e.toString());
+            ioe.setStackTrace(e.getStackTrace());
+            throw ioe;
+          }
+        }
+      };
+  }
+
+  private static void log(String value) {
+    if (value!= null && value.length() > 55)
+      value = value.substring(0, 55)+"...";
+    LOG.info(value);
+  }
+  
+}

+ 291 - 0
src/java/org/apache/hadoop/ipc/Server.java

@@ -0,0 +1,291 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ipc.
+
+import java.io.IOException;
+import java.io.EOFException;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+
+import java.net.Socket;
+import java.net.ServerSocket;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+
+import java.util.LinkedList;
+import java.util.logging.Logger;
+import java.util.logging.Level;
+
+import org.apache.hadoop.util.LogFormatter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.UTF8;
+
+/** An abstract IPC service.  IPC calls take a single {@link Writable} as a
+ * parameter, and return a {@link Writable} as their value.  A service runs on
+ * a port and is defined by a parameter class and a value class.
+ * 
+ * @author Doug Cutting
+ * @see Client
+ */
+public abstract class Server {
+  public static final Logger LOG =
+    LogFormatter.getLogger("org.apache.hadoop.ipc.Server");
+
+  private int port;                               // port we listen on
+  private int handlerCount;                       // number of handler threads
+  private int maxQueuedCalls;                     // max number of queued calls
+  private Class paramClass;                       // class of call parameters
+
+  private int timeout;
+
+  private boolean running = true;                 // true while server runs
+  private LinkedList callQueue = new LinkedList(); // queued calls
+  private Object callDequeued = new Object();     // used by wait/notify
+
+  /** A call queued for handling. */
+  private static class Call {
+    private int id;                               // the client's call id
+    private Writable param;                       // the parameter passed
+    private Connection connection;                // connection to client
+
+    public Call(int id, Writable param, Connection connection) {
+      this.id = id;
+      this.param = param;
+      this.connection = connection;
+    }
+  }
+
+  /** Listens on the socket, starting new connection threads. */
+  private class Listener extends Thread {
+    private ServerSocket socket;
+
+    public Listener() throws IOException {
+      this.socket = new ServerSocket(port);
+      socket.setSoTimeout(timeout);
+      this.setDaemon(true);
+      this.setName("Server listener on port " + port);
+    }
+
+    public void run() {
+      LOG.info(getName() + ": starting");
+      while (running) {
+        try {
+          new Connection(socket.accept()).start(); // start a new connection
+        } catch (SocketTimeoutException e) {      // ignore timeouts
+        } catch (Exception e) {                   // log all other exceptions
+          LOG.log(Level.INFO, getName() + " caught: " + e, e);
+        }
+      }
+      try {
+        socket.close();
+      } catch (IOException e) {
+        LOG.info(getName() + ": e=" + e);
+      }
+      LOG.info(getName() + ": exiting");
+    }
+  }
+
+  /** Reads calls from a connection and queues them for handling. */
+  private class Connection extends Thread {
+    private Socket socket;
+    private DataInputStream in;
+    private DataOutputStream out;
+
+    public Connection(Socket socket) throws IOException {
+      this.socket = socket;
+      socket.setSoTimeout(timeout);
+      this.in = new DataInputStream
+        (new BufferedInputStream(socket.getInputStream()));
+      this.out = new DataOutputStream
+        (new BufferedOutputStream(socket.getOutputStream()));
+      this.setDaemon(true);
+      this.setName("Server connection on port " + port + " from "
+                   + socket.getInetAddress().getHostAddress());
+    }
+
+    public void run() {
+      LOG.info(getName() + ": starting");
+      try {
+        while (running) {
+          int id;
+          try {
+            id = in.readInt();                    // try to read an id
+          } catch (SocketTimeoutException e) {
+            continue;
+          }
+        
+          if (LOG.isLoggable(Level.FINE))
+            LOG.fine(getName() + " got #" + id);
+        
+          Writable param = makeParam();           // read param
+          param.readFields(in);        
+        
+          Call call = new Call(id, param, this);
+        
+          synchronized (callQueue) {
+            callQueue.addLast(call);              // queue the call
+            callQueue.notify();                   // wake up a waiting handler
+          }
+        
+          while (running && callQueue.size() >= maxQueuedCalls) {
+            synchronized (callDequeued) {         // queue is full
+              callDequeued.wait(timeout);         // wait for a dequeue
+            }
+          }
+        }
+      } catch (EOFException eof) {
+          // This is what happens on linux when the other side shuts down
+      } catch (SocketException eof) {
+          // This is what happens on Win32 when the other side shuts down
+      } catch (Exception e) {
+        LOG.log(Level.INFO, getName() + " caught: " + e, e);
+      } finally {
+        try {
+          socket.close();
+        } catch (IOException e) {}
+        LOG.info(getName() + ": exiting");
+      }
+    }
+
+  }
+
+  /** Handles queued calls . */
+  private class Handler extends Thread {
+    public Handler(int instanceNumber) {
+      this.setDaemon(true);
+      this.setName("Server handler "+ instanceNumber + " on " + port);
+    }
+
+    public void run() {
+      LOG.info(getName() + ": starting");
+      while (running) {
+        try {
+          Call call;
+          synchronized (callQueue) {
+            while (running && callQueue.size()==0) { // wait for a call
+              callQueue.wait(timeout);
+            }
+            if (!running) break;
+            call = (Call)callQueue.removeFirst(); // pop the queue
+          }
+
+          synchronized (callDequeued) {           // tell others we've dequeued
+            callDequeued.notify();
+          }
+
+          if (LOG.isLoggable(Level.FINE))
+            LOG.fine(getName() + ": has #" + call.id + " from " +
+                     call.connection.socket.getInetAddress().getHostAddress());
+          
+          String error = null;
+          Writable value = null;
+          try {
+            value = call(call.param);             // make the call
+          } catch (IOException e) {
+            LOG.log(Level.INFO, getName() + " call error: " + e, e);
+            error = e.getMessage();
+          } catch (Exception e) {
+            LOG.log(Level.INFO, getName() + " call error: " + e, e);
+            error = e.toString();
+          }
+            
+          DataOutputStream out = call.connection.out;
+          synchronized (out) {
+            out.writeInt(call.id);                // write call id
+            out.writeBoolean(error!=null);        // write error flag
+            if (error != null)
+              value = new UTF8(error);
+            value.write(out);                     // write value
+            out.flush();
+          }
+
+        } catch (Exception e) {
+          LOG.log(Level.INFO, getName() + " caught: " + e, e);
+        }
+      }
+      LOG.info(getName() + ": exiting");
+    }
+  }
+  
+  /** Constructs a server listening on the named port.  Parameters passed must
+   * be of the named class.  The <code>handlerCount</handlerCount> determines
+   * the number of handler threads that will be used to process calls.
+   */
+  protected Server(int port, Class paramClass, int handlerCount, Configuration conf) {
+    this.port = port;
+    this.paramClass = paramClass;
+    this.handlerCount = handlerCount;
+    this.maxQueuedCalls = handlerCount;
+    this.timeout = conf.getInt("ipc.client.timeout",10000); 
+  }
+
+  /** Sets the timeout used for network i/o. */
+  public void setTimeout(int timeout) { this.timeout = timeout; }
+
+  /** Starts the service.  Must be called before any calls will be handled. */
+  public synchronized void start() throws IOException {
+    Listener listener = new Listener();
+    listener.start();
+    
+    for (int i = 0; i < handlerCount; i++) {
+      Handler handler = new Handler(i);
+      handler.start();
+    }
+  }
+
+  /** Stops the service.  No new calls will be handled after this is called.  All
+   * subthreads will likely be finished after this returns.
+   */
+  public synchronized void stop() {
+    LOG.info("Stopping server on " + port);
+    running = false;
+    try {
+      Thread.sleep(timeout);     //  inexactly wait for pending requests to finish
+    } catch (InterruptedException e) {}
+    notifyAll();
+  }
+
+  /** Wait for the server to be stopped.
+   * Does not wait for all subthreads to finish.
+   *  See {@link #stop()}.
+   */
+  public synchronized void join() throws InterruptedException {
+    while (running) {
+      wait();
+    }
+  }
+
+  /** Called for each call. */
+  public abstract Writable call(Writable param) throws IOException;
+
+  
+  private Writable makeParam() {
+    Writable param;                               // construct param
+    try {
+      param = (Writable)paramClass.newInstance();
+    } catch (InstantiationException e) {
+      throw new RuntimeException(e.toString());
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException(e.toString());
+    }
+    return param;
+  }
+
+}

+ 5 - 0
src/java/org/apache/hadoop/ipc/package.html

@@ -0,0 +1,5 @@
+<html>
+<body>
+Client/Server code used by distributed search.
+</body>
+</html>

+ 83 - 0
src/java/org/apache/hadoop/mapred/CombiningCollector.java

@@ -0,0 +1,83 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+
+/** Implements partial value reduction during mapping.  This can minimize the
+ * size of intermediate data.  Buffers a list of values for each unique key,
+ * then invokes the combiner's reduce method to merge some values before
+ * they're transferred to a reduce node. */
+class CombiningCollector implements OutputCollector {
+  private int limit;
+
+  private int count = 0;
+  private Map keyToValues;                        // the buffer
+
+  private JobConf job;
+  private OutputCollector out;
+  private Reducer combiner;
+  private Reporter reporter;
+
+  public CombiningCollector(JobConf job, OutputCollector out,
+                            Reporter reporter) {
+    this.job = job;
+    this.out = out;
+    this.reporter = reporter;
+    this.combiner = (Reducer)job.newInstance(job.getCombinerClass());
+    this.keyToValues = new TreeMap(job.getOutputKeyComparator());
+    this.limit = job.getInt("mapred.combine.buffer.size", 100000);
+  }
+
+  public synchronized void collect(WritableComparable key, Writable value)
+    throws IOException {
+
+    // buffer new value in map
+    ArrayList values = (ArrayList)keyToValues.get(key);
+    if (values == null) {                         // no values yet for this key
+      values = new ArrayList(1);                  // make a new list
+      values.add(value);                          // add this value
+      keyToValues.put(key, values);               // add to map
+    } else {
+      values.add(value);                          // other values: just add new
+    }
+
+    count++;
+
+    if (count >= this.limit) {                         // time to flush
+      flush();
+    }
+  }
+
+  public synchronized void flush() throws IOException {
+    Iterator pairs = keyToValues.entrySet().iterator();
+    while (pairs.hasNext()) {
+      Map.Entry pair = (Map.Entry)pairs.next();
+      combiner.reduce((WritableComparable)pair.getKey(),
+                      ((ArrayList)pair.getValue()).iterator(),
+                      out, reporter);
+    }
+    keyToValues.clear();
+    count = 0;
+  }
+
+}

+ 77 - 0
src/java/org/apache/hadoop/mapred/FileSplit.java

@@ -0,0 +1,77 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+import java.io.IOException;
+import java.io.File;
+import java.io.DataInput;
+import java.io.DataOutput;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.fs.NutchFileSystem;
+
+/** A section of an input file.  Returned by {@link
+ * InputFormat#getSplits(NutchFileSystem, JobConf, int)} and passed to
+ * InputFormat#getRecordReader(NutchFileSystem,FileSplit,JobConf,Reporter). */
+public class FileSplit implements Writable {
+  private File file;
+  private long start;
+  private long length;
+  
+  FileSplit() {}
+
+  /** Constructs a split.
+   *
+   * @param file the file name
+   * @param start the position of the first byte in the file to process
+   * @param length the number of bytes in the file to process
+   */
+  public FileSplit(File file, long start, long length) {
+    this.file = file;
+    this.start = start;
+    this.length = length;
+  }
+  
+  /** The file containing this split's data. */
+  public File getFile() { return file; }
+  
+  /** The position of the first byte in the file to process. */
+  public long getStart() { return start; }
+  
+  /** The number of bytes in the file to process. */
+  public long getLength() { return length; }
+
+  public String toString() { return file + ":" + start + "+" + length; }
+
+  ////////////////////////////////////////////
+  // Writable methods
+  ////////////////////////////////////////////
+
+  public void write(DataOutput out) throws IOException {
+    UTF8.writeString(out, file.toString());
+    out.writeLong(start);
+    out.writeLong(length);
+  }
+  public void readFields(DataInput in) throws IOException {
+    file = new File(UTF8.readString(in));
+    start = in.readLong();
+    length = in.readLong();
+  }
+
+
+}

+ 51 - 0
src/java/org/apache/hadoop/mapred/InputFormat.java

@@ -0,0 +1,51 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+import java.io.IOException;
+import java.io.File;
+
+import org.apache.hadoop.fs.NutchFileSystem;
+
+/** An input data format.  Input files are stored in a {@link NutchFileSystem}.
+ * 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. */
+public interface InputFormat {
+
+  /** Splits a set of input files.  One split is created per map task.
+   *
+   * @param fs the filesystem containing the files to be split
+   * @param job the job whose input files are to be split
+   * @param numSplits the desired number of splits
+   * @return the splits
+   */
+  FileSplit[] getSplits(NutchFileSystem fs, JobConf job, int numSplits)
+    throws IOException;
+
+  /** Construct a {@link RecordReader} for a {@link FileSplit}.
+   *
+   * @param fs the {@link NutchFileSystem}
+   * @param split the {@link FileSplit}
+   * @param job the job that this split belongs to
+   * @return a {@link RecordReader}
+   */
+  RecordReader getRecordReader(NutchFileSystem fs, FileSplit split,
+                               JobConf job, Reporter reporter)
+    throws IOException;
+}
+

+ 134 - 0
src/java/org/apache/hadoop/mapred/InputFormatBase.java

@@ -0,0 +1,134 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+import java.io.IOException;
+import java.io.File;
+
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.logging.Logger;
+
+import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.util.LogFormatter;
+
+/** A base class for {@link InputFormat}. */
+public abstract class InputFormatBase implements InputFormat {
+
+  public static final Logger LOG =
+    LogFormatter.getLogger("org.apache.hadoop.mapred.InputFormatBase");
+
+  private static final double SPLIT_SLOP = 0.1;   // 10% slop
+
+  private int minSplitSize = 1;
+
+  protected void setMinSplitSize(int minSplitSize) {
+    this.minSplitSize = minSplitSize;
+  }
+
+  public abstract RecordReader getRecordReader(NutchFileSystem fs,
+                                               FileSplit split,
+                                               JobConf job,
+                                               Reporter reporter)
+    throws IOException;
+
+  /** List input directories.
+   * Subclasses may override to, e.g., select only files matching a regular
+   * expression.
+   * Property mapred.input.subdir, if set, names a subdirectory that
+   * is appended to all input dirs specified by job, and if the given fs
+   * lists those too, each is added to the returned array of File.
+   * @param fs
+   * @param job
+   * @return array of File objects, never zero length.
+   * @throws IOException if zero items.
+   */
+  protected File[] listFiles(NutchFileSystem fs, JobConf job)
+    throws IOException {
+    File[] dirs = job.getInputDirs();
+    String subdir = job.get("mapred.input.subdir");
+    ArrayList result = new ArrayList();
+    for (int i = 0; i < dirs.length; i++) {
+      File[] dir = fs.listFiles(dirs[i]);
+      if (dir != null) {
+        for (int j = 0; j < dir.length; j++) {
+          File file = dir[j];
+          if (subdir != null) {
+            File[] subFiles = fs.listFiles(new File(file, subdir));
+            if (subFiles != null) {
+              for (int k = 0; k < subFiles.length; k++) {
+                result.add(subFiles[k]);
+              }
+            }
+          } else {
+            result.add(file);
+          }
+        }
+      }
+    }
+
+    if (result.size() == 0) {
+      throw new IOException("No input directories specified in: "+job);
+    }
+    return (File[])result.toArray(new File[result.size()]);
+  }
+
+  /** Splits files returned by {#listFiles(NutchFileSystem,JobConf) when
+   * they're too big.*/ 
+  public FileSplit[] getSplits(NutchFileSystem fs, JobConf job, int numSplits)
+    throws IOException {
+
+    File[] files = listFiles(fs, job);
+
+    for (int i = 0; i < files.length; i++) {      // check we have valid files
+      File file = files[i];
+      if (fs.isDirectory(file) || !fs.exists(file)) {
+        throw new IOException("Not a file: "+files[i]);
+      }
+    }
+
+    long totalSize = 0;                           // compute total size
+    for (int i = 0; i < files.length; i++) {
+      totalSize += fs.getLength(files[i]);
+    }
+
+    long bytesPerSplit = Math.max(totalSize / numSplits, minSplitSize);
+    long maxPerSplit = bytesPerSplit + (long)(bytesPerSplit*SPLIT_SLOP);
+
+    //LOG.info("bytesPerSplit = " + bytesPerSplit);
+    //LOG.info("maxPerSplit = " + maxPerSplit);
+
+    ArrayList splits = new ArrayList(numSplits);  // generate splits
+    for (int i = 0; i < files.length; i++) {
+      File file = files[i];
+      long length = fs.getLength(file);
+
+      long bytesRemaining = length;
+      while (bytesRemaining >= maxPerSplit) {
+        splits.add(new FileSplit(file, length-bytesRemaining, bytesPerSplit));
+        bytesRemaining -= bytesPerSplit;
+      }
+      
+      if (bytesRemaining != 0) {
+        splits.add(new FileSplit(file, length-bytesRemaining, bytesRemaining));
+      }
+    }
+    return (FileSplit[])splits.toArray(new FileSplit[splits.size()]);
+  }
+
+}
+

+ 64 - 0
src/java/org/apache/hadoop/mapred/InterTrackerProtocol.java

@@ -0,0 +1,64 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+import java.io.*;
+import org.apache.hadoop.io.*;
+
+/** 
+ * Protocol that a TaskTracker and the central JobTracker use to communicate.
+ * The JobTracker is the Server, which implements this protocol.
+ */ 
+public interface InterTrackerProtocol {
+  public final static int TRACKERS_OK = 0;
+  public final static int UNKNOWN_TASKTRACKER = 1;
+
+  /** 
+   * Called regularly by the task tracker to update the status of its tasks
+   * within the job tracker.  JobTracker responds with a code that tells the 
+   * TaskTracker whether all is well.
+   *
+   * TaskTracker must also indicate whether this is the first interaction
+   * (since state refresh)
+   */
+  int emitHeartbeat(TaskTrackerStatus status, boolean initialContact);
+
+  /** Called to get new tasks from from the job tracker for this tracker.*/
+  Task pollForNewTask(String trackerName);
+
+  /** Called to find which tasks that have been run by this tracker should now
+   * be closed because their job is complete.  This is used to, e.g., 
+   * notify a map task that its output is no longer needed and may 
+   * be removed. */
+  String pollForTaskWithClosedJob(String trackerName);
+
+  /** Called by a reduce task to find which map tasks are completed.
+   *
+   * @param taskId the reduce task id
+   * @param mapTasksNeeded an array of UTF8 naming map task ids whose output is needed.
+   * @return an array of MapOutputLocation
+   */
+  MapOutputLocation[] locateMapOutputs(String taskId, String[][] mapTasksNeeded);
+
+  /**
+   * The task tracker calls this once, to discern where it can find
+   * files referred to by the JobTracker
+   */
+  public String getFilesystemName() throws IOException;
+}
+
+

+ 378 - 0
src/java/org/apache/hadoop/mapred/JobClient.java

@@ -0,0 +1,378 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.
+
+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 java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.logging.*;
+
+/*******************************************************
+ * JobClient interacts with the JobTracker network interface.
+ * This object implements the job-control interface, and
+ * should be the primary method by which user programs interact
+ * with the networked job system.
+ *
+ * @author Mike Cafarella
+ *******************************************************/
+public class JobClient implements MRConstants {
+    private static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.mapred.JobClient");
+
+    static long MAX_JOBPROFILE_AGE = 1000 * 2;
+
+    /**
+     * A NetworkedJob is an implementation of RunningJob.  It holds
+     * a JobProfile object to provide some info, and interacts with the
+     * remote service to provide certain functionality.
+     */
+    class NetworkedJob implements RunningJob {
+        JobProfile profile;
+        JobStatus status;
+        long statustime;
+
+        /**
+         * We store a JobProfile and a timestamp for when we last
+         * acquired the job profile.  If the job is null, then we cannot
+         * perform any of the tasks.  The job might be null if the JobTracker
+         * has completely forgotten about the job.  (eg, 24 hours after the
+         * job completes.)
+         */
+        public NetworkedJob(JobStatus job) throws IOException {
+            this.status = job;
+            this.profile = jobSubmitClient.getJobProfile(job.getJobId());
+            this.statustime = System.currentTimeMillis();
+        }
+
+        /**
+         * Some methods rely on having a recent job profile object.  Refresh
+         * it, if necessary
+         */
+        synchronized void ensureFreshStatus() throws IOException {
+            if (System.currentTimeMillis() - statustime > MAX_JOBPROFILE_AGE) {
+                this.status = jobSubmitClient.getJobStatus(profile.getJobId());
+                this.statustime = System.currentTimeMillis();
+            }
+        }
+
+        /**
+         * An identifier for the job
+         */
+        public String getJobID() {
+            return profile.getJobId();
+        }
+
+        /**
+         * The name of the job file
+         */
+        public String getJobFile() {
+            return profile.getJobFile();
+        }
+
+        /**
+         * A URL where the job's status can be seen
+         */
+        public String getTrackingURL() {
+            return profile.getURL().toString();
+        }
+
+        /**
+         * A float between 0.0 and 1.0, indicating the % of map work
+         * completed.
+         */
+        public float mapProgress() throws IOException {
+            ensureFreshStatus();
+            return status.mapProgress();
+        }
+
+        /**
+         * A float between 0.0 and 1.0, indicating the % of reduce work
+         * completed.
+         */
+        public float reduceProgress() throws IOException {
+            ensureFreshStatus();
+            return status.reduceProgress();
+        }
+
+        /**
+         * Returns immediately whether the whole job is done yet or not.
+         */
+        public synchronized boolean isComplete() throws IOException {
+            ensureFreshStatus();
+            return (status.getRunState() == JobStatus.SUCCEEDED ||
+                    status.getRunState() == JobStatus.FAILED);
+        }
+
+        /**
+         * True iff job completed successfully.
+         */
+        public synchronized boolean isSuccessful() throws IOException {
+            ensureFreshStatus();
+            return status.getRunState() == JobStatus.SUCCEEDED;
+        }
+
+        /**
+         * Blocks until the job is finished
+         */
+        public synchronized void waitForCompletion() throws IOException {
+            while (! isComplete()) {
+                try {
+                    Thread.sleep(5000);
+                } catch (InterruptedException ie) {
+                }
+            }
+        }
+
+        /**
+         * Tells the service to terminate the current job.
+         */
+        public synchronized void killJob() throws IOException {
+            jobSubmitClient.killJob(getJobID());
+        }
+
+        /**
+         * Dump stats to screen
+         */
+        public String toString() {
+            try {
+                ensureFreshStatus();
+            } catch (IOException e) {
+            }
+            return "Job: " + profile.getJobId() + "\n" + 
+                "file: " + profile.getJobFile() + "\n" + 
+                "tracking URL: " + profile.getURL() + "\n" + 
+                "map() completion: " + status.mapProgress() + "\n" + 
+                "reduce() completion: " + status.reduceProgress();
+        }
+    }
+
+    JobSubmissionProtocol jobSubmitClient;
+    NutchFileSystem fs = null;
+
+    private Configuration conf;
+    static Random r = new Random();
+
+    /**
+     * Build a job client, connect to the default job tracker
+     */
+    public JobClient(Configuration conf) throws IOException {
+      this.conf = conf;
+      String tracker = conf.get("mapred.job.tracker", "local");
+      if ("local".equals(tracker)) {
+        this.jobSubmitClient = new LocalJobRunner(conf);
+      } else {
+        this.jobSubmitClient = (JobSubmissionProtocol) 
+          RPC.getProxy(JobSubmissionProtocol.class,
+                       JobTracker.getAddress(conf), conf);
+      }
+    }
+  
+    /**
+     * Build a job client, connect to the indicated job tracker.
+     */
+    public JobClient(InetSocketAddress jobTrackAddr, Configuration conf) throws IOException {
+        this.jobSubmitClient = (JobSubmissionProtocol) 
+            RPC.getProxy(JobSubmissionProtocol.class, jobTrackAddr, conf);
+    }
+
+
+    /**
+     */
+    public synchronized void close() throws IOException {
+        if (fs != null) {
+            fs.close();
+            fs = null;
+        }
+    }
+
+    /**
+     * Get a filesystem handle.  We need this to prepare jobs
+     * for submission to the MapReduce system.
+     */
+    public synchronized NutchFileSystem getFs() throws IOException {
+      if (this.fs == null) {
+        String fsName = jobSubmitClient.getFilesystemName();
+        this.fs = NutchFileSystem.getNamed(fsName, this.conf);
+      }
+      return fs;
+    }
+
+    /**
+     * Submit a job to the MR system
+     */
+    public RunningJob submitJob(String jobFile) throws IOException {
+        // Load in the submitted job details
+        JobConf job = new JobConf(jobFile);
+        return submitJob(job);
+    }
+
+    /**
+     * Submit a job to the MR system
+     */
+    public RunningJob submitJob(JobConf job) throws IOException {
+        //
+        // First figure out what fs the JobTracker is using.  Copy the
+        // job to it, under a temporary name.  This allows DFS to work,
+        // and under the local fs also provides UNIX-like object loading 
+        // semantics.  (that is, if the job file is deleted right after
+        // submission, we can still run the submission to completion)
+        //
+
+        // Create a number of filenames in the JobTracker's fs namespace
+        File submitJobDir = new File(job.getSystemDir(), "submit_" + Integer.toString(Math.abs(r.nextInt()), 36));
+        File submitJobFile = new File(submitJobDir, "job.xml");
+        File submitJarFile = new File(submitJobDir, "job.jar");
+
+        String originalJarPath = job.getJar();
+
+        if (originalJarPath != null) {           // Copy jar to JobTracker's fs
+          job.setJar(submitJarFile.toString());
+          getFs().copyFromLocalFile(new File(originalJarPath), submitJarFile);
+        }
+
+        // Write job file to JobTracker's fs
+        NFSDataOutputStream out = getFs().create(submitJobFile);
+        try {
+          job.write(out);
+        } finally {
+          out.close();
+        }
+
+        //
+        // Now, actually submit the job (using the submit name)
+        //
+        JobStatus status = jobSubmitClient.submitJob(submitJobFile.getPath());
+        if (status != null) {
+            return new NetworkedJob(status);
+        } else {
+            throw new IOException("Could not launch job");
+        }
+    }
+
+    /**
+     * Get an RunningJob object to track an ongoing job.  Returns
+     * null if the id does not correspond to any known job.
+     */
+    public RunningJob getJob(String jobid) throws IOException {
+        JobStatus status = jobSubmitClient.getJobStatus(jobid);
+        if (status != null) {
+            return new NetworkedJob(status);
+        } else {
+            return null;
+        }
+    }
+
+    /** Utility that submits a job, then polls for progress until the job is
+     * complete. */
+    public static void runJob(JobConf job) throws IOException {
+      JobClient jc = new JobClient(job);
+      boolean error = true;
+      RunningJob running = null;
+      String lastReport = null;
+      try {
+        running = jc.submitJob(job);
+        String jobId = running.getJobID();
+        LOG.info("Running job: " + jobId);
+        while (!running.isComplete()) {
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException e) {}
+          running = jc.getJob(jobId);
+          String report = null;
+          report = " map "+Math.round(running.mapProgress()*100)+"%  reduce " + Math.round(running.reduceProgress()*100)+"%";
+          if (!report.equals(lastReport)) {
+            LOG.info(report);
+            lastReport = report;
+          }
+        }
+        if (!running.isSuccessful()) {
+          throw new IOException("Job failed!");
+        }
+        LOG.info("Job complete: " + jobId);
+        error = false;
+      } finally {
+        if (error && (running != null)) {
+          running.killJob();
+        }
+        jc.close();
+      }
+    }
+
+
+    /**
+     */
+    public static void main(String argv[]) throws IOException {
+        if (argv.length < 2) {
+            System.out.println("JobClient -submit <job> | -status <id> | -kill <id>");
+            System.exit(-1);
+        }
+
+        // Process args
+        String jobTrackerStr = argv[0];
+        String submitJobFile = null;
+        String jobid = null;
+        boolean getStatus = false;
+        boolean killJob = false;
+
+        for (int i = 0; i < argv.length; i++) {
+            if ("-submit".equals(argv[i])) {
+                submitJobFile = argv[i+1];
+                i+=2;
+            } else if ("-status".equals(argv[i])) {
+                jobid = argv[i+1];
+                getStatus = true;
+                i++;
+            } else if ("-kill".equals(argv[i])) {
+                jobid = argv[i+1];
+                killJob = true;
+                i++;
+            }
+        }
+
+        // Submit the request
+        JobClient jc = new JobClient(new Configuration());
+        try {
+            if (submitJobFile != null) {
+                RunningJob job = jc.submitJob(submitJobFile);
+                System.out.println("Created job " + job.getJobID());
+            } else if (getStatus) {
+                RunningJob job = jc.getJob(jobid);
+                if (job == null) {
+                    System.out.println("Could not find job " + jobid);
+                } else {
+                    System.out.println();
+                    System.out.println(job);
+                }
+            } else if (killJob) {
+                RunningJob job = jc.getJob(jobid);
+                if (job == null) {
+                    System.out.println("Could not find job " + jobid);
+                } else {
+                    job.killJob();
+                    System.out.println("Killed job " + jobid);
+                }
+            }
+        } finally {
+            jc.close();
+        }
+    }
+}
+

+ 284 - 0
src/java/org/apache/hadoop/mapred/JobConf.java

@@ -0,0 +1,284 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+
+import java.io.IOException;
+import java.io.File;
+import java.io.InputStream;
+import java.io.FileNotFoundException;
+import java.net.URL;
+
+import java.util.Properties;
+import java.util.jar.JarFile;
+import java.util.jar.JarEntry;
+import java.util.StringTokenizer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Collections;
+
+import org.apache.hadoop.fs.NutchFileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.UTF8;
+
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
+import org.apache.hadoop.mapred.lib.HashPartitioner;
+
+/** A map/reduce job configuration.  This names the {@link Mapper}, combiner
+ * (if any), {@link Partitioner}, {@link Reducer}, {@link InputFormat}, and
+ * {@link OutputFormat} implementations to be used.  It also indicates the set
+ * of input files, and where the output files should be written. */
+public class JobConf extends Configuration {
+
+  public JobConf() {
+    super();
+  }
+    
+  /**
+   * Construct a map/reduce job configuration.
+   * 
+   * @param conf
+   *          a Configuration whose settings will be inherited.
+   */
+  public JobConf(Configuration conf) {
+    super(conf);
+    addConfResource("mapred-default.xml");
+  }
+
+
+  /** Construct a map/reduce configuration.
+   *
+   * @param config a Configuration-format XML job description file
+   */
+  public JobConf(String config) {
+    this(new File(config));
+  }
+
+  /** Construct a map/reduce configuration.
+   *
+   * @param config a Configuration-format XML job description file
+   */
+  public JobConf(File config) {
+    super();
+    addConfResource("mapred-default.xml");
+    addConfResource(config);
+  }
+
+  public String getJar() { return get("mapred.jar"); }
+  public void setJar(String jar) { set("mapred.jar", jar); }
+
+  public File getSystemDir() {
+    return new File(get("mapred.system.dir",
+                                        "/tmp/nutch/mapred/system"));
+  }
+
+  public String[] getLocalDirs() throws IOException {
+    return getStrings("mapred.local.dir");
+  }
+
+  public void deleteLocalFiles() throws IOException {
+    String[] localDirs = getLocalDirs();
+    for (int i = 0; i < localDirs.length; i++) {
+      FileUtil.fullyDelete(new File(localDirs[i]), this);
+    }
+  }
+
+  public void deleteLocalFiles(String subdir) throws IOException {
+    String[] localDirs = getLocalDirs();
+    for (int i = 0; i < localDirs.length; i++) {
+      FileUtil.fullyDelete(new File(localDirs[i], subdir), this);
+    }
+  }
+
+  /** Constructs a local file name.  Files are distributed among configured
+   * local directories.*/
+  public File getLocalFile(String subdir, String name) throws IOException {
+      String param[] = new String[1];
+      param[0] = name;
+      return getLocalFile(subdir, param, "", false);
+  }
+  // REMIND - mjc - rename this!  getLocalFile() is not quite the same.
+  public File getLocalFile(String subdir, String names[], String ending) throws IOException {
+      return getLocalFile(subdir, names, ending, true);
+  }
+  File getLocalFile(String subdir, String names[], String ending, boolean existingFileTest) throws IOException {
+    String[] localDirs = getLocalDirs();
+    for (int k = 0; k < names.length; k++) {
+        String path = subdir + File.separator + names[k] + ending;
+        int hashCode = path.hashCode();
+        for (int i = 0; i < localDirs.length; i++) {  // try each local dir
+            int index = (hashCode+i & Integer.MAX_VALUE) % localDirs.length;
+            File file = new File(localDirs[index], path);
+            File dir = file.getParentFile();
+            if (existingFileTest) {
+                if (file.exists()) {
+                    return file;
+                }
+            } else {
+                if (dir.exists() || dir.mkdirs()) {
+                    return file;
+                }
+            }
+        }
+    }
+    throw new IOException("No valid local directories.");
+  }
+
+  public void setInputDir(File dir) { set("mapred.input.dir", dir); }
+
+  public void addInputDir(File dir) {
+    String dirs = get("mapred.input.dir");
+    set("mapred.input.dir", dirs == null ? dir.toString() : dirs + "," + dir);
+  }
+  public File[] getInputDirs() {
+    String dirs = get("mapred.input.dir", "");
+    ArrayList list = Collections.list(new StringTokenizer(dirs, ","));
+    File[] result = new File[list.size()];
+    for (int i = 0; i < list.size(); i++) {
+      result[i] = new File((String)list.get(i));
+    }
+    return result;
+  }
+
+  public File getOutputDir() { return new File(get("mapred.output.dir")); }
+  public void setOutputDir(File dir) { set("mapred.output.dir", dir); }
+
+  public InputFormat getInputFormat() {
+    return (InputFormat)newInstance(getClass("mapred.input.format.class",
+                                             TextInputFormat.class,
+                                             InputFormat.class));
+  }
+  public void setInputFormat(Class theClass) {
+    setClass("mapred.input.format.class", theClass, InputFormat.class);
+  }
+  public OutputFormat getOutputFormat() {
+    return (OutputFormat)newInstance(getClass("mapred.output.format.class",
+                                              TextOutputFormat.class,
+                                              OutputFormat.class));
+  }
+  public void setOutputFormat(Class theClass) {
+    setClass("mapred.output.format.class", theClass, OutputFormat.class);
+  }
+  
+  public Class getInputKeyClass() {
+    return getClass("mapred.input.key.class",
+                    LongWritable.class, WritableComparable.class);
+  }
+  public void setInputKeyClass(Class theClass) {
+    setClass("mapred.input.key.class", theClass, WritableComparable.class);
+  }
+
+  public Class getInputValueClass() {
+    return getClass("mapred.input.value.class", UTF8.class, Writable.class);
+  }
+  public void setInputValueClass(Class theClass) {
+    setClass("mapred.input.value.class", theClass, Writable.class);
+  }
+
+  public Class getOutputKeyClass() {
+    return getClass("mapred.output.key.class",
+                    LongWritable.class, WritableComparable.class);
+  }
+  public void setOutputKeyClass(Class theClass) {
+    setClass("mapred.output.key.class", theClass, WritableComparable.class);
+  }
+
+  public WritableComparator getOutputKeyComparator() {
+    Class theClass = getClass("mapred.output.key.comparator.class", null,
+                              WritableComparator.class);
+    if (theClass != null)
+      return (WritableComparator)newInstance(theClass);
+    return WritableComparator.get(getOutputKeyClass());
+  }
+
+  public void setOutputKeyComparatorClass(Class theClass) {
+    setClass("mapred.output.key.comparator.class",
+             theClass, WritableComparator.class);
+  }
+
+  public Class getOutputValueClass() {
+    return getClass("mapred.output.value.class", UTF8.class, Writable.class);
+  }
+  public void setOutputValueClass(Class theClass) {
+    setClass("mapred.output.value.class", theClass, Writable.class);
+  }
+
+  public Class getMapperClass() {
+    return getClass("mapred.mapper.class", IdentityMapper.class, Mapper.class);
+  }
+  public void setMapperClass(Class theClass) {
+    setClass("mapred.mapper.class", theClass, Mapper.class);
+  }
+
+  public Class getMapRunnerClass() {
+    return getClass("mapred.map.runner.class",
+                    MapRunner.class, MapRunnable.class);
+  }
+  public void setMapRunnerClass(Class theClass) {
+    setClass("mapred.map.runner.class", theClass, MapRunnable.class);
+  }
+
+  public Class getPartitionerClass() {
+    return getClass("mapred.partitioner.class",
+                    HashPartitioner.class, Partitioner.class);
+  }
+  public void setPartitionerClass(Class theClass) {
+    setClass("mapred.partitioner.class", theClass, Partitioner.class);
+  }
+
+  public Class getReducerClass() {
+    return getClass("mapred.reducer.class",
+                    IdentityReducer.class, Reducer.class);
+  }
+  public void setReducerClass(Class theClass) {
+    setClass("mapred.reducer.class", theClass, Reducer.class);
+  }
+
+  public Class getCombinerClass() {
+    return getClass("mapred.combiner.class", null, Reducer.class);
+  }
+  public void setCombinerClass(Class theClass) {
+    setClass("mapred.combiner.class", theClass, Reducer.class);
+  }
+  
+  public int getNumMapTasks() { return getInt("mapred.map.tasks", 1); }
+  public void setNumMapTasks(int n) { setInt("mapred.map.tasks", n); }
+
+  public int getNumReduceTasks() { return getInt("mapred.reduce.tasks", 1); }
+  public void setNumReduceTasks(int n) { setInt("mapred.reduce.tasks", n); }
+
+  public Object newInstance(Class theClass) {
+    Object result;
+    try {
+      result = theClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    if (result instanceof JobConfigurable)
+      ((JobConfigurable)result).configure(this);
+    return result;
+  }
+
+}
+

+ 26 - 0
src/java/org/apache/hadoop/mapred/JobConfigurable.java

@@ -0,0 +1,26 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+/** That what may be configured. */
+public interface JobConfigurable {
+  /** Initializes a new instance from a {@link JobConf}.
+   *
+   * @param job the configuration
+   */
+  void configure(JobConf job);
+}

+ 437 - 0
src/java/org/apache/hadoop/mapred/JobInProgress.java

@@ -0,0 +1,437 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.
+
+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 java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.logging.*;
+
+///////////////////////////////////////////////////////
+// JobInProgress maintains all the info for keeping
+// a Job on the straight and narrow.  It keeps its JobProfile
+// and its latest JobStatus, plus a set of tables for 
+// doing bookkeeping of its Tasks.
+///////////////////////////////////////////////////////
+public class JobInProgress {
+    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.mapred.JobInProgress");
+
+    JobProfile profile;
+    JobStatus status;
+    File localJobFile = null;
+
+    TaskInProgress maps[] = null;
+    TaskInProgress reduces[] = null;
+    int numMapTasks = 0;
+    int numReduceTasks = 0;
+
+    JobTracker jobtracker = null;
+
+    long startTime;
+    long finishTime;
+    String deleteUponCompletion = null;
+
+    Configuration conf;
+    boolean tasksInited = false;
+
+    /**
+     * Create a JobInProgress with the given job file, plus a handle
+     * to the tracker.
+     */
+    public JobInProgress(String jobFile, JobTracker jobtracker, Configuration conf) throws IOException {
+        String jobid = "job_" + jobtracker.createUniqueId();
+        String url = "http://" + jobtracker.getJobTrackerMachine() + ":" + jobtracker.getInfoPort() + "/jobdetails.jsp?jobid=" + jobid;
+        this.conf = conf;
+        this.jobtracker = jobtracker;
+        this.profile = new JobProfile(jobid, jobFile, url);
+        this.status = new JobStatus(jobid, 0.0f, 0.0f, JobStatus.RUNNING);
+        this.startTime = System.currentTimeMillis();
+
+        this.localJobFile = new JobConf(conf).getLocalFile(JobTracker.SUBDIR, jobid + ".xml");
+        NutchFileSystem fs = NutchFileSystem.get(conf);
+        fs.copyToLocalFile(new File(jobFile), localJobFile);
+
+        JobConf jd = new JobConf(localJobFile);
+        this.numMapTasks = jd.getNumMapTasks();
+        this.numReduceTasks = jd.getNumReduceTasks();
+
+        //
+        // If a jobFile is in the systemDir, we can delete it (and
+        // its JAR) upon completion
+        //
+        File systemDir = jd.getSystemDir();
+        if (jobFile.startsWith(systemDir.getPath())) {
+            this.deleteUponCompletion = jobFile;
+        }
+    }
+
+    /**
+     * Construct the splits, etc
+     */
+    void initTasks() throws IOException {
+        if (tasksInited) {
+            return;
+        }
+
+        //
+        // construct input splits
+        //
+        String jobid = profile.getJobId();
+        String jobFile = profile.getJobFile();
+
+        JobConf jd = new JobConf(localJobFile);
+        NutchFileSystem fs = NutchFileSystem.get(conf);
+        FileSplit[] splits =
+            jd.getInputFormat().getSplits(fs, jd, numMapTasks);
+
+        //
+        // sort splits by decreasing length, to reduce job's tail
+        //
+        Arrays.sort(splits, new Comparator() {
+            public int compare(Object a, Object b) {
+                long diff =
+                    ((FileSplit)b).getLength() - ((FileSplit)a).getLength();
+                return diff==0 ? 0 : (diff > 0 ? 1 : -1);
+            }
+        });
+
+        //
+        // adjust number of map tasks to actual number of splits
+        //
+        this.numMapTasks = splits.length;
+        // create a map task for each split
+        this.maps = new TaskInProgress[numMapTasks];
+        for (int i = 0; i < numMapTasks; i++) {
+            maps[i] = new TaskInProgress(jobFile, splits[i], jobtracker, conf, this);
+        }
+
+        //
+        // Create reduce tasks
+        //
+        this.reduces = new TaskInProgress[numReduceTasks];
+        for (int i = 0; i < numReduceTasks; i++) {
+            reduces[i] = new TaskInProgress(jobFile, maps, i, jobtracker, conf, this);
+        }
+
+        tasksInited = true;
+    }
+
+    /////////////////////////////////////////////////////
+    // Accessors for the JobInProgress
+    /////////////////////////////////////////////////////
+    public JobProfile getProfile() {
+        return profile;
+    }
+    public JobStatus getStatus() {
+        return status;
+    }
+    public long getStartTime() {
+        return startTime;
+    }
+    public long getFinishTime() {
+        return finishTime;
+    }
+    public int desiredMaps() {
+        return numMapTasks;
+    }
+    public int finishedMaps() {
+        int finishedCount = 0;
+        for (int i = 0; i < maps.length; i++) {
+            if (maps[i].isComplete()) {
+                finishedCount++;
+            }
+        }
+        return finishedCount;
+    }
+    public int desiredReduces() {
+        return numReduceTasks;
+    }
+    public int finishedReduces() {
+        int finishedCount = 0;
+        for (int i = 0; i < reduces.length; i++) {
+            if (reduces[i].isComplete()) {
+                finishedCount++;
+            }
+        }
+        return finishedCount;
+    }
+
+    /**
+     * Return a treeset of completed TaskInProgress objects
+     */
+    public Vector reportTasksInProgress(boolean shouldBeMap, boolean shouldBeComplete) {
+        Vector results = new Vector();
+        TaskInProgress tips[] = null;
+        if (shouldBeMap) {
+            tips = maps;
+        } else {
+            tips = reduces;
+        }
+        for (int i = 0; i < tips.length; i++) {
+            if (tips[i].isComplete() == shouldBeComplete) {
+                results.add(tips[i]);
+            }
+        }
+        return results;
+    }
+
+    ////////////////////////////////////////////////////
+    // Status update methods
+    ////////////////////////////////////////////////////
+    public void updateTaskStatus(TaskInProgress tip, TaskStatus status) {
+        tip.updateStatus(status);
+
+        //
+        // Update JobInProgress status
+        //
+        if (maps.length == 0) {
+            this.status.setMapProgress(1.0f);
+        } else {
+            double reportedProgress = 0;
+            for (int i = 0; i < maps.length; i++) {
+                reportedProgress += maps[i].getProgress();
+            }
+            this.status.setMapProgress((float) (reportedProgress / maps.length));
+        }
+        if (reduces.length == 0) {
+            this.status.setReduceProgress(1.0f);
+        } else {
+            double reportedProgress = 0;
+            for (int i = 0; i < reduces.length; i++) {
+                reportedProgress += reduces[i].getProgress();
+            }
+            this.status.setReduceProgress((float) (reportedProgress / reduces.length));
+        }
+    }
+
+    /////////////////////////////////////////////////////
+    // Create/manage tasks
+    /////////////////////////////////////////////////////
+    /**
+     * Return a MapTask, if appropriate, to run on the given tasktracker
+     */
+    public Task obtainNewMapTask(String taskTracker, TaskTrackerStatus tts) {
+        if (! tasksInited) {
+            try {
+                initTasks();
+            } catch (IOException ie) {
+                ie.printStackTrace();
+                LOG.info("Cannot create task split for " + profile.getJobId());
+            }
+        }
+
+        Task t = null;
+        int cacheTarget = -1;
+        int stdTarget = -1;
+        int specTarget = -1;
+        double totalProgress = 0;
+
+        //
+        // We end up creating two tasks for the same bucket, because
+        // we call obtainNewMapTask() really fast, twice in a row.
+        // There's not enough time for the "recentTasks"
+        //
+        for (int i = 0; i < maps.length; i++) {
+            if (maps[i].hasTaskWithCacheHit(taskTracker, tts)) {
+                if (cacheTarget < 0) {
+                    cacheTarget = i;
+                }
+            } else if (maps[i].hasTask()) {
+                if (stdTarget < 0) {
+                    stdTarget = i;
+                }
+            }
+            totalProgress += maps[i].getProgress();
+        }
+        double avgProgress = totalProgress / maps.length;
+
+        for (int i = 0; i < maps.length; i++) {        
+            if (maps[i].hasSpeculativeTask(avgProgress)) {
+                if (specTarget < 0) {
+                    specTarget = i;
+                }
+            }
+        }
+        
+        if (cacheTarget >= 0) {
+            t = maps[cacheTarget].getTaskToRun(taskTracker, tts, avgProgress);
+        } else if (stdTarget >= 0) {
+            t = maps[stdTarget].getTaskToRun(taskTracker, tts, avgProgress);
+        } else if (specTarget >= 0) {
+            t = maps[specTarget].getTaskToRun(taskTracker, tts, avgProgress);
+        }
+        return t;
+    }
+
+    /**
+     * Return a ReduceTask, if appropriate, to run on the given tasktracker.
+     * We don't have cache-sensitivity for reduce tasks, as they
+     *  work on temporary MapRed files.  
+     */
+    public Task obtainNewReduceTask(String taskTracker, TaskTrackerStatus tts) {
+        if (! tasksInited) {
+            try {
+                initTasks();
+            } catch (IOException ie) {
+                ie.printStackTrace();
+                LOG.info("Cannot create task split for " + profile.getJobId());
+            }
+        }
+
+        Task t = null;
+        int stdTarget = -1;
+        int specTarget = -1;
+        int totalProgress = 0;
+        for (int i = 0; i < reduces.length; i++) {
+            totalProgress += reduces[i].getProgress();
+        }
+        double avgProgress = (1.0 * totalProgress) / reduces.length;
+
+        for (int i = 0; i < reduces.length; i++) {
+            if (reduces[i].hasTask()) {
+                if (stdTarget < 0) {
+                    stdTarget = i;
+                }
+            } else if (reduces[i].hasSpeculativeTask(avgProgress)) {
+                if (specTarget < 0) {
+                    specTarget = i;
+                }
+            }
+        }
+        
+        if (stdTarget >= 0) {
+            t = reduces[stdTarget].getTaskToRun(taskTracker, tts, avgProgress);
+        } else if (specTarget >= 0) {
+            t = reduces[specTarget].getTaskToRun(taskTracker, tts, avgProgress);
+        }
+        return t;
+    }
+
+    /**
+     * A taskid assigned to this JobInProgress has reported in successfully.
+     */
+    public synchronized void completedTask(TaskInProgress tip, String taskid) {
+        LOG.info("Taskid '" + taskid + "' has finished successfully.");
+        tip.completed(taskid);
+
+        //
+        // Figure out whether the Job is done
+        //
+        boolean allDone = true;
+        for (int i = 0; i < maps.length; i++) {
+            if (! maps[i].isComplete()) {
+                allDone = false;
+                break;
+            }
+        }
+        if (allDone) {
+            for (int i = 0; i < reduces.length; i++) {
+                if (! reduces[i].isComplete()) {
+                    allDone = false;
+                    break;
+                }
+            }
+        }
+
+        //
+        // If all tasks are complete, then the job is done!
+        //
+        if (status.getRunState() == JobStatus.RUNNING && allDone) {
+            this.status = new JobStatus(status.getJobId(), 1.0f, 1.0f, JobStatus.SUCCEEDED);
+            this.finishTime = System.currentTimeMillis();
+        }
+    }
+
+    /**
+     * Kill the job and all its component tasks.
+     */
+    public synchronized void kill() {
+        if (status.getRunState() != JobStatus.FAILED) {
+            this.status = new JobStatus(status.getJobId(), 1.0f, 1.0f, JobStatus.FAILED);
+            this.finishTime = System.currentTimeMillis();
+
+            //
+            // kill all TIPs.
+            //
+            for (int i = 0; i < maps.length; i++) {
+                maps[i].kill();
+            }
+            for (int i = 0; i < reduces.length; i++) {
+                reduces[i].kill();
+            }
+        }
+    }
+
+    /**
+     * A task assigned to this JobInProgress has reported in as failed.
+     * Most of the time, we'll just reschedule execution.  However, after
+     * many repeated failures we may instead decide to allow the entire 
+     * job to fail.
+     *
+     * Even if a task has reported as completed in the past, it might later
+     * be reported as failed.  That's because the TaskTracker that hosts a map
+     * task might die before the entire job can complete.  If that happens,
+     * we need to schedule reexecution so that downstream reduce tasks can 
+     * obtain the map task's output.
+     */
+    public void failedTask(TaskInProgress tip, String taskid, String trackerName) {
+        tip.failedSubTask(taskid, trackerName);
+            
+        //
+        // Check if we need to kill the job because of too many failures
+        //
+        if (tip.isFailed()) {
+            LOG.info("Aborting job " + profile.getJobId());
+            kill();
+        }
+    }
+
+    /**
+     * The job is dead.  We're now GC'ing it, getting rid of the job
+     * from all tables.  Be sure to remove all of this job's tasks
+     * from the various tables.
+     */
+    public synchronized void garbageCollect() throws IOException {
+        //
+        // Remove this job from all tables
+        //
+
+        // Definitely remove the local-disk copy of the job file
+        if (localJobFile != null) {
+            localJobFile.delete();
+            localJobFile = null;
+        }
+
+        //
+        // If the job file was in the temporary system directory,
+        // we should delete it upon garbage collect.
+        //
+        if (deleteUponCompletion != null) {
+            JobConf jd = new JobConf(deleteUponCompletion);
+            NutchFileSystem fs = NutchFileSystem.get(conf);
+            fs.delete(new File(jd.getJar()));
+            fs.delete(new File(deleteUponCompletion));
+            deleteUponCompletion = null;
+        }
+    }
+}
+

+ 86 - 0
src/java/org/apache/hadoop/mapred/JobProfile.java

@@ -0,0 +1,86 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.
+
+import org.apache.hadoop.io.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+
+/**************************************************
+ * A JobProfile is a MapReduce primitive.  Tracks a job,
+ * whether living or dead.
+ *
+ * @author Mike Cafarella
+ **************************************************/
+public class JobProfile implements Writable {
+    String jobid;
+    String jobFile;
+    String url;
+
+    /**
+     */
+    public JobProfile() {
+    }
+
+    /**
+     */
+    public JobProfile(String jobid, String jobFile, String url) {
+        this.jobid = jobid;
+        this.jobFile = jobFile;
+        this.url = url;
+    }
+
+    /**
+     */
+    public String getJobId() {
+        return jobid;
+    }
+
+    /**
+     */
+    public String getJobFile() {
+        return jobFile;
+    }
+
+
+    /**
+     */
+    public URL getURL() {
+        try {
+            return new URL(url.toString());
+        } catch (IOException ie) {
+            return null;
+        }
+    }
+
+    ///////////////////////////////////////
+    // Writable
+    ///////////////////////////////////////
+    public void write(DataOutput out) throws IOException {
+        UTF8.writeString(out, jobid);
+        UTF8.writeString(out, jobFile);
+        UTF8.writeString(out, url);
+    }
+    public void readFields(DataInput in) throws IOException {
+        this.jobid = UTF8.readString(in);
+        this.jobFile = UTF8.readString(in);
+        this.url = UTF8.readString(in);
+    }
+}
+
+

+ 79 - 0
src/java/org/apache/hadoop/mapred/JobStatus.java

@@ -0,0 +1,79 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.
+
+import org.apache.hadoop.io.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+
+/**************************************************
+ * Describes the current status of a job.  This is
+ * not intended to be a comprehensive piece of data.
+ * For that, look at JobProfile.
+ *
+ * @author Mike Cafarella
+ **************************************************/
+public class JobStatus implements Writable {
+    public static final int RUNNING = 1;
+    public static final int SUCCEEDED = 2;
+    public static final int FAILED = 3;
+
+    String jobid;
+    float mapProgress;
+    float reduceProgress;
+    int runState;
+
+    /**
+     */
+    public JobStatus() {
+    }
+
+    /**
+     */
+    public JobStatus(String jobid, float mapProgress, float reduceProgress, int runState) {
+        this.jobid = jobid;
+        this.mapProgress = mapProgress;
+        this.reduceProgress = reduceProgress;
+        this.runState = runState;
+    }
+
+    /**
+     */
+    public String getJobId() { return jobid; }
+    public float mapProgress() { return mapProgress; }
+    public void setMapProgress(float p) { this.mapProgress = p; }
+    public float reduceProgress() { return reduceProgress; }
+    public void setReduceProgress(float p) { this.reduceProgress = p; }
+    public int getRunState() { return runState; }
+
+    ///////////////////////////////////////
+    // Writable
+    ///////////////////////////////////////
+    public void write(DataOutput out) throws IOException {
+        UTF8.writeString(out, jobid);
+        out.writeFloat(mapProgress);
+        out.writeFloat(reduceProgress);
+        out.writeInt(runState);
+    }
+    public void readFields(DataInput in) throws IOException {
+        this.jobid = UTF8.readString(in);
+        this.mapProgress = in.readFloat();
+        this.reduceProgress = in.readFloat();
+        this.runState = in.readInt();
+    }
+}

+ 63 - 0
src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java

@@ -0,0 +1,63 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+import java.io.*;
+import java.util.*;
+import org.apache.hadoop.io.*;
+
+/** 
+ * Protocol that a JobClient and the central JobTracker use to communicate.  The
+ * JobClient can use these methods to submit a Job for execution, and learn about
+ * the current system status.
+ */ 
+public interface JobSubmissionProtocol {
+    /**
+     * Submit a Job for execution.  Returns the latest profile for
+     * that job.
+     */
+    public JobStatus submitJob(String jobFile) throws IOException;
+
+    /**
+     * Kill the indicated job
+     */
+    public void killJob(String jobid);
+
+    /**
+     * Grab a handle to a job that is already known to the JobTracker
+     */
+    public JobProfile getJobProfile(String jobid);
+
+    /**
+     * Grab a handle to a job that is already known to the JobTracker
+     */
+    public JobStatus getJobStatus(String jobid);
+
+    /**
+     * Grab a bunch of info on the tasks that make up the job
+     */
+    public Vector[] getMapTaskReport(String jobid);
+    public Vector[] getReduceTaskReport(String jobid);
+
+    /**
+     * A MapReduce system always operates on a single filesystem.  This 
+     * function returns the fs name.  ('local' if the localfs; 'addr:port' 
+     * if dfs).  The client can then copy files into the right locations 
+     * prior to submitting the job.
+     */
+    public String getFilesystemName() throws IOException;
+}

+ 816 - 0
src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -0,0 +1,816 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.
+
+
+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 java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.logging.*;
+
+/*******************************************************
+ * JobTracker is the central location for submitting and 
+ * tracking MR jobs in a network environment.
+ *
+ * @author Mike Cafarella
+ *******************************************************/
+public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmissionProtocol {
+    static long RETIRE_JOB_INTERVAL;
+    static long RETIRE_JOB_CHECK_INTERVAL;
+    static float TASK_ALLOC_EPSILON;
+    static float PAD_FRACTION;
+    static float MIN_SLOTS_FOR_PADDING;
+
+    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.mapred.JobTracker");
+
+    private static JobTracker tracker = null;
+    public static void startTracker(Configuration conf) throws IOException {
+      if (tracker != null)
+        throw new IOException("JobTracker already running.");
+      while (true) {
+        try {
+          tracker = new JobTracker(conf);
+          break;
+        } catch (IOException e) {
+          LOG.log(Level.WARNING, "Starting tracker", e);
+        }
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+        }
+      }
+      tracker.offerService();
+    }
+
+    public static JobTracker getTracker() {
+        return tracker;
+    }
+
+    ///////////////////////////////////////////////////////
+    // Used to expire TaskTrackers that have gone down
+    ///////////////////////////////////////////////////////
+    class ExpireTrackers implements Runnable {
+        boolean shouldRun = true;
+        public ExpireTrackers() {
+        }
+        /**
+         * The run method lives for the life of the JobTracker, and removes TaskTrackers
+         * that have not checked in for some time.
+         */
+        public void run() {
+            while (shouldRun) {
+                //
+                // Thread runs periodically to check whether trackers should be expired.
+                // The sleep interval must be no more than half the maximum expiry time
+                // for a task tracker.
+                //
+                try {
+                    Thread.sleep(TASKTRACKER_EXPIRY_INTERVAL / 3);
+                } catch (InterruptedException ie) {
+                }
+
+                //
+                // Loop through all expired items in the queue
+                //
+                synchronized (taskTrackers) {
+                    synchronized (trackerExpiryQueue) {
+                        long now = System.currentTimeMillis();
+                        TaskTrackerStatus leastRecent = null;
+                        while ((trackerExpiryQueue.size() > 0) &&
+                               ((leastRecent = (TaskTrackerStatus) trackerExpiryQueue.first()) != null) &&
+                               (now - leastRecent.getLastSeen() > TASKTRACKER_EXPIRY_INTERVAL)) {
+
+                            // Remove profile from head of queue
+                            trackerExpiryQueue.remove(leastRecent);
+
+                            // Figure out if last-seen time should be updated, or if tracker is dead
+                            TaskTrackerStatus newProfile = (TaskTrackerStatus) taskTrackers.get(leastRecent.getTrackerName());
+                            // Items might leave the taskTracker set through other means; the
+                            // status stored in 'taskTrackers' might be null, which means the
+                            // tracker has already been destroyed.
+                            if (newProfile != null) {
+                                if (now - newProfile.getLastSeen() > TASKTRACKER_EXPIRY_INTERVAL) {
+                                    // Remove completely
+
+                                    TaskTrackerStatus oldStatus = (TaskTrackerStatus) taskTrackers.remove(leastRecent.getTrackerName());
+                                    if (oldStatus != null) {
+                                        totalMaps -= oldStatus.countMapTasks();
+                                        totalReduces -= oldStatus.countReduceTasks();
+                                    }
+                                    lostTaskTracker(leastRecent.getTrackerName());
+                                } else {
+                                    // Update time by inserting latest profile
+                                    trackerExpiryQueue.add(newProfile);
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+        }
+        
+        /**
+         * Stop the tracker on next iteration
+         */
+        public void stopTracker() {
+            shouldRun = false;
+        }
+    }
+
+    ///////////////////////////////////////////////////////
+    // Used to remove old finished Jobs that have been around for too long
+    ///////////////////////////////////////////////////////
+    class RetireJobs implements Runnable {
+        boolean shouldRun = true;
+        public RetireJobs() {
+        }
+
+        /**
+         * The run method lives for the life of the JobTracker,
+         * and removes Jobs that are not still running, but which
+         * finished a long time ago.
+         */
+        public void run() {
+            while (shouldRun) {
+                try {
+                    Thread.sleep(RETIRE_JOB_CHECK_INTERVAL);
+                } catch (InterruptedException ie) {
+                }
+                
+                synchronized (jobs) {
+                    for (Iterator it = jobs.keySet().iterator(); it.hasNext(); ) {
+                        String jobid = (String) it.next();
+                        JobInProgress job = (JobInProgress) jobs.get(jobid);
+
+                        if (job.getStatus().getRunState() != JobStatus.RUNNING &&
+                            (job.getFinishTime() + RETIRE_JOB_INTERVAL < System.currentTimeMillis())) {
+                            it.remove();
+                            jobsByArrival.remove(job);
+                        }
+                    }
+                }
+            }
+        }
+        public void stopRetirer() {
+            shouldRun = false;
+        }
+    }
+
+    /////////////////////////////////////////////////////////////////
+    // The real JobTracker
+    ////////////////////////////////////////////////////////////////
+    int port;
+    String localMachine;
+    long startTime;
+    int totalSubmissions = 0;
+    Random r = new Random();
+
+    private int maxCurrentTasks;
+
+    //
+    // Properties to maintain while running Jobs and Tasks:
+    //
+    // 1.  Each Task is always contained in a single Job.  A Job succeeds when all its 
+    //     Tasks are complete.
+    //
+    // 2.  Every running or successful Task is assigned to a Tracker.  Idle Tasks are not.
+    //
+    // 3.  When a Tracker fails, all of its assigned Tasks are marked as failures.
+    //
+    // 4.  A Task might need to be reexecuted if it (or the machine it's hosted on) fails
+    //     before the Job is 100% complete.  Sometimes an upstream Task can fail without
+    //     reexecution if all downstream Tasks that require its output have already obtained
+    //     the necessary files.
+    //
+
+    // All the known jobs.  (jobid->JobInProgress)
+    TreeMap jobs = new TreeMap();
+    Vector jobsByArrival = new Vector();
+
+    // All the known TaskInProgress items, mapped to by taskids (taskid->TIP)
+    TreeMap taskidToTIPMap = new TreeMap();
+
+    // (taskid --> trackerID) 
+    TreeMap taskidToTrackerMap = new TreeMap();
+
+    // (trackerID->TreeSet of taskids running at that tracker)
+    TreeMap trackerToTaskMap = new TreeMap();
+
+    //
+    // Watch and expire TaskTracker objects using these structures.
+    // We can map from Name->TaskTrackerStatus, or we can expire by time.
+    //
+    int totalMaps = 0;
+    int totalReduces = 0;
+    TreeMap taskTrackers = new TreeMap();
+    ExpireTrackers expireTrackers = new ExpireTrackers();
+    RetireJobs retireJobs = new RetireJobs();
+
+    /**
+     * It might seem like a bug to maintain a TreeSet of status objects,
+     * which can be updated at any time.  But that's not what happens!  We
+     * only update status objects in the taskTrackers table.  Status objects
+     * are never updated once they enter the expiry queue.  Instead, we wait
+     * for them to expire and remove them from the expiry queue.  If a status
+     * object has been updated in the taskTracker table, the latest status is 
+     * reinserted.  Otherwise, we assume the tracker has expired.
+     */
+    TreeSet trackerExpiryQueue = new TreeSet(new Comparator() {
+        public int compare(Object o1, Object o2) {
+            TaskTrackerStatus p1 = (TaskTrackerStatus) o1;
+            TaskTrackerStatus p2 = (TaskTrackerStatus) o2;
+            if (p1.getLastSeen() < p2.getLastSeen()) {
+                return -1;
+            } else if (p1.getLastSeen() > p2.getLastSeen()) {
+                return 1;
+            } else {
+                return (p1.getTrackerName().compareTo(p2.getTrackerName()));
+            }
+        }
+    });
+
+    // Used to provide an HTML view on Job, Task, and TaskTracker structures
+    JobTrackerInfoServer infoServer;
+    int infoPort;
+
+    Server interTrackerServer;
+
+    // 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;
+    File systemDir;
+    private Configuration conf;
+
+    /**
+     * Start the JobTracker process, listen on the indicated port
+     */
+    JobTracker(Configuration conf) throws IOException {
+        //
+        // Grab some static constants
+        //
+        maxCurrentTasks = conf.getInt("mapred.tasktracker.tasks.maximum", 2);
+        RETIRE_JOB_INTERVAL = conf.getLong("mapred.jobtracker.retirejob.interval", 24 * 60 * 60 * 1000);
+        RETIRE_JOB_CHECK_INTERVAL = conf.getLong("mapred.jobtracker.retirejob.check", 60 * 1000);
+        TASK_ALLOC_EPSILON = conf.getFloat("mapred.jobtracker.taskalloc.loadbalance.epsilon", 0.2f);
+        PAD_FRACTION = conf.getFloat("mapred.jobtracker.taskalloc.capacitypad", 0.1f);
+        MIN_SLOTS_FOR_PADDING = 3 * maxCurrentTasks;
+
+        // This is a directory of temporary submission files.  We delete it
+        // on startup, and can delete any files that we're done with
+        this.conf = conf;
+        JobConf jobConf = new JobConf(conf);
+        this.systemDir = jobConf.getSystemDir();
+        this.fs = NutchFileSystem.get(conf);
+        FileUtil.fullyDelete(fs, systemDir);
+        fs.mkdirs(systemDir);
+
+        // Same with 'localDir' except it's always on the local disk.
+        jobConf.deleteLocalFiles(SUBDIR);
+
+        // Set ports, start RPC servers, etc.
+        InetSocketAddress addr = getAddress(conf);
+        this.localMachine = addr.getHostName();
+        this.port = addr.getPort();
+        this.interTrackerServer = RPC.getServer(this, addr.getPort(), 10, false, conf);
+        this.interTrackerServer.start();
+	Properties p = System.getProperties();
+	for (Iterator it = p.keySet().iterator(); it.hasNext(); ) {
+	    String key = (String) it.next();
+	    String val = (String) p.getProperty(key);
+	    LOG.info("Property '" + key + "' is " + val);
+	}
+
+        this.infoPort = conf.getInt("mapred.job.tracker.info.port", 50030);
+        this.infoServer = new JobTrackerInfoServer(this, infoPort);
+        this.infoServer.start();
+
+        this.startTime = System.currentTimeMillis();
+
+        new Thread(this.expireTrackers).start();
+        new Thread(this.retireJobs).start();
+    }
+
+    public static InetSocketAddress getAddress(Configuration conf) {
+      String jobTrackerStr =
+        conf.get("mapred.job.tracker", "localhost:8012");
+      int colon = jobTrackerStr.indexOf(":");
+      if (colon < 0) {
+        throw new RuntimeException("Bad mapred.job.tracker: "+jobTrackerStr);
+      }
+      String jobTrackerName = jobTrackerStr.substring(0, colon);
+      int jobTrackerPort = Integer.parseInt(jobTrackerStr.substring(colon+1));
+      return new InetSocketAddress(jobTrackerName, jobTrackerPort);
+    }
+
+
+    /**
+     * Run forever
+     */
+    public void offerService() {
+        try {
+            this.interTrackerServer.join();
+        } catch (InterruptedException ie) {
+        }
+    }
+
+    ///////////////////////////////////////////////////////
+    // Maintain lookup tables; called by JobInProgress
+    // and TaskInProgress
+    ///////////////////////////////////////////////////////
+    void createTaskEntry(String taskid, String taskTracker, TaskInProgress tip) {
+        LOG.info("Adding task '" + taskid + "' to tip " + tip.getTIPId() + ", for tracker '" + taskTracker + "'");
+
+        // taskid --> tracker
+        taskidToTrackerMap.put(taskid, taskTracker);
+
+        // tracker --> taskid
+        TreeSet taskset = (TreeSet) trackerToTaskMap.get(taskTracker);
+        if (taskset == null) {
+            taskset = new TreeSet();
+            trackerToTaskMap.put(taskTracker, taskset);
+        }
+        taskset.add(taskid);
+
+        // taskid --> TIP
+        taskidToTIPMap.put(taskid, tip);
+    }
+    void removeTaskEntry(String taskid) {
+        // taskid --> tracker
+        String tracker = (String) taskidToTrackerMap.remove(taskid);
+
+        // tracker --> taskid
+        TreeSet trackerSet = (TreeSet) trackerToTaskMap.get(tracker);
+        if (trackerSet != null) {
+            trackerSet.remove(taskid);
+        }
+
+        // taskid --> TIP
+        taskidToTIPMap.remove(taskid);
+    }
+
+    ///////////////////////////////////////////////////////
+    // Accessors for objects that want info on jobs, tasks,
+    // trackers, etc.
+    ///////////////////////////////////////////////////////
+    public int getTotalSubmissions() {
+        return totalSubmissions;
+    }
+    public String getJobTrackerMachine() {
+        return localMachine;
+    }
+    public int getTrackerPort() {
+        return port;
+    }
+    public int getInfoPort() {
+        return infoPort;
+    }
+    public long getStartTime() {
+        return startTime;
+    }
+    public Vector runningJobs() {
+        Vector v = new Vector();
+        for (Iterator it = jobs.values().iterator(); it.hasNext(); ) {
+            JobInProgress jip = (JobInProgress) it.next();
+            JobStatus status = jip.getStatus();
+            if (status.getRunState() == JobStatus.RUNNING) {
+                v.add(jip);
+            }
+        }
+        return v;
+    }
+    public Vector failedJobs() {
+        Vector v = new Vector();
+        for (Iterator it = jobs.values().iterator(); it.hasNext(); ) {
+            JobInProgress jip = (JobInProgress) it.next();
+            JobStatus status = jip.getStatus();
+            if (status.getRunState() == JobStatus.FAILED) {
+                v.add(jip);
+            }
+        }
+        return v;
+    }
+    public Vector completedJobs() {
+        Vector v = new Vector();
+        for (Iterator it = jobs.values().iterator(); it.hasNext(); ) {
+            JobInProgress jip = (JobInProgress) it.next();
+            JobStatus status = jip.getStatus();
+            if (status.getRunState() == JobStatus.SUCCEEDED) {
+                v.add(jip);
+            }
+        }
+        return v;
+    }
+    public Collection taskTrackers() {
+        return taskTrackers.values();
+    }
+    public TaskTrackerStatus getTaskTracker(String trackerID) {
+        return (TaskTrackerStatus) taskTrackers.get(trackerID);
+    }
+
+    ////////////////////////////////////////////////////
+    // InterTrackerProtocol
+    ////////////////////////////////////////////////////
+    public void initialize(String taskTrackerName) {
+        if (taskTrackers.get(taskTrackerName) != null) {
+            TaskTrackerStatus oldStatus = (TaskTrackerStatus) taskTrackers.remove(taskTrackerName);
+            totalMaps -= oldStatus.countMapTasks();
+            totalReduces -= oldStatus.countReduceTasks();
+
+            lostTaskTracker(taskTrackerName);
+        }
+    }
+
+    /**
+     * Process incoming heartbeat messages from the task trackers.
+     */
+    public synchronized int emitHeartbeat(TaskTrackerStatus trackerStatus, boolean initialContact) {
+        String trackerName = trackerStatus.getTrackerName();
+        trackerStatus.setLastSeen(System.currentTimeMillis());
+
+        synchronized (taskTrackers) {
+            synchronized (trackerExpiryQueue) {
+                if (initialContact) {
+                    // If it's first contact, then clear out any state hanging around
+                    if (taskTrackers.get(trackerName) != null) {
+                        TaskTrackerStatus oldStatus = (TaskTrackerStatus) taskTrackers.remove(trackerName);
+                        totalMaps -= oldStatus.countMapTasks();
+                        totalReduces -= oldStatus.countReduceTasks();
+                        lostTaskTracker(trackerName);
+                    }
+                } else {
+                    // If not first contact, there should be some record of the tracker
+                    if (taskTrackers.get(trackerName) == null) {
+                        return InterTrackerProtocol.UNKNOWN_TASKTRACKER;
+                    }
+                }
+
+                // Store latest state.  If first contact, then save current
+                // state in expiry queue
+                totalMaps += trackerStatus.countMapTasks();
+                totalReduces += trackerStatus.countReduceTasks();
+                taskTrackers.put(trackerName, trackerStatus);
+                if (initialContact) {
+                    trackerExpiryQueue.add(trackerStatus);
+                }
+            }
+        }
+
+        updateTaskStatuses(trackerStatus);
+        //LOG.info("Got heartbeat from "+trackerName);
+        return InterTrackerProtocol.TRACKERS_OK;
+    }
+
+    /**
+     * A tracker wants to know if there's a Task to run.  Returns
+     * a task we'd like the TaskTracker to execute right now.
+     *
+     * Eventually this function should compute load on the various TaskTrackers,
+     * and incorporate knowledge of DFS file placement.  But for right now, it
+     * just grabs a single item out of the pending task list and hands it back.
+     */
+    public synchronized Task pollForNewTask(String taskTracker) {
+        //
+        // Compute average map and reduce task numbers across pool
+        //
+        int avgMaps = 0;
+        int avgReduces = 0;
+        if (taskTrackers.size() > 0) {
+            avgMaps = totalMaps / taskTrackers.size();
+            avgReduces = totalReduces / taskTrackers.size();
+        }
+        int totalCapacity = taskTrackers.size() * maxCurrentTasks;
+
+        //
+        // Get map + reduce counts for the current tracker.
+        //
+        TaskTrackerStatus tts = (TaskTrackerStatus) taskTrackers.get(taskTracker);
+        int numMaps = tts.countMapTasks();
+        int numReduces = tts.countReduceTasks();
+
+        //
+        // In the below steps, we allocate first a map task (if appropriate),
+        // and then a reduce task if appropriate.  We go through all jobs
+        // in order of job arrival; jobs only get serviced if their 
+        // predecessors are serviced, too.
+        //
+
+        //
+        // We hand a task to the current taskTracker if the given machine 
+        // has a workload that's equal to or less than the averageMaps 
+        // +/- TASK_ALLOC_EPSILON.  (That epsilon is in place in case
+        // there is an odd machine that is failing for some reason but 
+        // has not yet been removed from the pool, making capacity seem
+        // larger than it really is.)
+        //
+        if ((numMaps < maxCurrentTasks) &&
+            (numMaps <= (avgMaps + TASK_ALLOC_EPSILON))) {
+
+            int totalNeededMaps = 0;
+            for (Iterator it = jobsByArrival.iterator(); it.hasNext(); ) {
+                JobInProgress job = (JobInProgress) it.next();
+                if (job.getStatus().getRunState() != JobStatus.RUNNING) {
+                    continue;
+                }
+
+                Task t = job.obtainNewMapTask(taskTracker, tts);
+                if (t != null) {
+                    return t;
+                }
+
+                //
+                // Beyond the highest-priority task, reserve a little 
+                // room for failures and speculative executions; don't 
+                // schedule tasks to the hilt.
+                //
+                totalNeededMaps += job.desiredMaps();
+                double padding = 0;
+                if (totalCapacity > MIN_SLOTS_FOR_PADDING) {
+                    padding = Math.min(maxCurrentTasks, totalNeededMaps * PAD_FRACTION);
+                }
+                if (totalNeededMaps + padding >= totalCapacity) {
+                    break;
+                }
+            }
+        }
+
+        //
+        // Same thing, but for reduce tasks
+        //
+        if ((numReduces < maxCurrentTasks) &&
+            (numReduces <= (avgReduces + TASK_ALLOC_EPSILON))) {
+
+            int totalNeededReduces = 0;
+            for (Iterator it = jobsByArrival.iterator(); it.hasNext(); ) {
+                JobInProgress job = (JobInProgress) it.next();
+                if (job.getStatus().getRunState() != JobStatus.RUNNING) {
+                    continue;
+                }
+
+                Task t = job.obtainNewReduceTask(taskTracker, tts);
+                if (t != null) {
+                    return t;
+                }
+
+                //
+                // Beyond the highest-priority task, reserve a little 
+                // room for failures and speculative executions; don't 
+                // schedule tasks to the hilt.
+                //
+                totalNeededReduces += job.desiredReduces();
+                double padding = 0;
+                if (totalCapacity > MIN_SLOTS_FOR_PADDING) {
+                    padding = Math.min(maxCurrentTasks, totalNeededReduces * PAD_FRACTION);
+                }
+                if (totalNeededReduces + padding >= totalCapacity) {
+                    break;
+                }
+            }
+        }
+        return null;
+    }
+
+    /**
+     * A tracker wants to know if any of its Tasks have been
+     * closed (because the job completed, whether successfully or not)
+     */
+    public String pollForTaskWithClosedJob(String taskTracker) {
+        TreeSet taskIds = (TreeSet) trackerToTaskMap.get(taskTracker);
+        if (taskIds != null) {
+            for (Iterator it = taskIds.iterator(); it.hasNext(); ) {
+                String taskId = (String) it.next();
+                TaskInProgress tip = (TaskInProgress) taskidToTIPMap.get(taskId);
+                if (tip.shouldCloseForClosedJob(taskId)) {
+                    // 
+                    // This is how the JobTracker ends a task at the TaskTracker.
+                    // It may be successfully completed, or may be killed in
+                    // mid-execution.
+                    //
+                    return taskId;
+                }
+            }
+        }
+        return null;
+    }
+
+    /**
+     * A TaskTracker wants to know the physical locations of completed, but not
+     * yet closed, tasks.  This exists so the reduce task thread can locate
+     * map task outputs.
+     */
+    public synchronized MapOutputLocation[] locateMapOutputs(String taskId, String[][] mapTasksNeeded) {
+        ArrayList v = new ArrayList();
+        for (int i = 0; i < mapTasksNeeded.length; i++) {
+            for (int j = 0; j < mapTasksNeeded[i].length; j++) {
+                TaskInProgress tip = (TaskInProgress) taskidToTIPMap.get(mapTasksNeeded[i][j]);
+                if (tip != null && tip.isComplete(mapTasksNeeded[i][j])) {
+                    String trackerId = (String) taskidToTrackerMap.get(mapTasksNeeded[i][j]);
+                    TaskTrackerStatus tracker = (TaskTrackerStatus) taskTrackers.get(trackerId);
+                    v.add(new MapOutputLocation(mapTasksNeeded[i][j], tracker.getHost(), tracker.getPort()));
+                    break;
+                }
+            }
+        }
+        // randomly shuffle results to load-balance map output requests
+        Collections.shuffle(v);
+
+        return (MapOutputLocation[]) v.toArray(new MapOutputLocation[v.size()]);
+    }
+
+    /**
+     * Grab the local fs name
+     */
+    public synchronized String getFilesystemName() throws IOException {
+        return fs.getName();
+    }
+
+    ////////////////////////////////////////////////////
+    // JobSubmissionProtocol
+    ////////////////////////////////////////////////////
+    public synchronized JobStatus submitJob(String jobFile) throws IOException {
+        totalSubmissions++;
+        JobInProgress job = createJob(jobFile);
+        return job.getStatus();
+    }
+
+    public synchronized void killJob(String jobid) {
+        JobInProgress job = (JobInProgress) jobs.get(jobid);
+        job.kill();
+    }
+
+    public synchronized JobProfile getJobProfile(String jobid) {
+        JobInProgress job = (JobInProgress) jobs.get(jobid);
+        if (job != null) {
+            return job.getProfile();
+        } else {
+            return null;
+        }
+    }
+    public synchronized JobStatus getJobStatus(String jobid) {
+        JobInProgress job = (JobInProgress) jobs.get(jobid);
+        if (job != null) {
+            return job.getStatus();
+        } else {
+            return null;
+        }
+    }
+    public synchronized Vector[] getMapTaskReport(String jobid) {
+        JobInProgress job = (JobInProgress) jobs.get(jobid);
+        if (job == null) {
+            return new Vector[0];
+        } else {
+            Vector reports = new Vector();
+            Vector completeMapTasks = job.reportTasksInProgress(true, true);
+            for (Iterator it = completeMapTasks.iterator(); it.hasNext(); ) {
+                TaskInProgress tip = (TaskInProgress) it.next();
+                reports.add(tip.generateSingleReport());
+            }
+            Vector incompleteMapTasks = job.reportTasksInProgress(true, false);
+            for (Iterator it = incompleteMapTasks.iterator(); it.hasNext(); ) {
+                TaskInProgress tip = (TaskInProgress) it.next();
+                reports.add(tip.generateSingleReport());
+            }
+            return (Vector[]) reports.toArray(new Vector[reports.size()]);
+        }
+    }
+
+    public synchronized Vector[] getReduceTaskReport(String jobid) {
+        JobInProgress job = (JobInProgress) jobs.get(jobid);
+        if (job == null) {
+            return new Vector[0];
+        } else {
+            Vector reports = new Vector();
+            Vector completeReduceTasks = job.reportTasksInProgress(false, true);
+            for (Iterator it = completeReduceTasks.iterator(); it.hasNext(); ) {
+                TaskInProgress tip = (TaskInProgress) it.next();
+                reports.add(tip.generateSingleReport());
+            }
+            Vector incompleteReduceTasks = job.reportTasksInProgress(false, false);
+            for (Iterator it = incompleteReduceTasks.iterator(); it.hasNext(); ) {
+                TaskInProgress tip = (TaskInProgress) it.next();
+                reports.add(tip.generateSingleReport());
+            }
+            return (Vector[]) reports.toArray(new Vector[reports.size()]);
+        }
+    }
+
+    ///////////////////////////////////////////////////////////////
+    // JobTracker methods
+    ///////////////////////////////////////////////////////////////
+    public JobInProgress getJob(String jobid) {
+        return (JobInProgress) jobs.get(jobid);
+    }
+    /**
+     * Grab random num for task id
+     */
+    String createUniqueId() {
+        return "" + Integer.toString(Math.abs(r.nextInt()),36);
+    }
+
+    /**
+     * JobProfile createJob() kicks off a new job.  
+     * This function creates a job profile and also decomposes it into
+     * tasks.  The tasks are added to the unassignedTasks structure.  
+     * (The precise structure will change as we get more sophisticated about 
+     * task allocation.)
+     *
+     * Create a 'JobInProgress' object, which contains both JobProfile
+     * and JobStatus.  Those two sub-objects are sometimes shipped outside
+     * of the JobTracker.  But JobInProgress adds info that's useful for
+     * the JobTracker alone.
+     */
+    JobInProgress createJob(String jobFile) throws IOException {
+        JobInProgress job = new JobInProgress(jobFile, this, this.conf);
+        jobs.put(job.getProfile().getJobId(), job);
+        jobsByArrival.add(job);
+        return job;
+    }
+
+    ////////////////////////////////////////////////////
+    // Methods to track all the TaskTrackers
+    ////////////////////////////////////////////////////
+    /**
+     * Accept and process a new TaskTracker profile.  We might
+     * have known about the TaskTracker previously, or it might
+     * be brand-new.  All task-tracker structures have already
+     * been updated.  Just process the contained tasks and any
+     * jobs that might be affected.
+     */
+    void updateTaskStatuses(TaskTrackerStatus status) {
+        for (Iterator it = status.taskReports(); it.hasNext(); ) {
+            TaskStatus report = (TaskStatus) it.next();
+            TaskInProgress tip = (TaskInProgress) taskidToTIPMap.get(report.getTaskId());
+            if (tip == null) {
+                LOG.info("Serious problem.  While updating status, cannot find taskid " + report.getTaskId());
+            } else {
+                JobInProgress job = tip.getJob();
+                job.updateTaskStatus(tip, report);
+
+                if (report.getRunState() == TaskStatus.SUCCEEDED) {
+                    job.completedTask(tip, report.getTaskId());
+                } else if (report.getRunState() == TaskStatus.FAILED) {
+                    // Tell the job to fail the relevant task
+                    job.failedTask(tip, report.getTaskId(), status.getTrackerName());
+                }
+            }
+        }
+    }
+
+    /**
+     * We lost the task tracker!  All task-tracker structures have 
+     * already been updated.  Just process the contained tasks and any
+     * jobs that might be affected.
+     */
+    void lostTaskTracker(String trackerName) {
+        LOG.info("Lost tracker '" + trackerName + "'");
+        TreeSet lostTasks = (TreeSet) trackerToTaskMap.get(trackerName);
+        trackerToTaskMap.remove(trackerName);
+
+        if (lostTasks != null) {
+            for (Iterator it = lostTasks.iterator(); it.hasNext(); ) {
+                String taskId = (String) it.next();
+                TaskInProgress tip = (TaskInProgress) taskidToTIPMap.get(taskId);
+
+                // Tell the job to fail the relevant task
+                JobInProgress job = tip.getJob();
+                job.failedTask(tip, taskId, trackerName);
+            }
+        }
+    }
+
+    ////////////////////////////////////////////////////////////
+    // main()
+    ////////////////////////////////////////////////////////////
+
+    /**
+     * Start the JobTracker process.  This is used only for debugging.  As a rule,
+     * JobTracker should be run as part of the DFS Namenode process.
+     */
+    public static void main(String argv[]) throws IOException, InterruptedException {
+        if (argv.length != 0) {
+          System.out.println("usage: JobTracker");
+          System.exit(-1);
+        }
+
+        startTracker(new Configuration());
+    }
+}

+ 128 - 0
src/java/org/apache/hadoop/mapred/JobTrackerInfoServer.java

@@ -0,0 +1,128 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.
+
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.ipc.*;
+import org.apache.hadoop.conf.*;
+
+import org.mortbay.util.*;
+import org.mortbay.http.*;
+import org.mortbay.http.handler.*;
+import org.mortbay.jetty.servlet.*;
+import org.mortbay.jetty.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.logging.*;
+
+import javax.servlet.*;
+import javax.servlet.http.*;
+
+/*******************************************************
+ * JobTrackerInfoServer provides stats about the JobTracker
+ * via HTTP.  It's useful for clients that want to track
+ * their jobs' progress.
+ *
+ * @author Mike Cafarella
+ *******************************************************/
+public class JobTrackerInfoServer {
+
+    public static class RedirectHandler extends AbstractHttpHandler {
+        public void handle(String pathInContext, String pathParams, HttpRequest request, HttpResponse response) throws HttpException, IOException {
+            response.sendRedirect("/jobtracker");
+            request.setHandled(true);
+        }
+    }
+
+    /////////////////////////////////////
+    // The actual JobTrackerInfoServer
+    /////////////////////////////////////
+    static JobTracker jobTracker;
+    org.mortbay.jetty.Server server;
+
+    /**
+     * We need the jobTracker to grab stats, and the port to 
+     * know where to listen.
+     */
+    private static final boolean WINDOWS = System.getProperty("os.name").startsWith("Windows");
+    public JobTrackerInfoServer(JobTracker jobTracker, int port) throws IOException {
+        this.jobTracker = jobTracker;
+        this.server = new org.mortbay.jetty.Server();
+	URL url = JobTrackerInfoServer.class.getClassLoader().getResource("webapps");
+	String path = url.getPath();
+	if (WINDOWS && path.startsWith("/")) {
+	    path = path.substring(1);
+	    try {
+		path = URLDecoder.decode(path, "UTF-8");
+	    } catch (UnsupportedEncodingException e) {
+	    }
+	}
+	File jobtracker = new File(path, "jobtracker");
+        WebApplicationContext context = server.addWebApplication(null, "/", jobtracker.getCanonicalPath());
+
+        SocketListener socketListener = new SocketListener();
+        socketListener.setPort(port);
+        this.server.addListener(socketListener);
+
+        //
+        // REMIND - mjc - I can't figure out how to get request redirect to work.
+        // I've tried adding an additional default handler to the context, but
+        // it doesn't seem to work.  The handler has its handle() function called
+        // even when the JSP is processed correctly!  I just want to add a redirect
+        // page, when the URL is incorrectly typed.
+        //
+        // context.addHandler(new LocalNotFoundHandler());
+    }
+
+    /**
+     * The thread class we need to kick off the HTTP server async-style.
+     */
+    class HTTPStarter implements Runnable {
+        public void run() {
+            try {
+                server.start();
+            } catch (Exception me) {
+                me.printStackTrace();
+            }
+        }
+    }
+
+    /**
+     * Launch the HTTP server
+     */
+    public void start() throws IOException {
+        new Thread(new HTTPStarter()).start();
+        try {
+            Thread.sleep(1000);
+        } catch (InterruptedException ie) {
+        }
+        if (! server.isStarted()) {
+            throw new IOException("Could not start HTTP server");
+        }
+    }
+
+    /**
+     * Stop the HTTP server
+     */
+    public void stop() {
+        try {
+            this.server.stop();
+        } catch (InterruptedException ie) {
+        }
+    }
+}

+ 188 - 0
src/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -0,0 +1,188 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+import java.io.*;
+import java.util.*;
+import java.util.logging.*;
+
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.util.LogFormatter;
+
+/** Implements MapReduce locally, in-process, for debugging. */ 
+public class LocalJobRunner implements JobSubmissionProtocol {
+  public static final Logger LOG =
+    LogFormatter.getLogger("org.apache.hadoop.mapred.LocalJobRunner");
+
+  private NutchFileSystem fs;
+  private HashMap jobs = new HashMap();
+  private Configuration conf;
+
+  private class Job extends Thread
+    implements TaskUmbilicalProtocol {
+    private String file;
+    private String id;
+    private JobConf job;
+
+    private JobStatus status = new JobStatus();
+    private ArrayList mapIds = new ArrayList();
+    private MapOutputFile mapoutputFile;
+
+    public Job(String file, Configuration conf) throws IOException {
+      this.file = file;
+      this.id = "job_" + newId();
+      this.mapoutputFile = new MapOutputFile();
+      this.mapoutputFile.setConf(conf);
+
+      File localFile = new JobConf(conf).getLocalFile("localRunner", id+".xml");
+      fs.copyToLocalFile(new File(file), localFile);
+      this.job = new JobConf(localFile);
+      
+      
+      this.status.jobid = id;
+      this.status.runState = JobStatus.RUNNING;
+
+      jobs.put(id, this);
+
+      this.start();
+    }
+
+    public void run() {
+      try {
+        // split input into minimum number of splits
+        FileSplit[] splits = job.getInputFormat().getSplits(fs, job, 1);
+
+        // run a map task for each split
+        job.setNumReduceTasks(1);                 // force a single reduce task
+        for (int i = 0; i < splits.length; i++) {
+          mapIds.add("map_" + newId());
+          MapTask map = new MapTask(file, (String)mapIds.get(i), splits[i]);
+          map.setConf(job);
+          map.run(job, this);
+        }
+
+        // move map output to reduce input
+        String reduceId = "reduce_" + newId();
+        for (int i = 0; i < mapIds.size(); i++) {
+          String mapId = (String)mapIds.get(i);
+          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))
+            throw new IOException("Couldn't rename " + mapOut);
+          this.mapoutputFile.removeAll(mapId);
+        }
+
+        // run a single reduce task
+        String mapDependencies[][] = new String[mapIds.size()][1];
+        for (int i = 0; i < mapIds.size(); i++) {
+            mapDependencies[i][0] = (String) mapIds.get(i);
+        }
+        ReduceTask reduce =
+          new ReduceTask(file, reduceId,
+                         mapDependencies,
+                         0);
+        reduce.setConf(job);
+        reduce.run(job, this);
+        this.mapoutputFile.removeAll(reduceId);
+        
+        this.status.runState = JobStatus.SUCCEEDED;
+
+      } catch (Throwable t) {
+        this.status.runState = JobStatus.FAILED;
+        t.printStackTrace();
+      }
+    }
+
+    private String newId() {
+      return Integer.toString(Math.abs(new Random().nextInt()),36);
+    }
+
+    // TaskUmbilicalProtocol methods
+
+    public Task getTask(String taskid) { return null; }
+
+    public void progress(String taskId, float progress, String state) {
+      LOG.info(state);
+      float taskIndex = mapIds.indexOf(taskId);
+      if (taskIndex >= 0) {                       // mapping
+        float numTasks = mapIds.size();
+        status.mapProgress = (taskIndex/numTasks)+(progress/numTasks);
+      } else {
+        status.reduceProgress = progress;
+      }
+    }
+
+    public void reportDiagnosticInfo(String taskid, String trace) {
+      // Ignore for now
+    }
+
+    public void ping(String taskid) throws IOException {}
+
+    public void done(String taskId) throws IOException {
+      int taskIndex = mapIds.indexOf(taskId);
+      if (taskIndex >= 0) {                       // mapping
+        status.mapProgress = 1.0f;
+      } else {
+        status.reduceProgress = 1.0f;
+      }
+    }
+
+    public synchronized void fsError(String message) throws IOException {
+      LOG.severe("FSError: "+ message);
+    }
+
+  }
+
+  public LocalJobRunner(Configuration conf) throws IOException {
+    this.fs = NutchFileSystem.get(conf);
+    this.conf = conf;
+  }
+
+  // JobSubmissionProtocol methods
+
+  public JobStatus submitJob(String jobFile) throws IOException {
+    return new Job(jobFile, this.conf).status;
+  }
+
+  public void killJob(String id) {
+    ((Thread)jobs.get(id)).stop();
+  }
+
+  public JobProfile getJobProfile(String id) {
+    Job job = (Job)jobs.get(id);
+    return new JobProfile(id, job.file, "http://localhost:8080/");
+  }
+
+  public Vector[] getMapTaskReport(String id) {
+    return new Vector[0];
+  }
+  public Vector[] getReduceTaskReport(String id) {
+    return new Vector[0];
+  }
+
+  public JobStatus getJobStatus(String id) {
+    Job job = (Job)jobs.get(id);
+    return job.status;
+  }
+
+  public String getFilesystemName() throws IOException {
+    return fs.getName();
+  }
+}

+ 35 - 0
src/java/org/apache/hadoop/mapred/MRConstants.java

@@ -0,0 +1,35 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.
+
+/*******************************
+ * Some handy constants
+ * 
+ * @author Mike Cafarella
+ *******************************/
+public interface MRConstants {
+    //
+    // Timeouts, constants
+    //
+    public static final long HEARTBEAT_INTERVAL = 3 * 1000;
+    public static final long TASKTRACKER_EXPIRY_INTERVAL = 60 * 1000;
+
+    //
+    // Result codes
+    //
+    public static int SUCCESS = 0;
+    public static int FILE_NOT_FOUND = -1;
+}

+ 80 - 0
src/java/org/apache/hadoop/mapred/MapFileOutputFormat.java

@@ -0,0 +1,80 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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.io.MapFile;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.conf.Configuration;
+
+public class MapFileOutputFormat implements OutputFormat {
+
+  public RecordWriter getRecordWriter(NutchFileSystem fs, JobConf job,
+                                      String name) throws IOException {
+
+    File file = new File(job.getOutputDir(), name);
+
+    final MapFile.Writer out =
+      new MapFile.Writer(fs, file.toString(),
+                         job.getOutputKeyClass(),
+                         job.getOutputValueClass(),
+                         job.getBoolean("mapred.output.compress", false));
+
+    return new RecordWriter() {
+
+        public void write(WritableComparable key, Writable value)
+          throws IOException {
+
+          out.append(key, value);
+        }
+
+        public void close(Reporter reporter) throws IOException { out.close();}
+      };
+  }
+
+  /** Open the output generated by this format. */
+  public static MapFile.Reader[] getReaders(NutchFileSystem fs, File dir, Configuration conf)
+    throws IOException {
+    File[] names = fs.listFiles(dir);
+    
+    // sort names, so that hash partitioning works
+    Arrays.sort(names);
+    
+    MapFile.Reader[] parts = new MapFile.Reader[names.length];
+    for (int i = 0; i < names.length; i++) {
+      parts[i] = new MapFile.Reader(fs, names[i].toString(), conf);
+    }
+    return parts;
+  }
+    
+  /** Get an entry from output generated by this class. */
+  public static Writable getEntry(MapFile.Reader[] readers,
+                                  Partitioner partitioner,
+                                  WritableComparable key,
+                                  Writable value) throws IOException {
+    int part = partitioner.getPartition(key, value, readers.length);
+    return readers[part].get(key, value);
+  }
+
+}
+

+ 146 - 0
src/java/org/apache/hadoop/mapred/MapOutputFile.java

@@ -0,0 +1,146 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+import java.io.IOException;
+
+import java.io.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.*;
+
+/** A local file to be transferred via the {@link MapOutputProtocol}. */ 
+public class MapOutputFile implements Writable, Configurable {
+  private String mapTaskId;
+  private String reduceTaskId;
+  private int partition;
+  
+  /** Permits reporting of file copy progress. */
+  public interface ProgressReporter {
+    void progress(float progress) throws IOException;
+  }
+
+  private ThreadLocal REPORTERS = new ThreadLocal();
+  private JobConf jobConf;
+  
+  public void setProgressReporter(ProgressReporter reporter) {
+    REPORTERS.set(reporter);
+  }
+
+  /** Create a local map output file name.
+   * @param mapTaskId a map task id
+   * @param partition a reduce partition
+   */
+  public File getOutputFile(String mapTaskId, int partition)
+    throws IOException {
+    return this.jobConf.getLocalFile(mapTaskId, "part-"+partition+".out");
+  }
+
+  /** Create a local reduce input file name.
+   * @param mapTaskId a map task id
+   * @param reduceTaskId a reduce task id
+   */
+  public File getInputFile(String mapTaskId, String reduceTaskId)
+    throws IOException {
+    return this.jobConf.getLocalFile(reduceTaskId, mapTaskId+".out");
+  }
+  public File getInputFile(String mapTaskId[], String reduceTaskId)
+    throws IOException {
+    return this.jobConf.getLocalFile(reduceTaskId, mapTaskId, ".out");
+  }
+
+  /** Removes all of the files related to a task. */
+  public void removeAll(String taskId) throws IOException {
+    this.jobConf.deleteLocalFiles(taskId);
+  }
+
+  /** 
+   * Removes all contents of temporary storage.  Called upon 
+   * startup, to remove any leftovers from previous run.
+   */
+  public void cleanupStorage() throws IOException {
+    this.jobConf.deleteLocalFiles();
+  }
+
+  /** Construct a file for transfer. */
+  public MapOutputFile() { 
+  }
+  
+  public MapOutputFile(String mapTaskId, String reduceTaskId, int partition) {
+    this.mapTaskId = mapTaskId;
+    this.reduceTaskId = reduceTaskId;
+    this.partition = partition;
+  }
+
+  public void write(DataOutput out) throws IOException {
+    UTF8.writeString(out, mapTaskId);
+    UTF8.writeString(out, reduceTaskId);
+    out.writeInt(partition);
+    
+    // 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);
+    try {
+      byte[] buffer = new byte[8192];
+      int l;
+      while ((l = in.read(buffer)) != -1) {
+        out.write(buffer, 0, l);
+      }
+    } finally {
+      in.close();
+    }
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    this.mapTaskId = UTF8.readString(in);
+    this.reduceTaskId = UTF8.readString(in);
+    this.partition = in.readInt();
+
+    ProgressReporter reporter = (ProgressReporter)REPORTERS.get();
+
+    // read the length-prefixed file content into a local file
+    File file = getInputFile(mapTaskId, reduceTaskId);
+    long length = in.readLong();
+    float progPerByte = 1.0f / length;
+    long unread = length;
+    NFSDataOutputStream out = NutchFileSystem.getNamed("local", this.jobConf).create(file);
+    try {
+      byte[] buffer = new byte[8192];
+      while (unread > 0) {
+          int bytesToRead = (int)Math.min(unread, buffer.length);
+          in.readFully(buffer, 0, bytesToRead);
+          out.write(buffer, 0, bytesToRead);
+          unread -= bytesToRead;
+          if (reporter != null) {
+            reporter.progress((length-unread)*progPerByte);
+          }
+      }
+    } finally {
+      out.close();
+    }
+  }
+
+  public void setConf(Configuration conf) {
+    this.jobConf = new JobConf(conf);
+  }
+
+  public Configuration getConf() {
+    return this.jobConf;
+  }
+
+}

+ 68 - 0
src/java/org/apache/hadoop/mapred/MapOutputLocation.java

@@ -0,0 +1,68 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred.
+
+import java.io.IOException;
+
+import java.io.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.conf.*;
+
+/** The location of a map output file, as passed to a reduce task via the
+ * {@link InterTrackerProtocol}. */ 
+public class MapOutputLocation implements Writable {
+  private String mapTaskId;
+  private String host;
+  private int port;
+
+  /** RPC constructor **/
+  public MapOutputLocation() {
+  }
+
+  /** Construct a location. */
+  public MapOutputLocation(String mapTaskId, String host, int port) {
+    this.mapTaskId = mapTaskId;
+    this.host = host;
+    this.port = port;
+  }
+
+  /** The map task id. */
+  public String getMapTaskId() { return mapTaskId; }
+
+  /** The host the task completed on. */
+  public String getHost() { return host; }
+
+  /** The port listening for {@link MapOutputProtocol} connections. */
+  public int getPort() { return port; }
+
+  public void write(DataOutput out) throws IOException {
+    UTF8.writeString(out, mapTaskId);
+    UTF8.writeString(out, host);
+    out.writeInt(port);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    this.mapTaskId = UTF8.readString(in);
+    this.host = UTF8.readString(in);
+    this.port = in.readInt();
+  }
+
+  public String toString() {
+    return mapTaskId+"@"+host+":"+port;
+  }
+
+}

Some files were not shown because too many files changed in this diff