Index: src/java/org/apache/lucene/index/DocInverterPerField.java
===================================================================
--- src/java/org/apache/lucene/index/DocInverterPerField.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocInverterPerField.java	(working copy)
@@ -35,20 +35,20 @@
 
 final class DocInverterPerField extends DocFieldConsumerPerField {
 
-  final private DocInverterPerThread perThread;
-  final private FieldInfo fieldInfo;
+  final private DocInverter parent;
+  final FieldInfo fieldInfo;
   final InvertedDocConsumerPerField consumer;
   final InvertedDocEndConsumerPerField endConsumer;
   final DocumentsWriter.DocState docState;
   final FieldInvertState fieldState;
 
-  public DocInverterPerField(DocInverterPerThread perThread, FieldInfo fieldInfo) {
-    this.perThread = perThread;
+  public DocInverterPerField(DocInverter parent, FieldInfo fieldInfo) {
+    this.parent = parent;
     this.fieldInfo = fieldInfo;
-    docState = perThread.docState;
-    fieldState = perThread.fieldState;
-    this.consumer = perThread.consumer.addField(this, fieldInfo);
-    this.endConsumer = perThread.endConsumer.addField(this, fieldInfo);
+    docState = parent.docState;
+    fieldState = parent.fieldState;
+    this.consumer = parent.consumer.addField(this, fieldInfo);
+    this.endConsumer = parent.endConsumer.addField(this, fieldInfo);
   }
 
   @Override
@@ -84,8 +84,8 @@
         if (!field.isTokenized()) {		  // un-tokenized field
           String stringValue = field.stringValue();
           final int valueLength = stringValue.length();
-          perThread.singleToken.reinit(stringValue, 0, valueLength);
-          fieldState.attributeSource = perThread.singleToken;
+          parent.singleToken.reinit(stringValue, 0, valueLength);
+          fieldState.attributeSource = parent.singleToken;
           consumer.start(field);
 
           boolean success = false;
@@ -119,8 +119,8 @@
               if (stringValue == null) {
                 throw new IllegalArgumentException("field must have either TokenStream, String or Reader value");
               }
-              perThread.stringReader.init(stringValue);
-              reader = perThread.stringReader;
+              parent.stringReader.init(stringValue);
+              reader = parent.stringReader;
             }
           
             // Tokenize field and add to postingTable
@@ -208,4 +208,9 @@
     consumer.finish();
     endConsumer.finish();
   }
+
+  @Override
+  FieldInfo getFieldInfo() {
+    return this.fieldInfo;
+  }
 }
Index: src/java/org/apache/lucene/index/NormsWriterPerField.java
===================================================================
--- src/java/org/apache/lucene/index/NormsWriterPerField.java	(revision 934193)
+++ src/java/org/apache/lucene/index/NormsWriterPerField.java	(working copy)
@@ -27,7 +27,6 @@
 
 final class NormsWriterPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsWriterPerField> {
 
-  final NormsWriterPerThread perThread;
   final FieldInfo fieldInfo;
   final DocumentsWriter.DocState docState;
 
@@ -45,10 +44,9 @@
     upto = 0;
   }
 
-  public NormsWriterPerField(final DocInverterPerField docInverterPerField, final NormsWriterPerThread perThread, final FieldInfo fieldInfo) {
-    this.perThread = perThread;
+  public NormsWriterPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
     this.fieldInfo = fieldInfo;
-    docState = perThread.docState;
+    docState = docInverterPerField.docState;
     fieldState = docInverterPerField.fieldState;
   }
 
Index: src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java	(working copy)
@@ -1,27 +0,0 @@
-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 TermsHashConsumerPerThread {
-  abstract void startDocument() throws IOException;
-  abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
-  abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo);
-  abstract public void abort();
-}
Index: src/java/org/apache/lucene/index/InvertedDocConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocConsumer.java	(revision 934193)
+++ src/java/org/apache/lucene/index/InvertedDocConsumer.java	(working copy)
@@ -17,24 +17,26 @@
  * limitations under the License.
  */
 
-import java.util.Collection;
+import java.io.IOException;
 import java.util.Map;
