You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2008/07/18 11:20:14 UTC

svn commit: r677865 [1/5] - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/store/ src/java/org/apache/lucene/util/ src/test/org/apache/lucene/ src/test/org/apache/lucene/index/ src/test/org/apache/lucene/search/

Author: mikemccand
Date: Fri Jul 18 02:20:12 2008
New Revision: 677865

URL: http://svn.apache.org/viewvc?rev=677865&view=rev
Log:
LUCENE-1301: refactor DocumentsWriter into a package-private indexing chain, as a first step towards flexible indexing

Added:
    lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumerPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumer.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocInverter.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxFieldMergeState.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/IntBlockPool.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumer.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriter.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/NormsWriterPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/RawPostingList.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriter.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriterPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/TermsHash.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashConsumer.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashConsumerPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashPerField.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/index/TermsHashPerThread.java   (with props)
    lucene/java/trunk/src/java/org/apache/lucene/util/ArrayUtil.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java   (with props)
Removed:
    lucene/java/trunk/src/java/org/apache/lucene/index/AbortException.java
    lucene/java/trunk/src/java/org/apache/lucene/index/BufferedNorms.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriterFieldData.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriterFieldMergeState.java
    lucene/java/trunk/src/java/org/apache/lucene/index/Posting.java
    lucene/java/trunk/src/java/org/apache/lucene/index/PostingVector.java
Modified:
    lucene/java/trunk/LICENSE.txt
    lucene/java/trunk/src/java/org/apache/lucene/index/ByteBlockPool.java
    lucene/java/trunk/src/java/org/apache/lucene/index/ByteSliceReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/CharBlockPool.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriterThreadState.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/store/RAMOutputStream.java
    lucene/java/trunk/src/test/org/apache/lucene/TestDemo.java
    lucene/java/trunk/src/test/org/apache/lucene/TestSnapshotDeletionPolicy.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestDocumentWriter.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestStressIndexing2.java
    lucene/java/trunk/src/test/org/apache/lucene/search/TestTermVectors.java

Modified: lucene/java/trunk/LICENSE.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/LICENSE.txt?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/LICENSE.txt (original)
+++ lucene/java/trunk/LICENSE.txt Fri Jul 18 02:20:12 2008
@@ -229,3 +229,12 @@
  * for internal or external distribution as long as this notice
  * remains attached.
  */
+
+
+Some code in src/java/org/apache/lucene/util/ArrayUtil.java was
+derived from Python 2.4.2 sources available at
+http://www.python.org. Full license is here:
+
+  http://www.python.org/download/releases/2.4.2/license/
+
+

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/ByteBlockPool.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/ByteBlockPool.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/ByteBlockPool.java Fri Jul 18 02:20:12 2008
@@ -104,10 +104,10 @@
   }
 
   // Size of each slice.  These arrays should be at most 16
-  // elements.  First array is just a compact way to encode
-  // X+1 with a max.  Second array is the length of each
-  // slice, ie first slice is 5 bytes, next slice is 14
-  // bytes, etc.
+  // elements (index is encoded with 4 bits).  First array
+  // is just a compact way to encode X+1 with a max.  Second
+  // array is the length of each slice, ie first slice is 5
+  // bytes, next slice is 14 bytes, etc.
   final static int[] nextLevelArray = {1, 2, 3, 4, 5, 6, 7, 8, 9, 9};
   final static int[] levelSizeArray = {5, 14, 20, 30, 40, 40, 80, 80, 120, 200};
   final static int FIRST_LEVEL_SIZE = levelSizeArray[0];

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/ByteSliceReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/ByteSliceReader.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/ByteSliceReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/ByteSliceReader.java Fri Jul 18 02:20:12 2008
@@ -39,7 +39,9 @@
 
   public void init(ByteBlockPool pool, int startIndex, int endIndex) {
 
-    assert endIndex-startIndex > 0;
+    assert endIndex-startIndex >= 0;
+    assert startIndex >= 0;
+    assert endIndex >= 0;
 
     this.pool = pool;
     this.endIndex = endIndex;
@@ -59,9 +61,14 @@
       limit = upto+firstSize-4;
   }
 
