ソースを参照

HADOOP-2951. Add a contrib module that provides a utility to build or update Lucene indexes using MapReduce. Contributed by Ning Li.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@639138 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 17 年 前
コミット
b94166a8bb
40 ファイル変更4622 行追加0 行削除
  1. 3 0
      CHANGES.txt
  2. 2 0
      build.xml
  3. 43 0
      src/contrib/index/README
  4. 80 0
      src/contrib/index/build.xml
  5. 48 0
      src/contrib/index/conf/index-config.xml.template
  6. BIN
      src/contrib/index/lib/lucene-core-2.3.1.jar
  7. 10 0
      src/contrib/index/sample/data.txt
  8. 10 0
      src/contrib/index/sample/data2.txt
  9. 56 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/HashingDistributionPolicy.java
  10. 57 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/IdentityLocalAnalysis.java
  11. 46 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/LineDocInputFormat.java
  12. 80 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/LineDocLocalAnalysis.java
  13. 231 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/LineDocRecordReader.java
  14. 92 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/LineDocTextAndOp.java
  15. 58 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/RoundRobinDistributionPolicy.java
  16. 349 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/FileSystemDirectory.java
  17. 55 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/LuceneIndexFileNameFilter.java
  18. 112 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/LuceneUtil.java
  19. 49 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/MixedDeletionPolicy.java
  20. 185 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/MixedDirectory.java
  21. 119 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/RAMDirectoryUtil.java
  22. 233 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/ShardWriter.java
  23. 273 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/main/UpdateIndex.java
  24. 208 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/DocumentAndOp.java
  25. 89 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/DocumentID.java
  26. 50 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IDistributionPolicy.java
  27. 46 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IIndexUpdater.java
  28. 32 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/ILocalAnalysis.java
  29. 77 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateCombiner.java
  30. 238 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateConfiguration.java
  31. 199 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateMapper.java
  32. 65 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateOutputFormat.java
  33. 60 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdatePartitioner.java
  34. 143 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateReducer.java
  35. 151 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdater.java
  36. 239 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IntermediateForm.java
  37. 240 0
      src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/Shard.java
  38. 105 0
      src/contrib/index/src/test/org/apache/hadoop/contrib/index/lucene/TestMixedDirectory.java
  39. 234 0
      src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestDistributionPolicy.java
  40. 255 0
      src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestIndexUpdater.java

+ 3 - 0
CHANGES.txt

@@ -56,6 +56,9 @@ Trunk (unreleased changes)
     HADOOP-2346. Utilities to support timeout while writing to sockets.
     DFSClient and DataNode sockets have 10min write timeout. (rangadi)
     
+    HADOOP-2951.  Add a contrib module that provides a utility to
+    build or update Lucene indexes using Map/Reduce.  (Ning Li via cutting)
+
   IMPROVEMENTS
 
     HADOOP-2655. Copy on write for data and metadata files in the 

+ 2 - 0
build.xml

@@ -698,6 +698,7 @@
 
     	<packageset dir="src/contrib/streaming/src/java"/>
     	<packageset dir="src/contrib/data_join/src/java"/>
+    	<packageset dir="src/contrib/index/src/java"/>
 
         <link href="${javadoc.link.java}"/>
 
@@ -714,6 +715,7 @@
 
        <group title="contrib: Streaming" packages="org.apache.hadoop.streaming*"/>
        <group title="contrib: DataJoin" packages="org.apache.hadoop.contrib.utils.join*"/>
+       <group title="contrib: Index" packages="org.apache.hadoop.contrib.index*"/>
 
     </javadoc>
   </target>	

+ 43 - 0
src/contrib/index/README

@@ -0,0 +1,43 @@
+This contrib package provides a utility to build or update an index
+using Map/Reduce.
+
+A distributed "index" is partitioned into "shards". Each shard corresponds
+to a Lucene instance. org.apache.hadoop.contrib.index.main.UpdateIndex
+contains the main() method which uses a Map/Reduce job to analyze documents
+and update Lucene instances in parallel.
+
+The Map phase of the Map/Reduce job formats, analyzes and parses the input
+(in parallel), while the Reduce phase collects and applies the updates to
+each Lucene instance (again in parallel). The updates are applied using the
+local file system where a Reduce task runs and then copied back to HDFS.
+For example, if the updates caused a new Lucene segment to be created, the
+new segment would be created on the local file system first, and then
+copied back to HDFS.
+
+When the Map/Reduce job completes, a "new version" of the index is ready
+to be queried. It is important to note that the new version of the index
+is not derived from scratch. By leveraging Lucene's update algorithm, the
+new version of each Lucene instance will share as many files as possible
+as the previous version.
+
+The main() method in UpdateIndex requires the following information for
+updating the shards:
+  - Input formatter. This specifies how to format the input documents.
+  - Analysis. This defines the analyzer to use on the input. The analyzer
+    determines whether a document is being inserted, updated, or deleted.
+    For inserts or updates, the analyzer also converts each input document
+    into a Lucene document.
+  - Input paths. This provides the location(s) of updated documents,
+    e.g., HDFS files or directories, or HBase tables.
+  - Shard paths, or index path with the number of shards. Either specify
+    the path for each shard, or specify an index path and the shards are
+    the sub-directories of the index directory.
+  - Output path. When the update to a shard is done, a message is put here.
+  - Number of map tasks.
+
+All of the information can be specified in a configuration file. All but
+the first two can also be specified as command line options. Check out
+conf/index-config.xml.template for other configurable parameters.
+
+Note: Because of the parallel nature of Map/Reduce, the behaviour of
+multiple inserts, deletes or updates to the same document is undefined.

+ 80 - 0
src/contrib/index/build.xml

@@ -0,0 +1,80 @@
+<?xml version="1.0"?>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You 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.
+-->
+
+<project name="index" default="jar">
+
+  <import file="../build-contrib.xml"/>
+
+  <!-- Override jar target to specify main class -->
+  <target name="jar" depends="compile" unless="skip.contrib">
+    <echo message="contrib: ${name}"/>
+    <jar
+      jarfile="${build.dir}/hadoop-${version}-${name}.jar"
+      basedir="${build.classes}"
+    >
+      <manifest>
+        <attribute name="Main-Class" value="org.apache.hadoop.contrib.index.main.UpdateIndex"/>
+      </manifest>
+    </jar>
+  </target>
+
+  <!-- Override test target to copy sample data -->
+  <target name="test" depends="compile-test, compile, compile-examples" if="test.available">
+    <echo message="contrib: ${name}"/>
+    <delete dir="${hadoop.log.dir}"/>
+    <mkdir dir="${hadoop.log.dir}"/>
+    <delete dir="${build.test}/sample"/>
+    <mkdir dir="${build.test}/sample"/>
+    <copy todir="${build.test}/sample">
+      <fileset dir="${root}/sample"/>
+    </copy>
+    <junit
+      printsummary="yes" showoutput="${test.output}" 
+      haltonfailure="no" fork="yes" maxmemory="256m"
+      errorProperty="tests.failed" failureProperty="tests.failed"
+      timeout="${test.timeout}">
+      
+      <sysproperty key="test.build.data" value="${build.test}/data"/>
+      <sysproperty key="build.test" value="${build.test}"/>
+      <sysproperty key="contrib.name" value="${name}"/>
+      
+      <!-- requires fork=yes for: 
+        relative File paths to use the specified user.dir 
+        classpath to use build/contrib/*.jar
+      -->
+      <sysproperty key="user.dir" value="${build.test}/data"/>
+      
+      <sysproperty key="fs.default.name" value="${fs.default.name}"/>
+      <sysproperty key="hadoop.test.localoutputfile" value="${hadoop.test.localoutputfile}"/>
+      <sysproperty key="hadoop.log.dir" value="${hadoop.log.dir}"/>
+      <classpath refid="test.classpath"/>
+      <formatter type="${test.junit.output.format}" />
+      <batchtest todir="${build.test}" unless="testcase">
+        <fileset dir="${src.test}"
+                 includes="**/Test*.java" excludes="**/${test.exclude}.java" />
+      </batchtest>
+      <batchtest todir="${build.test}" if="testcase">
+        <fileset dir="${src.test}" includes="**/${testcase}.java"/>
+      </batchtest>
+    </junit>
+    <fail if="tests.failed">Tests failed!</fail>
+
+  </target>
+
+</project>

+ 48 - 0
src/contrib/index/conf/index-config.xml.template

@@ -0,0 +1,48 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+<property>
+  <name>sea.distribution.policy</name>
+  <value>org.apache.hadoop.contrib.index.example.HashingDistributionPolicy</value>
+</property>
+
+<property>
+  <name>sea.document.analyzer</name>
+  <value>org.apache.lucene.analysis.standard.StandardAnalyzer</value>
+</property>
+
+<property>
+  <name>sea.input.format</name>
+  <value>org.apache.hadoop.contrib.index.example.LineDocInputFormat</value>
+</property>
+
+<property>
+  <name>sea.index.updater</name>
+  <value>org.apache.hadoop.contrib.index.mapred.IndexUpdater</value>
+</property>
+
+<property>
+  <name>sea.local.analysis</name>
+  <value>org.apache.hadoop.contrib.index.example.LineDocLocalAnalysis</value>
+</property>
+
+<property>
+  <name>sea.max.field.length</name>
+  <value>2000000</value>
+</property>
+
+<property>
+  <name>sea.max.num.segments</name>
+  <value>10</value>
+</property>
+
+<property>
+  <name>sea.use.compound.file</name>
+  <value>true</value>
+</property>
+
+</configuration>

BIN
src/contrib/index/lib/lucene-core-2.3.1.jar


+ 10 - 0
src/contrib/index/sample/data.txt

@@ -0,0 +1,10 @@
+0 ins apache dot org
+1 ins apache
+2 ins apache
+3 ins apache
+4 ins apache
+5 ins apache
+6 ins apache
+7 ins apache
+8 ins apache
+9 ins apache

+ 10 - 0
src/contrib/index/sample/data2.txt

@@ -0,0 +1,10 @@
+0 del
+1 upd hadoop
+2 del
+3 upd hadoop
+4 del
+5 upd hadoop
+6 del
+7 upd hadoop
+8 del
+9 upd hadoop

+ 56 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/HashingDistributionPolicy.java