-import java.io.IOException;
 
 abstract class InvertedDocConsumer {
 
-  /** Add a new thread */
-  abstract InvertedDocConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
-
   /** Abort (called after hitting AbortException) */
   abstract void abort();
 
   /** Flush a new segment */
-  abstract void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
+  abstract void flush(Map<FieldInfo, InvertedDocConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
 
   /** Close doc stores */
   abstract void closeDocStore(SegmentWriteState state) throws IOException;
 
+  abstract InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
+  
+  abstract void startDocument() throws IOException;
+  
+  abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
+  
   /** Attempt to free RAM, returning true if any RAM was
    *  freed */
   abstract boolean freeRAM();
Index: src/java/org/apache/lucene/index/DocFieldConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumer.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocFieldConsumer.java	(working copy)
@@ -18,7 +18,6 @@
  */
 
 import java.io.IOException;
-import java.util.Collection;
 import java.util.Map;
 
 abstract class DocFieldConsumer {
@@ -27,7 +26,7 @@
 
   /** Called when DocumentsWriter decides to create a new
    *  segment */
-  abstract void flush(Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
+  abstract void flush(Map<FieldInfo, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
 
   /** Called when DocumentsWriter decides to close the doc
    *  stores */
@@ -36,14 +35,17 @@
   /** 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();
+  
+  abstract void startDocument() throws IOException;
 
+  abstract DocFieldConsumerPerField addField(FieldInfo fi);
+  
+  abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
+  
   void setFieldInfos(FieldInfos fieldInfos) {
     this.fieldInfos = fieldInfos;
   }
Index: src/java/org/apache/lucene/index/DocFieldConsumerPerField.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumerPerField.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocFieldConsumerPerField.java	(working copy)
@@ -24,4 +24,5 @@
   /** Processes all occurrences of a single field */
   abstract void processFields(Fieldable[] fields, int count) throws IOException;
   abstract void abort();
+  abstract FieldInfo getFieldInfo();
 }
Index: src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java	(working copy)
@@ -1,393 +0,0 @@
-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;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
-
-/**
- * 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 StoredFieldsWriterPerThread fieldsWriter;
-
-  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);
-    fieldsWriter = docFieldProcessor.fieldsWriter.addThread(docState);
-  }
-
-  @Override
-  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;
-      }
-    }
-    fieldsWriter.abort();
-    consumer.abort();
-  }
-
-  public Collection<DocFieldConsumerPerField> fields() {
-    Collection<DocFieldConsumerPerField> fields = new HashSet<DocFieldConsumerPerField>();
-    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(SegmentWriteState 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.infoStream != null) {
-            state.infoStream.println("  purge field=" + perField.fieldInfo.name);
-          }
-
-          totalFieldCount--;
-
-        } else {
-          // Reset
-          perField.lastGen = -1;
-          lastPerField = perField;
-        }
-
-        perField = perField.next;
-      }
-    }
-  }
-
-  private void rehash() {
-    final int newHashSize = (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;
-  }
-
-  @Override
-  public DocumentsWriter.DocWriter processDocument() throws IOException {
-
-    consumer.startDocument();
-    fieldsWriter.startDocument();
-
-    final Document doc = docState.doc;
-
-    assert docFieldProcessor.docWriter.writer.testPoint("DocumentsWriter.ThreadState.init start");
-
-    fieldCount = 0;
-    
-    final int thisFieldGen = fieldGen++;
-
-    final List<Fieldable> 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 = 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, field.getOmitTermFreqAndPositions());
-
-        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, field.getOmitTermFreqAndPositions());
-
-      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 (field.isStored()) {
-        fieldsWriter.addField(field, fp.fieldInfo);
-      }
-    }
-
-    // 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 (whose UTF8 encoding is longer than the max length " + DocumentsWriter.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped.  Please correct the analyzer to not produce such terms.  The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'"); 
-      docState.maxTermPrefix = null;
-    }
-
-    final DocumentsWriter.DocWriter one = fieldsWriter.finishDocument();
-    final DocumentsWriter.DocWriter two = consumer.finishDocument();
-    if (one == null) {
-      return two;
-    } else if (two == null) {
-      return one;
-    } else {
-      PerDoc both = getPerDoc();
-      both.docID = docState.docID;
-      assert one.docID == docState.docID;
-      assert two.docID == docState.docID;
-      both.one = one;
-      both.two = two;
-      return both;
-    }
-  }
-
-  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);
-  }
-
-  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.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      }
-      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;
-
-    @Override
-    public long sizeInBytes() {
-      return one.sizeInBytes() + two.sizeInBytes();
-    }
-
-    @Override
-    public void finish() throws IOException {
-      try {
-        try {
-          one.finish();
-        } finally {
-          two.finish();
-        }
-      } finally {
-        freePerDoc(this);
-      }
-    }
-
-    @Override
-    public void abort() {
-      try {
-        try {
-          one.abort();
-        } finally {
-          two.abort();
-        }
-      } finally {
-        freePerDoc(this);
-      }
-    }
-  }
-}
\ No newline at end of file
Index: src/java/org/apache/lucene/index/DocFieldConsumers.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumers.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocFieldConsumers.java	(working copy)
@@ -17,12 +17,9 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
 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;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -33,10 +30,12 @@
 final class DocFieldConsumers extends DocFieldConsumer {
   final DocFieldConsumer one;
   final DocFieldConsumer two;
+  final DocumentsWriter.DocState docState;
 
-  public DocFieldConsumers(DocFieldConsumer one, DocFieldConsumer two) {
+  public DocFieldConsumers(DocFieldProcessor processor, DocFieldConsumer one, DocFieldConsumer two) {
     this.one = one;
     this.two = two;
+    this.docState = processor.docState;
   }
 
   @Override
@@ -47,33 +46,19 @@
   }
 
   @Override
-  public void flush(Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
+  public void flush(Map<FieldInfo, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
 
-    Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> oneThreadsAndFields = new HashMap<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>>();
-    Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> twoThreadsAndFields = new HashMap<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>>();
+    Map<FieldInfo, DocFieldConsumerPerField> oneFieldsToFlush = new HashMap<FieldInfo, DocFieldConsumerPerField>();
+    Map<FieldInfo, DocFieldConsumerPerField> twoFieldsToFlush = new HashMap<FieldInfo, DocFieldConsumerPerField>();
 
-    for (Map.Entry<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> entry : threadsAndFields.entrySet()) {
-
-      final DocFieldConsumersPerThread perThread = (DocFieldConsumersPerThread) entry.getKey();
-
-      final Collection<DocFieldConsumerPerField> fields = entry.getValue();
-
-      Iterator<DocFieldConsumerPerField> fieldsIt = fields.iterator();
-      Collection<DocFieldConsumerPerField> oneFields = new HashSet<DocFieldConsumerPerField>();
-      Collection<DocFieldConsumerPerField> twoFields = new HashSet<DocFieldConsumerPerField>();
-      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);
+    for (Map.Entry<FieldInfo, DocFieldConsumerPerField> fieldToFlush : fieldsToFlush.entrySet()) {
+      DocFieldConsumersPerField perField = (DocFieldConsumersPerField) fieldToFlush.getValue();
+      oneFieldsToFlush.put(fieldToFlush.getKey(), perField.one);
+      twoFieldsToFlush.put(fieldToFlush.getKey(), perField.two);
     }
-    
 
-    one.flush(oneThreadsAndFields, state);
-    two.flush(twoThreadsAndFields, state);
+    one.flush(oneFieldsToFlush, state);
+    two.flush(twoFieldsToFlush, state);
   }
 
   @Override
@@ -101,11 +86,6 @@
     return any;
   }
 
-  @Override
-  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;
@@ -166,4 +146,35 @@
       }
     }
   }
+  
+  @Override
+  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 = getPerDoc();
+      both.docID = docState.docID;
+      assert oneDoc.docID == docState.docID;
+      assert twoDoc.docID == docState.docID;
+      both.writerOne = oneDoc;
+      both.writerTwo = twoDoc;
+      return both;
+    }
+  }
+  
+  @Override
+  public void startDocument() throws IOException {
+    one.startDocument();
+    two.startDocument();
+  }
+  
+  @Override
+  public DocFieldConsumerPerField addField(FieldInfo fi) {
+    return new DocFieldConsumersPerField(this, fi, one.addField(fi), two.addField(fi));
+  }
+
 }
Index: src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocEndConsumer.java	(revision 934193)
+++ src/java/org/apache/lucene/index/InvertedDocEndConsumer.java	(working copy)
@@ -17,14 +17,15 @@
  * limitations under the License.
  */
 
-import java.util.Collection;
+import java.io.IOException;
 import java.util.Map;
-import java.io.IOException;
 
 abstract class InvertedDocEndConsumer {
-  abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
-  abstract void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
+  abstract void flush(Map<FieldInfo, InvertedDocEndConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
   abstract void closeDocStore(SegmentWriteState state) throws IOException;
   abstract void abort();
   abstract void setFieldInfos(FieldInfos fieldInfos);
+  abstract InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
+  abstract void startDocument() throws IOException;
+  abstract void finishDocument() throws IOException;
 }
Index: src/java/org/apache/lucene/index/TermsHashConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHashConsumer.java	(revision 934193)
+++ src/java/org/apache/lucene/index/TermsHashConsumer.java	(working copy)
@@ -18,15 +18,16 @@
  */
 
 import java.io.IOException;
-import java.util.Collection;
 import java.util.Map;
 
 abstract class TermsHashConsumer {
-  abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
-  abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
+  abstract void flush(Map<FieldInfo, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException;
   abstract void abort();
   abstract void closeDocStore(SegmentWriteState state) throws IOException;
 
+  abstract void startDocument() throws IOException;
+  abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
+  abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo);
   FieldInfos fieldInfos;
 
   void setFieldInfos(FieldInfos fieldInfos) {
Index: src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java	(working copy)
@@ -1,79 +0,0 @@
-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.store.IndexOutput;
-import org.apache.lucene.document.Fieldable;
-
-final class StoredFieldsWriterPerThread {
-
-  final FieldsWriter localFieldsWriter;
-  final StoredFieldsWriter storedFieldsWriter;
-  final DocumentsWriter.DocState docState;
-
-  StoredFieldsWriter.PerDoc doc;
-
-  public StoredFieldsWriterPerThread(DocumentsWriter.DocState docState, StoredFieldsWriter storedFieldsWriter) throws IOException {
-    this.storedFieldsWriter = storedFieldsWriter;
-    this.docState = docState;
-    localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null, storedFieldsWriter.fieldInfos);
-  }
-
-  public void startDocument() {
-    if (doc != null) {
-      // Only happens if previous document hit non-aborting
-      // exception while writing stored fields into
-      // localFieldsWriter:
-      doc.reset();
-      doc.docID = docState.docID;
-    }
-  }
-
-  public void addField(Fieldable field, FieldInfo fieldInfo) throws IOException {
-    if (doc == null) {
-      doc = storedFieldsWriter.getPerDoc();
-      doc.docID = docState.docID;
-      localFieldsWriter.setFieldsStream(doc.fdt);
-      assert doc.numStoredFields == 0: "doc.numStoredFields=" + doc.numStoredFields;
-      assert 0 == doc.fdt.length();
-      assert 0 == doc.fdt.getFilePointer();
-    }
-
-    localFieldsWriter.writeField(fieldInfo, field);
-    assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField");
-    doc.numStoredFields++;
-  }
-
-  public DocumentsWriter.DocWriter finishDocument() {
-    // If there were any stored fields in this doc, doc will
-    // be non-null; else it's null.
-    try {
-      return doc;
-    } finally {
-      doc = null;
-    }
-  }
-
-  public void abort() {
-    if (doc != null) {
-      doc.abort();
-      doc = null;
-    }
-  }
-}
Index: src/java/org/apache/lucene/index/DocFieldConsumersPerField.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumersPerField.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocFieldConsumersPerField.java	(working copy)
@@ -24,12 +24,14 @@
 
   final DocFieldConsumerPerField one;
   final DocFieldConsumerPerField two;
-  final DocFieldConsumersPerThread perThread;
+  final DocFieldConsumers parent;
+  final FieldInfo fieldInfo;
 
-  public DocFieldConsumersPerField(DocFieldConsumersPerThread perThread, DocFieldConsumerPerField one, DocFieldConsumerPerField two) {
-    this.perThread = perThread;
+  public DocFieldConsumersPerField(DocFieldConsumers parent, FieldInfo fi, DocFieldConsumerPerField one, DocFieldConsumerPerField two) {
+    this.parent = parent;
     this.one = one;
     this.two = two;
+    this.fieldInfo = fi;
   }
 
   @Override
@@ -46,4 +48,9 @@
       two.abort();
     }
   }
+
+  @Override
+  FieldInfo getFieldInfo() {
+    return fieldInfo;
+  }
 }
Index: src/java/org/apache/lucene/index/StoredFieldsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/StoredFieldsWriter.java	(revision 934193)
+++ src/java/org/apache/lucene/index/StoredFieldsWriter.java	(working copy)
@@ -18,6 +18,9 @@
  */
 
 import java.io.IOException;
+
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -26,22 +29,35 @@
 final class StoredFieldsWriter {
 
   FieldsWriter fieldsWriter;
+  final FieldsWriter localFieldsWriter;
   final DocumentsWriter docWriter;
   final FieldInfos fieldInfos;
   int lastDocID;
 
   PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;
+  
+  PerDoc doc;
+  final DocumentsWriter.DocState docState;
 
   public StoredFieldsWriter(DocumentsWriter docWriter, FieldInfos fieldInfos) {
     this.docWriter = docWriter;
     this.fieldInfos = fieldInfos;
+    this.docState = docWriter.docState;
+    localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null, fieldInfos);
   }
 
-  public StoredFieldsWriterPerThread addThread(DocumentsWriter.DocState docState) throws IOException {
-    return new StoredFieldsWriterPerThread(docState, this);
+  public void startDocument() {
+    if (doc != null) {
+      // Only happens if previous document hit non-aborting
+      // exception while writing stored fields into
+      // localFieldsWriter:
+      doc.reset();
+      doc.docID = docState.docID;
+    }
   }
 
+
   synchronized public void flush(SegmentWriteState state) throws IOException {
 
     if (state.numDocsInStore > 0) {
@@ -115,7 +131,22 @@
       return docFreeList[--freeCount];
   }
 
+  public DocumentsWriter.DocWriter finishDocument() {
+    // If there were any stored fields in this doc, doc will
+    // be non-null; else it's null.
+    try {
+      return doc;
+    } finally {
+      doc = null;
+    }
+  }
+
   synchronized void abort() {
+    if (doc != null) {
+      doc.abort();
+      doc = null;
+    }
+
     if (fieldsWriter != null) {
       try {
         fieldsWriter.close();
@@ -153,6 +184,21 @@
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
   }
 
+  public void addField(Fieldable field, FieldInfo fieldInfo) throws IOException {
+    if (doc == null) {
+      doc = getPerDoc();
+      doc.docID = docState.docID;
+      localFieldsWriter.setFieldsStream(doc.fdt);
+      assert doc.numStoredFields == 0: "doc.numStoredFields=" + doc.numStoredFields;
+      assert 0 == doc.fdt.length();
+      assert 0 == doc.fdt.getFilePointer();
+    }
+
+    localFieldsWriter.writeField(fieldInfo, field);
+    assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField");
+    doc.numStoredFields++;
+  }
+  
   public boolean freeRAM() {
     return false;
   }
@@ -177,7 +223,7 @@
     }
 
     @Override
-    void abort() {
+    public void abort() {
       reset();
       free(this);
     }
Index: src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java	(working copy)
@@ -1,27 +0,0 @@
-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();
-}
Index: src/java/org/apache/lucene/index/DocConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocConsumerPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocConsumerPerThread.java	(working copy)
@@ -1,33 +0,0 @@
-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();
-}
Index: src/java/org/apache/lucene/index/FreqProxFieldMergeState.java
===================================================================
--- src/java/org/apache/lucene/index/FreqProxFieldMergeState.java	(revision 934193)
+++ src/java/org/apache/lucene/index/FreqProxFieldMergeState.java	(working copy)
@@ -1,113 +0,0 @@
-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.Comparator;
-import org.apache.lucene.util.BytesRef;
-
-import org.apache.lucene.index.FreqProxTermsWriterPerField.FreqProxPostingsArray;
-
-// TODO FI: some of this is "generic" to TermsHash* so we
-// should factor it out so other consumers don't have to
-// duplicate this code
-
-/** Used by DocumentsWriter to merge the postings from
- *  multiple ThreadStates when creating a segment */
-final class FreqProxFieldMergeState {
-
-  final FreqProxTermsWriterPerField field;
-  final int numPostings;
-  private final ByteBlockPool bytePool;
-  final int[] termIDs;
-  final FreqProxPostingsArray postings;
-  int currentTermID;
-  
-  final BytesRef text = new BytesRef();
-
-  private int postingUpto = -1;
-
-  final ByteSliceReader freq = new ByteSliceReader();
-  final ByteSliceReader prox = new ByteSliceReader();
-
-  int docID;
-  int termFreq;
-
-  public FreqProxFieldMergeState(FreqProxTermsWriterPerField field, Comparator<BytesRef> termComp) {
-    this.field = field;
-    this.numPostings = field.termsHashPerField.numPostings;
-    this.bytePool = field.perThread.termsHashPerThread.bytePool;
-    this.termIDs = field.termsHashPerField.sortPostings(termComp);
-    this.postings = (FreqProxPostingsArray) field.termsHashPerField.postingsArray;
-  }
-
-  boolean nextTerm() throws IOException {
-    postingUpto++;
-    if (postingUpto == numPostings) {
-      return false;
-    }
-
-    currentTermID = termIDs[postingUpto];
-    docID = 0;
-
-    // Get BytesRef
-    final int textStart = postings.textStarts[currentTermID];
-    bytePool.setBytesRef(text, textStart);
-
-    field.termsHashPerField.initReader(freq, currentTermID, 0);
-    if (!field.fieldInfo.omitTermFreqAndPositions) {
-      field.termsHashPerField.initReader(prox, currentTermID, 1);
-    }
-
-    // Should always be true
-    boolean result = nextDoc();
-    assert result;
-
-    return true;
-  }
-
-  public boolean nextDoc() throws IOException {
-    if (freq.eof()) {
-      if (postings.lastDocCodes[currentTermID] != -1) {
-        // Return last doc
-        docID = postings.lastDocIDs[currentTermID];
-        if (!field.omitTermFreqAndPositions)
-          termFreq = postings.docFreqs[currentTermID];
-        postings.lastDocCodes[currentTermID] = -1;
-        return true;
-      } else
-        // EOF
-        return false;
-    }
-
-    final int code = freq.readVInt();
-    if (field.omitTermFreqAndPositions)
-      docID += code;
-    else {
-      docID += code >>> 1;
-      if ((code & 1) != 0)
-        termFreq = 1;
-      else
-        termFreq = freq.readVInt();
-    }
-
-    assert docID != postings.lastDocIDs[currentTermID];
-
-    return true;
-  }
-}
Index: src/java/org/apache/lucene/index/TermsHash.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHash.java	(revision 934193)
+++ src/java/org/apache/lucene/index/TermsHash.java	(working copy)
@@ -18,12 +18,15 @@
  */
 
 import java.io.IOException;
-import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 
+import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.index.DocumentsWriter.DocWriter;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.UnicodeUtil;
+
 /** This class implements {@link InvertedDocConsumer}, which
  *  is passed each token produced by the analyzer on each
  *  field.  It stores these tokens in a hash table, and
@@ -37,25 +40,61 @@
   final TermsHashConsumer consumer;
   final TermsHash nextTermsHash;
   final DocumentsWriter docWriter;
+  
+  final IntBlockPool intPool;
+  final ByteBlockPool bytePool;
+  ByteBlockPool termBytePool;
 
+  final boolean primary;
+  final DocumentsWriter.DocState docState;
+
+  // Used when comparing postings via termRefComp, in TermsHashPerField
+  final BytesRef tr1 = new BytesRef();
+  final BytesRef tr2 = new BytesRef();
+
+  // Used by perField:
+  final BytesRef utf8 = new BytesRef(10);
+  
   boolean trackAllocations;
 
+  final LegacyTermAttributeWrapper legacyTermAttributeWrapper = new LegacyTermAttributeWrapper();
+  
+  /** This class is used to wrap a legacy TermAttribute without support for {@link TermToBytesRefAttribute}. */
+  @Deprecated
+  static class LegacyTermAttributeWrapper implements TermToBytesRefAttribute {
+    private TermAttribute termAtt = null;
+  
+    void setTermAttribute(TermAttribute termAtt) {
+      this.termAtt = termAtt;
+    }
+  
+    public int toBytesRef(BytesRef target) {
+      assert target.bytes != null : "target byteref must be != null, because utf8 is used here";
+      return UnicodeUtil.UTF16toUTF8WithHash(termAtt.termBuffer(), 0, termAtt.termLength(), target);
+    }
+  }
+  
   public TermsHash(final DocumentsWriter docWriter, boolean trackAllocations, final TermsHashConsumer consumer, final TermsHash nextTermsHash) {
+    this.docState = docWriter.docState;
     this.docWriter = docWriter;
     this.consumer = consumer;
     this.nextTermsHash = nextTermsHash;
     this.trackAllocations = trackAllocations;
-  }
+    
+    intPool = new IntBlockPool(docWriter, trackAllocations);
+    bytePool = new ByteBlockPool(docWriter.byteBlockAllocator, trackAllocations);
+    
+    if (nextTermsHash != null) {
+      // We are primary
+      primary = true;
+      termBytePool = bytePool;
+      nextTermsHash.termBytePool = bytePool;
+    } else {
+      primary = false;
+    }
 
-  @Override
-  InvertedDocConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) {
-    return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, null);
   }
 
-  TermsHashPerThread addThread(DocInverterPerThread docInverterPerThread, TermsHashPerThread primaryPerThread) {
-    return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, primaryPerThread);
-  }
-
   @Override
   void setFieldInfos(FieldInfos fieldInfos) {
     this.fieldInfos = fieldInfos;
@@ -64,11 +103,24 @@
 
   @Override
   synchronized public void abort() {
+    reset();
     consumer.abort();
-    if (nextTermsHash != null)
+    if (nextTermsHash != null) {
       nextTermsHash.abort();
+    }
   }
+  
+  // Clear all state
+  void reset() {
+    intPool.reset();
+    bytePool.reset();
 
+    if (primary) {
+      bytePool.reset();
+    }
+  }
+
+
   @Override
   synchronized void closeDocStore(SegmentWriteState state) throws IOException {
     consumer.closeDocStore(state);
@@ -77,50 +129,64 @@
   }
 
   @Override
-  synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
-    Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> childThreadsAndFields = new HashMap<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>();
-    Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> nextThreadsAndFields;
+  synchronized void flush(Map<FieldInfo,InvertedDocConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
+    Map<FieldInfo,TermsHashConsumerPerField> childFields = new HashMap<FieldInfo,TermsHashConsumerPerField>();
+    Map<FieldInfo,InvertedDocConsumerPerField> nextChildFields;
 
-    if (nextTermsHash != null)
-      nextThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
-    else
-      nextThreadsAndFields = null;
+    if (nextTermsHash != null) {
+      nextChildFields = new HashMap<FieldInfo,InvertedDocConsumerPerField>();
+    } else {
+      nextChildFields = null;
+    }
 
-    for (final Map.Entry<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> entry : threadsAndFields.entrySet()) {
-
-      TermsHashPerThread perThread = (TermsHashPerThread) entry.getKey();
-
-      Collection<InvertedDocConsumerPerField> fields = entry.getValue();
-
-      Iterator<InvertedDocConsumerPerField> fieldsIt = fields.iterator();
-      Collection<TermsHashConsumerPerField> childFields = new HashSet<TermsHashConsumerPerField>();
-      Collection<InvertedDocConsumerPerField> nextChildFields;
-
-      if (nextTermsHash != null)
-        nextChildFields = new HashSet<InvertedDocConsumerPerField>();
-      else
-        nextChildFields = null;
-
-      while(fieldsIt.hasNext()) {
-        TermsHashPerField perField = (TermsHashPerField) fieldsIt.next();
-        childFields.add(perField.consumer);
-        if (nextTermsHash != null)
-          nextChildFields.add(perField.nextPerField);
-      }
-
-      childThreadsAndFields.put(perThread.consumer, childFields);
-      if (nextTermsHash != null)
-        nextThreadsAndFields.put(perThread.nextPerThread, nextChildFields);
+    for (final Map.Entry<FieldInfo,InvertedDocConsumerPerField> entry : fieldsToFlush.entrySet()) {
+        TermsHashPerField perField = (TermsHashPerField) entry.getValue();
+        childFields.put(entry.getKey(), perField.consumer);
+        if (nextTermsHash != null) {
+          nextChildFields.put(entry.getKey(), perField.nextPerField);
+        }
     }
     
-    consumer.flush(childThreadsAndFields, state);
+    consumer.flush(childFields, state);
 
-    if (nextTermsHash != null)
-      nextTermsHash.flush(nextThreadsAndFields, state);
+    if (nextTermsHash != null) {
+      nextTermsHash.flush(nextChildFields, state);
+    }
   }
+  
+  @Override
+  InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
+    return new TermsHashPerField(docInverterPerField, this, nextTermsHash, fieldInfo);
+  }
 
   @Override
   synchronized public boolean freeRAM() {
     return false;
   }
+
+  @Override
+  DocWriter finishDocument() throws IOException {
+    final DocumentsWriter.DocWriter doc = consumer.finishDocument();
+
+    final DocumentsWriter.DocWriter doc2;
+    if (nextTermsHash != null) {
+      doc2 = nextTermsHash.consumer.finishDocument();
+    } else {
+      doc2 = null;
+    }
+    if (doc == null) {
+      return doc2;
+    } else {
+      doc.setNext(doc2);
+      return doc;
+    }
+  }
+
+  @Override
+  void startDocument() throws IOException {
+    consumer.startDocument();
+    if (nextTermsHash != null) {
+      nextTermsHash.consumer.startDocument();
+    }
+  }
 }
Index: src/java/org/apache/lucene/index/TermsHashPerField.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHashPerField.java	(revision 934193)
+++ src/java/org/apache/lucene/index/TermsHashPerField.java	(working copy)
@@ -31,8 +31,9 @@
 
   final TermsHashConsumerPerField consumer;
 
+  final TermsHash termsHash;
+  
   final TermsHashPerField nextPerField;
-  final TermsHashPerThread perThread;
   final DocumentsWriter.DocState docState;
   final FieldInvertState fieldState;
   TermToBytesRefAttribute termAtt;
@@ -58,28 +59,28 @@
   private final BytesRef utf8;
   private Comparator<BytesRef> termComp;
 
-  public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHashPerThread perThread, final TermsHashPerThread nextPerThread, final FieldInfo fieldInfo) {
-    this.perThread = perThread;
-    intPool = perThread.intPool;
-    bytePool = perThread.bytePool;
-    termBytePool = perThread.termBytePool;
-    docState = perThread.docState;
+  public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHash termsHash, final TermsHash nextTermsHash, final FieldInfo fieldInfo) {
+    intPool = termsHash.intPool;
+    bytePool = termsHash.bytePool;
+    termBytePool = termsHash.termBytePool;
+    docState = termsHash.docState;
+    this.termsHash = termsHash;
 
     postingsHash = new int[postingsHashSize];
     Arrays.fill(postingsHash, -1);
     bytesUsed(postingsHashSize * RamUsageEstimator.NUM_BYTES_INT);
 
     fieldState = docInverterPerField.fieldState;
-    this.consumer = perThread.consumer.addField(this, fieldInfo);
+    this.consumer = termsHash.consumer.addField(this, fieldInfo);
     initPostingsArray();
     bytesUsed(postingsArray.size * postingsArray.bytesPerPosting());
 
     streamCount = consumer.getStreamCount();
     numPostingInt = 2*streamCount;
-    utf8 = perThread.utf8;
+    utf8 = termsHash.utf8;
     this.fieldInfo = fieldInfo;
-    if (nextPerThread != null)
-      nextPerField = (TermsHashPerField) nextPerThread.addField(docInverterPerField, fieldInfo);
+    if (nextTermsHash != null)
+      nextPerField = (TermsHashPerField) nextTermsHash.addField(docInverterPerField, fieldInfo);
     else
       nextPerField = null;
   }
@@ -90,8 +91,8 @@
 
   // sugar: just forwards to DW
   private void bytesUsed(long size) {
-    if (perThread.termsHash.trackAllocations) {
-      perThread.termsHash.docWriter.bytesUsed(size);
+    if (termsHash.trackAllocations) {
+      termsHash.docWriter.bytesUsed(size);
     }
   }
   
@@ -252,10 +253,10 @@
       return 0;
     }
 
-    termBytePool.setBytesRef(perThread.tr1, postingsArray.textStarts[term1]);
-    termBytePool.setBytesRef(perThread.tr2, postingsArray.textStarts[term2]);
+    termBytePool.setBytesRef(termsHash.tr1, postingsArray.textStarts[term1]);
+    termBytePool.setBytesRef(termsHash.tr2, postingsArray.textStarts[term2]);
 
-    return termComp.compare(perThread.tr1, perThread.tr2);
+    return termComp.compare(termsHash.tr1, termsHash.tr2);
   }
 
   /** Test whether the text for current RawPostingList p equals
@@ -299,8 +300,8 @@
     if (fieldState.attributeSource.hasAttribute(TermToBytesRefAttribute.class)) {
       termAtt = fieldState.attributeSource.getAttribute(TermToBytesRefAttribute.class);
     } else if (fieldState.attributeSource.hasAttribute(TermAttribute.class)) {
-      perThread.legacyTermAttributeWrapper.setTermAttribute(fieldState.attributeSource.getAttribute(TermAttribute.class));
-      termAtt = perThread.legacyTermAttributeWrapper;
+      termsHash.legacyTermAttributeWrapper.setTermAttribute(fieldState.attributeSource.getAttribute(TermAttribute.class));
+      termAtt = termsHash.legacyTermAttributeWrapper;
     } else {
       throw new IllegalArgumentException("Could not find a term attribute (that implements TermToBytesRefAttribute) in the TokenStream");
     }
@@ -580,7 +581,7 @@
       int termID = postingsHash[i];
       if (termID != -1) {
         int code;
-        if (perThread.primary) {
+        if (termsHash.primary) {
           final int textStart = postingsArray.textStarts[termID];
           final int start = textStart & DocumentsWriter.BYTE_BLOCK_MASK;
           final byte[] text = bytePool.buffers[textStart >> DocumentsWriter.BYTE_BLOCK_SHIFT];
Index: src/java/org/apache/lucene/index/DocFieldProcessor.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldProcessor.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocFieldProcessor.java	(working copy)
@@ -19,10 +19,17 @@
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
-import java.util.HashMap;
 
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
+
 /**
  * This is a DocConsumer that gathers all fields under the
  * same name, and calls per-field consumers to process field
@@ -33,13 +40,27 @@
 
 final class DocFieldProcessor extends DocConsumer {
 
-  final DocumentsWriter docWriter;
   final FieldInfos fieldInfos = new FieldInfos();
   final DocFieldConsumer consumer;
   final StoredFieldsWriter fieldsWriter;
 
+  // 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;
+
+  
+  float docBoost;
+  int fieldGen;
+  final DocumentsWriter.DocState docState;
+
+
   public DocFieldProcessor(DocumentsWriter docWriter, DocFieldConsumer consumer) {
-    this.docWriter = docWriter;
+    this.docState = docWriter.docState;
     this.consumer = consumer;
     consumer.setFieldInfos(fieldInfos);
     fieldsWriter = new StoredFieldsWriter(docWriter, fieldInfos);
@@ -52,16 +73,17 @@
   }
 
   @Override
-  public void flush(Collection<DocConsumerPerThread> threads, SegmentWriteState state) throws IOException {
+  public void flush(SegmentWriteState state) throws IOException {
 
-    Map<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>> childThreadsAndFields = new HashMap<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>>();
-    for ( DocConsumerPerThread thread : threads) {
-      DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) thread;
-      childThreadsAndFields.put(perThread.consumer, perThread.fields());
-      perThread.trimFields(state);
+    Map<FieldInfo, DocFieldConsumerPerField> childFields = new HashMap<FieldInfo, DocFieldConsumerPerField>();
+    Collection<DocFieldConsumerPerField> fields = fields();
+    for (DocFieldConsumerPerField f : fields) {
+      childFields.put(f.getFieldInfo(), f);
     }
+    trimFields(state);
+
     fieldsWriter.flush(state);
-    consumer.flush(childThreadsAndFields, state);
+    consumer.flush(childFields, state);
 
     // Important to save after asking consumer to flush so
     // consumer can alter the FieldInfo* if necessary.  EG,
@@ -74,6 +96,15 @@
 
   @Override
   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;
+      }
+    }
+
     fieldsWriter.abort();
     consumer.abort();
   }
@@ -82,9 +113,317 @@
   public boolean freeRAM() {
     return consumer.freeRAM();
   }
+  
+  public Collection<DocFieldConsumerPerField> fields() {
+    Collection<DocFieldConsumerPerField> fields = new HashSet<DocFieldConsumerPerField>();
+    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(SegmentWriteState 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.infoStream != null) {
+            state.infoStream.println("  purge field=" + perField.fieldInfo.name);
+          }
+
+          totalFieldCount--;
+
+        } else {
+          // Reset
+          perField.lastGen = -1;
+          lastPerField = perField;
+        }
+
+        perField = perField.next;
+      }
+    }
+  }
+
+  private void rehash() {
+    final int newHashSize = (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;
+  }
+
   @Override
-  public DocConsumerPerThread addThread(DocumentsWriterThreadState threadState) throws IOException {
-    return new DocFieldProcessorPerThread(threadState, this);
+  public DocumentsWriter.DocWriter processDocument() throws IOException {
+
+    consumer.startDocument();
+    fieldsWriter.startDocument();
+
+    final Document doc = docState.doc;
+
+    fieldCount = 0;
+    
+    final int thisFieldGen = fieldGen++;
+
+    final List<Fieldable> 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 = 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, field.getOmitTermFreqAndPositions());
+
+        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, field.getOmitTermFreqAndPositions());
+      }
+      
+      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 (field.isStored()) {
+        fieldsWriter.addField(field, fp.fieldInfo);
+      }
+    }
+
+    // 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 (whose UTF8 encoding is longer than the max length " + DocumentsWriter.MAX_TERM_LENGTH_UTF8 + "), all of which were skipped.  Please correct the analyzer to not produce such terms.  The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'"); 
+      docState.maxTermPrefix = null;
+    }
+
+    final DocumentsWriter.DocWriter one = fieldsWriter.finishDocument();
+    final DocumentsWriter.DocWriter two = consumer.finishDocument();
+    if (one == null) {
+      return two;
+    } else if (two == null) {
+      return one;
+    } else {
+      PerDoc both = getPerDoc();
+      both.docID = docState.docID;
+      assert one.docID == docState.docID;
+      assert two.docID == docState.docID;
+      both.one = one;
+      both.two = two;
+      return both;
+    }
   }
+
+  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);
+  }
+
+  PerDoc[] docFreeList = new PerDoc[1];
+  int freeCount;
+  int allocCount;
+
+  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.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+      }
+      return new PerDoc();
+    } else
+      return docFreeList[--freeCount];
+  }
+
+  void freePerDoc(PerDoc perDoc) {
+    assert freeCount < docFreeList.length;
+    docFreeList[freeCount++] = perDoc;
+  }
+
+  class PerDoc extends DocumentsWriter.DocWriter {
+
+    DocumentsWriter.DocWriter one;
+    DocumentsWriter.DocWriter two;
+
+    @Override
+    public long sizeInBytes() {
+      return one.sizeInBytes() + two.sizeInBytes();
+    }
+
+    @Override
+    public void finish() throws IOException {
+      try {
+        try {
+          one.finish();
+        } finally {
+          two.finish();
+        }
+      } finally {
+        freePerDoc(this);
+      }
+    }
+
+    @Override
+    public void abort() {
+      try {
+        try {
+          one.abort();
+        } finally {
+          two.abort();
+        }
+      } finally {
+        freePerDoc(this);
+      }
+    }
+  }
 }
Index: src/java/org/apache/lucene/index/FreqProxTermsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/FreqProxTermsWriter.java	(revision 934193)
+++ src/java/org/apache/lucene/index/FreqProxTermsWriter.java	(working copy)
@@ -19,67 +19,54 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.Iterator;
+import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
-import java.util.Comparator;
 
-import org.apache.lucene.index.codecs.PostingsConsumer;
+import org.apache.lucene.index.DocumentsWriter.DocWriter;
 import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.PostingsConsumer;
 import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.util.BytesRef;
 
 final class FreqProxTermsWriter extends TermsHashConsumer {
 
   @Override
-  public TermsHashConsumerPerThread addThread(TermsHashPerThread perThread) {
-    return new FreqProxTermsWriterPerThread(perThread);
-  }
-
-  @Override
   void closeDocStore(SegmentWriteState state) {}
 
   @Override
   void abort() {}
 
-  private int flushedDocCount;
-
   // TODO: would be nice to factor out more of this, eg the
   // FreqProxFieldMergeState, and code to visit all Fields
   // under the same FieldInfo together, up into TermsHash*.
   // Other writers would presumably share alot of this...
 
   @Override
-  public void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
+  public void flush(Map<FieldInfo, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
 
     // Gather all FieldData's that have postings, across all
     // ThreadStates
     List<FreqProxTermsWriterPerField> allFields = new ArrayList<FreqProxTermsWriterPerField>();
-    
-    flushedDocCount = state.numDocs;
 
-    for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
-
-      Collection<TermsHashConsumerPerField> fields = entry.getValue();
-
-
-      for (final TermsHashConsumerPerField i : fields) {
-        final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) i;
-        if (perField.termsHashPerField.numPostings > 0)
+    for (TermsHashConsumerPerField f : fieldsToFlush.values()) {
+        final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) f;
+        if (perField.termsHashPerField.numPostings > 0) {
           allFields.add(perField);
-      }
+        }
     }
 
     final int numAllFields = allFields.size();
-
-    // Sort by field name
+    
+    // sort by field name
     Collections.sort(allFields);
 
     // TODO: allow Lucene user to customize this codec:
     final FieldsConsumer consumer = state.codec.fieldsConsumer(state);
 
+    TermsHash termsHash = null;
+    
     /*
     Current writer chain:
       FieldsConsumer
@@ -92,208 +79,44 @@
                     -> IMPL: FormatPostingsPositionsWriter
     */
 
-    int start = 0;
-    while(start < numAllFields) {
-      final FieldInfo fieldInfo = allFields.get(start).fieldInfo;
-      final String fieldName = fieldInfo.name;
-
-      int end = start+1;
-      while(end < numAllFields && allFields.get(end).fieldInfo.name.equals(fieldName))
-        end++;
+    for (int fieldNumber = 0; fieldNumber < numAllFields; fieldNumber++) {
+      final FieldInfo fieldInfo = allFields.get(fieldNumber).fieldInfo;
       
-      FreqProxTermsWriterPerField[] fields = new FreqProxTermsWriterPerField[end-start];
-      for(int i=start;i<end;i++) {
-        fields[i-start] = allFields.get(i);
+      FreqProxTermsWriterPerField fieldWriter = allFields.get(fieldNumber);
+      fieldInfo.storePayloads |= fieldWriter.hasPayloads;
 
-        // Aggregate the storePayload as seen by the same
-        // field across multiple threads
-        fieldInfo.storePayloads |= fields[i-start].hasPayloads;
-      }
-
       // If this field has postings then add them to the
       // segment
-      appendPostings(fields, consumer);
+      fieldWriter.flush(consumer, state);
 
-      for(int i=0;i<fields.length;i++) {
-        TermsHashPerField perField = fields[i].termsHashPerField;
-        int numPostings = perField.numPostings;
-        perField.reset();
-        perField.shrinkHash(numPostings);
-        fields[i].reset();
-      }
-
-      start = end;
+      TermsHashPerField perField = fieldWriter.termsHashPerField;
+      assert termsHash == null || termsHash == perField.termsHash;
+      termsHash = perField.termsHash;
+      int numPostings = perField.numPostings;
+      perField.reset();
+      perField.shrinkHash(numPostings);
+      fieldWriter.reset();
     }
 
-    for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
-      FreqProxTermsWriterPerThread perThread = (FreqProxTermsWriterPerThread) entry.getKey();
-      perThread.termsHashPerThread.reset(true);
+    if (termsHash != null) {
+      termsHash.reset();
     }
     consumer.close();
   }
 
   BytesRef payload;
 
-  /* Walk through all unique text tokens (Posting
-   * instances) found in this field and serialize them
-   * into a single RAM segment. */
-  void appendPostings(FreqProxTermsWriterPerField[] fields,
-                      FieldsConsumer consumer)
-    throws CorruptIndexException, IOException {
+  @Override
+  public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
+    return new FreqProxTermsWriterPerField(termsHashPerField, this, fieldInfo);
+  }
 
-    int numFields = fields.length;
+  @Override
+  DocWriter finishDocument() throws IOException {
+    return null;
+  }
 
-    final BytesRef text = new BytesRef();
-
-    final FreqProxFieldMergeState[] mergeStates = new FreqProxFieldMergeState[numFields];
-
-    final TermsConsumer termsConsumer = consumer.addField(fields[0].fieldInfo);
-    final Comparator<BytesRef> termComp = termsConsumer.getComparator();
-
-    for(int i=0;i<numFields;i++) {
-      FreqProxFieldMergeState fms = mergeStates[i] = new FreqProxFieldMergeState(fields[i], termComp);
-
-      assert fms.field.fieldInfo == fields[0].fieldInfo;
-
-      // Should always be true
-      boolean result = fms.nextTerm();
-      assert result;
-    }
-
-    FreqProxFieldMergeState[] termStates = new FreqProxFieldMergeState[numFields];
-
-    final boolean currentFieldOmitTermFreqAndPositions = fields[0].fieldInfo.omitTermFreqAndPositions;
-    //System.out.println("flush terms field=" + fields[0].fieldInfo.name);
-
-    // TODO: really TermsHashPerField should take over most
-    // of this loop, including merge sort of terms from
-    // multiple threads and interacting with the
-    // TermsConsumer, only calling out to us (passing us the
-    // DocsConsumer) to handle delivery of docs/positions
-    while(numFields > 0) {
-
-      // Get the next term to merge
-      termStates[0] = mergeStates[0];
-      int numToMerge = 1;
-
-      // TODO: pqueue
-      for(int i=1;i<numFields;i++) {
-        final int cmp = termComp.compare(mergeStates[i].text, termStates[0].text);
-        if (cmp < 0) {
-          termStates[0] = mergeStates[i];
-          numToMerge = 1;
-        } else if (cmp == 0) {
-          termStates[numToMerge++] = mergeStates[i];
-        }
-      }
-
-      // Need shallow copy here because termStates[0].text
-      // changes by the time we call finishTerm
-      text.bytes = termStates[0].text.bytes;
-      text.offset = termStates[0].text.offset;
-      text.length = termStates[0].text.length;  
-
-      //System.out.println("  term=" + text.toUnicodeString());
-      //System.out.println("  term=" + text.toString());
-
-      final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text);
-
-      // Now termStates has numToMerge FieldMergeStates
-      // which all share the same term.  Now we must
-      // interleave the docID streams.
-      int numDocs = 0;
-      while(numToMerge > 0) {
-        
-        FreqProxFieldMergeState minState = termStates[0];
-        for(int i=1;i<numToMerge;i++) {
-          if (termStates[i].docID < minState.docID) {
-            minState = termStates[i];
-          }
-        }
-
-        final int termDocFreq = minState.termFreq;
-        numDocs++;
-
-        assert minState.docID < flushedDocCount: "doc=" + minState.docID + " maxDoc=" + flushedDocCount;
-
-        postingsConsumer.startDoc(minState.docID, termDocFreq);
-
-        final ByteSliceReader prox = minState.prox;
-
-        // Carefully copy over the prox + payload info,
-        // changing the format to match Lucene's segment
-        // format.
-        if (!currentFieldOmitTermFreqAndPositions) {
-          // omitTermFreqAndPositions == false so we do write positions &
-          // payload          
-          int position = 0;
-          for(int j=0;j<termDocFreq;j++) {
-            final int code = prox.readVInt();
-            position += code >> 1;
-            //System.out.println("    pos=" + position);
-
-            final int payloadLength;
-            final BytesRef thisPayload;
-
-            if ((code & 1) != 0) {
-              // This position has a payload
-              payloadLength = prox.readVInt();  
-              
-              if (payload == null) {
-                payload = new BytesRef();
-                payload.bytes = new byte[payloadLength];
-              } else if (payload.bytes.length < payloadLength) {
-                payload.grow(payloadLength);
-              }
-
-              prox.readBytes(payload.bytes, 0, payloadLength);
-              payload.length = payloadLength;
-              thisPayload = payload;
-
-            } else {
-              payloadLength = 0;
-              thisPayload = null;
-            }
-
-            postingsConsumer.addPosition(position, thisPayload);
-          } //End for
-
-          postingsConsumer.finishDoc();
-        }
-
-        if (!minState.nextDoc()) {
-
-          // Remove from termStates
-          int upto = 0;
-          // TODO: inefficient O(N) where N = number of
-          // threads that had seen this term:
-          for(int i=0;i<numToMerge;i++) {
-            if (termStates[i] != minState) {
-              termStates[upto++] = termStates[i];
-            }
-          }
-          numToMerge--;
-          assert upto == numToMerge;
-
-          // Advance this state to the next term
-
-          if (!minState.nextTerm()) {
-            // OK, no more terms, so remove from mergeStates
-            // as well
-            upto = 0;
-            for(int i=0;i<numFields;i++)
-              if (mergeStates[i] != minState)
-                mergeStates[upto++] = mergeStates[i];
-            numFields--;
-            assert upto == numFields;
-          }
-        }
-      }
-
-      assert numDocs > 0;
-      termsConsumer.finishTerm(text, numDocs);
-    }
-
-    termsConsumer.finish();
+  @Override
+  void startDocument() throws IOException {
   }
 }