+  public boolean eof() {
+    assert upto + bufferOffset <= endIndex;
+    return upto + bufferOffset == endIndex;
+  }
+
   public byte readByte() {
-    // Assert that we are not @ EOF
-    assert upto + bufferOffset < endIndex;
+    assert !eof();
+    assert upto <= limit;
     if (upto == limit)
       nextSlice();
     return buffer[upto++];

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/CharBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/CharBlockPool.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/CharBlockPool.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/CharBlockPool.java Fri Jul 18 02:20:12 2008
@@ -23,11 +23,11 @@
   int numBuffer;
 
   int bufferUpto = -1;                        // Which buffer we are upto
-  public int byteUpto = DocumentsWriter.CHAR_BLOCK_SIZE;             // Where we are in head buffer
+  public int charUpto = DocumentsWriter.CHAR_BLOCK_SIZE;             // Where we are in head buffer
 
   public char[] buffer;                              // Current head buffer
-  public int byteOffset = -DocumentsWriter.CHAR_BLOCK_SIZE;          // Current head offset
-  private DocumentsWriter docWriter;
+  public int charOffset = -DocumentsWriter.CHAR_BLOCK_SIZE;          // Current head offset
+  final private DocumentsWriter docWriter;
 
   public CharBlockPool(DocumentsWriter docWriter) {
     this.docWriter = docWriter;
@@ -36,8 +36,8 @@
   public void reset() {
     docWriter.recycleCharBlocks(buffers, 1+bufferUpto);
     bufferUpto = -1;
-    byteUpto = DocumentsWriter.CHAR_BLOCK_SIZE;
-    byteOffset = -DocumentsWriter.CHAR_BLOCK_SIZE;
+    charUpto = DocumentsWriter.CHAR_BLOCK_SIZE;
+    charOffset = -DocumentsWriter.CHAR_BLOCK_SIZE;
   }
 
   public void nextBuffer() {
@@ -49,8 +49,8 @@
     buffer = buffers[1+bufferUpto] = docWriter.getCharBlock();
     bufferUpto++;
 
-    byteUpto = 0;
-    byteOffset += DocumentsWriter.CHAR_BLOCK_SIZE;
+    charUpto = 0;
+    charOffset += DocumentsWriter.CHAR_BLOCK_SIZE;
   }
 }
 

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,29 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+
+abstract class DocConsumer {
+  abstract DocConsumerPerThread addThread(DocumentsWriterThreadState perThread) throws IOException;
+  abstract void flush(final Collection threads, final DocumentsWriter.FlushState state) throws IOException;
+  abstract void closeDocStore(final DocumentsWriter.FlushState state) throws IOException;
+  abstract void abort();
+  abstract boolean freeRAM();
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumerPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumerPerThread.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumerPerThread.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumerPerThread.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,33 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+
+abstract class DocConsumerPerThread {
+
+  /** Process the document. If there is
+   *  something for this document to be done in docID order,
+   *  you should encapsulate that as a
+   *  DocumentsWriter.DocWriter and return it.
+   *  DocumentsWriter then calls finish() on this object
+   *  when it's its turn. */
+  abstract DocumentsWriter.DocWriter processDocument() throws IOException;
+
+  abstract void abort();
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocConsumerPerThread.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumer.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumer.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumer.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,49 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Map;
+
+abstract class DocFieldConsumer {
+
+  FieldInfos fieldInfos;
+
+  /** Called when DocumentsWriter decides to create a new
+   *  segment */
+  abstract void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException;
+
+  /** Called when DocumentsWriter decides to close the doc
+   *  stores */
+  abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
+  
+  /** Called when an aborting exception is hit */
+  abstract void abort();
+
+  /** Add a new thread */
+  abstract DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException;
+
+  /** Called when DocumentsWriter is using too much RAM.
+   *  The consumer should free RAM, if possible, returning
+   *  true if any RAM was in fact freed. */
+  abstract boolean freeRAM();
+
+  void setFieldInfos(FieldInfos fieldInfos) {
+    this.fieldInfos = fieldInfos;
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,27 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import org.apache.lucene.document.Fieldable;
+
+abstract class DocFieldConsumerPerField {
+  /** Processes all occurrences of a single field */
+  abstract void processFields(Fieldable[] fields, int count) throws IOException;
+  abstract void abort();
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,27 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+
+abstract class DocFieldConsumerPerThread {
+  abstract void startDocument() throws IOException;
+  abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
+  abstract DocFieldConsumerPerField addField(FieldInfo fi);
+  abstract void abort();
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,162 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.util.HashMap;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.HashSet;
+import java.io.IOException;
+
+import org.apache.lucene.util.ArrayUtil;
+
+/** This is just a "splitter" class: it lets you wrap two
+ *  DocFieldConsumer instances as a single consumer. */
+
+final class DocFieldConsumers extends DocFieldConsumer {
+  final DocFieldConsumer one;
+  final DocFieldConsumer two;
+
+  public DocFieldConsumers(DocFieldConsumer one, DocFieldConsumer two) {
+    this.one = one;
+    this.two = two;
+  }
+
+  void setFieldInfos(FieldInfos fieldInfos) {
+    super.setFieldInfos(fieldInfos);
+    one.setFieldInfos(fieldInfos);
+    two.setFieldInfos(fieldInfos);
+  }
+
+  public void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException {
+
+    Map oneThreadsAndFields = new HashMap();
+    Map twoThreadsAndFields = new HashMap();
+
+    Iterator it = threadsAndFields.entrySet().iterator();
+    while(it.hasNext()) {
+
+      Map.Entry entry = (Map.Entry) it.next();
+
+      DocFieldConsumersPerThread perThread = (DocFieldConsumersPerThread) entry.getKey();
+
+      Collection fields = (Collection) entry.getValue();
+
+      Iterator fieldsIt = fields.iterator();
+      Collection oneFields = new HashSet();
+      Collection twoFields = new HashSet();
+      while(fieldsIt.hasNext()) {
+        DocFieldConsumersPerField perField = (DocFieldConsumersPerField) fieldsIt.next();
+        oneFields.add(perField.one);
+        twoFields.add(perField.two);
+      }
+
+      oneThreadsAndFields.put(perThread.one, oneFields);
+      twoThreadsAndFields.put(perThread.two, twoFields);
+    }
+    
+
+    one.flush(oneThreadsAndFields, state);
+    two.flush(twoThreadsAndFields, state);
+  }
+
+  public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {      
+    try {
+      one.closeDocStore(state);
+    } finally {
+      two.closeDocStore(state);
+    }
+  }
+
+  public void abort() {
+    try {
+      one.abort();
+    } finally {
+      two.abort();
+    }
+  }
+
+  public boolean freeRAM() {
+    boolean any = one.freeRAM();
+    any |= two.freeRAM();
+    return any;
+  }
+
+  public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException {
+    return new DocFieldConsumersPerThread(docFieldProcessorPerThread, this, one.addThread(docFieldProcessorPerThread), two.addThread(docFieldProcessorPerThread));
+  }
+
+  PerDoc[] docFreeList = new PerDoc[1];
+  int freeCount;
+  int allocCount;
+
+  synchronized PerDoc getPerDoc() {
+    if (freeCount == 0) {
+      allocCount++;
+      if (allocCount > docFreeList.length) {
+        // Grow our free list up front to make sure we have
+        // enough space to recycle all outstanding PerDoc
+        // instances
+        assert allocCount == 1+docFreeList.length;
+        docFreeList = new PerDoc[ArrayUtil.getNextSize(allocCount)];
+      }
+      return new PerDoc();
+    } else
+      return docFreeList[--freeCount];
+  }
+
+  synchronized void freePerDoc(PerDoc perDoc) {
+    assert freeCount < docFreeList.length;
+    docFreeList[freeCount++] = perDoc;
+  }
+
+  class PerDoc extends DocumentsWriter.DocWriter {
+
+    DocumentsWriter.DocWriter one;
+    DocumentsWriter.DocWriter two;
+
+    public long sizeInBytes() {
+      return one.sizeInBytes() + two.sizeInBytes();
+    }
+
+    public void finish() throws IOException {
+      try {
+        try {
+          one.finish();
+        } finally {
+          two.finish();
+        }
+      } finally {
+        freePerDoc(this);
+      }
+    }
+
+    public void abort() {
+      try {
+        try {
+          one.abort();
+        } finally {
+          two.abort();
+        }
+      } finally {
+        freePerDoc(this);
+      }
+    }
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumers.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,47 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import org.apache.lucene.document.Fieldable;
+
+final class DocFieldConsumersPerField extends DocFieldConsumerPerField {
+
+  final DocFieldConsumerPerField one;
+  final DocFieldConsumerPerField two;
+  final DocFieldConsumersPerThread perThread;
+
+  public DocFieldConsumersPerField(DocFieldConsumersPerThread perThread, DocFieldConsumerPerField one, DocFieldConsumerPerField two) {
+    this.perThread = perThread;
+    this.one = one;
+    this.two = two;
+  }
+
+  public void processFields(Fieldable[] fields, int count) throws IOException {
+    one.processFields(fields, count);
+    two.processFields(fields, count);
+  }
+
+  public void abort() {
+    try {
+      one.abort();
+    } finally {
+      two.abort();
+    }
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,71 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+
+final class DocFieldConsumersPerThread extends DocFieldConsumerPerThread {
+
+  final DocFieldConsumerPerThread one;
+  final DocFieldConsumerPerThread two;
+  final DocFieldConsumers parent;
+  final DocumentsWriter.DocState docState;
+
+  public DocFieldConsumersPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread,
+                                    DocFieldConsumers parent, DocFieldConsumerPerThread one, DocFieldConsumerPerThread two) {
+    this.parent = parent;
+    this.one = one;
+    this.two = two;
+    docState = docFieldProcessorPerThread.docState;
+  }
+
+  public void startDocument() throws IOException {
+    one.startDocument();
+    two.startDocument();
+  }
+
+  public void abort() {
+    try {
+      one.abort();
+    } finally {
+      two.abort();
+    }
+  }
+
+  public DocumentsWriter.DocWriter finishDocument() throws IOException {
+    final DocumentsWriter.DocWriter oneDoc = one.finishDocument();
+    final DocumentsWriter.DocWriter twoDoc = two.finishDocument();
+    if (oneDoc == null)
+      return twoDoc;
+    else if (twoDoc == null)
+      return oneDoc;
+    else {
+      DocFieldConsumers.PerDoc both = parent.getPerDoc();
+      both.docID = docState.docID;
+      assert oneDoc.docID == docState.docID;
+      assert twoDoc.docID == docState.docID;
+      both.one = oneDoc;
+      both.two = twoDoc;
+      return both;
+    }
+  }
+
+  public DocFieldConsumerPerField addField(FieldInfo fi) {
+    return new DocFieldConsumersPerField(this, one.addField(fi), two.addField(fi));
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,80 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Iterator;
+
+/**
+ * This is a DocConsumer that gathers all fields under the
+ * same name, and calls per-field consumers to process field
+ * by field.  This class doesn't doesn't do any "real" work
+ * of its own: it just forwards the fields to a
+ * DocFieldConsumer.
+ */
+
+final class DocFieldProcessor extends DocConsumer {
+
+  final DocumentsWriter docWriter;
+  final FieldInfos fieldInfos = new FieldInfos();
+  final DocFieldConsumer consumer;
+
+  public DocFieldProcessor(DocumentsWriter docWriter, DocFieldConsumer consumer) {
+    this.docWriter = docWriter;
+    this.consumer = consumer;
+    consumer.setFieldInfos(fieldInfos);
+  }
+
+  public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
+    consumer.closeDocStore(state);
+  }
+
+  public void flush(Collection threads, DocumentsWriter.FlushState state) throws IOException {
+
+    Map childThreadsAndFields = new HashMap();
+    Iterator it = threads.iterator();
+    while(it.hasNext()) {
+      DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) it.next();
+      childThreadsAndFields.put(perThread.consumer, perThread.fields());
+      perThread.trimFields(state);
+    }
+
+    consumer.flush(childThreadsAndFields, state);
+
+    // Important to save after asking consumer to flush so
+    // consumer can alter the FieldInfo* if necessary.  EG,
+    // FreqProxTermsWriter does this with
+    // FieldInfo.storePayload.
+    fieldInfos.write(state.directory, state.segmentName + ".fnm");
+  }
+
+  public void abort() {
+    consumer.abort();
+  }
+
+  public boolean freeRAM() {
+    return consumer.freeRAM();
+  }
+
+  public DocConsumerPerThread addThread(DocumentsWriterThreadState threadState) throws IOException {
+    return new DocFieldProcessorPerThread(threadState, this);
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,45 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import org.apache.lucene.document.Fieldable;
+
+/**
+ * Holds all per thread, per field state.
+ */
+
+final class DocFieldProcessorPerField {
+
+  final DocFieldConsumerPerField consumer;
+  final FieldInfo fieldInfo;
+
+  DocFieldProcessorPerField next;
+  int lastGen = -1;
+
+  int fieldCount;
+  Fieldable[] fields = new Fieldable[1];
+
+  public DocFieldProcessorPerField(final DocFieldProcessorPerThread perThread, final FieldInfo fieldInfo) {
+    this.consumer = perThread.consumer.addField(fieldInfo);
+    this.fieldInfo = fieldInfo;
+  }
+
+  public void abort() {
+    consumer.abort();
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,302 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.io.IOException;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Fieldable;
+
+/**
+ * Gathers all Fieldables for a document under the same
+ * name, updates FieldInfos, and calls per-field consumers
+ * to process field by field.
+ *
+ * Currently, only a single thread visits the fields,
+ * sequentially, for processing.
+ */
+
+final class DocFieldProcessorPerThread extends DocConsumerPerThread {
+
+  float docBoost;
+  int fieldGen;
+  final DocFieldProcessor docFieldProcessor;
+  final FieldInfos fieldInfos;
+  final DocFieldConsumerPerThread consumer;
+
+  // Holds all fields seen in current doc
+  DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1];
+  int fieldCount;
+
+  // Hash table for all fields ever seen
+  DocFieldProcessorPerField[] fieldHash = new DocFieldProcessorPerField[2];
+  int hashMask = 1;
+  int totalFieldCount;
+
+  final DocumentsWriter.DocState docState;
+  
+  public DocFieldProcessorPerThread(DocumentsWriterThreadState threadState, DocFieldProcessor docFieldProcessor) throws IOException {
+    this.docState = threadState.docState;
+    this.docFieldProcessor = docFieldProcessor;
+    this.fieldInfos = docFieldProcessor.fieldInfos;
+    this.consumer = docFieldProcessor.consumer.addThread(this);
+  }
+
+  public void abort() {
+    for(int i=0;i<fieldHash.length;i++) {
+      DocFieldProcessorPerField field = fieldHash[i];
+      while(field != null) {
+        final DocFieldProcessorPerField next = field.next;
+        field.abort();
+        field = next;
+      }
+    }
+    consumer.abort();
+  }
+
+  public Collection fields() {
+    Collection fields = new HashSet();
+    for(int i=0;i<fieldHash.length;i++) {
+      DocFieldProcessorPerField field = fieldHash[i];
+      while(field != null) {
+        fields.add(field.consumer);
+        field = field.next;
+      }
+    }
+    assert fields.size() == totalFieldCount;
+    return fields;
+  }
+
+  /** If there are fields we've seen but did not see again
+   *  in the last run, then free them up. */
+
+  void trimFields(DocumentsWriter.FlushState state) {
+
+    for(int i=0;i<fieldHash.length;i++) {
+      DocFieldProcessorPerField perField = fieldHash[i];
+      DocFieldProcessorPerField lastPerField = null;
+
+      while (perField != null) {
+
+        if (perField.lastGen == -1) {
+
+          // This field was not seen since the previous
+          // flush, so, free up its resources now
+
+          // Unhash
+          if (lastPerField == null)
+            fieldHash[i] = perField.next;
+          else
+            lastPerField.next = perField.next;
+
+          if (state.docWriter.infoStream != null)
+            state.docWriter.infoStream.println("  purge field=" + perField.fieldInfo.name);
+
+          totalFieldCount--;
+
+        } else {
+          // Reset
+          perField.lastGen = -1;
+          lastPerField = perField;
+        }
+
+        perField = perField.next;
+      }
+    }
+  }
+
+  private void rehash() {
+    final int newHashSize = (int) (fieldHash.length*2);
+    assert newHashSize > fieldHash.length;
+
+    final DocFieldProcessorPerField newHashArray[] = new DocFieldProcessorPerField[newHashSize];
+
+    // Rehash
+    int newHashMask = newHashSize-1;
+    for(int j=0;j<fieldHash.length;j++) {
+      DocFieldProcessorPerField fp0 = fieldHash[j];
+      while(fp0 != null) {
+        final int hashPos2 = fp0.fieldInfo.name.hashCode() & newHashMask;
+        DocFieldProcessorPerField nextFP0 = fp0.next;
+        fp0.next = newHashArray[hashPos2];
+        newHashArray[hashPos2] = fp0;
+        fp0 = nextFP0;
+      }
+    }
+
+    fieldHash = newHashArray;
+    hashMask = newHashMask;
+  }
+
+  public DocumentsWriter.DocWriter processDocument() throws IOException {
+
+    consumer.startDocument();
+    final Document doc = docState.doc;
+
+    assert docFieldProcessor.docWriter.writer.testPoint("DocumentsWriter.ThreadState.init start");
+
+    fieldCount = 0;
+    
+    final int thisFieldGen = fieldGen++;
+
+    final List docFields = doc.getFields();
+    final int numDocFields = docFields.size();
+
+    // Absorb any new fields first seen in this document.
+    // Also absorb any changes to fields we had already
+    // seen before (eg suddenly turning on norms or
+    // vectors, etc.):
+
+    for(int i=0;i<numDocFields;i++) {
+      Fieldable field = (Fieldable) docFields.get(i);
+      final String fieldName = field.name();
+
+      // Make sure we have a PerField allocated
+      final int hashPos = fieldName.hashCode() & hashMask;
+      DocFieldProcessorPerField fp = fieldHash[hashPos];
+      while(fp != null && !fp.fieldInfo.name.equals(fieldName))
+        fp = fp.next;
+
+      if (fp == null) {
+
+        // TODO FI: we need to genericize the "flags" that a
+        // field holds, and, how these flags are merged; it
+        // needs to be more "pluggable" such that if I want
+        // to have a new "thing" my Fields can do, I can
+        // easily add it
+        FieldInfo fi = fieldInfos.add(fieldName, field.isIndexed(), field.isTermVectorStored(),
+                                      field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
+                                      field.getOmitNorms(), false);
+
+        fp = new DocFieldProcessorPerField(this, fi);
+        fp.next = fieldHash[hashPos];
+        fieldHash[hashPos] = fp;
+        totalFieldCount++;
+
+        if (totalFieldCount >= fieldHash.length/2)
+          rehash();
+      } else
+        fp.fieldInfo.update(field.isIndexed(), field.isTermVectorStored(),
+                            field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
+                            field.getOmitNorms(), false);
+
+      if (thisFieldGen != fp.lastGen) {
+
+        // First time we're seeing this field for this doc
+        fp.fieldCount = 0;
+
+        if (fieldCount == fields.length) {
+          final int newSize = fields.length*2;
+          DocFieldProcessorPerField newArray[] = new DocFieldProcessorPerField[newSize];
+          System.arraycopy(fields, 0, newArray, 0, fieldCount);
+          fields = newArray;
+        }
+
+        fields[fieldCount++] = fp;
+        fp.lastGen = thisFieldGen;
+      }
+
+      if (fp.fieldCount == fp.fields.length) {
+        Fieldable[] newArray = new Fieldable[fp.fields.length*2];
+        System.arraycopy(fp.fields, 0, newArray, 0, fp.fieldCount);
+        fp.fields = newArray;
+      }
+
+      fp.fields[fp.fieldCount++] = field;
+    }
+
+    // If we are writing vectors then we must visit
+    // fields in sorted order so they are written in
+    // sorted order.  TODO: we actually only need to
+    // sort the subset of fields that have vectors
+    // enabled; we could save [small amount of] CPU
+    // here.
+    quickSort(fields, 0, fieldCount-1);
+
+    for(int i=0;i<fieldCount;i++)
+      fields[i].consumer.processFields(fields[i].fields, fields[i].fieldCount);
+
+    if (docState.maxTermPrefix != null && docState.infoStream != null)
+      docState.infoStream.println("WARNING: document contains at least one immense term (longer than the max length " + DocumentsWriter.MAX_TERM_LENGTH + "), all of which were skipped.  Please correct the analyzer to not produce such terms.  The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'"); 
+
+    return consumer.finishDocument();
+  }
+
+  void quickSort(DocFieldProcessorPerField[] array, int lo, int hi) {
+    if (lo >= hi)
+      return;
+    else if (hi == 1+lo) {
+      if (array[lo].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) {
+        final DocFieldProcessorPerField tmp = array[lo];
+        array[lo] = array[hi];
+        array[hi] = tmp;
+      }
+      return;
+    }
+
+    int mid = (lo + hi) >>> 1;
+
+    if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) {
+      DocFieldProcessorPerField tmp = array[lo];
+      array[lo] = array[mid];
+      array[mid] = tmp;
+    }
+
+    if (array[mid].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) {
+      DocFieldProcessorPerField tmp = array[mid];
+      array[mid] = array[hi];
+      array[hi] = tmp;
+
+      if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) {
+        DocFieldProcessorPerField tmp2 = array[lo];
+        array[lo] = array[mid];
+        array[mid] = tmp2;
+      }
+    }
+
+    int left = lo + 1;
+    int right = hi - 1;
+
+    if (left >= right)
+      return;
+
+    DocFieldProcessorPerField partition = array[mid];
+
+    for (; ;) {
+      while (array[right].fieldInfo.name.compareTo(partition.fieldInfo.name) > 0)
+        --right;
+
+      while (left < right && array[left].fieldInfo.name.compareTo(partition.fieldInfo.name) <= 0)
+        ++left;
+
+      if (left < right) {
+        DocFieldProcessorPerField tmp = array[left];
+        array[left] = array[right];
+        array[right] = tmp;
+        --right;
+      } else {
+        break;
+      }
+    }
+
+    quickSort(array, lo, left);
+    quickSort(array, left + 1, hi);
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocInverter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocInverter.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocInverter.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocInverter.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,109 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.util.Map;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Collection;
+import java.util.Iterator;
+import java.io.IOException;
+
+/** This is a DocFieldConsumer that inverts each field,
+ *  separately, from a Document, and accepts a
+ *  InvertedTermsConsumer to process those terms. */
+
+final class DocInverter extends DocFieldConsumer {
+
+  final InvertedDocConsumer consumer;
+  final InvertedDocEndConsumer endConsumer;
+
+  public DocInverter(InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) {
+    this.consumer = consumer;
+    this.endConsumer = endConsumer;
+  }
+
+  void setFieldInfos(FieldInfos fieldInfos) {
+    super.setFieldInfos(fieldInfos);
+    consumer.setFieldInfos(fieldInfos);
+    endConsumer.setFieldInfos(fieldInfos);
+  }
+
+  void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException {
+
+    Map childThreadsAndFields = new HashMap();
+    Map endChildThreadsAndFields = new HashMap();
+
+    Iterator it = threadsAndFields.entrySet().iterator();
+    while(it.hasNext()) {
+
+      Map.Entry entry = (Map.Entry) it.next();
+
+      DocInverterPerThread perThread = (DocInverterPerThread) entry.getKey();
+
+      Collection fields = (Collection) entry.getValue();
+
+      Iterator fieldsIt = fields.iterator();
+      Collection childFields = new HashSet();
+      Collection endChildFields = new HashSet();
+      while(fieldsIt.hasNext()) {
+        DocInverterPerField perField = (DocInverterPerField) fieldsIt.next();
+        childFields.add(perField.consumer);
+        endChildFields.add(perField.endConsumer);
+      }
+
+      childThreadsAndFields.put(perThread.consumer, childFields);
+      endChildThreadsAndFields.put(perThread.endConsumer, endChildFields);
+    }
+    
+    consumer.flush(childThreadsAndFields, state);
+    endConsumer.flush(endChildThreadsAndFields, state);
+  }
+
+  public void closeDocStore(DocumentsWriter.FlushState state) throws IOException {
+    consumer.closeDocStore(state);
+    endConsumer.closeDocStore(state);
+  }
+
+  void abort() {
+    consumer.abort();
+    endConsumer.abort();
+  }
+
+  public boolean freeRAM() {
+    return consumer.freeRAM();
+  }
+
+  public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) {
+    return new DocInverterPerThread(docFieldProcessorPerThread, this);
+  }
+
+  final static class FieldInvertState {
+    int position;
+    int length;
+    int offset;
+    float boost;
+
+    void reset(float docBoost) {
+      position = 0;
+      length = 0;
+      offset = 0;
+      boost = docBoost;
+    }
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocInverter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerField.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerField.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerField.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,181 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+
+/**
+ * Holds state for inverting all occurrences of a single
+ * field in the document.  This class doesn't do anything
+ * itself; instead, it forwards the tokens produced by
+ * analysis to its own consumer
+ * (InvertedDocConsumerPerField).  It also interacts with an
+ * endConsumer (InvertedDocEndConsumerPerField).
+ */
+
+final class DocInverterPerField extends DocFieldConsumerPerField {
+
+  final private DocInverterPerThread perThread;
+  final private FieldInfo fieldInfo;
+  final InvertedDocConsumerPerField consumer;
+  final InvertedDocEndConsumerPerField endConsumer;
+  final DocumentsWriter.DocState docState;
+  final DocInverter.FieldInvertState fieldState;
+
+  public DocInverterPerField(DocInverterPerThread perThread, FieldInfo fieldInfo) {
+    this.perThread = perThread;
+    this.fieldInfo = fieldInfo;
+    docState = perThread.docState;
+    fieldState = perThread.fieldState;
+    this.consumer = perThread.consumer.addField(this, fieldInfo);
+    this.endConsumer = perThread.endConsumer.addField(this, fieldInfo);
+  }
+
+  void abort() {
+    consumer.abort();
+    endConsumer.abort();
+  }
+
+  public void processFields(final Fieldable[] fields,
+                            final int count) throws IOException {
+
+    fieldState.reset(docState.doc.getBoost());
+
+    final int maxFieldLength = docState.maxFieldLength;
+
+    final boolean doInvert = consumer.start(fields, count);
+
+    for(int i=0;i<count;i++) {
+
+      final Fieldable field = fields[i];
+
+      // TODO FI: this should be "genericized" to querying
+      // consumer if it wants to see this particular field
+      // tokenized.
+      if (field.isIndexed() && doInvert) {
+
+        if (fieldState.length > 0)
+          fieldState.position += docState.analyzer.getPositionIncrementGap(fieldInfo.name);
+
+        if (!field.isTokenized()) {		  // un-tokenized field
+          String stringValue = field.stringValue();
+          final int valueLength = stringValue.length();
+          Token token = perThread.localToken;
+          token.clear();
+          char[] termBuffer = token.termBuffer();
+          if (termBuffer.length < valueLength)
+            termBuffer = token.resizeTermBuffer(valueLength);
+          stringValue.getChars(0, valueLength, termBuffer, 0);
+          token.setTermLength(valueLength);
+          token.setStartOffset(fieldState.offset);
+          token.setEndOffset(fieldState.offset + stringValue.length());
+          boolean success = false;
+          try {
+            consumer.add(token);
+            success = true;
+          } finally {
+            if (!success)
+              docState.docWriter.setAborting();
+          }
+          fieldState.offset += stringValue.length();
+          fieldState.length++;
+          fieldState.position++;
+        } else {                                  // tokenized field
+          final TokenStream stream;
+          final TokenStream streamValue = field.tokenStreamValue();
+
+          if (streamValue != null) 
+            stream = streamValue;
+          else {
+            // the field does not have a TokenStream,
+            // so we have to obtain one from the analyzer
+            final Reader reader;			  // find or make Reader
+            final Reader readerValue = field.readerValue();
+
+            if (readerValue != null)
+              reader = readerValue;
+            else {
+              String stringValue = field.stringValue();
+              if (stringValue == null)
+                throw new IllegalArgumentException("field must have either TokenStream, String or Reader value");
+              perThread.stringReader.init(stringValue);
+              reader = perThread.stringReader;
+            }
+          
+            // Tokenize field and add to postingTable
+            stream = docState.analyzer.reusableTokenStream(fieldInfo.name, reader);
+          }
+
+          // reset the TokenStream to the first token
+          stream.reset();
+
+          try {
+            int offsetEnd = fieldState.offset-1;
+            final Token localToken = perThread.localToken;
+            for(;;) {
+
+              // If we hit an exception in stream.next below
+              // (which is fairly common, eg if analyzer
+              // chokes on a given document), then it's
+              // non-aborting and (above) this one document
+              // will be marked as deleted, but still
+              // consume a docID
+              Token token = stream.next(localToken);
+
+              if (token == null) break;
+              fieldState.position += (token.getPositionIncrement() - 1);
+              boolean success = false;
+              try {
+                // If we hit an exception in here, we abort
+                // all buffered documents since the last
+                // flush, on the likelihood that the
+                // internal state of the consumer is now
+                // corrupt and should not be flushed to a
+                // new segment:
+                consumer.add(token);
+                success = true;
+              } finally {
+                if (!success)
+                  docState.docWriter.setAborting();
+              }
+              fieldState.position++;
+              offsetEnd = fieldState.offset + token.endOffset();
+              if (++fieldState.length >= maxFieldLength) {
+                if (docState.infoStream != null)
+                  docState.infoStream.println("maxFieldLength " +maxFieldLength+ " reached for field " + fieldInfo.name + ", ignoring following tokens");
+                break;
+              }
+            }
+            fieldState.offset = offsetEnd+1;
+          } finally {
+            stream.close();
+          }
+        }
+
+        fieldState.boost *= field.getBoost();
+      }
+    }
+
+    consumer.finish();
+    endConsumer.finish();
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerField.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerThread.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerThread.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerThread.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,70 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.Token;
+
+/** This is a DocFieldConsumer that inverts each field,
+ *  separately, from a Document, and accepts a
+ *  InvertedTermsConsumer to process those terms. */
+
+final class DocInverterPerThread extends DocFieldConsumerPerThread {
+  final DocInverter docInverter;
+  final InvertedDocConsumerPerThread consumer;
+  final InvertedDocEndConsumerPerThread endConsumer;
+  final Token localToken = new Token();
+  final DocumentsWriter.DocState docState;
+
+  final DocInverter.FieldInvertState fieldState = new DocInverter.FieldInvertState();
+
+  // Used to read a string value for a field
+  final ReusableStringReader stringReader = new ReusableStringReader();
+
+  public DocInverterPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread, DocInverter docInverter) {
+    this.docInverter = docInverter;
+    docState = docFieldProcessorPerThread.docState;
+    consumer = docInverter.consumer.addThread(this);
+    endConsumer = docInverter.endConsumer.addThread(this);
+  }
+
+  public void startDocument() throws IOException {
+    consumer.startDocument();
+    endConsumer.startDocument();
+  }
+
+  public DocumentsWriter.DocWriter finishDocument() throws IOException {
+    // TODO: allow endConsumer.finishDocument to also return
+    // a DocWriter
+    endConsumer.finishDocument();
+    return consumer.finishDocument();
+  }
+
+  void abort() {
+    try {
+      consumer.abort();
+    } finally {
+      endConsumer.abort();
+    }
+  }
+
+  public DocFieldConsumerPerField addField(FieldInfo fi) {
+    return new DocInverterPerField(this, fi);
+  }
+}

Propchange: lucene/java/trunk/src/java/org/apache/lucene/index/DocInverterPerThread.java
------------------------------------------------------------------------------
    svn:eol-style = native