@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.example;
+
+import org.apache.hadoop.contrib.index.mapred.DocumentID;
+import org.apache.hadoop.contrib.index.mapred.IDistributionPolicy;
+import org.apache.hadoop.contrib.index.mapred.Shard;
+
+/**
+ * Choose a shard for each insert or delete based on document id hashing. Do
+ * NOT use this distribution policy when the number of shards changes.
+ */
+public class HashingDistributionPolicy implements IDistributionPolicy {
+
+  private int numShards;
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.contrib.index.mapred.IDistributionPolicy#init(org.apache.hadoop.contrib.index.mapred.Shard[])
+   */
+  public void init(Shard[] shards) {
+    numShards = shards.length;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.contrib.index.mapred.IDistributionPolicy#chooseShardForInsert(org.apache.hadoop.contrib.index.mapred.DocumentID)
+   */
+  public int chooseShardForInsert(DocumentID key) {
+    int hashCode = key.hashCode();
+    return hashCode >= 0 ? hashCode % numShards : (-hashCode) % numShards;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.contrib.index.mapred.IDistributionPolicy#chooseShardForDelete(org.apache.hadoop.contrib.index.mapred.DocumentID)
+   */
+  public int chooseShardForDelete(DocumentID key) {
+    int hashCode = key.hashCode();
+    return hashCode >= 0 ? hashCode % numShards : (-hashCode) % numShards;
+  }
+
+}

+ 57 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/IdentityLocalAnalysis.java

@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.example;
+
+import java.io.IOException;
+
+import org.apache.hadoop.contrib.index.mapred.DocumentAndOp;
+import org.apache.hadoop.contrib.index.mapred.DocumentID;
+import org.apache.hadoop.contrib.index.mapred.ILocalAnalysis;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * Identity local analysis maps inputs directly into outputs.
+ */
+public class IdentityLocalAnalysis implements
+    ILocalAnalysis<DocumentID, DocumentAndOp> {
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.Mapper#map(java.lang.Object, java.lang.Object, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)
+   */
+  public void map(DocumentID key, DocumentAndOp value,
+      OutputCollector<DocumentID, DocumentAndOp> output, Reporter reporter)
+      throws IOException {
+    output.collect(key, value);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.JobConfigurable#configure(org.apache.hadoop.mapred.JobConf)
+   */
+  public void configure(JobConf job) {
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Closeable#close()
+   */
+  public void close() throws IOException {
+  }
+
+}

+ 46 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/LineDocInputFormat.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.example;
+
+import java.io.IOException;
+
+import org.apache.hadoop.contrib.index.mapred.DocumentID;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * An InputFormat for LineDoc for plain text files where each line is a doc.
+ */
+public class LineDocInputFormat extends
+    FileInputFormat<DocumentID, LineDocTextAndOp> {
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.FileInputFormat#getRecordReader(org.apache.hadoop.mapred.InputSplit, org.apache.hadoop.mapred.JobConf, org.apache.hadoop.mapred.Reporter)
+   */
+  public RecordReader<DocumentID, LineDocTextAndOp> getRecordReader(
+      InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    reporter.setStatus(split.toString());
+    return new LineDocRecordReader(job, (FileSplit) split);
+  }
+
+}

+ 80 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/LineDocLocalAnalysis.java

@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.example;
+
+import java.io.IOException;
+
+import org.apache.hadoop.contrib.index.mapred.DocumentAndOp;
+import org.apache.hadoop.contrib.index.mapred.DocumentID;
+import org.apache.hadoop.contrib.index.mapred.ILocalAnalysis;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.Term;
+
+/**
+ * Convert LineDocTextAndOp to DocumentAndOp as required by ILocalAnalysis.
+ */
+public class LineDocLocalAnalysis implements
+    ILocalAnalysis<DocumentID, LineDocTextAndOp> {
+
+  private static String docidFieldName = "id";
+  private static String contentFieldName = "content";
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.Mapper#map(java.lang.Object, java.lang.Object, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)
+   */
+  public void map(DocumentID key, LineDocTextAndOp value,
+      OutputCollector<DocumentID, DocumentAndOp> output, Reporter reporter)
+      throws IOException {
+
+    DocumentAndOp.Op op = value.getOp();
+    Document doc = null;
+    Term term = null;
+
+    if (op == DocumentAndOp.Op.INSERT || op == DocumentAndOp.Op.UPDATE) {
+      doc = new Document();
+      doc.add(new Field(docidFieldName, key.getText().toString(),
+          Field.Store.YES, Field.Index.UN_TOKENIZED));
+      doc.add(new Field(contentFieldName, value.getText().toString(),
+          Field.Store.NO, Field.Index.TOKENIZED));
+    }
+
+    if (op == DocumentAndOp.Op.DELETE || op == DocumentAndOp.Op.UPDATE) {
+      term = new Term(docidFieldName, key.getText().toString());
+    }
+
+    output.collect(key, new DocumentAndOp(op, doc, term));
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.JobConfigurable#configure(org.apache.hadoop.mapred.JobConf)
+   */
+  public void configure(JobConf job) {
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Closeable#close()
+   */
+  public void close() throws IOException {
+  }
+
+}

+ 231 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/LineDocRecordReader.java

@@ -0,0 +1,231 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.example;
+
+import java.io.BufferedInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.contrib.index.mapred.DocumentAndOp;
+import org.apache.hadoop.contrib.index.mapred.DocumentID;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.RecordReader;
+
+/**
+ * A simple RecordReader for LineDoc for plain text files where each line is a
+ * doc. Each line is as follows: documentID<SPACE>op<SPACE>content<EOF>,
+ * where op can be "i", "ins" or "insert" for insert, "d", "del" or "delete"
+ * for delete, or "u", "upd" or "update" for update.
+ */
+public class LineDocRecordReader implements
+    RecordReader<DocumentID, LineDocTextAndOp> {
+  private static final char SPACE = ' ';
+  private static final char EOL = '\n';
+
+  private long start;
+  private long pos;
+  private long end;
+  private BufferedInputStream in;
+  private ByteArrayOutputStream buffer = new ByteArrayOutputStream(256);
+
+  /**
+   * Provide a bridge to get the bytes from the ByteArrayOutputStream without
+   * creating a new byte array.
+   */
+  private static class TextStuffer extends OutputStream {
+    public Text target;
+
+    public void write(int b) {
+      throw new UnsupportedOperationException("write(byte) not supported");
+    }
+
+    public void write(byte[] data, int offset, int len) throws IOException {
+      target.set(data, offset, len);
+    }
+  }
+
+  private TextStuffer bridge = new TextStuffer();
+
+  /**
+   * Constructor
+   * @param job
+   * @param split  
+   * @throws IOException
+   */
+  public LineDocRecordReader(Configuration job, FileSplit split)
+      throws IOException {
+    long start = split.getStart();
+    long end = start + split.getLength();
+    final Path file = split.getPath();
+
+    // open the file and seek to the start of the split
+    FileSystem fs = file.getFileSystem(job);
+    FSDataInputStream fileIn = fs.open(split.getPath());
+    InputStream in = fileIn;
+    boolean skipFirstLine = false;
+    if (start != 0) {
+      skipFirstLine = true; // wait till BufferedInputStream to skip
+      --start;
+      fileIn.seek(start);
+    }
+
+    this.in = new BufferedInputStream(in);
+    if (skipFirstLine) { // skip first line and re-establish "start".
+      start += LineDocRecordReader.readData(this.in, null, EOL);
+    }
+    this.start = start;
+    this.pos = start;
+    this.end = end;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.RecordReader#close()
+   */
+  public void close() throws IOException {
+    in.close();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.RecordReader#createKey()
+   */
+  public DocumentID createKey() {
+    return new DocumentID();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.RecordReader#createValue()
+   */
+  public LineDocTextAndOp createValue() {
+    return new LineDocTextAndOp();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.RecordReader#getPos()
+   */
+  public long getPos() throws IOException {
+    return pos;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.RecordReader#getProgress()
+   */
+  public float getProgress() throws IOException {
+    if (start == end) {
+      return 0.0f;
+    } else {
+      return Math.min(1.0f, (pos - start) / (float) (end - start));
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.RecordReader#next(java.lang.Object, java.lang.Object)
+   */
+  public synchronized boolean next(DocumentID key, LineDocTextAndOp value)
+      throws IOException {
+    if (pos >= end) {
+      return false;
+    }
+
+    // key is document id, which are bytes until first space
+    if (!readInto(key.getText(), SPACE)) {
+      return false;
+    }
+
+    // read operation: i/d/u, or ins/del/upd, or insert/delete/update
+    Text opText = new Text();
+    if (!readInto(opText, SPACE)) {
+      return false;
+    }
+    String opStr = opText.toString();
+    DocumentAndOp.Op op;
+    if (opStr.equals("i") || opStr.equals("ins") || opStr.equals("insert")) {
+      op = DocumentAndOp.Op.INSERT;
+    } else if (opStr.equals("d") || opStr.equals("del")
+        || opStr.equals("delete")) {
+      op = DocumentAndOp.Op.DELETE;
+    } else if (opStr.equals("u") || opStr.equals("upd")
+        || opStr.equals("update")) {
+      op = DocumentAndOp.Op.UPDATE;
+    } else {
+      // default is insert
+      op = DocumentAndOp.Op.INSERT;
+    }
+    value.setOp(op);
+
+    if (op == DocumentAndOp.Op.DELETE) {
+      return true;
+    } else {
+      // read rest of the line
+      return readInto(value.getText(), EOL);
+    }
+  }
+
+  private boolean readInto(Text text, char delimiter) throws IOException {
+    buffer.reset();
+    long bytesRead = readData(in, buffer, delimiter);
+    if (bytesRead == 0) {
+      return false;
+    }
+    pos += bytesRead;
+    bridge.target = text;
+    buffer.writeTo(bridge);
+    return true;
+  }
+
+  private static long readData(InputStream in, OutputStream out, char delimiter)
+      throws IOException {
+    long bytes = 0;
+    while (true) {
+
+      int b = in.read();
+      if (b == -1) {
+        break;
+      }
+      bytes += 1;
+
+      byte c = (byte) b;
+      if (c == EOL || c == delimiter) {
+        break;
+      }
+
+      if (c == '\r') {
+        in.mark(1);
+        byte nextC = (byte) in.read();
+        if (nextC != EOL || c == delimiter) {
+          in.reset();
+        } else {
+          bytes += 1;
+        }
+        break;
+      }
+
+      if (out != null) {
+        out.write(c);
+      }
+    }
+    return bytes;
+  }
+}

+ 92 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/LineDocTextAndOp.java

@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.example;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.contrib.index.mapred.DocumentAndOp;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * This class represents an operation. The operation can be an insert, a delete
+ * or an update. If the operation is an insert or an update, a (new) document,
+ * which is in the form of text, is specified.
+ */
+public class LineDocTextAndOp implements Writable {
+  private DocumentAndOp.Op op;
+  private Text doc;
+
+  /**
+   * Constructor
+   */
+  public LineDocTextAndOp() {
+    doc = new Text();
+  }
+
+  /**
+   * Set the type of the operation.
+   * @param op  the type of the operation
+   */
+  public void setOp(DocumentAndOp.Op op) {
+    this.op = op;
+  }
+
+  /**
+   * Get the type of the operation.
+   * @return the type of the operation
+   */
+  public DocumentAndOp.Op getOp() {
+    return op;
+  }
+
+  /**
+   * Get the text that represents a document.
+   * @return the text that represents a document
+   */
+  public Text getText() {
+    return doc;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  public String toString() {
+    return this.getClass().getName() + "[op=" + op + ", text=" + doc + "]";
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
+   */
+  public void write(DataOutput out) throws IOException {
+    throw new IOException(this.getClass().getName()
+        + ".write should never be called");
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
+   */
+  public void readFields(DataInput in) throws IOException {
+    throw new IOException(this.getClass().getName()
+        + ".readFields should never be called");
+  }
+
+}

+ 58 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/example/RoundRobinDistributionPolicy.java

@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.example;
+
+import org.apache.hadoop.contrib.index.mapred.DocumentID;
+import org.apache.hadoop.contrib.index.mapred.IDistributionPolicy;
+import org.apache.hadoop.contrib.index.mapred.Shard;
+
+/**
+ * Choose a shard for each insert in a round-robin fashion. Choose all the
+ * shards for each delete because we don't know where it is stored.
+ */
+public class RoundRobinDistributionPolicy implements IDistributionPolicy {
+
+  private int numShards;
+  private int rr; // round-robin implementation
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.contrib.index.mapred.IDistributionPolicy#init(org.apache.hadoop.contrib.index.mapred.Shard[])
+   */
+  public void init(Shard[] shards) {
+    numShards = shards.length;
+    rr = 0;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.contrib.index.mapred.IDistributionPolicy#chooseShardForInsert(org.apache.hadoop.contrib.index.mapred.DocumentID)
+   */
+  public int chooseShardForInsert(DocumentID key) {
+    int chosen = rr;
+    rr = (rr + 1) % numShards;
+    return chosen;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.contrib.index.mapred.IDistributionPolicy#chooseShardForDelete(org.apache.hadoop.contrib.index.mapred.DocumentID)
+   */
+  public int chooseShardForDelete(DocumentID key) {
+    // -1 represents all the shards
+    return -1;
+  }
+}

+ 349 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/FileSystemDirectory.java

@@ -0,0 +1,349 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.lucene;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.lucene.store.BufferedIndexInput;
+import org.apache.lucene.store.BufferedIndexOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.Lock;
+
+/**
+ * This class implements a Lucene Directory on top of a general FileSystem.
+ * Currently it does not support locking.
+ */
+public class FileSystemDirectory extends Directory {
+
+  private final FileSystem fs;
+  private final Path directory;
+  private final int ioFileBufferSize;
+
+  /**
+   * Constructor
+   * @param fs
+   * @param directory
+   * @param create
+   * @param conf
+   * @throws IOException
+   */
+  public FileSystemDirectory(FileSystem fs, Path directory, boolean create,
+      Configuration conf) throws IOException {
+
+    this.fs = fs;
+    this.directory = directory;
+    this.ioFileBufferSize = conf.getInt("io.file.buffer.size", 4096);
+
+    if (create) {
+      create();
+    }
+
+    boolean isDir = false;
+    try {
+      FileStatus status = fs.getFileStatus(directory);
+      if (status != null) {
+        isDir = status.isDir();
+      }
+    } catch (IOException e) {
+      // file does not exist, isDir already set to false
+    }
+    if (!isDir) {
+      throw new IOException(directory + " is not a directory");
+    }
+  }
+
+  private void create() throws IOException {
+    if (!fs.exists(directory)) {
+      fs.mkdirs(directory);
+    }
+
+    boolean isDir = false;
+    try {
+      FileStatus status = fs.getFileStatus(directory);
+      if (status != null) {
+        isDir = status.isDir();
+      }
+    } catch (IOException e) {
+      // file does not exist, isDir already set to false
+    }
+    if (!isDir) {
+      throw new IOException(directory + " is not a directory");
+    }
+
+    // clear old index files
+    FileStatus[] fileStatus =
+        fs.listStatus(directory, LuceneIndexFileNameFilter.getFilter());
+    for (int i = 0; i < fileStatus.length; i++) {
+      if (!fs.delete(fileStatus[i].getPath())) {
+        throw new IOException("Cannot delete index file "
+            + fileStatus[i].getPath());
+      }
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#list()
+   */
+  public String[] list() throws IOException {
+    FileStatus[] fileStatus =
+        fs.listStatus(directory, LuceneIndexFileNameFilter.getFilter());
+    String[] result = new String[fileStatus.length];
+    for (int i = 0; i < fileStatus.length; i++) {
+      result[i] = fileStatus[i].getPath().getName();
+    }
+    return result;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#fileExists(java.lang.String)
+   */
+  public boolean fileExists(String name) throws IOException {
+    return fs.exists(new Path(directory, name));
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#fileModified(java.lang.String)
+   */
+  public long fileModified(String name) {
+    throw new UnsupportedOperationException();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#touchFile(java.lang.String)
+   */
+  public void touchFile(String name) {
+    throw new UnsupportedOperationException();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#fileLength(java.lang.String)
+   */
+  public long fileLength(String name) throws IOException {
+    return fs.getFileStatus(new Path(directory, name)).getLen();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#deleteFile(java.lang.String)
+   */
+  public void deleteFile(String name) throws IOException {
+    if (!fs.delete(new Path(directory, name))) {
+      throw new IOException("Cannot delete index file " + name);
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#renameFile(java.lang.String, java.lang.String)
+   */
+  public void renameFile(String from, String to) throws IOException {
+    fs.rename(new Path(directory, from), new Path(directory, to));
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#createOutput(java.lang.String)
+   */
+  public IndexOutput createOutput(String name) throws IOException {
+    Path file = new Path(directory, name);
+    if (fs.exists(file) && !fs.delete(file)) {
+      // delete the existing one if applicable
+      throw new IOException("Cannot overwrite index file " + file);
+    }
+
+    return new FileSystemIndexOutput(file, ioFileBufferSize);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#openInput(java.lang.String)
+   */
+  public IndexInput openInput(String name) throws IOException {
+    return openInput(name, ioFileBufferSize);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#openInput(java.lang.String, int)
+   */
+  public IndexInput openInput(String name, int bufferSize) throws IOException {
+    return new FileSystemIndexInput(new Path(directory, name), bufferSize);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#makeLock(java.lang.String)
+   */
+  public Lock makeLock(final String name) {
+    return new Lock() {
+      public boolean obtain() {
+        return true;
+      }
+
+      public void release() {
+      }
+
+      public boolean isLocked() {
+        throw new UnsupportedOperationException();
+      }
+
+      public String toString() {
+        return "Lock@" + new Path(directory, name);
+      }
+    };
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.lucene.store.Directory#close()
+   */
+  public void close() throws IOException {
+    // do not close the file system
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  public String toString() {
+    return this.getClass().getName() + "@" + directory;
+  }
+
+  private class FileSystemIndexInput extends BufferedIndexInput {
+
+    // shared by clones
+    private class Descriptor {
+      public final FSDataInputStream in;
+      public long position; // cache of in.getPos()
+
+      public Descriptor(Path file, int ioFileBufferSize) throws IOException {
+        this.in = fs.open(file, ioFileBufferSize);
+      }
+    }
+
+    private final Path filePath; // for debugging
+    private final Descriptor descriptor;
+    private final long length;
+    private boolean isOpen;
+    private boolean isClone;
+
+    public FileSystemIndexInput(Path path, int ioFileBufferSize)
+        throws IOException {
+      filePath = path;
+      descriptor = new Descriptor(path, ioFileBufferSize);
+      length = fs.getFileStatus(path).getLen();
+      isOpen = true;
+    }
+
+    protected void readInternal(byte[] b, int offset, int len)
+        throws IOException {
+      synchronized (descriptor) {
+        long position = getFilePointer();
+        if (position != descriptor.position) {
+          descriptor.in.seek(position);
+          descriptor.position = position;
+        }
+        int total = 0;
+        do {
+          int i = descriptor.in.read(b, offset + total, len - total);
+          if (i == -1) {
+            throw new IOException("Read past EOF");
+          }
+          descriptor.position += i;
+          total += i;
+        } while (total < len);
+      }
+    }
+
+    public void close() throws IOException {
+      if (!isClone) {
+        if (isOpen) {
+          descriptor.in.close();
+          isOpen = false;
+        } else {
+          throw new IOException("Index file " + filePath + " already closed");
+        }
+      }
+    }
+
+    protected void seekInternal(long position) {
+      // handled in readInternal()
+    }
+
+    public long length() {
+      return length;
+    }
+
+    protected void finalize() throws IOException {
+      if (!isClone && isOpen) {
+        close(); // close the file
+      }
+    }
+
+    public Object clone() {
+      FileSystemIndexInput clone = (FileSystemIndexInput) super.clone();
+      clone.isClone = true;
+      return clone;
+    }
+  }
+
+  private class FileSystemIndexOutput extends BufferedIndexOutput {
+
+    private final Path filePath; // for debugging
+    private final FSDataOutputStream out;
+    private boolean isOpen;
+
+    public FileSystemIndexOutput(Path path, int ioFileBufferSize)
+        throws IOException {
+      filePath = path;
+      // overwrite is true by default
+      out = fs.create(path, true, ioFileBufferSize);
+      isOpen = true;
+    }
+
+    public void flushBuffer(byte[] b, int offset, int size) throws IOException {
+      out.write(b, offset, size);
+    }
+
+    public void close() throws IOException {
+      if (isOpen) {
+        super.close();
+        out.close();
+        isOpen = false;
+      } else {
+        throw new IOException("Index file " + filePath + " already closed");
+      }
+    }
+
+    public void seek(long pos) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    public long length() throws IOException {
+      return out.getPos();
+    }
+
+    protected void finalize() throws IOException {
+      if (isOpen) {
+        close(); // close the file
+      }
+    }
+  }
+
+}

+ 55 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/LuceneIndexFileNameFilter.java

@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.lucene;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.lucene.index.IndexFileNameFilter;
+
+/**
+ * A wrapper class to convert an IndexFileNameFilter which implements
+ * java.io.FilenameFilter to an org.apache.hadoop.fs.PathFilter.
+ */
+class LuceneIndexFileNameFilter implements PathFilter {
+
+  private static final LuceneIndexFileNameFilter singleton =
+      new LuceneIndexFileNameFilter();
+
+  /**
+   * Get a static instance.
+   * @return the static instance
+   */
+  public static LuceneIndexFileNameFilter getFilter() {
+    return singleton;
+  }
+
+  private final IndexFileNameFilter luceneFilter;
+
+  private LuceneIndexFileNameFilter() {
+    luceneFilter = IndexFileNameFilter.getFilter();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.fs.PathFilter#accept(org.apache.hadoop.fs.Path)
+   */
+  public boolean accept(Path path) {
+    return luceneFilter.accept(null, path.getName());
+  }
+
+}

+ 112 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/LuceneUtil.java

@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.lucene;
+
+import java.io.IOException;
+
+import org.apache.lucene.store.Directory;
+
+/**
+ * This class copies some methods from Lucene's SegmentInfos since that class
+ * is not public.
+ */
+public final class LuceneUtil {
+
+  static final class IndexFileNames {
+    /** Name of the index segment file */
+    static final String SEGMENTS = "segments";
+
+    /** Name of the generation reference file name */
+    static final String SEGMENTS_GEN = "segments.gen";
+  }
+
+  /**
+   * Check if the file is a segments_N file
+   * @param name
+   * @return true if the file is a segments_N file
+   */
+  public static boolean isSegmentsFile(String name) {
+    return name.startsWith(IndexFileNames.SEGMENTS)
+        && !name.equals(IndexFileNames.SEGMENTS_GEN);
+  }
+
+  /**
+   * Check if the file is the segments.gen file
+   * @param name
+   * @return true if the file is the segments.gen file
+   */
+  public static boolean isSegmentsGenFile(String name) {
+    return name.equals(IndexFileNames.SEGMENTS_GEN);
+  }
+
+  /**
+   * Get the generation (N) of the current segments_N file in the directory.
+   * 
+   * @param directory -- directory to search for the latest segments_N file
+   */
+  public static long getCurrentSegmentGeneration(Directory directory)
+      throws IOException {
+    String[] files = directory.list();
+    if (files == null)
+      throw new IOException("cannot read directory " + directory
+          + ": list() returned null");
+    return getCurrentSegmentGeneration(files);
+  }
+
+  /**
+   * Get the generation (N) of the current segments_N file from a list of
+   * files.
+   * 
+   * @param files -- array of file names to check
+   */
+  public static long getCurrentSegmentGeneration(String[] files) {
+    if (files == null) {
+      return -1;
+    }
+    long max = -1;
+    for (int i = 0; i < files.length; i++) {
+      String file = files[i];
+      if (file.startsWith(IndexFileNames.SEGMENTS)
+          && !file.equals(IndexFileNames.SEGMENTS_GEN)) {
+        long gen = generationFromSegmentsFileName(file);
+        if (gen > max) {
+          max = gen;
+        }
+      }
+    }
+    return max;
+  }
+
+  /**
+   * Parse the generation off the segments file name and return it.
+   */
+  public static long generationFromSegmentsFileName(String fileName) {
+    if (fileName.equals(IndexFileNames.SEGMENTS)) {
+      return 0;
+    } else if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+      return Long.parseLong(
+          fileName.substring(1 + IndexFileNames.SEGMENTS.length()),
+          Character.MAX_RADIX);
+    } else {
+      throw new IllegalArgumentException("fileName \"" + fileName
+          + "\" is not a segments file");
+    }
+  }
+
+}

+ 49 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/MixedDeletionPolicy.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.lucene;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.lucene.index.IndexCommitPoint;
+import org.apache.lucene.index.IndexDeletionPolicy;
+
+/**
+ * For mixed directory. Use KeepAllDeletionPolicy for the read-only directory
+ * (keep all from init) and use KeepOnlyLastCommitDeletionPolicy for the
+ * writable directory (initially empty, keep latest after init).
+ */
+class MixedDeletionPolicy implements IndexDeletionPolicy {
+
+  private int keepAllFromInit = 0;
+
+  public void onInit(List commits) throws IOException {
+    keepAllFromInit = commits.size();
+  }
+
+  public void onCommit(List commits) throws IOException {
+    int size = commits.size();
+    assert (size > keepAllFromInit);
+    // keep all from init and the latest, delete the rest
+    for (int i = keepAllFromInit; i < size - 1; i++) {
+      ((IndexCommitPoint) commits.get(i)).delete();
+    }
+  }
+
+}

+ 185 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/MixedDirectory.java

@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.lucene;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.NoLockFactory;
+
+/**
+ * The initial version of an index is stored in a read-only FileSystem dir
+ * (FileSystemDirectory). Index files created by newer versions are written to
+ * a writable local FS dir (Lucene's FSDirectory). We should use the general
+ * FileSystemDirectory for the writable dir as well. But have to use Lucene's
+ * FSDirectory because currently Lucene does randome write and
+ * FileSystemDirectory only supports sequential write.
+ * 
+ * Note: We may delete files from the read-only FileSystem dir because there
+ * can be some segment files from an uncommitted checkpoint. For the same
+ * reason, we may create files in the writable dir which already exist in the
+ * read-only dir and logically they overwrite the ones in the read-only dir.
+ */
+class MixedDirectory extends Directory {
+
+  private final Directory readDir; // FileSystemDirectory
+  private final Directory writeDir; // Lucene's FSDirectory
+
+  // take advantage of the fact that Lucene's FSDirectory.fileExists is faster
+
+  public MixedDirectory(FileSystem readFs, Path readPath, FileSystem writeFs,
+      Path writePath, Configuration conf) throws IOException {
+
+    try {
+      readDir = new FileSystemDirectory(readFs, readPath, false, conf);
+      // check writeFS is a local FS?
+      writeDir = FSDirectory.getDirectory(writePath.toString());
+
+    } catch (IOException e) {
+      try {
+        close();
+      } catch (IOException e1) {
+        // ignore this one, throw the original one
+      }
+      throw e;
+    }
+
+    lockFactory = new NoLockFactory();
+  }
+
+  // for debugging
+  MixedDirectory(Directory readDir, Directory writeDir) throws IOException {
+    this.readDir = readDir;
+    this.writeDir = writeDir;
+
+    lockFactory = new NoLockFactory();
+  }
+
+  @Override
+  public String[] list() throws IOException {
+    String[] readFiles = readDir.list();
+    String[] writeFiles = writeDir.list();
+
+    if (readFiles == null || readFiles.length == 0) {
+      return writeFiles;
+    } else if (writeFiles == null || writeFiles.length == 0) {
+      return readFiles;
+    } else {
+      String[] result = new String[readFiles.length + writeFiles.length];
+      System.arraycopy(readFiles, 0, result, 0, readFiles.length);
+      System.arraycopy(writeFiles, 0, result, readFiles.length,
+          writeFiles.length);
+      return result;
+    }
+  }
+
+  @Override
+  public void deleteFile(String name) throws IOException {
+    if (writeDir.fileExists(name)) {
+      writeDir.deleteFile(name);
+    }
+    if (readDir.fileExists(name)) {
+      readDir.deleteFile(name);
+    }
+  }
+
+  @Override
+  public boolean fileExists(String name) throws IOException {
+    return writeDir.fileExists(name) || readDir.fileExists(name);
+  }
+
+  @Override
+  public long fileLength(String name) throws IOException {
+    if (writeDir.fileExists(name)) {
+      return writeDir.fileLength(name);
+    } else {
+      return readDir.fileLength(name);
+    }
+  }
+
+  @Override
+  public long fileModified(String name) throws IOException {
+    if (writeDir.fileExists(name)) {
+      return writeDir.fileModified(name);
+    } else {
+      return readDir.fileModified(name);
+    }
+  }
+
+  @Override
+  public void renameFile(String from, String to) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void touchFile(String name) throws IOException {
+    if (writeDir.fileExists(name)) {
+      writeDir.touchFile(name);
+    } else {
+      readDir.touchFile(name);
+    }
+  }
+
+  @Override
+  public IndexOutput createOutput(String name) throws IOException {
+    return writeDir.createOutput(name);
+  }
+
+  @Override
+  public IndexInput openInput(String name) throws IOException {
+    if (writeDir.fileExists(name)) {
+      return writeDir.openInput(name);
+    } else {
+      return readDir.openInput(name);
+    }
+  }
+
+  @Override
+  public IndexInput openInput(String name, int bufferSize) throws IOException {
+    if (writeDir.fileExists(name)) {
+      return writeDir.openInput(name, bufferSize);
+    } else {
+      return readDir.openInput(name, bufferSize);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      if (readDir != null) {
+        readDir.close();
+      }
+    } finally {
+      if (writeDir != null) {
+        writeDir.close();
+      }
+    }
+  }
+
+  public String toString() {
+    return this.getClass().getName() + "@" + readDir + "&" + writeDir;
+  }
+
+}

+ 119 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/RAMDirectoryUtil.java

@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.lucene;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Text;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMDirectory;
+
+/**
+ * A utility class which writes an index in a ram dir into a DataOutput and
+ * read from a DataInput an index into a ram dir.
+ */
+public class RAMDirectoryUtil {
+  private static final int BUFFER_SIZE = 1024; // RAMOutputStream.BUFFER_SIZE;
+
+  /**
+   * Write a number of files from a ram directory to a data output.
+   * @param out  the data output
+   * @param dir  the ram directory
+   * @param names  the names of the files to write
+   * @throws IOException
+   */
+  public static void writeRAMFiles(DataOutput out, RAMDirectory dir,
+      String[] names) throws IOException {
+    out.writeInt(names.length);
+
+    for (int i = 0; i < names.length; i++) {
+      Text.writeString(out, names[i]);
+      long length = dir.fileLength(names[i]);
+      out.writeLong(length);
+
+      if (length > 0) {
+        // can we avoid the extra copy?
+        IndexInput input = null;
+        try {
+          input = dir.openInput(names[i], BUFFER_SIZE);
+
+          int position = 0;
+          byte[] buffer = new byte[BUFFER_SIZE];
+
+          while (position < length) {
+            int len =
+                position + BUFFER_SIZE <= length ? BUFFER_SIZE
+                    : (int) (length - position);
+            input.readBytes(buffer, 0, len);
+            out.write(buffer, 0, len);
+            position += len;
+          }
+        } finally {
+          if (input != null) {
+            input.close();
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Read a number of files from a data input to a ram directory.
+   * @param in  the data input
+   * @param dir  the ram directory
+   * @throws IOException
+   */
+  public static void readRAMFiles(DataInput in, RAMDirectory dir)
+      throws IOException {
+    int numFiles = in.readInt();
+
+    for (int i = 0; i < numFiles; i++) {
+      String name = Text.readString(in);
+      long length = in.readLong();
+
+      if (length > 0) {
+        // can we avoid the extra copy?
+        IndexOutput output = null;
+        try {
+          output = dir.createOutput(name);
+
+          int position = 0;
+          byte[] buffer = new byte[BUFFER_SIZE];
+
+          while (position < length) {
+            int len =
+                position + BUFFER_SIZE <= length ? BUFFER_SIZE
+                    : (int) (length - position);
+            in.readFully(buffer, 0, len);
+            output.writeBytes(buffer, 0, len);
+            position += len;
+          }
+        } finally {
+          if (output != null) {
+            output.close();
+          }
+        }
+      }
+    }
+  }
+
+}

+ 233 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/lucene/ShardWriter.java

@@ -0,0 +1,233 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.lucene;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.contrib.index.mapred.IndexUpdateConfiguration;
+import org.apache.hadoop.contrib.index.mapred.IntermediateForm;
+import org.apache.hadoop.contrib.index.mapred.Shard;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+
+/**
+ * The initial version of an index is stored in the perm dir. Index files
+ * created by newer versions are written to a temp dir on the local FS. After
+ * successfully creating the new version in the temp dir, the shard writer
+ * moves the new files to the perm dir and deletes the temp dir in close().
+ */
+public class ShardWriter {
+  static final Log LOG = LogFactory.getLog(ShardWriter.class);
+
+  private final FileSystem fs;
+  private final FileSystem localFs;
+  private final Path perm;
+  private final Path temp;
+  private final Directory dir;
+  private final IndexWriter writer;
+  private int maxNumSegments;
+  private long numForms = 0;
+
+  /**
+   * Constructor
+   * @param fs
+   * @param shard
+   * @param tempDir
+   * @param iconf
+   * @throws IOException
+   */
+  public ShardWriter(FileSystem fs, Shard shard, String tempDir,
+      IndexUpdateConfiguration iconf) throws IOException {
+    LOG.info("Construct a shard writer");
+
+    this.fs = fs;
+    localFs = FileSystem.getLocal(iconf.getConfiguration());
+    perm = new Path(shard.getDirectory());
+    temp = new Path(tempDir);
+
+    long initGeneration = shard.getGeneration();
+    if (!fs.exists(perm)) {
+      assert (initGeneration < 0);
+      fs.mkdirs(perm);
+    } else {
+      restoreGeneration(fs, perm, initGeneration);
+    }
+    dir =
+        new MixedDirectory(fs, perm, localFs, fs.startLocalOutput(perm, temp),
+            iconf.getConfiguration());
+
+    // analyzer is null because we only use addIndexes, not addDocument
+    writer =
+        new IndexWriter(dir, false, null,
+            initGeneration < 0 ? new KeepOnlyLastCommitDeletionPolicy()
+                : new MixedDeletionPolicy());
+    setParameters(iconf);
+  }
+
+  /**
+   * Process an intermediate form by carrying out, on the Lucene instance of
+   * the shard, the deletes and the inserts (a ram index) in the form. 
+   * @param form  the intermediate form containing deletes and a ram index
+   * @throws IOException
+   */
+  public void process(IntermediateForm form) throws IOException {
+    // first delete
+    Iterator<Term> iter = form.deleteTermIterator();
+    while (iter.hasNext()) {
+      writer.deleteDocuments(iter.next());
+    }
+    // then insert
+    writer.addIndexesNoOptimize(new Directory[] { form.getDirectory() });
+    numForms++;
+  }
+
+  /**
+   * Close the shard writer. Optimize the Lucene instance of the shard before
+   * closing if necessary, and copy the files created in the temp directory
+   * to the permanent directory after closing.
+   * @throws IOException
+   */
+  public void close() throws IOException {
+    LOG.info("Closing the shard writer, processed " + numForms + " forms");
+    try {
+      try {
+        if (maxNumSegments > 0) {
+          writer.optimize(maxNumSegments);
+          LOG.info("Optimized the shard into at most " + maxNumSegments
+              + " segments");
+        }
+      } finally {
+        writer.close();
+        LOG.info("Closed Lucene index writer");
+      }
+
+      moveFromTempToPerm();
+      LOG.info("Moved new index files to " + perm);
+
+    } finally {
+      dir.close();
+      LOG.info("Closed the shard writer");
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  public String toString() {
+    return this.getClass().getName() + "@" + perm + "&" + temp;
+  }
+
+  private void setParameters(IndexUpdateConfiguration iconf) {
+    int maxFieldLength = iconf.getIndexMaxFieldLength();
+    if (maxFieldLength > 0) {
+      writer.setMaxFieldLength(maxFieldLength);
+    }
+    writer.setUseCompoundFile(iconf.getIndexUseCompoundFile());
+    maxNumSegments = iconf.getIndexMaxNumSegments();
+
+    if (maxFieldLength > 0) {
+      LOG.info("sea.max.field.length = " + writer.getMaxFieldLength());
+    }
+    LOG.info("sea.use.compound.file = " + writer.getUseCompoundFile());
+    LOG.info("sea.max.num.segments = " + maxNumSegments);
+  }
+
+  // in case a previous reduce task fails, restore the generation to
+  // the original starting point by deleting the segments.gen file
+  // and the segments_N files whose generations are greater than the
+  // starting generation; rest of the unwanted files will be deleted
+  // once the unwanted segments_N files are deleted
+  private void restoreGeneration(FileSystem fs, Path perm, long startGen)
+      throws IOException {
+
+    FileStatus[] fileStatus = fs.listStatus(perm, new PathFilter() {
+      public boolean accept(Path path) {
+        return LuceneUtil.isSegmentsFile(path.getName());
+      }
+    });
+
+    // remove the segments_N files whose generation are greater than
+    // the starting generation
+    for (int i = 0; i < fileStatus.length; i++) {
+      Path path = fileStatus[i].getPath();
+      if (startGen < LuceneUtil.generationFromSegmentsFileName(path.getName())) {
+        fs.delete(path);
+      }
+    }
+
+    // always remove segments.gen in case last failed try removed segments_N
+    // but not segments.gen, and segments.gen will be overwritten anyway.
+    Path segmentsGenFile = new Path(LuceneUtil.IndexFileNames.SEGMENTS_GEN);
+    if (fs.exists(segmentsGenFile)) {
+      fs.delete(segmentsGenFile);
+    }
+  }
+
+  // move the files created in the temp dir into the perm dir
+  // and then delete the temp dir from the local FS
+  private void moveFromTempToPerm() throws IOException {
+    try {
+      FileStatus[] fileStatus =
+          localFs.listStatus(temp, LuceneIndexFileNameFilter.getFilter());
+      Path segmentsPath = null;
+      Path segmentsGenPath = null;
+
+      // move the files created in temp dir except segments_N and segments.gen
+      for (int i = 0; i < fileStatus.length; i++) {
+        Path path = fileStatus[i].getPath();
+        String name = path.getName();
+
+        if (LuceneUtil.isSegmentsGenFile(name)) {
+          assert (segmentsGenPath == null);
+          segmentsGenPath = path;
+        } else if (LuceneUtil.isSegmentsFile(name)) {
+          assert (segmentsPath == null);
+          segmentsPath = path;
+        } else {
+          fs.completeLocalOutput(new Path(perm, name), path);
+        }
+      }
+
+      // move the segments_N file
+      if (segmentsPath != null) {
+        fs.completeLocalOutput(new Path(perm, segmentsPath.getName()),
+            segmentsPath);
+      }
+
+      // move the segments.gen file
+      if (segmentsGenPath != null) {
+        fs.completeLocalOutput(new Path(perm, segmentsGenPath.getName()),
+            segmentsGenPath);
+      }
+    } finally {
+      // finally delete the temp dir (files should have been deleted)
+      localFs.delete(temp);
+    }
+  }
+
+}

+ 273 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/main/UpdateIndex.java

@@ -0,0 +1,273 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.main;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.contrib.index.mapred.IndexUpdateConfiguration;
+import org.apache.hadoop.contrib.index.mapred.IIndexUpdater;
+import org.apache.hadoop.contrib.index.mapred.Shard;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * A distributed "index" is partitioned into "shards". Each shard corresponds
+ * to a Lucene instance. This class contains the main() method which uses a
+ * Map/Reduce job to analyze documents and update Lucene instances in parallel.
+ * 
+ * The main() method in UpdateIndex requires the following information for
+ * updating the shards:
+ *   - Input formatter. This specifies how to format the input documents.
+ *   - Analysis. This defines the analyzer to use on the input. The analyzer
+ *     determines whether a document is being inserted, updated, or deleted.
+ *     For inserts or updates, the analyzer also converts each input document
+ *     into a Lucene document.
+ *   - Input paths. This provides the location(s) of updated documents,
+ *     e.g., HDFS files or directories, or HBase tables.
+ *   - Shard paths, or index path with the number of shards. Either specify
+ *     the path for each shard, or specify an index path and the shards are
+ *     the sub-directories of the index directory.
+ *   - Output path. When the update to a shard is done, a message is put here.
+ *   - Number of map tasks.
+ *
+ * All of the information can be specified in a configuration file. All but
+ * the first two can also be specified as command line options. Check out
+ * conf/index-config.xml.template for other configurable parameters.
+ *
+ * Note: Because of the parallel nature of Map/Reduce, the behaviour of
+ * multiple inserts, deletes or updates to the same document is undefined.
+ */
+public class UpdateIndex {
+  public static final Log LOG = LogFactory.getLog(UpdateIndex.class);
+
+  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
+  static {
+    NUMBER_FORMAT.setMinimumIntegerDigits(5);
+    NUMBER_FORMAT.setGroupingUsed(false);
+  }
+
+  private static long now() {
+    return System.currentTimeMillis();
+  }
+
+  private static void printUsage(String cmd) {
+    System.err.println("Usage: java " + UpdateIndex.class.getName() + "\n"
+        + "                        -inputPaths <inputPath,inputPath>\n"
+        + "                        -outputPath <outputPath>\n"
+        + "                        -shards     <shardDir,shardDir>\n"
+        + "                        -indexPath  <indexPath>\n"
+        + "                        -numShards  <num>\n"
+        + "                        -numMapTasks <num>\n"
+        + "                        -conf       <confPath>\n"
+        + "Note: Do not use both -shards option and -indexPath option.");
+  }
+
+  private static String getIndexPath(Configuration conf) {
+    return conf.get("sea.index.path");
+  }
+
+  private static int getNumShards(Configuration conf) {
+    return conf.getInt("sea.num.shards", 1);
+  }
+
+  private static Shard[] createShards(String indexPath, int numShards,
+      Configuration conf) throws IOException {
+
+    String parent = Shard.normalizePath(indexPath) + Path.SEPARATOR;
+    long versionNumber = -1;
+    long generation = -1;
+
+    FileSystem fs = FileSystem.get(conf);
+    Path path = new Path(indexPath);
+
+    if (fs.exists(path)) {
+      FileStatus[] fileStatus = fs.listStatus(path);
+      String[] shardNames = new String[fileStatus.length];
+      int count = 0;
+      for (int i = 0; i < fileStatus.length; i++) {
+        if (fileStatus[i].isDir()) {
+          shardNames[count] = fileStatus[i].getPath().getName();
+          count++;
+        }
+      }
+      Arrays.sort(shardNames, 0, count);
+
+      Shard[] shards = new Shard[count >= numShards ? count : numShards];
+      for (int i = 0; i < count; i++) {
+        shards[i] =
+            new Shard(versionNumber, parent + shardNames[i], generation);
+      }
+
+      int number = count;
+      for (int i = count; i < numShards; i++) {
+        String shardPath;
+        while (true) {
+          shardPath = parent + NUMBER_FORMAT.format(number++);
+          if (!fs.exists(new Path(shardPath))) {
+            break;
+          }
+        }
+        shards[i] = new Shard(versionNumber, shardPath, generation);
+      }
+      return shards;
+    } else {
+      Shard[] shards = new Shard[numShards];
+      for (int i = 0; i < shards.length; i++) {
+        shards[i] =
+            new Shard(versionNumber, parent + NUMBER_FORMAT.format(i),
+                generation);
+      }
+      return shards;
+    }
+  }
+
+  /**
+   * The main() method
+   * @param argv
+   */
+  public static void main(String[] argv) {
+    if (argv.length == 0) {
+      printUsage("");
+      System.exit(-1);
+    }
+
+    String inputPathsString = null;
+    Path outputPath = null;
+    String shardsString = null;
+    String indexPath = null;
+    int numShards = -1;
+    int numMapTasks = -1;
+    Configuration conf = new Configuration();
+    String confPath = null;
+
+    // parse the command line
+    for (int i = 0; i < argv.length; i++) { // parse command line
+      if (argv[i].equals("-inputPaths")) {
+        inputPathsString = argv[++i];
+      } else if (argv[i].equals("-outputPath")) {
+        outputPath = new Path(argv[++i]);
+      } else if (argv[i].equals("-shards")) {
+        shardsString = argv[++i];
+      } else if (argv[i].equals("-indexPath")) {
+        indexPath = argv[++i];
+      } else if (argv[i].equals("-numShards")) {
+        numShards = Integer.parseInt(argv[++i]);
+      } else if (argv[i].equals("-numMapTasks")) {
+        numMapTasks = Integer.parseInt(argv[++i]);
+      } else if (argv[i].equals("-conf")) {
+        // add as a local FS resource
+        confPath = argv[++i];
+        conf.addResource(new Path(confPath));
+      } else {
+        System.out.println("Unknown option " + argv[i] + " w/ value "
+            + argv[++i]);
+      }
+    }
+    LOG.info("inputPaths = " + inputPathsString);
+    LOG.info("outputPath = " + outputPath);
+    LOG.info("shards     = " + shardsString);
+    LOG.info("indexPath  = " + indexPath);
+    LOG.info("numShards  = " + numShards);
+    LOG.info("numMapTasks= " + numMapTasks);
+    LOG.info("confPath   = " + confPath);
+
+    Path[] inputPaths = null;
+    Shard[] shards = null;
+
+    JobConf jobConf = new JobConf(conf);
+    IndexUpdateConfiguration iconf = new IndexUpdateConfiguration(jobConf);
+
+    if (inputPathsString != null) {
+      jobConf.set("mapred.input.dir", inputPathsString);
+    }
+    inputPaths = jobConf.getInputPaths();
+    if (inputPaths.length == 0) {
+      inputPaths = null;
+    }
+
+    if (outputPath == null) {
+      outputPath = jobConf.getOutputPath();
+    }
+
+    if (inputPaths == null || outputPath == null) {
+      System.err.println("InputPaths and outputPath must be specified.");
+      printUsage("");
+      System.exit(-1);
+    }
+
+    if (shardsString != null) {
+      iconf.setIndexShards(shardsString);
+    }
+    shards = Shard.getIndexShards(iconf);
+    if (shards != null && shards.length == 0) {
+      shards = null;
+    }
+
+    if (indexPath == null) {
+      indexPath = getIndexPath(conf);
+    }
+    if (numShards <= 0) {
+      numShards = getNumShards(conf);
+    }
+
+    if (shards == null && indexPath == null) {
+      System.err.println("Either shards or indexPath must be specified.");
+      printUsage("");
+      System.exit(-1);
+    }
+
+    if (numMapTasks <= 0) {
+      numMapTasks = jobConf.getNumMapTasks();
+    }
+
+    try {
+      // create shards and set their directories if necessary
+      if (shards == null) {
+        shards = createShards(indexPath, numShards, conf);
+      }
+
+      long startTime = now();
+      try {
+        IIndexUpdater updater =
+            (IIndexUpdater) ReflectionUtils.newInstance(
+                iconf.getIndexUpdaterClass(), conf);
+        LOG.info("sea.index.updater = "
+            + iconf.getIndexUpdaterClass().getName());
+
+        updater.run(conf, inputPaths, outputPath, numMapTasks, shards);
+        LOG.info("Index update job is done");
+
+      } finally {
+        long elapsedTime = now() - startTime;
+        LOG.info("Elapsed time is  " + (elapsedTime / 1000) + "s");
+        System.out.println("Elapsed time is " + (elapsedTime / 1000) + "s");
+      }
+    } catch (Exception e) {
+      e.printStackTrace(System.err);
+    }
+  }
+}

+ 208 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/DocumentAndOp.java

@@ -0,0 +1,208 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.Term;
+
+/**
+ * This class represents an indexing operation. The operation can be an insert,
+ * a delete or an update. If the operation is an insert or an update, a (new)
+ * document must be specified. If the operation is a delete or an update, a
+ * delete term must be specified.
+ */
+public class DocumentAndOp implements Writable {
+
+  /**
+   * This class represents the type of an operation - an insert, a delete or
+   * an update.
+   */
+  public static final class Op {
+    public static final Op INSERT = new Op("INSERT");
+    public static final Op DELETE = new Op("DELETE");
+    public static final Op UPDATE = new Op("UPDATE");
+
+    private String name;
+
+    private Op(String name) {
+      this.name = name;
+    }
+
+    public String toString() {
+      return name;
+    }
+  }
+
+  private Op op;
+  private Document doc;
+  private Term term;
+
+  /**
+   * Constructor for no operation.
+   */
+  public DocumentAndOp() {
+  }
+
+  /**
+   * Constructor for an insert operation.
+   * @param op
+   * @param doc
+   */
+  public DocumentAndOp(Op op, Document doc) {
+    assert (op == Op.INSERT);
+    this.op = op;
+    this.doc = doc;
+    this.term = null;
+  }
+
+  /**
+   * Constructor for a delete operation.
+   * @param op
+   * @param term
+   */
+  public DocumentAndOp(Op op, Term term) {
+    assert (op == Op.DELETE);
+    this.op = op;
+    this.doc = null;
+    this.term = term;
+  }
+
+  /**
+   * Constructor for an insert, a delete or an update operation.
+   * @param op
+   * @param doc
+   * @param term
+   */
+  public DocumentAndOp(Op op, Document doc, Term term) {
+    if (op == Op.INSERT) {
+      assert (doc != null);
+      assert (term == null);
+    } else if (op == Op.DELETE) {
+      assert (doc == null);
+      assert (term != null);
+    } else {
+      assert (op == Op.UPDATE);
+      assert (doc != null);
+      assert (term != null);
+    }
+    this.op = op;
+    this.doc = doc;
+    this.term = term;
+  }
+
+  /**
+   * Set the instance to be an insert operation.
+   * @param doc
+   */
+  public void setInsert(Document doc) {
+    this.op = Op.INSERT;
+    this.doc = doc;
+    this.term = null;
+  }
+
+  /**
+   * Set the instance to be a delete operation.
+   * @param term
+   */
+  public void setDelete(Term term) {
+    this.op = Op.DELETE;
+    this.doc = null;
+    this.term = term;
+  }
+
+  /**
+   * Set the instance to be an update operation.
+   * @param doc
+   * @param term
+   */
+  public void setUpdate(Document doc, Term term) {
+    this.op = Op.UPDATE;
+    this.doc = doc;
+    this.term = term;
+  }
+
+  /**
+   * Get the type of operation.
+   * @return the type of the operation.
+   */
+  public Op getOp() {
+    return op;
+  }
+
+  /**
+   * Get the document.
+   * @return the document
+   */
+  public Document getDocument() {
+    return doc;
+  }
+
+  /**
+   * Get the term.
+   * @return the term
+   */
+  public Term getTerm() {
+    return term;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  public String toString() {
+    StringBuilder buffer = new StringBuilder();
+    buffer.append(this.getClass().getName());
+    buffer.append("[op=");
+    buffer.append(op);
+    buffer.append(", doc=");
+    if (doc != null) {
+      buffer.append(doc);
+    } else {
+      buffer.append("null");
+    }
+    buffer.append(", term=");
+    if (term != null) {
+      buffer.append(term);
+    } else {
+      buffer.append("null");
+    }
+    buffer.append("]");
+    return buffer.toString();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
+   */
+  public void write(DataOutput out) throws IOException {
+    throw new IOException(this.getClass().getName()
+        + ".write should never be called");
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
+   */
+  public void readFields(DataInput in) throws IOException {
+    throw new IOException(this.getClass().getName()
+        + ".readFields should never be called");
+  }
+}

+ 89 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/DocumentID.java

@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * The class represents a document id, which is of type text.
+ */
+public class DocumentID implements WritableComparable {
+  private final Text docID;
+
+  /**
+   * Constructor.
+   */
+  public DocumentID() {
+    docID = new Text();
+  }
+
+  /**
+   * The text of the document id.
+   * @return the text
+   */
+  public Text getText() {
+    return docID;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Comparable#compareTo(java.lang.Object)
+   */
+  public int compareTo(Object obj) {
+    if (this == obj) {
+      return 0;
+    } else {
+      return docID.compareTo(((DocumentID) obj).docID);
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#hashCode()
+   */
+  public int hashCode() {
+    return docID.hashCode();
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  public String toString() {
+    return this.getClass().getName() + "[" + docID + "]";
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
+   */
+  public void write(DataOutput out) throws IOException {
+    throw new IOException(this.getClass().getName()
+        + ".write should never be called");
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
+   */
+  public void readFields(DataInput in) throws IOException {
+    throw new IOException(this.getClass().getName()
+        + ".readFields should never be called");
+  }
+}

+ 50 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IDistributionPolicy.java

@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+/**
+ * A distribution policy decides, given a document with a document id, which
+ * one shard the request should be sent to if the request is an insert, and
+ * which shard(s) the request should be sent to if the request is a delete.
+ */
+public interface IDistributionPolicy {
+
+  /**
+   * Initialization. It must be called before any chooseShard() is called.
+   * @param shards
+   */
+  void init(Shard[] shards);
+
+  /**
+   * Choose a shard to send an insert request.
+   * @param key
+   * @return the index of the chosen shard
+   */
+  int chooseShardForInsert(DocumentID key);
+
+  /**
+   * Choose a shard or all shards to send a delete request. E.g. a round-robin
+   * distribution policy would send a delete request to all the shards.
+   * -1 represents all the shards.
+   * @param key
+   * @return the index of the chosen shard, -1 if all the shards are chosen
+   */
+  int chooseShardForDelete(DocumentID key);
+
+}

+ 46 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IIndexUpdater.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A class implements an index updater interface should create a Map/Reduce job
+ * configuration and run the Map/Reduce job to analyze documents and update
+ * Lucene instances in parallel.
+ */
+public interface IIndexUpdater {
+
+  /**
+   * Create a Map/Reduce job configuration and run the Map/Reduce job to
+   * analyze documents and update Lucene instances in parallel.
+   * @param conf
+   * @param inputPaths
+   * @param outputPath
+   * @param numMapTasks
+   * @param shards
+   * @throws IOException
+   */
+  void run(Configuration conf, Path[] inputPaths, Path outputPath,
+      int numMapTasks, Shard[] shards) throws IOException;
+
+}

+ 32 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/ILocalAnalysis.java

@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.Mapper;
+
+/**
+ * Application specific local analysis. The output type must be (DocumentID,
+ * DocumentAndOp).
+ */
+public interface ILocalAnalysis<K extends WritableComparable, V extends Writable>
+    extends Mapper<K, V, DocumentID, DocumentAndOp> {
+
+}

+ 77 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateCombiner.java

@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * This combiner combines multiple intermediate forms into one intermediate
+ * form. More specifically, the input intermediate forms are a single-document
+ * ram index and/or a single delete term. An output intermediate form contains
+ * a multi-document ram index and/or multiple delete terms.   
+ */
+public class IndexUpdateCombiner extends MapReduceBase implements
+    Reducer<Shard, IntermediateForm, Shard, IntermediateForm> {
+  static final Log LOG = LogFactory.getLog(IndexUpdateCombiner.class);
+
+  IndexUpdateConfiguration iconf;
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.Reducer#reduce(java.lang.Object, java.util.Iterator, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)
+   */
+  public void reduce(Shard key, Iterator<IntermediateForm> values,
+      OutputCollector<Shard, IntermediateForm> output, Reporter reporter)
+      throws IOException {
+
+    LOG.info("Construct a form writer for " + key);
+    IntermediateForm form = new IntermediateForm();
+    form.configure(iconf);
+    while (values.hasNext()) {
+      IntermediateForm singleDocForm = values.next();
+      form.process(singleDocForm);
+    }
+    form.closeWriter();
+    LOG.info("Closed the form writer for " + key + ", form = " + form);
+
+    output.collect(key, form);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.MapReduceBase#configure(org.apache.hadoop.mapred.JobConf)
+   */
+  public void configure(JobConf job) {
+    iconf = new IndexUpdateConfiguration(job);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.MapReduceBase#close()
+   */
+  public void close() throws IOException {
+  }
+
+}

+ 238 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateConfiguration.java

@@ -0,0 +1,238 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.contrib.index.example.HashingDistributionPolicy;
+import org.apache.hadoop.contrib.index.example.LineDocInputFormat;
+import org.apache.hadoop.contrib.index.example.LineDocLocalAnalysis;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+
+/**
+ * This class provides the getters and the setters to a number of parameters.
+ * Most of the parameters are related to the index update and the rest are
+ * from the existing Map/Reduce parameters.  
+ */
+public class IndexUpdateConfiguration {
+  final Configuration conf;
+
+  /**
+   * Constructor
+   * @param conf
+   */
+  public IndexUpdateConfiguration(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Get the underlying configuration object.
+   * @return the configuration
+   */
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  //
+  // existing map/reduce properties
+  //
+  // public int getIOFileBufferSize() {
+  // return getInt("io.file.buffer.size", 4096);
+  // }
+
+  /**
+   * Get the IO sort space in MB.
+   * @return the IO sort space in MB
+   */
+  public int getIOSortMB() {
+    return conf.getInt("io.sort.mb", 100);
+  }
+
+  /**
+   * Set the IO sort space in MB.
+   * @param mb  the IO sort space in MB
+   */
+  public void setIOSortMB(int mb) {
+    conf.setInt("io.sort.mb", mb);
+  }
+
+  /**
+   * Get the Map/Reduce temp directory.
+   * @return the Map/Reduce temp directory
+   */
+  public String getMapredTempDir() {
+    return conf.get("mapred.temp.dir");
+  }
+
+  //
+  // properties for index update
+  //
+  /**
+   * Get the distribution policy class.
+   * @return the distribution policy class
+   */
+  public Class<? extends IDistributionPolicy> getDistributionPolicyClass() {
+    return conf.getClass("sea.distribution.policy",
+        HashingDistributionPolicy.class, IDistributionPolicy.class);
+  }
+
+  /**
+   * Set the distribution policy class.
+   * @param theClass  the distribution policy class
+   */
+  public void setDistributionPolicyClass(
+      Class<? extends IDistributionPolicy> theClass) {
+    conf.setClass("sea.distribution.policy", theClass,
+        IDistributionPolicy.class);
+  }
+
+  /**
+   * Get the analyzer class.
+   * @return the analyzer class
+   */
+  public Class<? extends Analyzer> getDocumentAnalyzerClass() {
+    return conf.getClass("sea.document.analyzer", StandardAnalyzer.class,
+        Analyzer.class);
+  }
+
+  /**
+   * Set the analyzer class.
+   * @param theClass  the analyzer class
+   */
+  public void setDocumentAnalyzerClass(Class<? extends Analyzer> theClass) {
+    conf.setClass("sea.document.analyzer", theClass, Analyzer.class);
+  }
+
+  /**
+   * Get the index input format class.
+   * @return the index input format class
+   */
+  public Class<? extends InputFormat> getIndexInputFormatClass() {
+    return conf.getClass("sea.input.format", LineDocInputFormat.class,
+        InputFormat.class);
+  }
+
+  /**
+   * Set the index input format class.
+   * @param theClass  the index input format class
+   */
+  public void setIndexInputFormatClass(Class<? extends InputFormat> theClass) {
+    conf.setClass("sea.input.format", theClass, InputFormat.class);
+  }
+
+  /**
+   * Get the index updater class.
+   * @return the index updater class
+   */
+  public Class<? extends IIndexUpdater> getIndexUpdaterClass() {
+    return conf.getClass("sea.index.updater", IndexUpdater.class,
+        IIndexUpdater.class);
+  }
+
+  /**
+   * Set the index updater class.
+   * @param theClass  the index updater class
+   */
+  public void setIndexUpdaterClass(Class<? extends IIndexUpdater> theClass) {
+    conf.setClass("sea.index.updater", theClass, IIndexUpdater.class);
+  }
+
+  /**
+   * Get the local analysis class.
+   * @return the local analysis class
+   */
+  public Class<? extends ILocalAnalysis> getLocalAnalysisClass() {
+    return conf.getClass("sea.local.analysis", LineDocLocalAnalysis.class,
+        ILocalAnalysis.class);
+  }
+
+  /**
+   * Set the local analysis class.
+   * @param theClass  the local analysis class
+   */
+  public void setLocalAnalysisClass(Class<? extends ILocalAnalysis> theClass) {
+    conf.setClass("sea.local.analysis", theClass, ILocalAnalysis.class);
+  }
+
+  /**
+   * Get the string representation of a number of shards.
+   * @return the string representation of a number of shards
+   */
+  public String getIndexShards() {
+    return conf.get("sea.index.shards");
+  }
+
+  /**
+   * Set the string representation of a number of shards.
+   * @param shards  the string representation of a number of shards
+   */
+  public void setIndexShards(String shards) {
+    conf.set("sea.index.shards", shards);
+  }
+
+  /**
+   * Get the max field length for a Lucene instance.
+   * @return the max field length for a Lucene instance
+   */
+  public int getIndexMaxFieldLength() {
+    return conf.getInt("sea.max.field.length", -1);
+  }
+
+  /**
+   * Set the max field length for a Lucene instance.
+   * @param maxFieldLength  the max field length for a Lucene instance
+   */
+  public void setIndexMaxFieldLength(int maxFieldLength) {
+    conf.setInt("sea.max.field.length", maxFieldLength);
+  }
+
+  /**
+   * Get the max number of segments for a Lucene instance.
+   * @return the max number of segments for a Lucene instance
+   */
+  public int getIndexMaxNumSegments() {
+    return conf.getInt("sea.max.num.segments", -1);
+  }
+
+  /**
+   * Set the max number of segments for a Lucene instance.
+   * @param maxNumSegments  the max number of segments for a Lucene instance
+   */
+  public void setIndexMaxNumSegments(int maxNumSegments) {
+    conf.setInt("sea.max.num.segments", maxNumSegments);
+  }
+
+  /**
+   * Check whether to use the compound file format for a Lucene instance.
+   * @return true if using the compound file format for a Lucene instance
+   */
+  public boolean getIndexUseCompoundFile() {
+    return conf.getBoolean("sea.use.compound.file", false);
+  }
+
+  /**
+   * Set whether use the compound file format for a Lucene instance.
+   * @param useCompoundFile  whether to use the compound file format
+   */
+  public void setIndexUseCompoundFile(boolean useCompoundFile) {
+    conf.setBoolean("sea.use.compound.file", useCompoundFile);
+  }
+
+}

+ 199 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateMapper.java

@@ -0,0 +1,199 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.lucene.analysis.Analyzer;
+
+/**
+ * This class applies local analysis on a key-value pair and then convert the
+ * result docid-operation pair to a shard-and-intermediate form pair.
+ */
+public class IndexUpdateMapper<K extends WritableComparable, V extends Writable>
+    extends MapReduceBase implements Mapper<K, V, Shard, IntermediateForm> {
+  static final Log LOG = LogFactory.getLog(IndexUpdateMapper.class);
+
+  /**
+   * Get the map output key class.
+   * @return the map output key class
+   */
+  public static Class<? extends WritableComparable> getMapOutputKeyClass() {
+    return Shard.class;
+  }
+
+  /**
+   * Get the map output value class.
+   * @return the map output value class
+   */
+  public static Class<? extends Writable> getMapOutputValueClass() {
+    return IntermediateForm.class;
+  }
+
+  IndexUpdateConfiguration iconf;
+  private Analyzer analyzer;
+  private Shard[] shards;
+  private IDistributionPolicy distributionPolicy;
+
+  private ILocalAnalysis<K, V> localAnalysis;
+  private DocumentID tmpKey;
+  private DocumentAndOp tmpValue;
+
+  private OutputCollector<DocumentID, DocumentAndOp> tmpCollector =
+      new OutputCollector<DocumentID, DocumentAndOp>() {
+        public void collect(DocumentID key, DocumentAndOp value)
+            throws IOException {
+          tmpKey = key;
+          tmpValue = value;
+        }
+      };
+
+  /**
+   * Map a key-value pair to a shard-and-intermediate form pair. Internally,
+   * the local analysis is first applied to map the key-value pair to a
+   * document id-and-operation pair, then the docid-and-operation pair is
+   * mapped to a shard-intermediate form pair. The intermediate form is of the
+   * form of a single-document ram index and/or a single delete term.
+   */
+  public void map(K key, V value,
+      OutputCollector<Shard, IntermediateForm> output, Reporter reporter)
+      throws IOException {
+
+    synchronized (this) {
+      localAnalysis.map(key, value, tmpCollector, reporter);
+
+      if (tmpKey != null && tmpValue != null) {
+        DocumentAndOp doc = tmpValue;
+        IntermediateForm form = new IntermediateForm();
+        form.configure(iconf);
+        form.process(doc, analyzer);
+        form.closeWriter();
+
+        if (doc.getOp() == DocumentAndOp.Op.INSERT) {
+          int chosenShard = distributionPolicy.chooseShardForInsert(tmpKey);
+          if (chosenShard >= 0) {
+            // insert into one shard
+            output.collect(shards[chosenShard], form);
+          } else {
+            throw new IOException("Chosen shard for insert must be >= 0");
+          }
+
+        } else if (doc.getOp() == DocumentAndOp.Op.DELETE) {
+          int chosenShard = distributionPolicy.chooseShardForDelete(tmpKey);
+          if (chosenShard >= 0) {
+            // delete from one shard
+            output.collect(shards[chosenShard], form);
+          } else {
+            // broadcast delete to all shards
+            for (int i = 0; i < shards.length; i++) {
+              output.collect(shards[i], form);
+            }
+          }
+
+        } else { // UPDATE
+          int insertToShard = distributionPolicy.chooseShardForInsert(tmpKey);
+          int deleteFromShard =
+              distributionPolicy.chooseShardForDelete(tmpKey);
+
+          if (insertToShard >= 0) {
+            if (insertToShard == deleteFromShard) {
+              // update into one shard
+              output.collect(shards[insertToShard], form);
+            } else {
+              // prepare a deletion form
+              IntermediateForm deletionForm = new IntermediateForm();
+              deletionForm.configure(iconf);
+              deletionForm.process(new DocumentAndOp(DocumentAndOp.Op.DELETE,
+                  doc.getTerm()), analyzer);
+              deletionForm.closeWriter();
+
+              if (deleteFromShard >= 0) {
+                // delete from one shard
+                output.collect(shards[deleteFromShard], deletionForm);
+              } else {
+                // broadcast delete to all shards
+                for (int i = 0; i < shards.length; i++) {
+                  output.collect(shards[i], deletionForm);
+                }
+              }
+
+              // prepare an insertion form
+              IntermediateForm insertionForm = new IntermediateForm();
+              insertionForm.configure(iconf);
+              insertionForm.process(new DocumentAndOp(DocumentAndOp.Op.INSERT,
+                  doc.getDocument()), analyzer);
+              insertionForm.closeWriter();
+
+              // insert into one shard
+              output.collect(shards[insertToShard], insertionForm);
+            }
+          } else {
+            throw new IOException("Chosen shard for insert must be >= 0");
+          }
+        }
+      }
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.MapReduceBase#configure(org.apache.hadoop.mapred.JobConf)
+   */
+  public void configure(JobConf job) {
+    iconf = new IndexUpdateConfiguration(job);
+    analyzer =
+        (Analyzer) ReflectionUtils.newInstance(
+            iconf.getDocumentAnalyzerClass(), job);
+
+    localAnalysis =
+        (ILocalAnalysis) ReflectionUtils.newInstance(
+            iconf.getLocalAnalysisClass(), job);
+    localAnalysis.configure(job);
+
+    shards = Shard.getIndexShards(iconf);
+
+    distributionPolicy =
+        (IDistributionPolicy) ReflectionUtils.newInstance(
+            iconf.getDistributionPolicyClass(), job);
+    distributionPolicy.init(shards);
+
+    LOG.info("sea.document.analyzer = " + analyzer.getClass().getName());
+    LOG.info("sea.local.analysis = " + localAnalysis.getClass().getName());
+    LOG.info(shards.length + " shards = " + iconf.getIndexShards());
+    LOG.info("sea.distribution.policy = "
+        + distributionPolicy.getClass().getName());
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.MapReduceBase#close()
+   */
+  public void close() throws IOException {
+    localAnalysis.close();
+  }
+
+}

+ 65 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateOutputFormat.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputFormatBase;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * The record writer of this output format simply puts a message in an output
+ * path when a shard update is done.
+ */
+public class IndexUpdateOutputFormat extends OutputFormatBase<Shard, Text> {
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.OutputFormatBase#getRecordWriter(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.mapred.JobConf, java.lang.String, org.apache.hadoop.util.Progressable)
+   */
+  public RecordWriter<Shard, Text> getRecordWriter(final FileSystem fs,
+      JobConf job, String name, final Progressable progress)
+      throws IOException {
+
+    final Path perm = new Path(job.getOutputPath(), name);
+
+    return new RecordWriter<Shard, Text>() {
+      public void write(Shard key, Text value) throws IOException {
+        assert (IndexUpdateReducer.DONE.equals(value));
+
+        String shardName = key.getDirectory();
+        shardName = shardName.replace("/", "_");
+
+        Path doneFile =
+            new Path(perm, IndexUpdateReducer.DONE + "_" + shardName);
+        if (!fs.exists(doneFile)) {
+          fs.createNewFile(doneFile);
+        }
+      }
+
+      public void close(final Reporter reporter) throws IOException {
+      }
+    };
+  }
+}

+ 60 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdatePartitioner.java

@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Partitioner;
+
+/**
+ * This partitioner class puts the values of the same key - in this case the
+ * same shard - in the same partition.
+ */
+public class IndexUpdatePartitioner implements
+    Partitioner<Shard, IntermediateForm> {
+
+  private Shard[] shards;
+  private Map<Shard, Integer> map;
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.Partitioner#getPartition(java.lang.Object, java.lang.Object, int)
+   */
+  public int getPartition(Shard key, IntermediateForm value, int numPartitions) {
+    int partition = map.get(key).intValue();
+    if (partition < numPartitions) {
+      return partition;
+    } else {
+      return numPartitions - 1;
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.JobConfigurable#configure(org.apache.hadoop.mapred.JobConf)
+   */
+  public void configure(JobConf job) {
+    shards = Shard.getIndexShards(new IndexUpdateConfiguration(job));
+    map = new HashMap<Shard, Integer>();
+    for (int i = 0; i < shards.length; i++) {
+      map.put(shards[i], i);
+    }
+  }
+
+}

+ 143 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdateReducer.java

@@ -0,0 +1,143 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.contrib.index.lucene.ShardWriter;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Closeable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+
+/**
+ * This reducer applies to a shard the changes for it. A "new version" of
+ * a shard is created at the end of a reduce. It is important to note that
+ * the new version of the shard is not derived from scratch. By leveraging
+ * Lucene's update algorithm, the new version of each Lucene instance will
+ * share as many files as possible as the previous version. 
+ */
+public class IndexUpdateReducer extends MapReduceBase implements
+    Reducer<Shard, IntermediateForm, Shard, Text> {
+  static final Log LOG = LogFactory.getLog(IndexUpdateReducer.class);
+  static final Text DONE = new Text("done");
+
+  /**
+   * Get the reduce output key class.
+   * @return the reduce output key class
+   */
+  public static Class<? extends WritableComparable> getOutputKeyClass() {
+    return Shard.class;
+  }
+
+  /**
+   * Get the reduce output value class.
+   * @return the reduce output value class
+   */
+  public static Class<? extends Writable> getOutputValueClass() {
+    return Text.class;
+  }
+
+  private IndexUpdateConfiguration iconf;
+  private String mapredTempDir;
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.Reducer#reduce(java.lang.Object, java.util.Iterator, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)
+   */
+  public void reduce(Shard key, Iterator<IntermediateForm> values,
+      OutputCollector<Shard, Text> output, Reporter reporter)
+      throws IOException {
+
+    LOG.info("Construct a shard writer for " + key);
+    FileSystem fs = FileSystem.get(iconf.getConfiguration());
+    String temp =
+        mapredTempDir + Path.SEPARATOR + "shard_" + System.currentTimeMillis();
+    final ShardWriter writer = new ShardWriter(fs, key, temp, iconf);
+
+    // update the shard
+    while (values.hasNext()) {
+      IntermediateForm form = values.next();
+      writer.process(form);
+      reporter.progress();
+    }
+
+    // close the shard
+    final Reporter fReporter = reporter;
+    new Closeable() {
+      boolean closed = false;
+
+      public void close() throws IOException {
+        // spawn a thread to give progress heartbeats
+        Thread prog = new Thread() {
+          public void run() {
+            while (!closed) {
+              try {
+                fReporter.setStatus("closing");
+                Thread.sleep(1000);
+              } catch (InterruptedException e) {
+                continue;
+              } catch (Throwable e) {
+                return;
+              }
+            }
+          }
+        };
+
+        try {
+          prog.start();
+
+          if (writer != null) {
+            writer.close();
+          }
+        } finally {
+          closed = true;
+        }
+      }
+    }.close();
+    LOG.info("Closed the shard writer for " + key + ", writer = " + writer);
+
+    output.collect(key, DONE);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.MapReduceBase#configure(org.apache.hadoop.mapred.JobConf)
+   */
+  public void configure(JobConf job) {
+    iconf = new IndexUpdateConfiguration(job);
+    mapredTempDir = iconf.getMapredTempDir();
+    mapredTempDir = Shard.normalizePath(mapredTempDir);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.mapred.MapReduceBase#close()
+   */
+  public void close() throws IOException {
+  }
+
+}

+ 151 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IndexUpdater.java

@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.contrib.index.lucene.FileSystemDirectory;
+import org.apache.hadoop.contrib.index.lucene.LuceneUtil;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * An implementation of an index updater interface which creates a Map/Reduce
+ * job configuration and run the Map/Reduce job to analyze documents and update
+ * Lucene instances in parallel.
+ */
+public class IndexUpdater implements IIndexUpdater {
+  public static final Log LOG = LogFactory.getLog(IndexUpdater.class);
+
+  public IndexUpdater() {
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.contrib.index.mapred.IIndexUpdater#run(org.apache.hadoop.conf.Configuration, org.apache.hadoop.fs.Path[], org.apache.hadoop.fs.Path, int, org.apache.hadoop.contrib.index.mapred.Shard[])
+   */
+  public void run(Configuration conf, Path[] inputPaths, Path outputPath,
+      int numMapTasks, Shard[] shards) throws IOException {
+    JobConf jobConf =
+        createJob(conf, inputPaths, outputPath, numMapTasks, shards);
+    JobClient.runJob(jobConf);
+  }
+
+  JobConf createJob(Configuration conf, Path[] inputPaths, Path outputPath,
+      int numMapTasks, Shard[] shards) throws IOException {
+    // set the starting generation for each shard
+    // when a reduce task fails, a new reduce task
+    // has to know where to re-start
+    setShardGeneration(conf, shards);
+
+    // iconf.set sets properties in conf
+    IndexUpdateConfiguration iconf = new IndexUpdateConfiguration(conf);
+    Shard.setIndexShards(iconf, shards);
+
+    // MapTask.MapOutputBuffer uses "io.sort.mb" to decide its max buffer size
+    // (max buffer size = 1/2 * "io.sort.mb").
+    // Here we half-en "io.sort.mb" because we use the other half memory to
+    // build an intermediate form/index in Combiner.
+    iconf.setIOSortMB(iconf.getIOSortMB() / 2);
+
+    // create the job configuration
+    JobConf jobConf = new JobConf(conf, IndexUpdater.class);
+    jobConf.setJobName(this.getClass().getName() + "_"
+        + System.currentTimeMillis());
+
+    // provided by application
+    jobConf.setInputPath(inputPaths[0]);
+    for (int i = 1; i < inputPaths.length; i++) {
+      jobConf.addInputPath(inputPaths[i]);
+    }
+
+    jobConf.setOutputPath(outputPath);
+
+    jobConf.setNumMapTasks(numMapTasks);
+
+    // already set shards
+    jobConf.setNumReduceTasks(shards.length);
+
+    jobConf.setInputFormat(iconf.getIndexInputFormatClass());
+
+    Path[] inputs = jobConf.getInputPaths();
+    StringBuilder buffer = new StringBuilder(inputs[0].toString());
+    for (int i = 1; i < inputs.length; i++) {
+      buffer.append(",");
+      buffer.append(inputs[i].toString());
+    }
+    LOG.info("mapred.input.dir = " + buffer.toString());
+    LOG.info("mapred.output.dir = " + jobConf.getOutputPath().toString());
+    LOG.info("mapred.map.tasks = " + jobConf.getNumMapTasks());
+    LOG.info("mapred.reduce.tasks = " + jobConf.getNumReduceTasks());
+    LOG.info(shards.length + " shards = " + iconf.getIndexShards());
+    // better if we don't create the input format instance
+    LOG.info("mapred.input.format.class = "
+        + jobConf.getInputFormat().getClass().getName());
+
+    // set by the system
+    jobConf.setMapOutputKeyClass(IndexUpdateMapper.getMapOutputKeyClass());
+    jobConf.setMapOutputValueClass(IndexUpdateMapper.getMapOutputValueClass());
+    jobConf.setOutputKeyClass(IndexUpdateReducer.getOutputKeyClass());
+    jobConf.setOutputValueClass(IndexUpdateReducer.getOutputValueClass());
+
+    jobConf.setMapperClass(IndexUpdateMapper.class);
+    jobConf.setPartitionerClass(IndexUpdatePartitioner.class);
+    jobConf.setCombinerClass(IndexUpdateCombiner.class);
+    jobConf.setReducerClass(IndexUpdateReducer.class);
+
+    jobConf.setOutputFormat(IndexUpdateOutputFormat.class);
+
+    return jobConf;
+  }
+
+  void setShardGeneration(Configuration conf, Shard[] shards)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+
+    for (int i = 0; i < shards.length; i++) {
+      Path path = new Path(shards[i].getDirectory());
+      long generation = -1;
+
+      if (fs.exists(path)) {
+        FileSystemDirectory dir = null;
+
+        try {
+          dir = new FileSystemDirectory(fs, path, false, conf);
+          generation = LuceneUtil.getCurrentSegmentGeneration(dir);
+        } finally {
+          if (dir != null) {
+            dir.close();
+          }
+        }
+      }
+
+      if (generation != shards[i].getGeneration()) {
+        // set the starting generation for the shard
+        shards[i] =
+            new Shard(shards[i].getVersion(), shards[i].getDirectory(),
+                generation);
+      }
+    }
+  }
+}

+ 239 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/IntermediateForm.java

@@ -0,0 +1,239 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import org.apache.hadoop.contrib.index.lucene.RAMDirectoryUtil;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMDirectory;
+
+/**
+ * An intermediate form for one or more parsed Lucene documents and/or
+ * delete terms. It actually uses Lucene file format as the format for
+ * the intermediate form by using RAM dir files.
+ * 
+ * Note: If process(*) is ever called, closeWriter() should be called.
+ * Otherwise, no need to call closeWriter().
+ */
+public class IntermediateForm implements Writable {
+
+  private IndexUpdateConfiguration iconf = null;
+  private final Collection<Term> deleteList;
+  private RAMDirectory dir;
+  private IndexWriter writer;
+  private int numDocs;
+
+  /**
+   * Constructor
+   * @throws IOException
+   */
+  public IntermediateForm() throws IOException {
+    deleteList = new ConcurrentLinkedQueue<Term>();
+    dir = new RAMDirectory();
+    writer = null;
+    numDocs = 0;
+  }
+
+  /**
+   * Configure using an index update configuration.
+   * @param iconf  the index update configuration
+   */
+  public void configure(IndexUpdateConfiguration iconf) {
+    this.iconf = iconf;
+  }
+
+  /**
+   * Get the ram directory of the intermediate form.
+   * @return the ram directory
+   */
+  public Directory getDirectory() {
+    return dir;
+  }
+
+  /**
+   * Get an iterator for the delete terms in the intermediate form.
+   * @return an iterator for the delete terms
+   */
+  public Iterator<Term> deleteTermIterator() {
+    return deleteList.iterator();
+  }
+
+  /**
+   * This method is used by the index update mapper and process a document
+   * operation into the current intermediate form.
+   * @param doc  input document operation
+   * @param analyzer  the analyzer
+   * @throws IOException
+   */
+  public void process(DocumentAndOp doc, Analyzer analyzer) throws IOException {
+    if (doc.getOp() == DocumentAndOp.Op.DELETE
+        || doc.getOp() == DocumentAndOp.Op.UPDATE) {
+      deleteList.add(doc.getTerm());
+
+    }
+
+    if (doc.getOp() == DocumentAndOp.Op.INSERT
+        || doc.getOp() == DocumentAndOp.Op.UPDATE) {
+
+      if (writer == null) {
+        // analyzer is null because we specify an analyzer with addDocument
+        writer = createWriter();
+      }
+
+      writer.addDocument(doc.getDocument(), analyzer);
+      numDocs++;
+    }
+
+  }
+
+  /**
+   * This method is used by the index update combiner and process an
+   * intermediate form into the current intermediate form. More specifically,
+   * the input intermediate forms are a single-document ram index and/or a
+   * single delete term.
+   * @param form  the input intermediate form
+   * @throws IOException
+   */
+  public void process(IntermediateForm form) throws IOException {
+    if (form.deleteList.size() > 0) {
+      deleteList.addAll(form.deleteList);
+    }
+
+    if (form.dir.sizeInBytes() > 0) {
+      if (writer == null) {
+        writer = createWriter();
+      }
+
+      writer.addIndexesNoOptimize(new Directory[] { form.dir });
+      numDocs++;
+    }
+
+  }
+
+  /**
+   * Close the Lucene index writer associated with the intermediate form,
+   * if created. Do not close the ram directory. In fact, there is no need
+   * to close a ram directory.
+   * @throws IOException
+   */
+  public void closeWriter() throws IOException {
+    if (writer != null) {
+      writer.close();
+      writer = null;
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  public String toString() {
+    StringBuilder buffer = new StringBuilder();
+    buffer.append(this.getClass().getSimpleName());
+    buffer.append("[numDocs=");
+    buffer.append(numDocs);
+    buffer.append(", numDeletes=");
+    buffer.append(deleteList.size());
+    if (deleteList.size() > 0) {
+      buffer.append("(");
+      Iterator<Term> iter = deleteTermIterator();
+      while (iter.hasNext()) {
+        buffer.append(iter.next());
+        buffer.append(" ");
+      }
+      buffer.append(")");
+    }
+    buffer.append("]");
+    return buffer.toString();
+  }
+
+  private IndexWriter createWriter() throws IOException {
+    IndexWriter writer =
+        new IndexWriter(dir, false, null,
+            new KeepOnlyLastCommitDeletionPolicy());
+    writer.setUseCompoundFile(false);
+
+    if (iconf != null) {
+      int maxFieldLength = iconf.getIndexMaxFieldLength();
+      if (maxFieldLength > 0) {
+        writer.setMaxFieldLength(maxFieldLength);
+      }
+    }
+
+    return writer;
+  }
+
+  private void resetForm() throws IOException {
+    deleteList.clear();
+    if (dir.sizeInBytes() > 0) {
+      // it's ok if we don't close a ram directory
+      dir.close();
+      // an alternative is to delete all the files and reuse the ram directory
+      dir = new RAMDirectory();
+    }
+    assert (writer == null);
+    numDocs = 0;
+  }
+
+  // ///////////////////////////////////
+  // Writable
+  // ///////////////////////////////////
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(deleteList.size());
+    for (Term term : deleteList) {
+      Text.writeString(out, term.field());
+      Text.writeString(out, term.text());
+    }
+
+    String[] files = dir.list();
+    RAMDirectoryUtil.writeRAMFiles(out, dir, files);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
+   */
+  public void readFields(DataInput in) throws IOException {
+    resetForm();
+
+    int numDeleteTerms = in.readInt();
+    for (int i = 0; i < numDeleteTerms; i++) {
+      String field = Text.readString(in);
+      String text = Text.readString(in);
+      deleteList.add(new Term(field, text));
+    }
+
+    RAMDirectoryUtil.readRAMFiles(in, dir);
+  }
+
+}

+ 240 - 0
src/contrib/index/src/java/org/apache/hadoop/contrib/index/mapred/Shard.java

@@ -0,0 +1,240 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * This class represents the metadata of a shard. Version is the version number
+ * of the entire index. Directory is the directory where this shard resides in.
+ * Generation is the Lucene index's generation. Version and generation are
+ * reserved for future use.
+ * 
+ * Note: Currently the version number of the entire index is not used and
+ * defaults to -1.
+ */
+public class Shard implements WritableComparable {
+
+  // This method is copied from Path.
+  public static String normalizePath(String path) {
+    // remove double slashes & backslashes
+    path = path.replace("//", "/");
+    path = path.replace("\\", "/");
+
+    // trim trailing slash from non-root path (ignoring windows drive)
+    if (path.length() > 1 && path.endsWith("/")) {
+      path = path.substring(0, path.length() - 1);
+    }
+
+    return path;
+  }
+
+  public static void setIndexShards(IndexUpdateConfiguration conf,
+      Shard[] shards) {
+    StringBuilder shardsString = new StringBuilder(shards[0].toString());
+    for (int i = 1; i < shards.length; i++) {
+      shardsString.append(",");
+      shardsString.append(shards[i].toString());
+    }
+    conf.setIndexShards(shardsString.toString());
+  }
+
+  public static Shard[] getIndexShards(IndexUpdateConfiguration conf) {
+    String shards = conf.getIndexShards();
+    if (shards != null) {
+      ArrayList<Object> list =
+          Collections.list(new StringTokenizer(shards, ","));
+      Shard[] result = new Shard[list.size()];
+      for (int i = 0; i < list.size(); i++) {
+        result[i] = Shard.createShardFromString((String) list.get(i));
+      }
+      return result;
+    } else {
+      return null;
+    }
+  }
+
+  // assume str is formatted correctly as a shard string
+  private static Shard createShardFromString(String str) {
+    int first = str.indexOf("@");
+    int second = str.indexOf("@", first + 1);
+    long version = Long.parseLong(str.substring(0, first));
+    String dir = str.substring(first + 1, second);
+    long gen = Long.parseLong(str.substring(second + 1));
+    return new Shard(version, dir, gen);
+  }
+
+  // index/shard version
+  // the shards in the same version of an index have the same version number
+  private long version;
+  private String dir;
+  private long gen; // Lucene's generation
+
+  /**
+   * Constructor.
+   */
+  public Shard() {
+    this.version = -1;
+    this.dir = null;
+    this.gen = -1;
+  }
+
+  /**
+   * Construct a shard from a versio number, a directory and a generation
+   * number.
+   * @param version  the version number of the entire index
+   * @param dir  the directory where this shard resides
+   * @param gen  the generation of the Lucene instance
+   */
+  public Shard(long version, String dir, long gen) {
+    this.version = version;
+    this.dir = normalizePath(dir);
+    this.gen = gen;
+  }
+
+  /**
+   * Construct using a shard object.
+   * @param shard  the shard used by the constructor
+   */
+  public Shard(Shard shard) {
+    this.version = shard.version;
+    this.dir = shard.dir;
+    this.gen = shard.gen;
+  }
+
+  /**
+   * Get the version number of the entire index.
+   * @return the version number of the entire index
+   */
+  public long getVersion() {
+    return version;
+  }
+
+  /**
+   * Get the directory where this shard resides.
+   * @return the directory where this shard resides
+   */
+  public String getDirectory() {
+    return dir;
+  }
+
+  /**
+   * Get the generation of the Lucene instance.
+   * @return the generation of the Lucene instance
+   */
+  public long getGeneration() {
+    return gen;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  public String toString() {
+    return version + "@" + dir + "@" + gen;
+  }
+
+  // ///////////////////////////////////
+  // Writable
+  // ///////////////////////////////////
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(version);
+    Text.writeString(out, dir);
+    out.writeLong(gen);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
+   */
+  public void readFields(DataInput in) throws IOException {
+    version = in.readLong();
+    dir = Text.readString(in);
+    gen = in.readLong();
+  }
+
+  // ///////////////////////////////////
+  // Comparable
+  // ///////////////////////////////////
+  /* (non-Javadoc)
+   * @see java.lang.Comparable#compareTo(java.lang.Object)
+   */
+  public int compareTo(Object o) {
+    return compareTo((Shard) o);
+  }
+
+  /**
+   * Compare to another shard.
+   * @param other  another shard
+   * @return compare version first, then directory and finally generation
+   */
+  public int compareTo(Shard other) {
+    // compare version
+    if (version < other.version) {
+      return -1;
+    } else if (version > other.version) {
+      return 1;
+    }
+    // compare dir
+    int result = dir.compareTo(other.dir);
+    if (result != 0) {
+      return result;
+    }
+    // compare gen
+    if (gen < other.gen) {
+      return -1;
+    } else if (gen == other.gen) {
+      return 0;
+    } else {
+      return 1;
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#equals(java.lang.Object)
+   */
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof Shard)) {
+      return false;
+    }
+    Shard other = (Shard) o;
+    return version == other.version && dir.equals(other.dir)
+        && gen == other.gen;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#hashCode()
+   */
+  public int hashCode() {
+    return (int) version ^ dir.hashCode() ^ (int) gen;
+  }
+
+}

+ 105 - 0
src/contrib/index/src/test/org/apache/hadoop/contrib/index/lucene/TestMixedDirectory.java

@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.lucene;
+
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexDeletionPolicy;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Hits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMDirectory;
+
+public class TestMixedDirectory extends TestCase {
+  private int numDocsPerUpdate = 10;
+  private int maxBufferedDocs = 2;
+
+  public void testMixedDirectoryAndPolicy() throws IOException {
+    Directory readDir = new RAMDirectory();
+    updateIndex(readDir, 0, numDocsPerUpdate,
+        new KeepOnlyLastCommitDeletionPolicy());
+
+    verify(readDir, numDocsPerUpdate);
+
+    IndexOutput out =
+        readDir.createOutput("_" + (numDocsPerUpdate / maxBufferedDocs + 2)
+            + ".cfs");
+    out.writeInt(0);
+    out.close();
+
+    Directory writeDir = new RAMDirectory();
+    Directory mixedDir = new MixedDirectory(readDir, writeDir);
+    updateIndex(mixedDir, numDocsPerUpdate, numDocsPerUpdate,
+        new MixedDeletionPolicy());
+
+    verify(readDir, numDocsPerUpdate);
+    verify(mixedDir, 2 * numDocsPerUpdate);
+  }
+
+  public void updateIndex(Directory dir, int base, int numDocs,
+      IndexDeletionPolicy policy) throws IOException {
+    IndexWriter writer =
+        new IndexWriter(dir, false, new StandardAnalyzer(), policy);
+    writer.setMaxBufferedDocs(maxBufferedDocs);
+    writer.setMergeFactor(1000);
+    for (int i = 0; i < numDocs; i++) {
+      addDoc(writer, base + i);
+    }
+    writer.close();
+  }
+
+  private void addDoc(IndexWriter writer, int id) throws IOException {
+    Document doc = new Document();
+    doc.add(new Field("id", String.valueOf(id), Field.Store.YES,
+        Field.Index.UN_TOKENIZED));
+    doc.add(new Field("content", "apache", Field.Store.NO,
+        Field.Index.TOKENIZED));
+    writer.addDocument(doc);
+  }
+
+  private void verify(Directory dir, int expectedHits) throws IOException {
+    IndexSearcher searcher = new IndexSearcher(dir);
+    Hits hits = searcher.search(new TermQuery(new Term("content", "apache")));
+    int numHits = hits.length();
+
+    assertEquals(expectedHits, numHits);
+
+    int[] docs = new int[numHits];
+    for (int i = 0; i < numHits; i++) {
+      Document hit = hits.doc(i);
+      docs[Integer.parseInt(hit.get("id"))]++;
+    }
+    for (int i = 0; i < numHits; i++) {
+      assertEquals(1, docs[i]);
+    }
+
+    searcher.close();
+  }
+
+}

+ 234 - 0
src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestDistributionPolicy.java

@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.NumberFormat;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.contrib.index.example.HashingDistributionPolicy;
+import org.apache.hadoop.contrib.index.example.RoundRobinDistributionPolicy;
+import org.apache.hadoop.contrib.index.lucene.FileSystemDirectory;
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Hits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+
+import junit.framework.TestCase;
+
+public class TestDistributionPolicy extends TestCase {
+
+  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
+  static {
+    NUMBER_FORMAT.setMinimumIntegerDigits(5);
+    NUMBER_FORMAT.setGroupingUsed(false);
+  }
+
+  // however, "we only allow 0 or 1 reducer in local mode" - from
+  // LocalJobRunner
+  private Configuration conf;
+  private Path localInputPath = new Path(System.getProperty("build.test") + "/sample/data.txt");
+  private Path localUpdatePath =
+      new Path(System.getProperty("build.test") + "/sample/data2.txt");
+  private Path inputPath = new Path("/myexample/data.txt");
+  private Path updatePath = new Path("/myexample/data2.txt");
+  private Path outputPath = new Path("/myoutput");
+  private Path indexPath = new Path("/myindex");
+  private int numShards = 3;
+  private int numMapTasks = 5;
+
+  private int numDataNodes = 3;
+  private int numTaskTrackers = 3;
+
+  private int numDocsPerRun = 10; // num of docs in local input path
+
+  private FileSystem fs;
+  private MiniDFSCluster dfsCluster;
+  private MiniMRCluster mrCluster;
+
+  public TestDistributionPolicy() throws IOException {
+    super();
+    if (System.getProperty("hadoop.log.dir") == null) {
+      String base = new File(".").getPath(); // getAbsolutePath();
+      System.setProperty("hadoop.log.dir", new Path(base).toString() + "/logs");
+    }
+    conf = new Configuration();
+  }
+
+  protected void setUp() throws Exception {
+    super.setUp();
+    try {
+      dfsCluster =
+          new MiniDFSCluster(conf, numDataNodes, true, (String[]) null);
+
+      fs = dfsCluster.getFileSystem();
+      if (fs.exists(inputPath)) {
+        fs.delete(inputPath);
+      }
+      fs.copyFromLocalFile(localInputPath, inputPath);
+      if (fs.exists(updatePath)) {
+        fs.delete(updatePath);
+      }
+      fs.copyFromLocalFile(localUpdatePath, updatePath);
+
+      if (fs.exists(outputPath)) {
+        // do not create, mapred will create
+        fs.delete(outputPath);
+      }
+
+      if (fs.exists(indexPath)) {
+        fs.delete(indexPath);
+      }
+
+      mrCluster =
+          new MiniMRCluster(numTaskTrackers, fs.getUri().toString(), 1);
+
+    } catch (IOException e) {
+      if (dfsCluster != null) {
+        dfsCluster.shutdown();
+        dfsCluster = null;
+      }
+
+      if (fs != null) {
+        fs.close();
+        fs = null;
+      }
+
+      if (mrCluster != null) {
+        mrCluster.shutdown();
+        mrCluster = null;
+      }
+
+      throw e;
+    }
+
+  }
+
+  protected void tearDown() throws Exception {
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+      dfsCluster = null;
+    }
+
+    if (fs != null) {
+      fs.close();
+      fs = null;
+    }
+
+    if (mrCluster != null) {
+      mrCluster.shutdown();
+      mrCluster = null;
+    }
+
+    super.tearDown();
+  }
+
+  public void testDistributionPolicy() throws IOException {
+    IndexUpdateConfiguration iconf = new IndexUpdateConfiguration(conf);
+
+    // test hashing distribution policy
+    iconf.setDistributionPolicyClass(HashingDistributionPolicy.class);
+    onetest();
+
+    if (fs.exists(indexPath)) {
+      fs.delete(indexPath);
+    }
+
+    // test round-robin distribution policy
+    iconf.setDistributionPolicyClass(RoundRobinDistributionPolicy.class);
+    onetest();
+  }
+
+  private void onetest() throws IOException {
+    long versionNumber = -1;
+    long generation = -1;
+
+    Shard[] shards = new Shard[numShards];
+    for (int j = 0; j < shards.length; j++) {
+      shards[j] =
+          new Shard(versionNumber,
+              new Path(indexPath, NUMBER_FORMAT.format(j)).toString(),
+              generation);
+    }
+
+    if (fs.exists(outputPath)) {
+      fs.delete(outputPath);
+    }
+
+    IIndexUpdater updater = new IndexUpdater();
+    updater.run(conf, new Path[] { inputPath }, outputPath, numMapTasks,
+        shards);
+
+    if (fs.exists(outputPath)) {
+      fs.delete(outputPath);
+    }
+
+    // delete docs w/ even docids, update docs w/ odd docids
+    updater.run(conf, new Path[] { updatePath }, outputPath, numMapTasks,
+        shards);
+
+    verify(shards);
+  }
+
+  private void verify(Shard[] shards) throws IOException {
+    // verify the index
+    IndexReader[] readers = new IndexReader[shards.length];
+    for (int i = 0; i < shards.length; i++) {
+      Directory dir =
+          new FileSystemDirectory(fs, new Path(shards[i].getDirectory()),
+              false, conf);
+      readers[i] = IndexReader.open(dir);
+    }
+
+    IndexReader reader = new MultiReader(readers);
+    IndexSearcher searcher = new IndexSearcher(reader);
+    Hits hits = searcher.search(new TermQuery(new Term("content", "apache")));
+    assertEquals(0, hits.length());
+
+    hits = searcher.search(new TermQuery(new Term("content", "hadoop")));
+    assertEquals(numDocsPerRun / 2, hits.length());
+
+    int[] counts = new int[numDocsPerRun];
+    for (int i = 0; i < hits.length(); i++) {
+      Document doc = hits.doc(i);
+      counts[Integer.parseInt(doc.get("id"))]++;
+    }
+
+    for (int i = 0; i < numDocsPerRun; i++) {
+      if (i % 2 == 0) {
+        assertEquals(0, counts[i]);
+      } else {
+        assertEquals(1, counts[i]);
+      }
+    }
+
+    searcher.close();
+    reader.close();
+  }
+
+}

+ 255 - 0
src/contrib/index/src/test/org/apache/hadoop/contrib/index/mapred/TestIndexUpdater.java

@@ -0,0 +1,255 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.contrib.index.mapred;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.NumberFormat;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.contrib.index.lucene.FileSystemDirectory;
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Hits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+
+import junit.framework.TestCase;
+
+public class TestIndexUpdater extends TestCase {
+
+  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
+  static {
+    NUMBER_FORMAT.setMinimumIntegerDigits(5);
+    NUMBER_FORMAT.setGroupingUsed(false);
+  }
+
+  // however, "we only allow 0 or 1 reducer in local mode" - from
+  // LocalJobRunner
+  private Configuration conf;
+  private Path localInputPath = new Path(System.getProperty("build.test") + "/sample/data.txt");
+  private Path inputPath = new Path("/myexample/data.txt");
+  private Path outputPath = new Path("/myoutput");
+  private Path indexPath = new Path("/myindex");
+  private int initNumShards = 3;
+  private int numMapTasks = 5;
+
+  private int numDataNodes = 3;
+  private int numTaskTrackers = 3;
+
+  private int numRuns = 3;
+  private int numDocsPerRun = 10; // num of docs in local input path
+
+  private FileSystem fs;
+  private MiniDFSCluster dfsCluster;
+  private MiniMRCluster mrCluster;
+
+  public TestIndexUpdater() throws IOException {
+    super();
+    if (System.getProperty("hadoop.log.dir") == null) {
+      String base = new File(".").getPath(); // getAbsolutePath();
+      System.setProperty("hadoop.log.dir", new Path(base).toString() + "/logs");
+    }
+    conf = new Configuration();
+  }
+
+  protected void setUp() throws Exception {
+    super.setUp();
+    try {
+      dfsCluster =
+          new MiniDFSCluster(conf, numDataNodes, true, (String[]) null);
+
+      fs = dfsCluster.getFileSystem();
+      if (fs.exists(inputPath)) {
+        fs.delete(inputPath);
+      }
+      fs.copyFromLocalFile(localInputPath, inputPath);
+
+      if (fs.exists(outputPath)) {
+        // do not create, mapred will create
+        fs.delete(outputPath);
+      }
+
+      if (fs.exists(indexPath)) {
+        fs.delete(indexPath);
+      }
+
+      mrCluster =
+          new MiniMRCluster(numTaskTrackers, fs.getUri().toString(), 1);
+
+    } catch (IOException e) {
+      if (dfsCluster != null) {
+        dfsCluster.shutdown();
+        dfsCluster = null;
+      }
+
+      if (fs != null) {
+        fs.close();
+        fs = null;
+      }
+
+      if (mrCluster != null) {
+        mrCluster.shutdown();
+        mrCluster = null;
+      }
+
+      throw e;
+    }
+
+  }
+
+  protected void tearDown() throws Exception {
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+      dfsCluster = null;
+    }
+
+    if (fs != null) {
+      fs.close();
+      fs = null;
+    }
+
+    if (mrCluster != null) {
+      mrCluster.shutdown();
+      mrCluster = null;
+    }
+
+    super.tearDown();
+  }
+
+  public void testIndexUpdater() throws IOException {
+    IndexUpdateConfiguration iconf = new IndexUpdateConfiguration(conf);
+    // max field length, compound file and number of segments will be checked
+    // later
+    iconf.setIndexMaxFieldLength(2);
+    iconf.setIndexUseCompoundFile(true);
+    iconf.setIndexMaxNumSegments(1);
+
+    long versionNumber = -1;
+    long generation = -1;
+
+    for (int i = 0; i < numRuns; i++) {
+      if (fs.exists(outputPath)) {
+        fs.delete(outputPath);
+      }
+
+      Shard[] shards = new Shard[initNumShards + i];
+      for (int j = 0; j < shards.length; j++) {
+        shards[j] =
+            new Shard(versionNumber, new Path(indexPath,
+                NUMBER_FORMAT.format(j)).toString(), generation);
+      }
+      run(i + 1, shards);
+    }
+  }
+
+  private void run(int numRuns, Shard[] shards) throws IOException {
+    IIndexUpdater updater = new IndexUpdater();
+    updater.run(conf, new Path[] { inputPath }, outputPath, numMapTasks,
+        shards);
+
+    // verify the done files
+    Path[] doneFileNames = new Path[shards.length];
+    int count = 0;
+    FileStatus[] fileStatus = fs.listStatus(outputPath);
+    for (int i = 0; i < fileStatus.length; i++) {
+      FileStatus[] doneFiles = fs.listStatus(fileStatus[i].getPath());
+      for (int j = 0; j < doneFiles.length; j++) {
+        doneFileNames[count++] = doneFiles[j].getPath();
+      }
+    }
+    assertEquals(shards.length, count);
+    for (int i = 0; i < count; i++) {
+      assertTrue(doneFileNames[i].getName().startsWith(
+          IndexUpdateReducer.DONE.toString()));
+    }
+
+    // verify the index
+    IndexReader[] readers = new IndexReader[shards.length];
+    for (int i = 0; i < shards.length; i++) {
+      Directory dir =
+          new FileSystemDirectory(fs, new Path(shards[i].getDirectory()),
+              false, conf);
+      readers[i] = IndexReader.open(dir);
+    }
+
+    IndexReader reader = new MultiReader(readers);
+    IndexSearcher searcher = new IndexSearcher(reader);
+    Hits hits = searcher.search(new TermQuery(new Term("content", "apache")));
+
+    assertEquals(numRuns * numDocsPerRun, hits.length());
+
+    int[] counts = new int[numDocsPerRun];
+    for (int i = 0; i < hits.length(); i++) {
+      Document doc = hits.doc(i);
+      counts[Integer.parseInt(doc.get("id"))]++;
+    }
+
+    for (int i = 0; i < numDocsPerRun; i++) {
+      assertEquals(numRuns, counts[i]);
+    }
+
+    // max field length is 2, so "dot" is also indexed but not "org"
+    hits = searcher.search(new TermQuery(new Term("content", "dot")));
+    assertEquals(numRuns, hits.length());
+
+    hits = searcher.search(new TermQuery(new Term("content", "org")));
+    assertEquals(0, hits.length());
+
+    searcher.close();
+    reader.close();
+
+    // open and close an index writer with KeepOnlyLastCommitDeletionPolicy
+    // to remove earlier checkpoints
+    for (int i = 0; i < shards.length; i++) {
+      Directory dir =
+          new FileSystemDirectory(fs, new Path(shards[i].getDirectory()),
+              false, conf);
+      IndexWriter writer =
+          new IndexWriter(dir, false, null,
+              new KeepOnlyLastCommitDeletionPolicy());
+      writer.close();
+    }
+
+    // verify the number of segments, must be done after an writer with
+    // KeepOnlyLastCommitDeletionPolicy so that earlier checkpoints are removed
+    for (int i = 0; i < shards.length; i++) {
+      PathFilter cfsFilter = new PathFilter() {
+        public boolean accept(Path path) {
+          return path.getName().endsWith(".cfs");
+        }
+      };
+      FileStatus[] cfsFiles =
+          fs.listStatus(new Path(shards[i].getDirectory()), cfsFilter);
+      assertEquals(1, cfsFiles.length);
+    }
+  }
+
+}