Index: src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldProcessorPerField.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocFieldProcessorPerField.java	(working copy)
@@ -34,8 +34,8 @@
   int fieldCount;
   Fieldable[] fields = new Fieldable[1];
 
-  public DocFieldProcessorPerField(final DocFieldProcessorPerThread perThread, final FieldInfo fieldInfo) {
-    this.consumer = perThread.consumer.addField(fieldInfo);
+  public DocFieldProcessorPerField(final DocFieldProcessor docFieldProcessor, final FieldInfo fieldInfo) {
+    this.consumer = docFieldProcessor.consumer.addField(fieldInfo);
     this.fieldInfo = fieldInfo;
   }
 
Index: src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
===================================================================
--- src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java	(revision 934193)
+++ src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java	(working copy)
@@ -18,16 +18,21 @@
  */
 
 import java.io.IOException;
+import java.util.Comparator;
 
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.PostingsConsumer;
+import org.apache.lucene.index.codecs.TermsConsumer;
+import org.apache.lucene.util.BytesRef;
 
 // TODO: break into separate freq and prox writers as
 // codecs; make separate container (tii/tis/skip/*) that can
 // be configured as any number of files 1..N
 final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implements Comparable<FreqProxTermsWriterPerField> {
 
-  final FreqProxTermsWriterPerThread perThread;
+  final FreqProxTermsWriter parent;
   final TermsHashPerField termsHashPerField;
   final FieldInfo fieldInfo;
   final DocumentsWriter.DocState docState;
@@ -35,9 +40,9 @@
   boolean omitTermFreqAndPositions;
   PayloadAttribute payloadAttribute;
 
-  public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriterPerThread perThread, FieldInfo fieldInfo) {
+  public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriter parent, FieldInfo fieldInfo) {
     this.termsHashPerField = termsHashPerField;
-    this.perThread = perThread;
+    this.parent = parent;
     this.fieldInfo = fieldInfo;
     docState = termsHashPerField.docState;
     fieldState = termsHashPerField.fieldState;
@@ -210,5 +215,133 @@
   }
   
   public void abort() {}
+  
+  BytesRef payload;
+  
+  /* Walk through all unique text tokens (Posting
+   * instances) found in this field and serialize them
+   * into a single RAM segment. */
+  void flush(FieldsConsumer consumer,  final SegmentWriteState state)
+    throws CorruptIndexException, IOException {
+
+    final TermsConsumer termsConsumer = consumer.addField(fieldInfo);
+    final Comparator<BytesRef> termComp = termsConsumer.getComparator();
+
+    final boolean currentFieldOmitTermFreqAndPositions = fieldInfo.omitTermFreqAndPositions;
+    
+    final int[] termIDs = termsHashPerField.sortPostings(termComp);
+    final int numTerms = termsHashPerField.numPostings;
+    final BytesRef text = new BytesRef();
+    final FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
+    final ByteSliceReader freq = new ByteSliceReader();
+    final ByteSliceReader prox = new ByteSliceReader();
+
+    
+    for (int i = 0; i < numTerms; i++) {
+      final int termID = termIDs[i];
+      // Get BytesRef
+      final int textStart = postings.textStarts[termID];
+      termsHashPerField.bytePool.setBytesRef(text, textStart);
+      
+      termsHashPerField.initReader(freq, termID, 0);
+      if (!fieldInfo.omitTermFreqAndPositions) {
+        termsHashPerField.initReader(prox, termID, 1);
+      }
+  
+      // TODO: really TermsHashPerField should take over most
+      // of this loop, including merge sort of terms from
+      // multiple threads and interacting with the
+      // TermsConsumer, only calling out to us (passing us the
+      // DocsConsumer) to handle delivery of docs/positions
+    
+      final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text);
+  
+      // Now termStates has numToMerge FieldMergeStates
+      // which all share the same term.  Now we must
+      // interleave the docID streams.
+      int numDocs = 0;
+      int docID = 0;
+      int termFreq = 0;
+      
+      while(true) {
+        if (freq.eof()) {
+          if (postings.lastDocCodes[termID] != -1) {
+            // Return last doc
+            docID = postings.lastDocIDs[termID];
+            if (!omitTermFreqAndPositions) {
+              termFreq = postings.docFreqs[termID];
+            }
+            postings.lastDocCodes[termID] = -1;
+          } else {
+            // EOF
+            break;
+          }
+        } else {
+          final int code = freq.readVInt();
+          if (omitTermFreqAndPositions) {
+            docID += code;
+          } else {
+            docID += code >>> 1;
+            if ((code & 1) != 0) {
+              termFreq = 1;
+            } else {
+              termFreq = freq.readVInt();
+            }
+          }
+    
+          assert docID != postings.lastDocIDs[termID];
+        }
+        
+        numDocs++;
+        assert docID < state.numDocs: "doc=" + docID + " maxDoc=" + state.numDocs;
+        final int termDocFreq = termFreq;
+        postingsConsumer.startDoc(docID, termDocFreq);
+    
+        // Carefully copy over the prox + payload info,
+        // changing the format to match Lucene's segment
+        // format.
+        if (!currentFieldOmitTermFreqAndPositions) {
+          // omitTermFreqAndPositions == false so we do write positions &
+          // payload          
+          int position = 0;
+          for(int j=0;j<termDocFreq;j++) {
+            final int code = prox.readVInt();
+            position += code >> 1;
+    
+            final int payloadLength;
+            final BytesRef thisPayload;
+    
+            if ((code & 1) != 0) {
+              // This position has a payload
+              payloadLength = prox.readVInt();  
+              
+              if (payload == null) {
+                payload = new BytesRef();
+                payload.bytes = new byte[payloadLength];
+              } else if (payload.bytes.length < payloadLength) {
+                payload.grow(payloadLength);
+              }
+    
+              prox.readBytes(payload.bytes, 0, payloadLength);
+              payload.length = payloadLength;
+              thisPayload = payload;
+    
+            } else {
+              payloadLength = 0;
+              thisPayload = null;
+            }
+    
+            postingsConsumer.addPosition(position, thisPayload);
+          } 
+    
+          postingsConsumer.finishDoc();
+        }
+      } 
+      termsConsumer.finishTerm(text, numDocs);
+    }
+  
+    termsConsumer.finish();
+  }
+
 }
 
Index: src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocFieldConsumersPerThread.java	(working copy)
@@ -1,75 +0,0 @@
-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;
-  }
-
-  @Override
-  public void startDocument() throws IOException {
-    one.startDocument();
-    two.startDocument();
-  }
-
-  @Override
-  public void abort() {
-    try {
-      one.abort();
-    } finally {
-      two.abort();
-    }
-  }
-
-  @Override
-  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.writerOne = oneDoc;
-      both.writerTwo = twoDoc;
-      return both;
-    }
-  }
-
-  @Override
-  public DocFieldConsumerPerField addField(FieldInfo fi) {
-    return new DocFieldConsumersPerField(this, one.addField(fi), two.addField(fi));
-  }
-}
Index: src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/TermVectorsTermsWriter.java	(revision 934193)
+++ src/java/org/apache/lucene/index/TermVectorsTermsWriter.java	(working copy)
@@ -17,16 +17,16 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.index.DocumentsWriter.DocWriter;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
 
-import java.io.IOException;
-import java.util.Collection;
-
-import java.util.Map;
-
 final class TermVectorsTermsWriter extends TermsHashConsumer {
 
   final DocumentsWriter docWriter;
@@ -37,19 +37,22 @@
   IndexOutput tvd;
   IndexOutput tvf;
   int lastDocID;
+  
+  final DocumentsWriter.DocState docState;
+  final BytesRef flushTerm = new BytesRef();
+  TermVectorsTermsWriter.PerDoc doc;
+  
+  // Used by perField when serializing the term vectors
+  final ByteSliceReader vectorSliceReader = new ByteSliceReader();
 
   public TermVectorsTermsWriter(DocumentsWriter docWriter) {
     this.docWriter = docWriter;
+    docState = docWriter.docState;
   }
 
   @Override
-  public TermsHashConsumerPerThread addThread(TermsHashPerThread termsHashPerThread) {
-    return new TermVectorsTermsWriterPerThread(termsHashPerThread, this);
-  }
+  synchronized void flush(Map<FieldInfo, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
 
-  @Override
-  synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
-
     if (tvx != null) {
 
       if (state.numDocsInStore > 0)
@@ -62,15 +65,10 @@
       tvf.flush();
     }
 
-    for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
-      for (final TermsHashConsumerPerField field : entry.getValue() ) {
-        TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field;
-        perField.termsHashPerField.reset();
-        perField.shrinkHash();
-      }
-
-      TermVectorsTermsWriterPerThread perThread = (TermVectorsTermsWriterPerThread) entry.getKey();
-      perThread.termsHashPerThread.reset(true);
+    for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) {
+      TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field;
+      perField.termsHashPerField.reset();
+      perField.shrinkHash();
     }
   }
 
@@ -210,6 +208,11 @@
 
   @Override
   public void abort() {
+    if (doc != null) {
+      doc.abort();
+      doc = null;
+    }
+
     if (tvx != null) {
       try {
         tvx.close();
@@ -232,6 +235,8 @@
       tvf = null;
     }
     lastDocID = 0;
+    
+
   }
 
   synchronized void free(PerDoc doc) {
@@ -256,7 +261,7 @@
     }
 
     @Override
-    void abort() {
+    public void abort() {
       reset();
       free(this);
     }
@@ -283,4 +288,47 @@
       finishDocument(this);
     }
   }
+
+  @Override
+  public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
+    return new TermVectorsTermsWriterPerField(termsHashPerField, this, fieldInfo);
+  }
+
+  @Override
+  DocWriter finishDocument() throws IOException {
+    try {
+      return doc;
+    } finally {
+      doc = null;
+    }
+  }
+
+  @Override
+  void startDocument() throws IOException {
+    assert clearLastVectorFieldName();
+    if (doc != null) {
+      doc.reset();
+      doc.docID = docState.docID;
+    }
+  }
+  
+  // Called only by assert
+  final boolean clearLastVectorFieldName() {
+    lastVectorFieldName = null;
+    return true;
+  }
+
+  // Called only by assert
+  String lastVectorFieldName;
+  final boolean vectorFieldsInOrder(FieldInfo fi) {
+    try {
+      if (lastVectorFieldName != null)
+        return lastVectorFieldName.compareTo(fi.name) < 0;
+      else
+        return true;
+    } finally {
+      lastVectorFieldName = fi.name;
+    }
+  }
+
 }
Index: src/java/org/apache/lucene/index/DocumentsWriterThreadState.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriterThreadState.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocumentsWriterThreadState.java	(working copy)
@@ -1,50 +0,0 @@
-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;
-
-/** Used by DocumentsWriter to maintain per-thread state.
- *  We keep a separate Posting hash and other state for each
- *  thread and then merge postings hashes from all threads
- *  when writing the segment. */
-final class DocumentsWriterThreadState {
-
-  boolean isIdle = true;                          // false if this is currently in use by a thread
-  int numThreads = 1;                             // Number of threads that share this instance
-  boolean doFlushAfter;                           // true if we should flush after processing current doc
-  final DocConsumerPerThread consumer;
-  final DocumentsWriter.DocState docState;
-
-  final DocumentsWriter docWriter;
-
-  public DocumentsWriterThreadState(DocumentsWriter docWriter) throws IOException {
-    this.docWriter = docWriter;
-    docState = new DocumentsWriter.DocState();
-    docState.maxFieldLength = docWriter.maxFieldLength;
-    docState.infoStream = docWriter.infoStream;
-    docState.similarity = docWriter.similarity;
-    docState.docWriter = docWriter;
-    consumer = docWriter.consumer.addThread(this);
-  }
-
-  void doAfterFlush() {
-    numThreads = 0;
-    doFlushAfter = false;
-  }
-}
Index: src/java/org/apache/lucene/index/NormsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/NormsWriterPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/NormsWriterPerThread.java	(working copy)
@@ -1,45 +0,0 @@
-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.
- */
-
-final class NormsWriterPerThread extends InvertedDocEndConsumerPerThread {
-  final NormsWriter normsWriter;
-  final DocumentsWriter.DocState docState;
-
-  public NormsWriterPerThread(DocInverterPerThread docInverterPerThread, NormsWriter normsWriter) {
-    this.normsWriter = normsWriter;
-    docState = docInverterPerThread.docState;
-  }
-
-  @Override
-  InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
-    return new NormsWriterPerField(docInverterPerField, this, fieldInfo);
-  }
-
-  @Override
-  void abort() {}
-
-  @Override
-  void startDocument() {}
-  @Override
-  void finishDocument() {}
-
-  boolean freeRAM() {
-    return false;
-  }
-}
Index: src/java/org/apache/lucene/index/TermsHashPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/TermsHashPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/TermsHashPerThread.java	(working copy)
@@ -1,133 +0,0 @@
-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.util.BytesRef;
-import org.apache.lucene.util.UnicodeUtil;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-
-import java.io.IOException;
-
-final class TermsHashPerThread extends InvertedDocConsumerPerThread {
-
-  final TermsHash termsHash;
-  final TermsHashConsumerPerThread consumer;
-  final TermsHashPerThread nextPerThread;
-
-  final IntBlockPool intPool;
-  final ByteBlockPool bytePool;
-  final ByteBlockPool termBytePool;
-  final boolean primary;
-  final DocumentsWriter.DocState docState;
-
-  // Used when comparing postings via termRefComp, in TermsHashPerField
-  final BytesRef tr1 = new BytesRef();
-  final BytesRef tr2 = new BytesRef();
-
-  // Used by perField:
-  final BytesRef utf8 = new BytesRef(10);
-  
-  final LegacyTermAttributeWrapper legacyTermAttributeWrapper = new LegacyTermAttributeWrapper();
-  
-  /** This class is used to wrap a legacy TermAttribute without support for {@link TermToBytesRefAttribute}. */
-  @Deprecated
-  static class LegacyTermAttributeWrapper implements TermToBytesRefAttribute {
-    private TermAttribute termAtt = null;
-  
-    void setTermAttribute(TermAttribute termAtt) {
-      this.termAtt = termAtt;
-    }
-  
-    public int toBytesRef(BytesRef target) {
-      assert target.bytes != null : "target byteref must be != null, because utf8 is used here";
-      return UnicodeUtil.UTF16toUTF8WithHash(termAtt.termBuffer(), 0, termAtt.termLength(), target);
-    }
-  }
-
-  public TermsHashPerThread(DocInverterPerThread docInverterPerThread, final TermsHash termsHash, final TermsHash nextTermsHash, final TermsHashPerThread primaryPerThread) {
-    docState = docInverterPerThread.docState;
-
-    this.termsHash = termsHash;
-    this.consumer = termsHash.consumer.addThread(this);
-
-    intPool = new IntBlockPool(termsHash.docWriter, termsHash.trackAllocations);
-    bytePool = new ByteBlockPool(termsHash.docWriter.byteBlockAllocator, termsHash.trackAllocations);
-
-    if (nextTermsHash != null) {
-      // We are primary
-      primary = true;
-      termBytePool = bytePool;
-    } else {
-      primary = false;
-      termBytePool = primaryPerThread.bytePool;
-    }
-
-    if (nextTermsHash != null)
-      nextPerThread = nextTermsHash.addThread(docInverterPerThread, this);
-    else
-      nextPerThread = null;
-  }
-
-  @Override
-  InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
-    return new TermsHashPerField(docInverterPerField, this, nextPerThread, fieldInfo);
-  }
-
-  @Override
-  synchronized public void abort() {
-    reset(true);
-    consumer.abort();
-    if (nextPerThread != null)
-      nextPerThread.abort();
-  }
-
-  @Override
-  public void startDocument() throws IOException {
-    consumer.startDocument();
-    if (nextPerThread != null)
-      nextPerThread.consumer.startDocument();
-  }
-
-  @Override
-  public DocumentsWriter.DocWriter finishDocument() throws IOException {
-    final DocumentsWriter.DocWriter doc = consumer.finishDocument();
-
-    final DocumentsWriter.DocWriter doc2;
-    if (nextPerThread != null)
-      doc2 = nextPerThread.consumer.finishDocument();
-    else
-      doc2 = null;
-    if (doc == null)
-      return doc2;
-    else {
-      doc.setNext(doc2);
-      return doc;
-    }
-  }
-
-  // Clear all state
-  void reset(boolean recyclePostings) {
-    intPool.reset();
-    bytePool.reset();
-
-    if (primary) {
-      bytePool.reset();
-    }
-  }
-}
Index: src/java/org/apache/lucene/index/DocumentsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriter.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocumentsWriter.java	(working copy)
@@ -22,10 +22,8 @@
 import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -36,14 +34,10 @@
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.Constants;
-import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.Constants;
 
 /**
  * This class accepts multiple added documents and directly
@@ -125,14 +119,8 @@
   private int numDocsInRAM;                       // # docs buffered in RAM
   int numDocsInStore;                     // # docs written to doc stores
 
-  // Max # ThreadState instances; if there are more threads
-  // than this they share ThreadStates
-  private DocumentsWriterThreadState[] threadStates = new DocumentsWriterThreadState[0];
-  private final HashMap<Thread,DocumentsWriterThreadState> threadBindings = new HashMap<Thread,DocumentsWriterThreadState>();
-
-  private int pauseThreads;               // Non-zero when we need all threads to
-                                          // pause (eg to flush)
   boolean flushPending;                   // True when a thread has decided to flush
+  boolean doFlushAfter;
   boolean bufferIsFull;                   // True when it's time to write segment
   private boolean aborting;               // True if an abort is pending
 
@@ -142,12 +130,10 @@
   int maxFieldLength = IndexWriterConfig.UNLIMITED_FIELD_LENGTH;
   Similarity similarity;
 
-  // max # simultaneous threads; if there are more than
-  // this, they wait for others to finish first
-  private final int maxThreadStates;
-
   List<String> newFiles;
 
+  final DocState docState;
+  
   static class DocState {
     DocumentsWriter docWriter;
     Analyzer analyzer;
@@ -202,7 +188,7 @@
     /**
      * Recycle the bytes used.
      */
-    synchronized void recycle() {
+    void recycle() {
       if (buffers.size() > 0) {
         setLength(0);
         
@@ -255,7 +241,7 @@
       final InvertedDocConsumer  termsHash = new TermsHash(documentsWriter, true, freqProxWriter,
                                                            new TermsHash(documentsWriter, false, termVectorsWriter, null));
       final NormsWriter normsWriter = new NormsWriter();
-      final DocInverter docInverter = new DocInverter(termsHash, normsWriter);
+      final DocInverter docInverter = new DocInverter(documentsWriter.docState, termsHash, normsWriter);
       return new DocFieldProcessor(documentsWriter, docInverter);
     }
   };
@@ -277,8 +263,6 @@
   // How much RAM we can use before flushing.  This is 0 if
   // we are flushing by doc count instead.
   private long ramBufferSize = (long) (IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB*1024*1024);
-  private long waitQueuePauseBytes = (long) (ramBufferSize*0.1);
-  private long waitQueueResumeBytes = (long) (ramBufferSize*0.05);
 
   // If we've allocated 5% over our RAM budget, we then
   // free down to 95%
@@ -290,24 +274,26 @@
 
   private int flushedDocCount;                      // How many docs already flushed to index
 
-  synchronized void updateFlushedDocCount(int n) {
+  void updateFlushedDocCount(int n) {
     flushedDocCount += n;
   }
-  synchronized int getFlushedDocCount() {
+  int getFlushedDocCount() {
     return flushedDocCount;
   }
-  synchronized void setFlushedDocCount(int n) {
+  void setFlushedDocCount(int n) {
     flushedDocCount = n;
   }
 
-  private boolean closed;
-
   DocumentsWriter(Directory directory, IndexWriter writer, IndexingChain indexingChain, int maxThreadStates) throws IOException {
     this.directory = directory;
     this.writer = writer;
     this.similarity = writer.getConfig().getSimilarity();
-    this.maxThreadStates = maxThreadStates;
     flushedDocCount = writer.maxDoc();
+    docState = new DocState();
+    docState.maxFieldLength = this.maxFieldLength;
+    docState.infoStream = this.infoStream;
+    docState.similarity = this.similarity;
+    docState.docWriter = this;
 
     consumer = indexingChain.getChain(this);
     if (consumer instanceof DocFieldProcessor) {
@@ -324,39 +310,29 @@
 
   /** If non-null, various details of indexing are printed
    *  here. */
-  synchronized void setInfoStream(PrintStream infoStream) {
+  void setInfoStream(PrintStream infoStream) {
     this.infoStream = infoStream;
-    for(int i=0;i<threadStates.length;i++)
-      threadStates[i].docState.infoStream = infoStream;
   }
 
-  synchronized void setMaxFieldLength(int maxFieldLength) {
+  void setMaxFieldLength(int maxFieldLength) {
     this.maxFieldLength = maxFieldLength;
-    for(int i=0;i<threadStates.length;i++)
-      threadStates[i].docState.maxFieldLength = maxFieldLength;
   }
 
-  synchronized void setSimilarity(Similarity similarity) {
+  void setSimilarity(Similarity similarity) {
     this.similarity = similarity;
-    for(int i=0;i<threadStates.length;i++)
-      threadStates[i].docState.similarity = similarity;
   }
 
   /** Set how much RAM we can use before flushing. */
-  synchronized void setRAMBufferSizeMB(double mb) {
+  void setRAMBufferSizeMB(double mb) {
     if (mb == IndexWriterConfig.DISABLE_AUTO_FLUSH) {
       ramBufferSize = IndexWriterConfig.DISABLE_AUTO_FLUSH;
-      waitQueuePauseBytes = 4*1024*1024;
-      waitQueueResumeBytes = 2*1024*1024;
     } else {
       ramBufferSize = (long) (mb*1024*1024);
-      waitQueuePauseBytes = (long) (ramBufferSize*0.1);
-      waitQueueResumeBytes = (long) (ramBufferSize*0.05);
       freeLevel = (long) (0.95 * ramBufferSize);
     }
   }
 
-  synchronized double getRAMBufferSizeMB() {
+  double getRAMBufferSizeMB() {
     if (ramBufferSize == IndexWriterConfig.DISABLE_AUTO_FLUSH) {
       return ramBufferSize;
     } else {
@@ -386,7 +362,7 @@
 
   /** Returns the current doc store segment we are writing
    *  to. */
-  synchronized String getDocStoreSegment() {
+  String getDocStoreSegment() {
     return docStoreSegment;
   }
 
@@ -399,10 +375,8 @@
   /** Closes the current open doc stores an returns the doc
    *  store segment name.  This returns null if there are *
    *  no buffered documents. */
-  synchronized String closeDocStore() throws IOException {
+  String closeDocStore() throws IOException {
     
-    assert allThreadsIdle();
-
     if (infoStream != null)
       message("closeDocStore: " + openFiles.size() + " files to flush to segment " + docStoreSegment + " numDocs=" + numDocsInStore);
     
@@ -447,27 +421,27 @@
   /* Returns Collection of files in use by this instance,
    * including any flushed segments. */
   @SuppressWarnings("unchecked")
-  synchronized List<String> openFiles() {
+  List<String> openFiles() {
     return (List<String>) ((ArrayList<String>) openFiles).clone();
   }
 
   @SuppressWarnings("unchecked")
-  synchronized List<String> closedFiles() {
+  List<String> closedFiles() {
     return (List<String>) ((ArrayList<String>) closedFiles).clone();
   }
 
-  synchronized void addOpenFile(String name) {
+  void addOpenFile(String name) {
     assert !openFiles.contains(name);
     openFiles.add(name);
   }
 
-  synchronized void removeOpenFile(String name) {
+  void removeOpenFile(String name) {
     assert openFiles.contains(name);
     openFiles.remove(name);
     closedFiles.add(name);
   }
 
-  synchronized void setAborting() {
+  void setAborting() {
     aborting = true;
   }
 
@@ -475,60 +449,37 @@
    *  updating the index files) and must discard all
    *  currently buffered docs.  This resets our state,
    *  discarding any docs added since last flush. */
-  synchronized void abort() throws IOException {
+  void abort() throws IOException {
 
     try {
       if (infoStream != null) {
         message("docWriter: now abort");
       }
 
-      // Forcefully remove waiting ThreadStates from line
-      waitQueue.abort();
-
-      // Wait for all other threads to finish with
-      // DocumentsWriter:
-      pauseAllThreads();
-
       try {
+        abortedFiles = openFiles();
+      } catch (Throwable t) {
+        abortedFiles = null;
+      }
 
-        assert 0 == waitQueue.numWaiting;
+      deletesInRAM.clear();
 
-        waitQueue.waitingBytes = 0;
+      openFiles.clear();
 
-        try {
-          abortedFiles = openFiles();
-        } catch (Throwable t) {
-          abortedFiles = null;
-        }
+      try {
+        consumer.abort();
+      } catch (Throwable t) {
+      }
 
-        deletesInRAM.clear();
+      docStoreSegment = null;
+      numDocsInStore = 0;
+      docStoreOffset = 0;
 
-        openFiles.clear();
+      // Reset all postings data
+      doAfterFlush();
 
-        for(int i=0;i<threadStates.length;i++)
-          try {
-            threadStates[i].consumer.abort();
-          } catch (Throwable t) {
-          }
-
-        try {
-          consumer.abort();
-        } catch (Throwable t) {
-        }
-
-        docStoreSegment = null;
-        numDocsInStore = 0;
-        docStoreOffset = 0;
-
-        // Reset all postings data
-        doAfterFlush();
-
-      } finally {
-        resumeAllThreads();
-      }
     } finally {
       aborting = false;
-      notifyAll();
       if (infoStream != null) {
         message("docWriter: done abort");
       }
@@ -537,55 +488,22 @@
 
   /** Reset after a flush */
   private void doAfterFlush() throws IOException {
-    // All ThreadStates should be idle when we are called
-    assert allThreadsIdle();
-    threadBindings.clear();
-    waitQueue.reset();
     segment = null;
     numDocsInRAM = 0;
     nextDocID = 0;
     bufferIsFull = false;
     flushPending = false;
-    for(int i=0;i<threadStates.length;i++)
-      threadStates[i].doAfterFlush();
+    doFlushAfter = false;
   }
 
-  // Returns true if an abort is in progress
-  synchronized boolean pauseAllThreads() {
-    pauseThreads++;
-    while(!allThreadsIdle()) {
-      try {
-        wait();
-      } catch (InterruptedException ie) {
-        throw new ThreadInterruptedException(ie);
-      }
-    }
-
-    return aborting;
-  }
-
-  synchronized void resumeAllThreads() {
-    pauseThreads--;
-    assert pauseThreads >= 0;
-    if (0 == pauseThreads)
-      notifyAll();
-  }
-
-  private synchronized boolean allThreadsIdle() {
-    for(int i=0;i<threadStates.length;i++)
-      if (!threadStates[i].isIdle)
-        return false;
-    return true;
-  }
-
-  synchronized boolean anyChanges() {
+  boolean anyChanges() {
     return numDocsInRAM != 0 ||
       deletesInRAM.numTerms != 0 ||
       deletesInRAM.docIDs.size() != 0 ||
       deletesInRAM.queries.size() != 0;
   }
 
-  synchronized private void initFlushState(boolean onlyDocStore) {
+  private void initFlushState(boolean onlyDocStore) {
     initSegmentName(onlyDocStore);
     flushState = new SegmentWriteState(infoStream, directory, segment, docFieldProcessor.fieldInfos,
                                        docStoreSegment, numDocsInRAM, numDocsInStore, writer.getConfig().getTermIndexInterval(),
@@ -598,15 +516,11 @@
   }
   
   /** Flush all pending docs to a new segment */
-  synchronized int flush(boolean closeDocStore) throws IOException {
+  int flush(boolean closeDocStore) throws IOException {
 
-    assert allThreadsIdle();
-
     assert numDocsInRAM > 0;
 
     assert nextDocID == numDocsInRAM;
-    assert waitQueue.numWaiting == 0;
-    assert waitQueue.waitingBytes == 0;
 
     initFlushState(false);
 
@@ -626,10 +540,7 @@
         flushState.numDocsInStore = 0;
       }
 
-      Collection<DocConsumerPerThread> threads = new HashSet<DocConsumerPerThread>();
-      for(int i=0;i<threadStates.length;i++)
-        threads.add(threadStates[i].consumer);
-      consumer.flush(threads, flushState);
+      consumer.flush(flushState);
 
       if (infoStream != null) {
         SegmentInfo si = new SegmentInfo(flushState.segmentName, flushState.numDocs, directory, flushState.codec);
@@ -654,8 +565,6 @@
       }
     }
 
-    assert waitQueue.waitingBytes == 0;
-
     return flushState.numDocs;
   }
 
@@ -676,7 +585,7 @@
    *  whether it was set. This is used by IndexWriter to
    *  trigger a single flush even when multiple threads are
    *  trying to do so. */
-  synchronized boolean setFlushPending() {
+  boolean setFlushPending() {
     if (flushPending)
       return false;
     else {
@@ -685,21 +594,16 @@
     }
   }
 
-  synchronized void clearFlushPending() {
+  void clearFlushPending() {
     bufferIsFull = false;
     flushPending = false;
   }
 
-  synchronized void pushDeletes() {
+  void pushDeletes() {
     deletesFlushed.update(deletesInRAM);
   }
 
-  synchronized void close() {
-    closed = true;
-    notifyAll();
-  }
-
-  synchronized void initSegmentName(boolean onlyDocStore) {
+  void initSegmentName(boolean onlyDocStore) {
     if (segment == null && (!onlyDocStore || docStoreSegment == null)) {
       segment = writer.newSegmentName();
       assert numDocsInRAM == 0;
@@ -715,56 +619,21 @@
    * flush is pending.  If delTerm is non-null then we
    * buffer this deleted term after the thread state has
    * been acquired. */
-  synchronized DocumentsWriterThreadState getThreadState(Document doc, Term delTerm) throws IOException {
+  void startDocument(Document doc, Term delTerm) throws IOException {
 
-    // First, find a thread state.  If this thread already
-    // has affinity to a specific ThreadState, use that one
-    // again.
-    DocumentsWriterThreadState state = threadBindings.get(Thread.currentThread());
-    if (state == null) {
-
-      // First time this thread has called us since last
-      // flush.  Find the least loaded thread state:
-      DocumentsWriterThreadState minThreadState = null;
-      for(int i=0;i<threadStates.length;i++) {
-        DocumentsWriterThreadState ts = threadStates[i];
-        if (minThreadState == null || ts.numThreads < minThreadState.numThreads)
-          minThreadState = ts;
-      }
-      if (minThreadState != null && (minThreadState.numThreads == 0 || threadStates.length >= maxThreadStates)) {
-        state = minThreadState;
-        state.numThreads++;
-      } else {
-        // Just create a new "private" thread state
-        DocumentsWriterThreadState[] newArray = new DocumentsWriterThreadState[1+threadStates.length];
-        if (threadStates.length > 0)
-          System.arraycopy(threadStates, 0, newArray, 0, threadStates.length);
-        state = newArray[threadStates.length] = new DocumentsWriterThreadState(this);
-        threadStates = newArray;
-      }
-      threadBindings.put(Thread.currentThread(), state);
-    }
-
-    // Next, wait until my thread state is idle (in case
-    // it's shared with other threads) and for threads to
-    // not be paused nor a flush pending:
-    waitReady(state);
-
     // Allocate segment name if this is the first doc since
     // last flush:
     initSegmentName(false);
 
-    state.isIdle = false;
-
     boolean success = false;
     try {
-      state.docState.docID = nextDocID;
+      docState.docID = nextDocID;
 
-      assert writer.testPoint("DocumentsWriter.ThreadState.init start");
+      assert writer.testPoint("DocumentsWriter.startDocument() start");
 
       if (delTerm != null) {
-        addDeleteTerm(delTerm, state.docState.docID);
-        state.doFlushAfter = timeToFlushDeletes();
+        addDeleteTerm(delTerm, docState.docID);
+        doFlushAfter = timeToFlushDeletes();
       }
 
       assert writer.testPoint("DocumentsWriter.ThreadState.init after delTerm");
@@ -779,23 +648,19 @@
           maxBufferedDocs != IndexWriterConfig.DISABLE_AUTO_FLUSH
           && numDocsInRAM >= maxBufferedDocs) {
         flushPending = true;
-        state.doFlushAfter = true;
+        doFlushAfter = true;
       }
 
       success = true;
     } finally {
       if (!success) {
         // Forcefully idle this ThreadState:
-        state.isIdle = true;
-        notifyAll();
-        if (state.doFlushAfter) {
-          state.doFlushAfter = false;
+        if (doFlushAfter) {
+          doFlushAfter = false;
           flushPending = false;
         }
       }
     }
-
-    return state;
   }
 
   /** Returns true if the caller (IndexWriter) should now
@@ -813,10 +678,8 @@
   boolean updateDocument(Document doc, Analyzer analyzer, Term delTerm)
     throws CorruptIndexException, IOException {
 
-    // This call is synchronized but fast
-    final DocumentsWriterThreadState state = getThreadState(doc, delTerm);
+    startDocument(doc, delTerm);
 
-    final DocState docState = state.docState;
     docState.doc = doc;
     docState.analyzer = analyzer;
 
@@ -824,70 +687,72 @@
     try {
       // This call is not synchronized and does all the
       // work
-      final DocWriter perDoc = state.consumer.processDocument();
+      final DocWriter perDoc = consumer.processDocument();
         
       // This call is synchronized but fast
-      finishDocument(state, perDoc);
+      finishDocument(perDoc);
       success = true;
     } finally {
       if (!success) {
-        synchronized(this) {
-
-          if (aborting) {
-            state.isIdle = true;
-            notifyAll();
-            abort();
-          } else {
-            skipDocWriter.docID = docState.docID;
-            boolean success2 = false;
-            try {
-              waitQueue.add(skipDocWriter);
-              success2 = true;
-            } finally {
-              if (!success2) {
-                state.isIdle = true;
-                notifyAll();
-                abort();
-                return false;
-              }
+        if (aborting) {
+          abort();
+        } else {
+          skipDocWriter.docID = docState.docID;
+          boolean success2 = false;
+          try {
+            writeDocument(skipDocWriter);
+            success2 = true;
+          } finally {
+            if (!success2) {
+              abort();
+              return false;
             }
+          }
 
-            state.isIdle = true;
-            notifyAll();
-
-            // If this thread state had decided to flush, we
-            // must clear it so another thread can flush
-            if (state.doFlushAfter) {
-              state.doFlushAfter = false;
-              flushPending = false;
-              notifyAll();
-            }
-
-            // Immediately mark this document as deleted
-            // since likely it was partially added.  This
-            // keeps indexing as "all or none" (atomic) when
-            // adding a document:
-            addDeleteDocID(state.docState.docID);
+          // If this thread state had decided to flush, we
+          // must clear it so another thread can flush
+          if (doFlushAfter) {
+            doFlushAfter = false;
+            flushPending = false;
           }
+
+          // Immediately mark this document as deleted
+          // since likely it was partially added.  This
+          // keeps indexing as "all or none" (atomic) when
+          // adding a document:
+          addDeleteDocID(docState.docID);
         }
       }
     }
 
-    return state.doFlushAfter || timeToFlushDeletes();
+    return doFlushAfter || timeToFlushDeletes();
   }
 
+  private void writeDocument(DocWriter doc) throws IOException {
+    boolean success = false;
+    try {
+      doc.finish();
+      numDocsInStore++;
+      success = true;
+    } finally {
+      if (!success)
+        setAborting();
+    }
+  }
+
+  
   // for testing
-  synchronized int getNumBufferedDeleteTerms() {
+  int getNumBufferedDeleteTerms() {
     return deletesInRAM.numTerms;
   }
 
   // for testing
-  synchronized Map<Term,BufferedDeletes.Num> getBufferedDeleteTerms() {
+  Map<Term,BufferedDeletes.Num> getBufferedDeleteTerms() {
     return deletesInRAM.terms;
   }
 
   /** Called whenever a merge has completed and the merged segments had deletions */
-  synchronized void remapDeletes(SegmentInfos infos, int[][] docMaps, int[] delCounts, MergePolicy.OneMerge merge, int mergeDocCount) {
+  void remapDeletes(SegmentInfos infos, int[][] docMaps, int[] delCounts, MergePolicy.OneMerge merge, int mergeDocCount) {
     if (docMaps == null)
       // The merged segments had no deletes so docIDs did not change and we have nothing to do
       return;
@@ -897,62 +762,36 @@
     flushedDocCount -= mapper.docShift;
   }
 
-  synchronized private void waitReady(DocumentsWriterThreadState state) {
-
-    while (!closed && ((state != null && !state.isIdle) || pauseThreads != 0 || flushPending || aborting)) {
-      try {
-        wait();
-      } catch (InterruptedException ie) {
-        throw new ThreadInterruptedException(ie);
-      }
-    }
-
-    if (closed)
-      throw new AlreadyClosedException("this IndexWriter is closed");
-  }
-
   boolean bufferDeleteTerms(Term[] terms) throws IOException {
-    synchronized(this) {
-      waitReady(null);
-      for (int i = 0; i < terms.length; i++)
-        addDeleteTerm(terms[i], numDocsInRAM);
-    }
+    for (int i = 0; i < terms.length; i++)
+      addDeleteTerm(terms[i], numDocsInRAM);
     return timeToFlushDeletes();
   }
 
   boolean bufferDeleteTerm(Term term) throws IOException {
-    synchronized(this) {
-      waitReady(null);
-      addDeleteTerm(term, numDocsInRAM);
-    }
+    addDeleteTerm(term, numDocsInRAM);
     return timeToFlushDeletes();
   }
 
   boolean bufferDeleteQueries(Query[] queries) throws IOException {
-    synchronized(this) {
-      waitReady(null);
-      for (int i = 0; i < queries.length; i++)
-        addDeleteQuery(queries[i], numDocsInRAM);
-    }
+    for (int i = 0; i < queries.length; i++)
+      addDeleteQuery(queries[i], numDocsInRAM);
     return timeToFlushDeletes();
   }
 
   boolean bufferDeleteQuery(Query query) throws IOException {
-    synchronized(this) {
-      waitReady(null);
-      addDeleteQuery(query, numDocsInRAM);
-    }
+    addDeleteQuery(query, numDocsInRAM);
     return timeToFlushDeletes();
   }
 
-  synchronized boolean deletesFull() {
+  boolean deletesFull() {
     return (ramBufferSize != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
             (deletesInRAM.bytesUsed + deletesFlushed.bytesUsed + numBytesUsed) >= ramBufferSize) ||
       (maxBufferedDeleteTerms != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
        ((deletesInRAM.size() + deletesFlushed.size()) >= maxBufferedDeleteTerms));
   }
 
-  synchronized boolean doApplyDeletes() {
+  boolean doApplyDeletes() {
     // Very similar to deletesFull(), except we don't count
     // numBytesUsed, because we are checking whether
     // deletes (alone) are consuming too many resources now
@@ -969,9 +808,7 @@
 
   private boolean timeToFlushDeletes() {
     balanceRAM();
-    synchronized(this) {
-      return (bufferIsFull || deletesFull()) && setFlushPending();
-    }
+    return (bufferIsFull || deletesFull()) && setFlushPending();
   }
 
   void setMaxBufferedDeleteTerms(int maxBufferedDeleteTerms) {
@@ -982,11 +819,11 @@
     return maxBufferedDeleteTerms;
   }
 
-  synchronized boolean hasDeletes() {
+  boolean hasDeletes() {
     return deletesFlushed.any();
   }
 
-  synchronized boolean applyDeletes(SegmentInfos infos) throws IOException {
+  boolean applyDeletes(SegmentInfos infos) throws IOException {
 
     if (!hasDeletes())
       return false;
@@ -1040,7 +877,7 @@
 
   // Apply buffered delete terms, queries and docIDs to the
   // provided reader
-  private final synchronized boolean applyDeletes(IndexReader reader, int docIDStart)
+  private final boolean applyDeletes(IndexReader reader, int docIDStart)
     throws CorruptIndexException, IOException {
 
     final int docEnd = docIDStart + reader.maxDoc();
@@ -1143,7 +980,7 @@
   // current number of documents buffered in ram so that the
   // delete term will be applied to those documents as well
   // as the disk segments.
-  synchronized private void addDeleteTerm(Term term, int docCount) {
+  private void addDeleteTerm(Term term, int docCount) {
     BufferedDeletes.Num num = deletesInRAM.terms.get(term);
     final int docIDUpto = flushedDocCount + docCount;
     if (num == null)
@@ -1157,77 +994,55 @@
 
   // Buffer a specific docID for deletion.  Currently only
   // used when we hit a exception when adding a document
-  synchronized private void addDeleteDocID(int docID) {
+  private void addDeleteDocID(int docID) {
     deletesInRAM.docIDs.add(Integer.valueOf(flushedDocCount+docID));
     deletesInRAM.addBytesUsed(BYTES_PER_DEL_DOCID);
   }
 
-  synchronized private void addDeleteQuery(Query query, int docID) {
+  private void addDeleteQuery(Query query, int docID) {
     deletesInRAM.queries.put(query, Integer.valueOf(flushedDocCount + docID));
     deletesInRAM.addBytesUsed(BYTES_PER_DEL_QUERY);
   }
 
   /** Does the synchronized work to finish/flush the
    *  inverted document. */
-  private void finishDocument(DocumentsWriterThreadState perThread, DocWriter docWriter) throws IOException {
+  private void finishDocument(DocWriter docWriter) throws IOException {
 
     // Must call this w/o holding synchronized(this) else
     // we'll hit deadlock:
     balanceRAM();
 
-    synchronized(this) {
+    assert docWriter == null || docWriter.docID == docState.docID;
 
-      assert docWriter == null || docWriter.docID == perThread.docState.docID;
+    if (aborting) {
 
-      if (aborting) {
-
-        // We are currently aborting, and another thread is
-        // waiting for me to become idle.  We just forcefully
-        // idle this threadState; it will be fully reset by
-        // abort()
-        if (docWriter != null)
-          try {
-            docWriter.abort();
-          } catch (Throwable t) {
-          }
-
-        perThread.isIdle = true;
-        notifyAll();
-        return;
-      }
-
-      final boolean doPause;
-
+      // We are currently aborting, and another thread is
+      // waiting for me to become idle.  We just forcefully
+      // idle this threadState; it will be fully reset by
+      // abort()
       if (docWriter != null)
-        doPause = waitQueue.add(docWriter);
-      else {
-        skipDocWriter.docID = perThread.docState.docID;
-        doPause = waitQueue.add(skipDocWriter);
-      }
+        try {
+          docWriter.abort();
+        } catch (Throwable t) {
+        }
 
-      if (doPause)
-        waitForWaitQueue();
+      return;
+    }
 
-      if (bufferIsFull && !flushPending) {
-        flushPending = true;
-        perThread.doFlushAfter = true;
-      }
+    if (docWriter != null)
+      writeDocument(docWriter);
+    else {
+      skipDocWriter.docID = docState.docID;
+      writeDocument(skipDocWriter);
+    }
 
-      perThread.isIdle = true;
-      notifyAll();
+
+    if (bufferIsFull && !flushPending) {
+      flushPending = true;
+      doFlushAfter = true;
     }
   }
 
-  synchronized void waitForWaitQueue() {
-    do {
-      try {
-        wait();
-      } catch (InterruptedException ie) {
-        throw new ThreadInterruptedException(ie);
-      }
-    } while (!waitQueue.doResume());
-  }
-
   private static class SkipDocWriter extends DocWriter {
     @Override
     void finish() {
@@ -1301,40 +1116,34 @@
     /* Allocate another byte[] from the shared pool */
     @Override
     byte[] getByteBlock(boolean trackAllocations) {
-      synchronized(DocumentsWriter.this) {
-        final int size = freeByteBlocks.size();
-        final byte[] b;
-        if (0 == size) {
-          b = new byte[blockSize];
-          // Always record a block allocated, even if
-          // trackAllocations is false.  This is necessary
-          // because this block will be shared between
-          // things that don't track allocations (term
-          // vectors) and things that do (freq/prox
-          // postings).
-          numBytesUsed += blockSize;
-        } else
-          b = freeByteBlocks.remove(size-1);
-        return b;
-      }
+      final int size = freeByteBlocks.size();
+      final byte[] b;
+      if (0 == size) {
+        b = new byte[blockSize];
+        // Always record a block allocated, even if
+        // trackAllocations is false.  This is necessary
+        // because this block will be shared between
+        // things that don't track allocations (term
+        // vectors) and things that do (freq/prox
+        // postings).
+        numBytesUsed += blockSize;
+      } else
+        b = freeByteBlocks.remove(size-1);
+      return b;
     }
 
     /* Return byte[]'s to the pool */
     @Override
     void recycleByteBlocks(byte[][] blocks, int start, int end) {
-      synchronized(DocumentsWriter.this) {
-        for(int i=start;i<end;i++)
-          freeByteBlocks.add(blocks[i]);
-      }
+      for(int i=start;i<end;i++)
+        freeByteBlocks.add(blocks[i]);
     }
 
     @Override
     void recycleByteBlocks(List<byte[]> blocks) {
-      synchronized(DocumentsWriter.this) {
-        final int size = blocks.size();
-        for(int i=0;i<size;i++)
-          freeByteBlocks.add(blocks.get(i));
-  }
+      final int size = blocks.size();
+      for(int i=0;i<size;i++)
+        freeByteBlocks.add(blocks.get(i));
     }
   }
 
@@ -1347,7 +1156,7 @@
   private ArrayList<int[]> freeIntBlocks = new ArrayList<int[]>();
 
   /* Allocate another int[] from the shared pool */
-  synchronized int[] getIntBlock(boolean trackAllocations) {
+  int[] getIntBlock(boolean trackAllocations) {
     final int size = freeIntBlocks.size();
     final int[] b;
     if (0 == size) {
@@ -1364,12 +1173,12 @@
     return b;
   }
 
-  synchronized void bytesUsed(long numBytes) {
+  void bytesUsed(long numBytes) {
     numBytesUsed += numBytes;
   }
 
   /* Return int[]s to the pool */
-  synchronized void recycleIntBlocks(int[][] blocks, int start, int end) {
+  void recycleIntBlocks(int[][] blocks, int start, int end) {
     for(int i=start;i<end;i++)
       freeIntBlocks.add(blocks[i]);
   }
@@ -1399,14 +1208,12 @@
     final boolean doBalance;
     final long deletesRAMUsed;
 
-    synchronized(this) {
-      if (ramBufferSize == IndexWriterConfig.DISABLE_AUTO_FLUSH || bufferIsFull) {
-        return;
-      }
-    
-      deletesRAMUsed = deletesInRAM.bytesUsed+deletesFlushed.bytesUsed;
-      doBalance = numBytesUsed+deletesRAMUsed >= ramBufferSize;
+    if (ramBufferSize == IndexWriterConfig.DISABLE_AUTO_FLUSH || bufferIsFull) {
+      return;
     }
+  
+    deletesRAMUsed = deletesInRAM.bytesUsed+deletesFlushed.bytesUsed;
+    doBalance = numBytesUsed+deletesRAMUsed >= ramBufferSize;
 
     if (doBalance) {
 
@@ -1429,39 +1236,37 @@
 
       while(numBytesUsed+deletesRAMUsed > freeLevel) {
       
-        synchronized(this) {
-          if (0 == perDocAllocator.freeByteBlocks.size() &&
-              0 == byteBlockAllocator.freeByteBlocks.size() &&
-              0 == freeIntBlocks.size() && !any) {
-            // Nothing else to free -- must flush now.
-            bufferIsFull = numBytesUsed+deletesRAMUsed > ramBufferSize;
-            if (infoStream != null) {
-              if (numBytesUsed+deletesRAMUsed > ramBufferSize)
-                message("    nothing to free; now set bufferIsFull");
-              else
-                message("    nothing to free");
-            }
-            break;
+        if (0 == perDocAllocator.freeByteBlocks.size() &&
+            0 == byteBlockAllocator.freeByteBlocks.size() &&
+            0 == freeIntBlocks.size() && !any) {
+          // Nothing else to free -- must flush now.
+          bufferIsFull = numBytesUsed+deletesRAMUsed > ramBufferSize;
+          if (infoStream != null) {
+            if (numBytesUsed+deletesRAMUsed > ramBufferSize)
+              message("    nothing to free; now set bufferIsFull");
+            else
+              message("    nothing to free");
           }
+          break;
+        }
 
-          if ((0 == iter % 4) && byteBlockAllocator.freeByteBlocks.size() > 0) {
-            byteBlockAllocator.freeByteBlocks.remove(byteBlockAllocator.freeByteBlocks.size()-1);
-            numBytesUsed -= BYTE_BLOCK_SIZE;
-          }
+        if ((0 == iter % 4) && byteBlockAllocator.freeByteBlocks.size() > 0) {
+          byteBlockAllocator.freeByteBlocks.remove(byteBlockAllocator.freeByteBlocks.size()-1);
+          numBytesUsed -= BYTE_BLOCK_SIZE;
+        }
 
-          if ((1 == iter % 4) && freeIntBlocks.size() > 0) {
-            freeIntBlocks.remove(freeIntBlocks.size()-1);
-            numBytesUsed -= INT_BLOCK_SIZE * INT_NUM_BYTE;
-          }
+        if ((1 == iter % 4) && freeIntBlocks.size() > 0) {
+          freeIntBlocks.remove(freeIntBlocks.size()-1);
+          numBytesUsed -= INT_BLOCK_SIZE * INT_NUM_BYTE;
+        }
 
-          if ((2 == iter % 4) && perDocAllocator.freeByteBlocks.size() > 0) {
-            // Remove upwards of 32 blocks (each block is 1K)
-            for (int i = 0; i < 32; ++i) {
-              perDocAllocator.freeByteBlocks.remove(perDocAllocator.freeByteBlocks.size() - 1);
-              numBytesUsed -= PER_DOC_BLOCK_SIZE;
-              if (perDocAllocator.freeByteBlocks.size() == 0) {
-                break;
-              }
+        if ((2 == iter % 4) && perDocAllocator.freeByteBlocks.size() > 0) {
+          // Remove upwards of 32 blocks (each block is 1K)
+          for (int i = 0; i < 32; ++i) {
+            perDocAllocator.freeByteBlocks.remove(perDocAllocator.freeByteBlocks.size() - 1);
+            numBytesUsed -= PER_DOC_BLOCK_SIZE;
+            if (perDocAllocator.freeByteBlocks.size() == 0) {
+              break;
             }
           }
         }
@@ -1477,119 +1282,4 @@
         message("    after free: freedMB=" + nf.format((startBytesUsed-numBytesUsed-deletesRAMUsed)/1024./1024.) + " usedMB=" + nf.format((numBytesUsed+deletesRAMUsed)/1024./1024.));
     }
   }
-
-  final WaitQueue waitQueue = new WaitQueue();
-
-  private class WaitQueue {
-    DocWriter[] waiting;
-    int nextWriteDocID;
-    int nextWriteLoc;
-    int numWaiting;
-    long waitingBytes;
-
-    public WaitQueue() {
-      waiting = new DocWriter[10];
-    }
-
-    synchronized void reset() {
-      // NOTE: nextWriteLoc doesn't need to be reset
-      assert numWaiting == 0;
-      assert waitingBytes == 0;
-      nextWriteDocID = 0;
-    }
-
-    synchronized boolean doResume() {
-      return waitingBytes <= waitQueueResumeBytes;
-    }
-
-    synchronized boolean doPause() {
-      return waitingBytes > waitQueuePauseBytes;
-    }
-
-    synchronized void abort() {
-      int count = 0;
-      for(int i=0;i<waiting.length;i++) {
-        final DocWriter doc = waiting[i];
-        if (doc != null) {
-          doc.abort();
-          waiting[i] = null;
-          count++;
-        }
-      }
-      waitingBytes = 0;
-      assert count == numWaiting;
-      numWaiting = 0;
-    }
-
-    private void writeDocument(DocWriter doc) throws IOException {
-      assert doc == skipDocWriter || nextWriteDocID == doc.docID;
-      boolean success = false;
-      try {
-        doc.finish();
-        nextWriteDocID++;
-        numDocsInStore++;
-        nextWriteLoc++;
-        assert nextWriteLoc <= waiting.length;
-        if (nextWriteLoc == waiting.length)
-          nextWriteLoc = 0;
-        success = true;
-      } finally {
-        if (!success)
-          setAborting();
-      }
-    }
-
-    synchronized public boolean add(DocWriter doc) throws IOException {
-
-      assert doc.docID >= nextWriteDocID;
-
-      if (doc.docID == nextWriteDocID) {
-        writeDocument(doc);
-        while(true) {
-          doc = waiting[nextWriteLoc];
-          if (doc != null) {
-            numWaiting--;
-            waiting[nextWriteLoc] = null;
-            waitingBytes -= doc.sizeInBytes();
-            writeDocument(doc);
-          } else
-            break;
-        }
-      } else {
-
-        // I finished before documents that were added
-        // before me.  This can easily happen when I am a
-        // small doc and the docs before me were large, or,
-        // just due to luck in the thread scheduling.  Just
-        // add myself to the queue and when that large doc
-        // finishes, it will flush me:
-        int gap = doc.docID - nextWriteDocID;
-        if (gap >= waiting.length) {
-          // Grow queue
-          DocWriter[] newArray = new DocWriter[ArrayUtil.oversize(gap, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-          assert nextWriteLoc >= 0;
-          System.arraycopy(waiting, nextWriteLoc, newArray, 0, waiting.length-nextWriteLoc);
-          System.arraycopy(waiting, 0, newArray, waiting.length-nextWriteLoc, nextWriteLoc);
-          nextWriteLoc = 0;
-          waiting = newArray;
-          gap = doc.docID - nextWriteDocID;
-        }
-
-        int loc = nextWriteLoc + gap;
-        if (loc >= waiting.length)
-          loc -= waiting.length;
-
-        // We should only wrap one time
-        assert loc < waiting.length;
-
-        // Nobody should be in my spot!
-        assert waiting[loc] == null;
-        waiting[loc] = doc;
-        numWaiting++;
-        waitingBytes += doc.sizeInBytes();
-      }
-      
-      return doPause();
-    }
-  }
 }
Index: src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
===================================================================
--- src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java	(revision 934193)
+++ src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java	(working copy)
@@ -26,7 +26,6 @@
 
 final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
 
-  final TermVectorsTermsWriterPerThread perThread;
   final TermsHashPerField termsHashPerField;
   final TermVectorsTermsWriter termsWriter;
   final FieldInfo fieldInfo;
@@ -40,10 +39,9 @@
   int maxNumPostings;
   OffsetAttribute offsetAttribute = null;
   
-  public TermVectorsTermsWriterPerField(TermsHashPerField termsHashPerField, TermVectorsTermsWriterPerThread perThread, FieldInfo fieldInfo) {
+  public TermVectorsTermsWriterPerField(TermsHashPerField termsHashPerField, TermVectorsTermsWriter termsWriter, FieldInfo fieldInfo) {
     this.termsHashPerField = termsHashPerField;
-    this.perThread = perThread;
-    this.termsWriter = perThread.termsWriter;
+    this.termsWriter = termsWriter;
     this.fieldInfo = fieldInfo;
     docState = termsHashPerField.docState;
     fieldState = termsHashPerField.fieldState;
@@ -70,14 +68,14 @@
     }
 
     if (doVectors) {
-      if (perThread.doc == null) {
-        perThread.doc = termsWriter.getPerDoc();
-        perThread.doc.docID = docState.docID;
-        assert perThread.doc.numVectorFields == 0;
-        assert 0 == perThread.doc.perDocTvf.length();
-        assert 0 == perThread.doc.perDocTvf.getFilePointer();
+      if (termsWriter.doc == null) {
+        termsWriter.doc = termsWriter.getPerDoc();
+        termsWriter.doc.docID = docState.docID;
+        assert termsWriter.doc.numVectorFields == 0;
+        assert 0 == termsWriter.doc.perDocTvf.length();
+        assert 0 == termsWriter.doc.perDocTvf.getFilePointer();
       } else {
-        assert perThread.doc.docID == docState.docID;
+        assert termsWriter.doc.docID == docState.docID;
 
         if (termsHashPerField.numPostings != 0)
           // Only necessary if previous doc hit a
@@ -106,7 +104,7 @@
 
     final int numPostings = termsHashPerField.numPostings;
 
-    final BytesRef flushTerm = perThread.flushTerm;
+    final BytesRef flushTerm = termsWriter.flushTerm;
 
     assert numPostings >= 0;
 
@@ -116,16 +114,16 @@
     if (numPostings > maxNumPostings)
       maxNumPostings = numPostings;
 
-    final IndexOutput tvf = perThread.doc.perDocTvf;
+    final IndexOutput tvf = termsWriter.doc.perDocTvf;
 
     // This is called once, after inverting all occurrences
     // of a given field in the doc.  At this point we flush
     // our hash into the DocWriter.
 
     assert fieldInfo.storeTermVector;
-    assert perThread.vectorFieldsInOrder(fieldInfo);
+    assert termsWriter.vectorFieldsInOrder(fieldInfo);
 
-    perThread.doc.addField(termsHashPerField.fieldInfo.number);
+    termsWriter.doc.addField(termsHashPerField.fieldInfo.number);
     TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
 
     // TODO: we may want to make this sort in same order
@@ -144,8 +142,8 @@
     byte[] lastBytes = null;
     int lastStart = 0;
       
-    final ByteSliceReader reader = perThread.vectorSliceReader;
-    final ByteBlockPool termBytePool = perThread.termsHashPerThread.termBytePool;
+    final ByteSliceReader reader = termsWriter.vectorSliceReader;
+    final ByteBlockPool termBytePool = termsHashPerField.termBytePool;
 
     for(int j=0;j<numPostings;j++) {
       final int termID = termIDs[j];
@@ -188,7 +186,7 @@
     }
 
     termsHashPerField.reset();
-    perThread.termsHashPerThread.reset(false);
+    termsHashPerField.termsHash.reset();
   }
 
   void shrinkHash() {
Index: src/java/org/apache/lucene/index/IndexWriter.java
===================================================================
--- src/java/org/apache/lucene/index/IndexWriter.java	(revision 934193)
+++ src/java/org/apache/lucene/index/IndexWriter.java	(working copy)
@@ -1755,13 +1755,13 @@
 
   private void closeInternal(boolean waitForMerges) throws CorruptIndexException, IOException {
 
-    docWriter.pauseAllThreads();
+//    docWriter.pauseAllThreads();
 
     try {
       if (infoStream != null)
         message("now flush at close");
 
-      docWriter.close();
+      //docWriter.close();
 
       // Only allow a new merge to be triggered if we are
       // going to wait for merges:
@@ -1811,8 +1811,9 @@
         closing = false;
         notifyAll();
         if (!closed) {
-          if (docWriter != null)
-            docWriter.resumeAllThreads();
+          if (docWriter != null) {
+            //docWriter.resumeAllThreads();
+          }
           if (infoStream != null)
             message("hit exception while closing");
         }
@@ -2815,7 +2816,7 @@
 
     boolean success = false;
 
-    docWriter.pauseAllThreads();
+    //docWriter.pauseAllThreads();
 
     try {
       finishMerges(false);
@@ -2866,7 +2867,6 @@
     } finally {
       synchronized(this) {
         if (!success) {
-          docWriter.resumeAllThreads();
           closing = false;
           notifyAll();
           if (infoStream != null)
@@ -2894,7 +2894,6 @@
    *    will receive {@link MergePolicy.MergeAbortedException}s.
    */
   public synchronized void deleteAll() throws IOException {
-    docWriter.pauseAllThreads();
     try {
 
       // Abort any running merges
@@ -2919,7 +2918,6 @@
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteAll");
     } finally {
-      docWriter.resumeAllThreads();
       if (infoStream != null) {
         message("hit exception during deleteAll");
       }
@@ -3109,8 +3107,6 @@
     noDupDirs(dirs);
 
     // Do not allow add docs or deletes while we are running:
-    docWriter.pauseAllThreads();
-
     try {
       if (infoStream != null)
         message("flush at addIndexesNoOptimize");
@@ -3172,7 +3168,7 @@
       handleOOM(oom, "addIndexesNoOptimize");
     } finally {
       if (docWriter != null) {
-        docWriter.resumeAllThreads();
+//        docWriter.resumeAllThreads();
       }
     }
   }
@@ -3281,9 +3277,6 @@
 
     ensureOpen();
 
-    // Do not allow add docs or deletes while we are running:
-    docWriter.pauseAllThreads();
-
     // We must pre-acquire a read lock here (and upgrade to
     // write lock in startTransaction below) so that no
     // other addIndexes is allowed to start up after we have
@@ -3415,7 +3408,7 @@
       handleOOM(oom, "addIndexes(IndexReader...)");
     } finally {
       if (docWriter != null) {
-        docWriter.resumeAllThreads();
+        //docWriter.resumeAllThreads();
       }
     }
   }
@@ -3651,10 +3644,10 @@
     // Make sure no threads are actively adding a document.
     // Returns true if docWriter is currently aborting, in
     // which case we skip flushing this segment
-    if (docWriter.pauseAllThreads()) {
-      docWriter.resumeAllThreads();
-      return false;
-    }
+//    if (docWriter.pauseAllThreads()) {
+//      docWriter.resumeAllThreads();
+//      return false;
+//    }
 
     try {
 
@@ -3788,7 +3781,7 @@
       return false;
     } finally {
       docWriter.clearFlushPending();
-      docWriter.resumeAllThreads();
+      //docWriter.resumeAllThreads();
     }
   }
 
Index: src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java	(working copy)
@@ -1,25 +0,0 @@
-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.
- */
-
-abstract class InvertedDocEndConsumerPerThread {
-  abstract void startDocument();
-  abstract InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
-  abstract void finishDocument();
-  abstract void abort();
-}
Index: src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java	(working copy)
@@ -1,45 +0,0 @@
-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.
- */
-
-final class FreqProxTermsWriterPerThread extends TermsHashConsumerPerThread {
-  final TermsHashPerThread termsHashPerThread;
-  final DocumentsWriter.DocState docState;
-
-  public FreqProxTermsWriterPerThread(TermsHashPerThread perThread) {
-    docState = perThread.docState;
-    termsHashPerThread = perThread;
-  }
-  
-  @Override
-  public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
-    return new FreqProxTermsWriterPerField(termsHashPerField, this, fieldInfo);
-  }
-
-  @Override
-  void startDocument() {
-  }
-
-  @Override
-  DocumentsWriter.DocWriter finishDocument() {
-    return null;
-  }
-
-  @Override
-  public void abort() {}
-}
Index: src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java	(working copy)
@@ -1,27 +0,0 @@
-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 InvertedDocConsumerPerThread {
-  abstract void startDocument() throws IOException;
-  abstract InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
-  abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
-  abstract void abort();
-}
Index: src/java/org/apache/lucene/index/DocConsumer.java
===================================================================
--- src/java/org/apache/lucene/index/DocConsumer.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocConsumer.java	(working copy)
@@ -18,11 +18,10 @@
  */
 
 import java.io.IOException;
-import java.util.Collection;
 
 abstract class DocConsumer {
-  abstract DocConsumerPerThread addThread(DocumentsWriterThreadState perThread) throws IOException;
-  abstract void flush(final Collection<DocConsumerPerThread> threads, final SegmentWriteState state) throws IOException;
+  abstract DocumentsWriter.DocWriter processDocument() throws IOException;
+  abstract void flush(final SegmentWriteState state) throws IOException;
   abstract void closeDocStore(final SegmentWriteState state) throws IOException;
   abstract void abort();
   abstract boolean freeRAM();
Index: src/java/org/apache/lucene/index/DocInverterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocInverterPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocInverterPerThread.java	(working copy)
@@ -1,92 +0,0 @@
-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.util.AttributeSource;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-
-/** 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 SingleTokenAttributeSource singleToken = new SingleTokenAttributeSource();
-  
-  static class SingleTokenAttributeSource extends AttributeSource {
-    final CharTermAttribute termAttribute;
-    final OffsetAttribute offsetAttribute;
-    
-    private SingleTokenAttributeSource() {
-      termAttribute = addAttribute(CharTermAttribute.class);
-      offsetAttribute = addAttribute(OffsetAttribute.class);
-    }
-    
-    public void reinit(String stringValue, int startOffset,  int endOffset) {
-      termAttribute.setEmpty().append(stringValue);
-      offsetAttribute.setOffset(startOffset, endOffset);
-    }
-  }
-  
-  final DocumentsWriter.DocState docState;
-
-  final FieldInvertState fieldState = new 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);
-  }
-
-  @Override
-  public void startDocument() throws IOException {
-    consumer.startDocument();
-    endConsumer.startDocument();
-  }
-
-  @Override
-  public DocumentsWriter.DocWriter finishDocument() throws IOException {
-    // TODO: allow endConsumer.finishDocument to also return
-    // a DocWriter
-    endConsumer.finishDocument();
-    return consumer.finishDocument();
-  }
-
-  @Override
-  void abort() {
-    try {
-      consumer.abort();
-    } finally {
-      endConsumer.abort();
-    }
-  }
-
-  @Override
-  public DocFieldConsumerPerField addField(FieldInfo fi) {
-    return new DocInverterPerField(this, fi);
-  }
-}
Index: src/java/org/apache/lucene/index/TermVectorsTermsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/TermVectorsTermsWriterPerThread.java	(revision 934193)
+++ src/java/org/apache/lucene/index/TermVectorsTermsWriterPerThread.java	(working copy)
@@ -1,89 +0,0 @@
-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.util.BytesRef;
-
-final class TermVectorsTermsWriterPerThread extends TermsHashConsumerPerThread {
-
-  final TermVectorsTermsWriter termsWriter;
-  final TermsHashPerThread termsHashPerThread;
-  final DocumentsWriter.DocState docState;
-  final BytesRef flushTerm = new BytesRef();
-
-  TermVectorsTermsWriter.PerDoc doc;
-
-  public TermVectorsTermsWriterPerThread(TermsHashPerThread termsHashPerThread, TermVectorsTermsWriter termsWriter) {
-    this.termsWriter = termsWriter;
-    this.termsHashPerThread = termsHashPerThread;
-    docState = termsHashPerThread.docState;
-  }
-  
-  // Used by perField when serializing the term vectors
-  final ByteSliceReader vectorSliceReader = new ByteSliceReader();
-
-  @Override
-  public void startDocument() {
-    assert clearLastVectorFieldName();
-    if (doc != null) {
-      doc.reset();
-      doc.docID = docState.docID;
-    }
-  }
-
-  @Override
-  public DocumentsWriter.DocWriter finishDocument() {
-    try {
-      return doc;
-    } finally {
-      doc = null;
-    }
-  }
-
-  @Override
-  public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
-    return new TermVectorsTermsWriterPerField(termsHashPerField, this, fieldInfo);
-  }
-
-  @Override
-  public void abort() {
-    if (doc != null) {
-      doc.abort();
-      doc = null;
-    }
-  }
-
-  // Called only by assert
-  final boolean clearLastVectorFieldName() {
-    lastVectorFieldName = null;
-    return true;
-  }
-
-  // Called only by assert
-  String lastVectorFieldName;
-  final boolean vectorFieldsInOrder(FieldInfo fi) {
-    try {
-      if (lastVectorFieldName != null)
-        return lastVectorFieldName.compareTo(fi.name) < 0;
-      else
-        return true;
-    } finally {
-      lastVectorFieldName = fi.name;
-    }
-  }
-}
Index: src/java/org/apache/lucene/index/DocInverter.java
===================================================================
--- src/java/org/apache/lucene/index/DocInverter.java	(revision 934193)
+++ src/java/org/apache/lucene/index/DocInverter.java	(working copy)
@@ -18,13 +18,14 @@
  */
 
 import java.io.IOException;
-import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
-
 import java.util.Map;
 
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.util.AttributeSource;
 
+
 /** This is a DocFieldConsumer that inverts each field,
  *  separately, from a Document, and accepts a
  *  InvertedTermsConsumer to process those terms. */
@@ -34,7 +35,32 @@
   final InvertedDocConsumer consumer;
   final InvertedDocEndConsumer endConsumer;
 
-  public DocInverter(InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) {
+  final DocumentsWriter.DocState docState;
+
+  final FieldInvertState fieldState = new FieldInvertState();
+
+  final SingleTokenAttributeSource singleToken = new SingleTokenAttributeSource();
+  
+  static class SingleTokenAttributeSource extends AttributeSource {
+    final CharTermAttribute termAttribute;
+    final OffsetAttribute offsetAttribute;
+    
+    private SingleTokenAttributeSource() {
+      termAttribute = addAttribute(CharTermAttribute.class);
+      offsetAttribute = addAttribute(OffsetAttribute.class);
+    }
+    
+    public void reinit(String stringValue, int startOffset,  int endOffset) {
+      termAttribute.setEmpty().append(stringValue);
+      offsetAttribute.setOffset(startOffset, endOffset);
+    }
+  }
+  
+  // Used to read a string value for a field
+  final ReusableStringReader stringReader = new ReusableStringReader();
+
+  public DocInverter(DocumentsWriter.DocState docState, InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) {
+    this.docState = docState;
     this.consumer = consumer;
     this.endConsumer = endConsumer;
   }
@@ -47,33 +73,37 @@
   }
 
   @Override
-  void flush(Map<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
+  void flush(Map<FieldInfo, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
 
-    Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> childThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
-    Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> endChildThreadsAndFields = new HashMap<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>>();
+    Map<FieldInfo, InvertedDocConsumerPerField> childFieldsToFlush = new HashMap<FieldInfo, InvertedDocConsumerPerField>();
+    Map<FieldInfo, InvertedDocEndConsumerPerField> endChildFieldsToFlush = new HashMap<FieldInfo, InvertedDocEndConsumerPerField>();
 
-    for (Map.Entry<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> entry : threadsAndFields.entrySet() ) {
-
-
-      DocInverterPerThread perThread = (DocInverterPerThread) entry.getKey();
-
-      Collection<InvertedDocConsumerPerField> childFields = new HashSet<InvertedDocConsumerPerField>();
-      Collection<InvertedDocEndConsumerPerField> endChildFields = new HashSet<InvertedDocEndConsumerPerField>();
-      for (final DocFieldConsumerPerField field: entry.getValue() ) {  
-        DocInverterPerField perField = (DocInverterPerField) field;
-        childFields.add(perField.consumer);
-        endChildFields.add(perField.endConsumer);
-      }
-
-      childThreadsAndFields.put(perThread.consumer, childFields);
-      endChildThreadsAndFields.put(perThread.endConsumer, endChildFields);
+    for (Map.Entry<FieldInfo, DocFieldConsumerPerField> fieldToFlush : fieldsToFlush.entrySet()) {
+      DocInverterPerField perField = (DocInverterPerField) fieldToFlush.getValue();
+      childFieldsToFlush.put(fieldToFlush.getKey(), perField.consumer);
+      endChildFieldsToFlush.put(fieldToFlush.getKey(), perField.endConsumer);
     }
     
-    consumer.flush(childThreadsAndFields, state);
-    endConsumer.flush(endChildThreadsAndFields, state);
+    consumer.flush(childFieldsToFlush, state);
+    endConsumer.flush(endChildFieldsToFlush, state);
   }
+  
+  @Override
+  public void startDocument() throws IOException {
+    consumer.startDocument();
+    endConsumer.startDocument();
+  }
 
   @Override
+  public DocumentsWriter.DocWriter finishDocument() throws IOException {
+    // TODO: allow endConsumer.finishDocument to also return
+    // a DocWriter
+    endConsumer.finishDocument();
+    return consumer.finishDocument();
+  }
+
+
+  @Override
   public void closeDocStore(SegmentWriteState state) throws IOException {
     consumer.closeDocStore(state);
     endConsumer.closeDocStore(state);
@@ -81,17 +111,21 @@
 
   @Override
   void abort() {
-    consumer.abort();
-    endConsumer.abort();
+    try {
+      consumer.abort();
+    } finally {
+      endConsumer.abort();
+    }
   }
 
   @Override
   public boolean freeRAM() {
     return consumer.freeRAM();
   }
-
+  
   @Override
-  public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) {
-    return new DocInverterPerThread(docFieldProcessorPerThread, this);
+  public DocFieldConsumerPerField addField(FieldInfo fi) {
+    return new DocInverterPerField(this, fi);
   }
+
 }
Index: src/java/org/apache/lucene/index/NormsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/NormsWriter.java	(revision 934193)
+++ src/java/org/apache/lucene/index/NormsWriter.java	(working copy)
@@ -19,14 +19,10 @@
 
 import java.io.IOException;
 import java.util.Collection;
-import java.util.Iterator;
-import java.util.HashMap;
 import java.util.Map;
-import java.util.List;
-import java.util.ArrayList;
 
-import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.search.Similarity;
+import org.apache.lucene.store.IndexOutput;
 
 // TODO FI: norms could actually be stored as doc store
 
@@ -39,10 +35,6 @@
 
   private static final byte defaultNorm = Similarity.getDefault().encodeNormValue(1.0f);
   private FieldInfos fieldInfos;
-  @Override
-  public InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) {
-    return new NormsWriterPerThread(docInverterPerThread, this);
-  }
 
   @Override
   public void abort() {}
@@ -58,36 +50,8 @@
   /** Produce _X.nrm if any document had a field with norms
    *  not disabled */
   @Override
-  public void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
+  public void flush(Map<FieldInfo,InvertedDocEndConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
 
-    final Map<FieldInfo,List<NormsWriterPerField>> byField = new HashMap<FieldInfo,List<NormsWriterPerField>>();
-
-    // Typically, each thread will have encountered the same
-    // field.  So first we collate by field, ie, all
-    // per-thread field instances that correspond to the
-    // same FieldInfo
-    for (final Map.Entry<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> entry : threadsAndFields.entrySet()) {
-      final Collection<InvertedDocEndConsumerPerField> fields = entry.getValue();
-      final Iterator<InvertedDocEndConsumerPerField> fieldsIt = fields.iterator();
-
-      while (fieldsIt.hasNext()) {
-        final NormsWriterPerField perField = (NormsWriterPerField) fieldsIt.next();
-
-        if (perField.upto > 0) {
-          // It has some norms
-          List<NormsWriterPerField> l = byField.get(perField.fieldInfo);
-          if (l == null) {
-            l = new ArrayList<NormsWriterPerField>();
-            byField.put(perField.fieldInfo, l);
-          }
-          l.add(perField);
-        } else
-          // Remove this field since we haven't seen it
-          // since the previous flush
-          fieldsIt.remove();
-      }
-    }
-
     final String normsFileName = IndexFileNames.segmentFileName(state.segmentName, IndexFileNames.NORMS_EXTENSION);
     state.flushedFiles.add(normsFileName);
     IndexOutput normsOut = state.directory.createOutput(normsFileName);
@@ -103,60 +67,26 @@
 
         final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
 
-        List<NormsWriterPerField> toMerge = byField.get(fieldInfo);
-        int upto = 0;
-        if (toMerge != null) {
+        NormsWriterPerField toWrite = (NormsWriterPerField) fieldsToFlush.get(fieldInfo);
 
-          final int numFields = toMerge.size();
-
+        int upto = 0;
+        if (toWrite != null && toWrite.upto > 0) {
           normCount++;
 
-          final NormsWriterPerField[] fields = new NormsWriterPerField[numFields];
-          int[] uptos = new int[numFields];
-
-          for(int j=0;j<numFields;j++)
-            fields[j] = toMerge.get(j);
-
-          int numLeft = numFields;
-              
-          while(numLeft > 0) {
-
-            assert uptos[0] < fields[0].docIDs.length : " uptos[0]=" + uptos[0] + " len=" + (fields[0].docIDs.length);
-
-            int minLoc = 0;
-            int minDocID = fields[0].docIDs[uptos[0]];
-
-            for(int j=1;j<numLeft;j++) {
-              final int docID = fields[j].docIDs[uptos[j]];
-              if (docID < minDocID) {
-                minDocID = docID;
-                minLoc = j;
-              }
-            }
-
-            assert minDocID < state.numDocs;
-
-            // Fill hole
-            for(;upto<minDocID;upto++)
+          int docID = 0;
+          for (; docID < state.numDocs; docID++) {
+            if (upto < toWrite.upto && toWrite.docIDs[upto] == docID) {
+              normsOut.writeByte(toWrite.norms[upto]);
+              upto++;
+            } else {
               normsOut.writeByte(defaultNorm);
-
-            normsOut.writeByte(fields[minLoc].norms[uptos[minLoc]]);
-            (uptos[minLoc])++;
-            upto++;
-
-            if (uptos[minLoc] == fields[minLoc].upto) {
-              fields[minLoc].reset();
-              if (minLoc != numLeft-1) {
-                fields[minLoc] = fields[numLeft-1];
-                uptos[minLoc] = uptos[numLeft-1];
-              }
-              numLeft--;
             }
           }
-          
-          // Fill final hole with defaultNorm
-          for(;upto<state.numDocs;upto++)
-            normsOut.writeByte(defaultNorm);
+
+          // we should have consumed every norm
+          assert upto == toWrite.upto;
+
+          toWrite.reset();
         } else if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
           normCount++;
           // Fill entire field with default norm:
@@ -174,4 +104,17 @@
 
   @Override
   void closeDocStore(SegmentWriteState state) {}
+
+  
+  @Override
+  void finishDocument() throws IOException {}
+
+  @Override
+  void startDocument() throws IOException {}
+
+  @Override
+  InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField,
+      FieldInfo fieldInfo) {
+    return new NormsWriterPerField(docInverterPerField, fieldInfo);
+  }
 }
