You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/05/02 15:51:22 UTC

svn commit: r1098566 [8/22] - in /lucene/dev/branches/docvalues: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/db/bdb-je/ dev-tools/idea/lucene/contrib/db/bdb/ dev-tools/idea/lucene/contrib...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon May  2 13:50:57 2011
@@ -28,20 +28,16 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BitVector;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.index.values.Bytes;
-import org.apache.lucene.index.values.Ints;
 import org.apache.lucene.index.values.DocValues;
-import org.apache.lucene.index.values.Floats;
-import org.apache.lucene.index.values.Type;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CloseableThreadLocal;
 
@@ -61,6 +57,9 @@ public class SegmentReader extends Index
   AtomicInteger deletedDocsRef = null;
   private boolean deletedDocsDirty = false;
   private boolean normsDirty = false;
+
+  // TODO: we should move this tracking into SegmentInfo;
+  // this way SegmentInfo.toString shows pending deletes
   private int pendingDeleteCount;
 
   private boolean rollbackHasChanges = false;
@@ -91,6 +90,7 @@ public class SegmentReader extends Index
     final FieldInfos fieldInfos;
 
     final FieldsProducer fields;
+    final PerDocValues perDocProducer;
     
     final Directory dir;
     final Directory cfsDir;
@@ -130,8 +130,10 @@ public class SegmentReader extends Index
         this.termsIndexDivisor = termsIndexDivisor;
         
         // Ask codec for its Fields
-        fields = segmentCodecs.codec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor));
+        final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor);
+        fields = segmentCodecs.codec().fieldsProducer(segmentReadState);
         assert fields != null;
+        perDocProducer = segmentCodecs.codec().docsProducer(segmentReadState);
         success = true;
       } finally {
         if (!success) {
@@ -169,6 +171,10 @@ public class SegmentReader extends Index
         if (fields != null) {
           fields.close();
         }
+        
+        if (perDocProducer != null) {
+          perDocProducer.close();
+        }
 
         if (termVectorsReaderOrig != null) {
           termVectorsReaderOrig.close();
@@ -808,8 +814,9 @@ public class SegmentReader extends Index
       oldRef.decrementAndGet();
     }
     deletedDocsDirty = true;
-    if (!deletedDocs.getAndSet(docNum))
+    if (!deletedDocs.getAndSet(docNum)) {
       pendingDeleteCount++;
+    }
   }
 
   @Override
@@ -1211,6 +1218,11 @@ public class SegmentReader extends Index
   
   @Override
   public DocValues docValues(String field) throws IOException {
-    return core.fields.docValues(field);
+    return core.perDocProducer.docValues(field);
+  }
+
+  @Override
+  public PerDocValues perDocValues() throws IOException {
+    return core.perDocProducer;
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java Mon May  2 13:50:57 2011
@@ -33,7 +33,6 @@ public class SegmentWriteState {
   public final FieldInfos fieldInfos;
   public final int numDocs;
   public boolean hasVectors;
-  public final AtomicLong bytesUsed;
 
   // Deletes to apply while we are flushing the segment.  A
   // Term is enrolled in here if it was deleted at one
@@ -56,7 +55,7 @@ public class SegmentWriteState {
   public int termIndexInterval;                   // TODO: this should be private to the codec, not settable here or in IWC
 
   public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
-      int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, BufferedDeletes segDeletes, AtomicLong bytesUsed) {
+      int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, BufferedDeletes segDeletes) {
     this.infoStream = infoStream;
     this.segDeletes = segDeletes;
     this.directory = directory;
@@ -66,7 +65,6 @@ public class SegmentWriteState {
     this.termIndexInterval = termIndexInterval;
     this.segmentCodecs = segmentCodecs;
     codecId = -1;
-    this.bytesUsed = bytesUsed;
   }
   
   /**
@@ -82,7 +80,6 @@ public class SegmentWriteState {
     segmentCodecs = state.segmentCodecs;
     this.codecId = codecId;
     segDeletes = state.segDeletes;
-    bytesUsed = state.bytesUsed;
   }
   
   public String codecIdAsString() {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java Mon May  2 13:50:57 2011
@@ -26,6 +26,7 @@ import org.apache.lucene.util.ReaderUtil
 
 import org.apache.lucene.index.DirectoryReader; // javadoc
 import org.apache.lucene.index.MultiReader; // javadoc
+import org.apache.lucene.index.codecs.PerDocValues;
 
 /**
  * This class forces a composite reader (eg a {@link
@@ -65,6 +66,11 @@ public final class SlowMultiReaderWrappe
   }
 
   @Override
+  public PerDocValues perDocValues() throws IOException {
+    return MultiPerDocValues.getPerDocs(in);
+  }
+
+  @Override
   public Bits getDeletedDocs() {
     return MultiFields.getDeletedDocs(in);
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Mon May  2 13:50:57 2011
@@ -18,7 +18,8 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import org.apache.lucene.store.RAMOutputStream;
+
+import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -26,22 +27,38 @@ import org.apache.lucene.util.RamUsageEs
 final class StoredFieldsWriter {
 
   FieldsWriter fieldsWriter;
-  final DocumentsWriter docWriter;
+  final DocumentsWriterPerThread docWriter;
   int lastDocID;
 
-  PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;
 
-  public StoredFieldsWriter(DocumentsWriter docWriter) {
+  final DocumentsWriterPerThread.DocState docState;
+
+  public StoredFieldsWriter(DocumentsWriterPerThread docWriter) {
     this.docWriter = docWriter;
+    this.docState = docWriter.docState;
+  }
+
+  private int numStoredFields;
+  private Fieldable[] storedFields;
+  private int[] fieldNumbers;
+
+  public void reset() {
+    numStoredFields = 0;
+    storedFields = new Fieldable[1];
+    fieldNumbers = new int[1];
   }
 
-  public StoredFieldsWriterPerThread addThread(DocumentsWriter.DocState docState) throws IOException {
-    return new StoredFieldsWriterPerThread(docState, this);
+  public void startDocument() {
+    reset();
   }
 
-  synchronized public void flush(SegmentWriteState state) throws IOException {
-    if (state.numDocs > lastDocID) {
+  public void flush(SegmentWriteState state) throws IOException {
+
+    if (state.numDocs > 0) {
+      // It's possible that all documents seen in this segment
+      // hit non-aborting exceptions, in which case we will
+      // not have yet init'd the FieldsWriter:
       initFieldsWriter();
       fill(state.numDocs);
     }
@@ -67,23 +84,9 @@ final class StoredFieldsWriter {
 
   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];
-    }
-  }
+  void abort() {
+    reset();
 
-  synchronized void abort() {
     if (fieldsWriter != null) {
       fieldsWriter.abort();
       fieldsWriter = null;
@@ -101,53 +104,40 @@ final class StoredFieldsWriter {
     }
   }
 
-  synchronized void finishDocument(PerDoc perDoc) throws IOException {
+  void finishDocument() throws IOException {
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument start");
+
     initFieldsWriter();
+    fill(docState.docID);
 
-    fill(perDoc.docID);
+    if (fieldsWriter != null && numStoredFields > 0) {
+      fieldsWriter.startDocument(numStoredFields);
+      for (int i = 0; i < numStoredFields; i++) {
+        fieldsWriter.writeField(fieldNumbers[i], storedFields[i]);
+      }
+      lastDocID++;
+    }
 
-    // Append stored fields to the real FieldsWriter:
-    fieldsWriter.flushDocument(perDoc.numStoredFields, perDoc.fdt);
-    lastDocID++;
-    perDoc.reset();
-    free(perDoc);
+    reset();
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
   }
 
-  synchronized void free(PerDoc perDoc) {
-    assert freeCount < docFreeList.length;
-    assert 0 == perDoc.numStoredFields;
-    assert 0 == perDoc.fdt.length();
-    assert 0 == perDoc.fdt.getFilePointer();
-    docFreeList[freeCount++] = perDoc;
-  }
-
-  class PerDoc extends DocumentsWriter.DocWriter {
-    final DocumentsWriter.PerDocBuffer buffer = docWriter.newPerDocBuffer();
-    RAMOutputStream fdt = new RAMOutputStream(buffer);
-    int numStoredFields;
-
-    void reset() {
-      fdt.reset();
-      buffer.recycle();
-      numStoredFields = 0;
+  public void addField(Fieldable field, FieldInfo fieldInfo) throws IOException {
+    if (numStoredFields == storedFields.length) {
+      int newSize = ArrayUtil.oversize(numStoredFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
+      Fieldable[] newArray = new Fieldable[newSize];
+      System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
+      storedFields = newArray;
     }
 
-    @Override
-    void abort() {
-      reset();
-      free(this);
+    if (numStoredFields == fieldNumbers.length) {
+      fieldNumbers = ArrayUtil.grow(fieldNumbers);
     }
 
-    @Override
-    public long sizeInBytes() {
-      return buffer.getSizeInBytes();
-    }
+    storedFields[numStoredFields] = field;
+    fieldNumbers[numStoredFields] = fieldInfo.number;
+    numStoredFields++;
 
-    @Override
-    public void finish() throws IOException {
-      finishDocument(this);
-    }
+    assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField");
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Mon May  2 13:50:57 2011
@@ -17,49 +17,48 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Map;
+
 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.IOUtils;
 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;
-  PerDoc[] docFreeList = new PerDoc[1];
+  final DocumentsWriterPerThread docWriter;
   int freeCount;
   IndexOutput tvx;
   IndexOutput tvd;
   IndexOutput tvf;
   int lastDocID;
+
+  final DocumentsWriterPerThread.DocState docState;
+  final BytesRef flushTerm = new BytesRef();
+
+  // Used by perField when serializing the term vectors
+  final ByteSliceReader vectorSliceReader = new ByteSliceReader();
   boolean hasVectors;
 
-  public TermVectorsTermsWriter(DocumentsWriter docWriter) {
+  public TermVectorsTermsWriter(DocumentsWriterPerThread docWriter) {
     this.docWriter = docWriter;
+    docState = docWriter.docState;
   }
 
   @Override
-  public TermsHashConsumerPerThread addThread(TermsHashPerThread termsHashPerThread) {
-    return new TermVectorsTermsWriterPerThread(termsHashPerThread, this);
-  }
-
-  @Override
-  synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
+  void flush(Map<FieldInfo, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
     if (tvx != null) {
       // At least one doc in this run had term vectors enabled
       fill(state.numDocs);
+      assert state.segmentName != null;
+      String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
       tvx.close();
       tvf.close();
       tvd.close();
       tvx = tvd = tvf = null;
-      assert state.segmentName != null;
-      String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
-      if (4 + ((long) state.numDocs) * 16 != state.directory.fileLength(idxName)) {
+      if (4+((long) state.numDocs)*16 != state.directory.fileLength(idxName)) {
         throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
       }
 
@@ -68,33 +67,10 @@ final class TermVectorsTermsWriter exten
       hasVectors = false;
     }
 
-    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);
-    }
-  }
-
-  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];
+    for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) {
+      TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field;
+      perField.termsHashPerField.reset();
+      perField.shrinkHash();
     }
   }
 
@@ -112,18 +88,17 @@ final class TermVectorsTermsWriter exten
     }
   }
 
-  synchronized void initTermVectorsWriter() throws IOException {        
+  private final void initTermVectorsWriter() throws IOException {
     if (tvx == null) {
 
       // If we hit an exception while init'ing the term
       // vector output files, we must abort this segment
       // because those files will be in an unknown
       // state:
-      hasVectors = true;
       tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
       tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
       tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
-      
+
       tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
       tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
       tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
@@ -132,39 +107,44 @@ final class TermVectorsTermsWriter exten
     }
   }
 
-  synchronized void finishDocument(PerDoc perDoc) throws IOException {
+  @Override
+  void finishDocument(TermsHash termsHash) throws IOException {
 
     assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument start");
 
+    if (!hasVectors) {
+      return;
+    }
+
     initTermVectorsWriter();
 
-    fill(perDoc.docID);
+    fill(docState.docID);
 
     // Append term vectors to the real outputs:
-    tvx.writeLong(tvd.getFilePointer());
+    long pointer = tvd.getFilePointer();
+    tvx.writeLong(pointer);
     tvx.writeLong(tvf.getFilePointer());
-    tvd.writeVInt(perDoc.numVectorFields);
-    if (perDoc.numVectorFields > 0) {
-      for(int i=0;i<perDoc.numVectorFields;i++) {
-        tvd.writeVInt(perDoc.fieldNumbers[i]);
-      }
-      assert 0 == perDoc.fieldPointers[0];
-      long lastPos = perDoc.fieldPointers[0];
-      for(int i=1;i<perDoc.numVectorFields;i++) {
-        long pos = perDoc.fieldPointers[i];
+    tvd.writeVInt(numVectorFields);
+    if (numVectorFields > 0) {
+      for(int i=0;i<numVectorFields;i++) {
+        tvd.writeVInt(perFields[i].fieldInfo.number);
+      }
+      long lastPos = tvf.getFilePointer();
+      perFields[0].finishDocument();
+      for(int i=1;i<numVectorFields;i++) {
+        long pos = tvf.getFilePointer();
         tvd.writeVLong(pos-lastPos);
         lastPos = pos;
+        perFields[i].finishDocument();
       }
-      perDoc.perDocTvf.writeTo(tvf);
-      perDoc.numVectorFields = 0;
     }
 
-    assert lastDocID == perDoc.docID;
+    assert lastDocID == docState.docID;
 
     lastDocID++;
 
-    perDoc.reset();
-    free(perDoc);
+    termsHash.reset();
+    reset();
     assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
   }
 
@@ -189,55 +169,58 @@ final class TermVectorsTermsWriter exten
     }
     tvx = tvd = tvf = null;
     lastDocID = 0;
-  }
 
-  synchronized void free(PerDoc doc) {
-    assert freeCount < docFreeList.length;
-    docFreeList[freeCount++] = doc;
+    reset();
   }
 
-  class PerDoc extends DocumentsWriter.DocWriter {
+  int numVectorFields;
 
-    final DocumentsWriter.PerDocBuffer buffer = docWriter.newPerDocBuffer();
-    RAMOutputStream perDocTvf = new RAMOutputStream(buffer);
+  TermVectorsTermsWriterPerField[] perFields;
 
-    int numVectorFields;
+  void reset() {
+    numVectorFields = 0;
+    perFields = new TermVectorsTermsWriterPerField[1];
+  }
 
-    int[] fieldNumbers = new int[1];
-    long[] fieldPointers = new long[1];
+  @Override
+  public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
+    return new TermVectorsTermsWriterPerField(termsHashPerField, this, fieldInfo);
+  }
 
-    void reset() {
-      perDocTvf.reset();
-      buffer.recycle();
-      numVectorFields = 0;
+  void addFieldToFlush(TermVectorsTermsWriterPerField fieldToFlush) {
+    if (numVectorFields == perFields.length) {
+      int newSize = ArrayUtil.oversize(numVectorFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
+      TermVectorsTermsWriterPerField[] newArray = new TermVectorsTermsWriterPerField[newSize];
+      System.arraycopy(perFields, 0, newArray, 0, numVectorFields);
+      perFields = newArray;
     }
 
-    @Override
-    void abort() {
-      reset();
-      free(this);
-    }
+    perFields[numVectorFields++] = fieldToFlush;
+  }
 
-    void addField(final int fieldNumber) {
-      if (numVectorFields == fieldNumbers.length) {
-        fieldNumbers = ArrayUtil.grow(fieldNumbers);
-      }
-      if (numVectorFields == fieldPointers.length) {
-        fieldPointers = ArrayUtil.grow(fieldPointers);
-      }
-      fieldNumbers[numVectorFields] = fieldNumber;
-      fieldPointers[numVectorFields] = perDocTvf.getFilePointer();
-      numVectorFields++;
-    }
+  @Override
+  void startDocument() throws IOException {
+    assert clearLastVectorFieldName();
+    reset();
+  }
 
-    @Override
-    public long sizeInBytes() {
-      return buffer.getSizeInBytes();
-    }
+  // Called only by assert
+  final boolean clearLastVectorFieldName() {
+    lastVectorFieldName = null;
+    return true;
+  }
 
-    @Override
-    public void finish() throws IOException {
-      finishDocument(this);
+  // 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;
     }
   }
+
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java Mon May  2 13:50:57 2011
@@ -28,11 +28,10 @@ import org.apache.lucene.util.RamUsageEs
 
 final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
 
-  final TermVectorsTermsWriterPerThread perThread;
   final TermsHashPerField termsHashPerField;
   final TermVectorsTermsWriter termsWriter;
   final FieldInfo fieldInfo;
-  final DocumentsWriter.DocState docState;
+  final DocumentsWriterPerThread.DocState docState;
   final FieldInvertState fieldState;
 
   boolean doVectors;
@@ -41,11 +40,10 @@ final class TermVectorsTermsWriterPerFie
 
   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;
@@ -72,22 +70,12 @@ final class TermVectorsTermsWriterPerFie
     }
 
     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();
-      }
-
-      assert perThread.doc.docID == docState.docID;
-
+      termsWriter.hasVectors = true;
       if (termsHashPerField.bytesHash.size() != 0) {
         // Only necessary if previous doc hit a
         // non-aborting exception while writing vectors in
         // this field:
         termsHashPerField.reset();
-        perThread.termsHashPerThread.reset(false);
       }
     }
 
@@ -95,42 +83,42 @@ final class TermVectorsTermsWriterPerFie
     //perThread.postingsCount = 0;
 
     return doVectors;
-  }     
+  }
 
   public void abort() {}
 
   /** Called once per field per document if term vectors
    *  are enabled, to write the vectors to
    *  RAMOutputStream, which is then quickly flushed to
-   *  the real term vectors files in the Directory. */
-  @Override
+   *  the real term vectors files in the Directory. */  @Override
   void finish() throws IOException {
+    if (!doVectors || termsHashPerField.bytesHash.size() == 0)
+      return;
 
+    termsWriter.addFieldToFlush(this);
+  }
+
+  void finishDocument() throws IOException {
     assert docState.testPoint("TermVectorsTermsWriterPerField.finish start");
 
     final int numPostings = termsHashPerField.bytesHash.size();
 
-    final BytesRef flushTerm = perThread.flushTerm;
+    final BytesRef flushTerm = termsWriter.flushTerm;
 
     assert numPostings >= 0;
 
-    if (!doVectors || numPostings == 0)
-      return;
-
     if (numPostings > maxNumPostings)
       maxNumPostings = numPostings;
 
-    final IndexOutput tvf = perThread.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);
     TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
+    final IndexOutput tvf = termsWriter.tvf;
 
     // TODO: we may want to make this sort in same order
     // as Codec's terms dict?
@@ -140,21 +128,21 @@ final class TermVectorsTermsWriterPerFie
     byte bits = 0x0;
     if (doVectorPositions)
       bits |= TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
-    if (doVectorOffsets) 
+    if (doVectorOffsets)
       bits |= TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
     tvf.writeByte(bits);
 
     int lastLen = 0;
     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];
       final int freq = postings.freqs[termID];
-          
+
       // Get BytesRef
       termBytePool.setBytesRef(flushTerm, postings.textStarts[termID]);
 
@@ -192,20 +180,13 @@ final class TermVectorsTermsWriterPerFie
     }
 
     termsHashPerField.reset();
-
-    // NOTE: we clear, per-field, at the thread level,
-    // because term vectors fully write themselves on each
-    // field; this saves RAM (eg if large doc has two large
-    // fields w/ term vectors on) because we recycle/reuse
-    // all RAM after each field:
-    perThread.termsHashPerThread.reset(false);
   }
 
   void shrinkHash() {
     termsHashPerField.shrinkHash(maxNumPostings);
     maxNumPostings = 0;
   }
-  
+
   @Override
   void start(Fieldable f) {
     if (doVectorOffsets) {
@@ -225,7 +206,7 @@ final class TermVectorsTermsWriterPerFie
     if (doVectorOffsets) {
       int startOffset = fieldState.offset + offsetAttribute.startOffset();
       int endOffset = fieldState.offset + offsetAttribute.endOffset();
-      
+
       termsHashPerField.writeVInt(1, startOffset);
       termsHashPerField.writeVInt(1, endOffset - startOffset);
       postings.lastOffsets[termID] = endOffset;
@@ -243,13 +224,13 @@ final class TermVectorsTermsWriterPerFie
     assert docState.testPoint("TermVectorsTermsWriterPerField.addTerm start");
 
     TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
-    
+
     postings.freqs[termID]++;
 
     if (doVectorOffsets) {
       int startOffset = fieldState.offset + offsetAttribute.startOffset();
       int endOffset = fieldState.offset + offsetAttribute.endOffset();
-      
+
       termsHashPerField.writeVInt(1, startOffset - postings.lastOffsets[termID]);
       termsHashPerField.writeVInt(1, endOffset - startOffset);
       postings.lastOffsets[termID] = endOffset;
@@ -280,7 +261,7 @@ final class TermVectorsTermsWriterPerFie
     int[] freqs;                                       // How many times this term occurred in the current doc
     int[] lastOffsets;                                 // Last offset we saw
     int[] lastPositions;                               // Last position where this term occurred
-    
+
     @Override
     ParallelPostingsArray newInstance(int size) {
       return new TermVectorsPostingsArray(size);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java Mon May  2 13:50:57 2011
@@ -20,12 +20,13 @@ package org.apache.lucene.index;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 
 import java.io.IOException;
 
 final class TermVectorsWriter {
-  
+
   private IndexOutput tvx = null, tvd = null, tvf = null;
   private FieldInfos fieldInfos;
 
@@ -46,7 +47,7 @@ final class TermVectorsWriter {
   /**
    * Add a complete document specified by all its term vectors. If document has no
    * term vectors, add value for tvx.
-   * 
+   *
    * @param vectors
    * @throws IOException
    */
@@ -99,7 +100,7 @@ final class TermVectorsWriter {
         final int[] freqs = vectors[i].getTermFrequencies();
 
         for (int j=0; j<numTerms; j++) {
-          
+
           int start = j == 0 ? 0 : StringHelper.bytesDifference(terms[j-1].bytes,
                                                    terms[j-1].length,
                                                    terms[j].bytes,
@@ -181,30 +182,11 @@ final class TermVectorsWriter {
     assert tvd.getFilePointer() == tvdPosition;
     assert tvf.getFilePointer() == tvfPosition;
   }
-  
+
   /** Close all streams. */
   final void close() throws IOException {
     // make an effort to close all streams we can but remember and re-throw
     // the first exception encountered in this process
-    IOException keep = null;
-    if (tvx != null)
-      try {
-        tvx.close();
-      } catch (IOException e) {
-        keep = e;
-      }
-    if (tvd != null)
-      try {
-        tvd.close();
-      } catch (IOException e) {
-        if (keep == null) keep = e;
-      }
-    if (tvf != null)
-      try {
-        tvf.close();
-      } catch (IOException e) {
-        if (keep == null) keep = e;
-      }
-    if (keep != null) throw (IOException) keep.fillInStackTrace();
+    IOUtils.closeSafely(tvx, tvd, tvf);
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java Mon May  2 13:50:57 2011
@@ -18,12 +18,12 @@ package org.apache.lucene.index;
  */
 
 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.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
+
 /** 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
@@ -36,78 +36,118 @@ final class TermsHash extends InvertedDo
 
   final TermsHashConsumer consumer;
   final TermsHash nextTermsHash;
-  final DocumentsWriter docWriter;
+  final DocumentsWriterPerThread docWriter;
+
+  final IntBlockPool intPool;
+  final ByteBlockPool bytePool;
+  ByteBlockPool termBytePool;
+
+  final boolean primary;
+  final DocumentsWriterPerThread.DocState docState;
+
+  // Used when comparing postings via termRefComp, in TermsHashPerField
+  final BytesRef tr1 = new BytesRef();
+  final BytesRef tr2 = new BytesRef();
 
-  boolean trackAllocations;
+  // Used by perField to obtain terms from the analysis chain
+  final BytesRef termBytesRef = new BytesRef(10);
 
-  public TermsHash(final DocumentsWriter docWriter, boolean trackAllocations, final TermsHashConsumer consumer, final TermsHash nextTermsHash) {
+  final boolean trackAllocations;
+
+
+  public TermsHash(final DocumentsWriterPerThread docWriter, final TermsHashConsumer consumer, boolean trackAllocations, final TermsHash nextTermsHash) {
+    this.docState = docWriter.docState;
     this.docWriter = docWriter;
     this.consumer = consumer;
+    this.trackAllocations = trackAllocations; 
     this.nextTermsHash = nextTermsHash;
-    this.trackAllocations = trackAllocations;
+    intPool = new IntBlockPool(docWriter);
+    bytePool = new ByteBlockPool(docWriter.byteBlockAllocator);
+
+    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);
+  public void abort() {
+    reset();
+    try {
+      consumer.abort();
+    } finally {
+      if (nextTermsHash != null) {
+        nextTermsHash.abort();
+      }
+    }
   }
 
-  TermsHashPerThread addThread(DocInverterPerThread docInverterPerThread, TermsHashPerThread primaryPerThread) {
-    return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, primaryPerThread);
-  }
+  // Clear all state
+  void reset() {
+    intPool.reset();
+    bytePool.reset();
 
-  @Override
-  public void abort() {
-    consumer.abort();
-    if (nextTermsHash != null)
-      nextTermsHash.abort();
+    if (primary) {
+      bytePool.reset();
+    }
   }
 
   @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;
-
-    if (nextTermsHash != null)
-      nextThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
-    else
-      nextThreadsAndFields = null;
+  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) {
+      nextChildFields = new HashMap<FieldInfo,InvertedDocConsumerPerField>();
+    } else {
+      nextChildFields = null;
+    }
 
-    for (final Map.Entry<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> entry : threadsAndFields.entrySet()) {
+    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);
+        }
+    }
 
-      TermsHashPerThread perThread = (TermsHashPerThread) entry.getKey();
+    consumer.flush(childFields, state);
 
-      Collection<InvertedDocConsumerPerField> fields = entry.getValue();
+    if (nextTermsHash != null) {
+      nextTermsHash.flush(nextChildFields, state);
+    }
+  }
 
-      Iterator<InvertedDocConsumerPerField> fieldsIt = fields.iterator();
-      Collection<TermsHashConsumerPerField> childFields = new HashSet<TermsHashConsumerPerField>();
-      Collection<InvertedDocConsumerPerField> nextChildFields;
+  @Override
+  InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
+    return new TermsHashPerField(docInverterPerField, this, nextTermsHash, fieldInfo);
+  }
 
-      if (nextTermsHash != null)
-        nextChildFields = new HashSet<InvertedDocConsumerPerField>();
-      else
-        nextChildFields = null;
+  @Override
+  public boolean freeRAM() {
+    return false;
+  }
 
-      while(fieldsIt.hasNext()) {
-        TermsHashPerField perField = (TermsHashPerField) fieldsIt.next();
-        childFields.add(perField.consumer);
-        if (nextTermsHash != null)
-          nextChildFields.add(perField.nextPerField);
+  @Override
+  void finishDocument() throws IOException {
+    try {
+      consumer.finishDocument(this);
+    } finally {
+      if (nextTermsHash != null) {
+        nextTermsHash.consumer.finishDocument(nextTermsHash);
       }
-
-      childThreadsAndFields.put(perThread.consumer, childFields);
-      if (nextTermsHash != null)
-        nextThreadsAndFields.put(perThread.nextPerThread, nextChildFields);
     }
-    
-    consumer.flush(childThreadsAndFields, state);
-
-    if (nextTermsHash != null)
-      nextTermsHash.flush(nextThreadsAndFields, state);
   }
 
   @Override
-  synchronized public boolean freeRAM() {
-    return false;
+  void startDocument() throws IOException {
+    consumer.startDocument();
+    if (nextTermsHash != null) {
+      nextTermsHash.consumer.startDocument();
+    }
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Mon May  2 13:50:57 2011
@@ -18,11 +18,12 @@ package org.apache.lucene.index;
  */
 
 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 startDocument() throws IOException;
+  abstract void finishDocument(TermsHash termsHash) throws IOException;
+  abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo);
+}

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java Mon May  2 13:50:57 2011
@@ -34,9 +34,10 @@ final class TermsHashPerField extends In
 
   final TermsHashConsumerPerField consumer;
 
+  final TermsHash termsHash;
+
   final TermsHashPerField nextPerField;
-  final TermsHashPerThread perThread;
-  final DocumentsWriter.DocState docState;
+  final DocumentsWriterPerThread.DocState docState;
   final FieldInvertState fieldState;
   TermToBytesRefAttribute termAtt;
   BytesRef termBytesRef;
@@ -52,27 +53,27 @@ final class TermsHashPerField extends In
   final FieldInfo fieldInfo;
 
   final BytesRefHash bytesHash;
- 
+
   ParallelPostingsArray postingsArray;
   private final AtomicLong bytesUsed;
 
-  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;
-    bytesUsed =  perThread.termsHash.trackAllocations?perThread.termsHash.docWriter.bytesUsed:new AtomicLong();
-
+  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;
+    bytesUsed = termsHash.trackAllocations ? termsHash.docWriter.bytesUsed
+        : new AtomicLong();
     fieldState = docInverterPerField.fieldState;
-    this.consumer = perThread.consumer.addField(this, fieldInfo);
+    this.consumer = termsHash.consumer.addField(this, fieldInfo);
     PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
-    bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts); 
+    bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
     streamCount = consumer.getStreamCount();
     numPostingInt = 2*streamCount;
     this.fieldInfo = fieldInfo;
-    if (nextPerThread != null)
-      nextPerField = (TermsHashPerField) nextPerThread.addField(docInverterPerField, fieldInfo);
+    if (nextTermsHash != null)
+      nextPerField = (TermsHashPerField) nextTermsHash.addField(docInverterPerField, fieldInfo);
     else
       nextPerField = null;
   }
@@ -80,7 +81,7 @@ final class TermsHashPerField extends In
   void shrinkHash(int targetSize) {
     // Fully free the bytesHash on each flush but keep the pool untouched
     // bytesHash.clear will clear the ByteStartArray and in turn the ParallelPostingsArray too
-    bytesHash.clear(false); 
+    bytesHash.clear(false);
   }
 
   public void reset() {
@@ -90,7 +91,7 @@ final class TermsHashPerField extends In
   }
 
   @Override
-  synchronized public void abort() {
+  public void abort() {
     reset();
     if (nextPerField != null)
       nextPerField.abort();
@@ -99,14 +100,13 @@ final class TermsHashPerField extends In
   public void initReader(ByteSliceReader reader, int termID, int stream) {
     assert stream < streamCount;
     int intStart = postingsArray.intStarts[termID];
-    final int[] ints = intPool.buffers[intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
-    final int upto = intStart & DocumentsWriter.INT_BLOCK_MASK;
+    final int[] ints = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
+    final int upto = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
     reader.init(bytePool,
                 postingsArray.byteStarts[termID]+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
                 ints[upto+stream]);
   }
 
-
   /** Collapse the hash table & sort in-place. */
   public int[] sortPostings(Comparator<BytesRef> termComp) {
    return bytesHash.sort(termComp);
@@ -124,7 +124,7 @@ final class TermsHashPerField extends In
       nextPerField.start(f);
     }
   }
-  
+
   @Override
   boolean start(Fieldable[] fields, int count) throws IOException {
     doCall = consumer.start(fields, count);
@@ -143,11 +143,12 @@ final class TermsHashPerField extends In
       // First time we are seeing this token since we last
       // flushed the hash.
       // Init stream slices
-      if (numPostingInt + intPool.intUpto > DocumentsWriter.INT_BLOCK_SIZE)
+      if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE)
         intPool.nextBuffer();
 
-      if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE)
+      if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) {
         bytePool.nextBuffer();
+      }
 
       intUptos = intPool.buffer;
       intUptoStart = intPool.intUpto;
@@ -166,8 +167,8 @@ final class TermsHashPerField extends In
     } else {
       termID = (-termID)-1;
       int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
-      intUptoStart = intStart & DocumentsWriter.INT_BLOCK_MASK;
+      intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
+      intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
   }
@@ -192,7 +193,7 @@ final class TermsHashPerField extends In
       if (docState.maxTermPrefix == null) {
         final int saved = termBytesRef.length;
         try {
-          termBytesRef.length = Math.min(30, DocumentsWriter.MAX_TERM_LENGTH_UTF8);
+          termBytesRef.length = Math.min(30, DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8);
           docState.maxTermPrefix = termBytesRef.toString();
         } finally {
           termBytesRef.length = saved;
@@ -204,7 +205,7 @@ final class TermsHashPerField extends In
     if (termID >= 0) {// New posting
       bytesHash.byteStart(termID);
       // Init stream slices
-      if (numPostingInt + intPool.intUpto > DocumentsWriter.INT_BLOCK_SIZE) {
+      if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE) {
         intPool.nextBuffer();
       }
 
@@ -229,8 +230,8 @@ final class TermsHashPerField extends In
     } else {
       termID = (-termID)-1;
       final int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
-      intUptoStart = intStart & DocumentsWriter.INT_BLOCK_MASK;
+      intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
+      intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
 
@@ -278,7 +279,7 @@ final class TermsHashPerField extends In
     if (nextPerField != null)
       nextPerField.finish();
   }
-  
+
   private static final class PostingsBytesStartArray extends BytesStartArray {
 
     private final TermsHashPerField perField;
@@ -289,10 +290,10 @@ final class TermsHashPerField extends In
       this.perField = perField;
       this.bytesUsed = bytesUsed;
     }
-    
+
     @Override
     public int[] init() {
-      if(perField.postingsArray == null) { 
+      if(perField.postingsArray == null) {
         perField.postingsArray = perField.consumer.createPostingsArray(2);
         bytesUsed.addAndGet(perField.postingsArray.size * perField.postingsArray.bytesPerPosting());
       }
@@ -312,7 +313,7 @@ final class TermsHashPerField extends In
     @Override
     public int[] clear() {
       if(perField.postingsArray != null) {
-        bytesUsed.addAndGet(-perField.postingsArray.size * perField.postingsArray.bytesPerPosting());
+        bytesUsed.addAndGet(-(perField.postingsArray.size * perField.postingsArray.bytesPerPosting()));
         perField.postingsArray = null;
       }
       return null;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java Mon May  2 13:50:57 2011
@@ -152,10 +152,10 @@ public class BlockTermsReader extends Fi
   }
 
   protected void readHeader(IndexInput input) throws IOException {
-    CodecUtil.checkHeader(in, BlockTermsWriter.CODEC_NAME,
+    CodecUtil.checkHeader(input, BlockTermsWriter.CODEC_NAME,
                           BlockTermsWriter.VERSION_START,
                           BlockTermsWriter.VERSION_CURRENT);
-    dirOffset = in.readLong();    
+    dirOffset = input.readLong();
   }
   
   protected void seekDir(IndexInput input, long dirOffset)
@@ -239,11 +239,6 @@ public class BlockTermsReader extends Fi
     public TermsEnum terms() throws IOException {
       return current.iterator();
     }
-
-    @Override
-    public DocValues docValues() throws IOException {
-      return null;
-    }
   }
 
   private class FieldReader extends Terms implements Closeable {
@@ -848,6 +843,11 @@ public class BlockTermsReader extends Fi
       private void decodeMetaData() throws IOException {
         //System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termCount + " state=" + state);
         if (!seekPending) {
+          // TODO: cutover to random-access API
+          // here.... really stupid that we have to decode N
+          // wasted term metadata just to get to the N+1th
+          // that we really need...
+
           // lazily catch up on metadata decode:
           final int limit = state.termCount;
           // We must set/incr state.termCount because

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/Codec.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/Codec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/Codec.java Mon May  2 13:50:57 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs;
 import java.io.IOException;
 import java.util.Set;
 
+import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
@@ -50,6 +51,14 @@ public abstract class Codec {
    *  returns, it must hold open any files it will need to
    *  use; else, those files may be deleted. */
   public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
+  
+  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
+    return null;
+  }
+  
+  public PerDocValues docsProducer(SegmentReadState state) throws IOException {
+    return null;
+  }
 
   /**
    * Gathers files associated with this segment

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java Mon May  2 13:50:57 2011
@@ -21,8 +21,6 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.codecs.docvalues.DocValuesConsumer;
-import org.apache.lucene.index.values.DocValues;
 
 import java.io.IOException;
 import java.io.Closeable;
@@ -39,11 +37,6 @@ public abstract class FieldsConsumer imp
   /** Add a new field */
   public abstract TermsConsumer addField(FieldInfo field) throws IOException;
   
-  /** Adds a new DocValuesField */
-  public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
-    throw new UnsupportedOperationException("docvalues are not supported");
-  }
-
   /** Called when we are done adding everything. */
   public abstract void close() throws IOException;
 
@@ -59,18 +52,6 @@ public abstract class FieldsConsumer imp
         final TermsConsumer termsConsumer = addField(mergeState.fieldInfo);
         termsConsumer.merge(mergeState, terms);
       }
-      if (mergeState.fieldInfo.hasDocValues()) {
-        final DocValues docValues = fieldsEnum.docValues();
-        if(docValues == null) { 
-          /* It is actually possible that a fieldInfo has a values type but no values are actually available.
-           * this can happen if there are already segments without values around.
-           */
-          continue; 
-        }
-        final DocValuesConsumer docValuesConsumer = addValuesField(mergeState.fieldInfo);
-        assert docValuesConsumer != null;
-        docValuesConsumer.merge(mergeState, docValues);
-      }
     }
   }
  

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsProducer.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsProducer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsProducer.java Mon May  2 13:50:57 2011
@@ -37,11 +37,6 @@ public abstract class FieldsProducer ext
   public abstract void close() throws IOException;
   public abstract void loadTermsIndex(int indexDivisor) throws IOException;
 
-  @Override
-  public DocValues docValues(String field) throws IOException {
-    return null;
-  }
-  
   public static final FieldsProducer EMPTY = new FieldsProducer() {
     
     @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java Mon May  2 13:50:57 2011
@@ -184,9 +184,21 @@ public abstract class MultiLevelSkipList
     }
   }
   
+  /** returns x == 0 ? 0 : Math.floor(Math.log(x) / Math.log(base)) */
+  static int log(int x, int base) {
+    assert base >= 2;
+    int ret = 0;
+    long n = base; // needs to be a long to avoid overflow
+    while (x >= n) {
+      n *= base;
+      ret++;
+    }
+    return ret;
+  }
+  
   /** Loads the skip levels  */
   private void loadSkipLevels() throws IOException {
-    numberOfSkipLevels = docCount == 0 ? 0 : (int) Math.floor(Math.log(docCount) / Math.log(skipInterval[0]));
+    numberOfSkipLevels = log(docCount, skipInterval[0]);
     if (numberOfSkipLevels > maxNumberOfSkipLevels) {
       numberOfSkipLevels = maxNumberOfSkipLevels;
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListWriter.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListWriter.java Mon May  2 13:50:57 2011
@@ -61,7 +61,7 @@ public abstract class MultiLevelSkipList
     this.skipInterval = skipInterval;
     
     // calculate the maximum number of skip levels for this document frequency
-    numberOfSkipLevels = df == 0 ? 0 : (int) Math.floor(Math.log(df) / Math.log(skipInterval));
+    numberOfSkipLevels = MultiLevelSkipListReader.log(df, skipInterval);
     
     // make sure it does not exceed maxSkipLevels
     if (numberOfSkipLevels > maxSkipLevels) {

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java?rev=1098566&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/PerDocConsumer.java Mon May  2 13:50:57 2011
@@ -0,0 +1,59 @@
+package org.apache.lucene.index.codecs;
+/**
+ * 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.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.codecs.docvalues.DocValuesConsumer;
+import org.apache.lucene.index.values.DocValues;
+
+/**
+ * nocommit - javadoc
+ * @experimental
+ *
+ */
+public abstract class PerDocConsumer implements Closeable{
+  /** Adds a new DocValuesField */
+  public abstract DocValuesConsumer addValuesField(FieldInfo field)
+      throws IOException;
+
+  public void merge(MergeState mergeState, PerDocValues producer)
+      throws IOException {
+    Iterable<String> fields = producer.fields();
+    for (String field : fields) {
+      mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field);
+      assert mergeState.fieldInfo != null : "FieldInfo for field is null: "
+          + field;
+      if (mergeState.fieldInfo.hasDocValues()) {
+        final DocValues docValues = producer.docValues(field);
+        if (docValues == null) {
+          /*
+           * It is actually possible that a fieldInfo has a values type but no
+           * values are actually available. this can happen if there are already
+           * segments without values around.
+           */
+          continue;
+        }
+        final DocValuesConsumer docValuesConsumer = addValuesField(mergeState.fieldInfo);
+        assert docValuesConsumer != null;
+        docValuesConsumer.merge(mergeState, docValues);
+      }
+    }
+
+  }
+}

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java?rev=1098566&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/PerDocValues.java Mon May  2 13:50:57 2011
@@ -0,0 +1,44 @@
+package org.apache.lucene.index.codecs;
+/**
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.index.values.DocValues;
+
+/**
+ * 
+ * nocommit javadoc
+ * @experimental
+ */
+public abstract class PerDocValues implements Closeable {
+  /**
+   * Returns {@link DocValues} for the current field.
+   * 
+   * @param field
+   *          the field name
+   * @return the {@link DocValues} for this field or <code>null</code> if not
+   *         applicable.
+   * @throws IOException
+   */
+  public abstract DocValues docValues(String field) throws IOException;
+
+  public static final PerDocValues[] EMPTY_ARRAY = new PerDocValues[0];
+
+  public abstract Collection<String> fields();
+}

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java Mon May  2 13:50:57 2011
@@ -18,6 +18,9 @@ package org.apache.lucene.index.codecs;
  */
 
 import java.io.IOException;
+import java.io.FileOutputStream;   // for toDot
+import java.io.OutputStreamWriter; // for toDot
+import java.io.Writer;             // for toDot
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -34,6 +37,7 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.automaton.fst.FST;
 import org.apache.lucene.util.automaton.fst.PositiveIntOutputs;
+import org.apache.lucene.util.automaton.fst.Util; // for toDot
 
 /** See {@link VariableGapTermsIndexWriter}
  * 
@@ -52,11 +56,12 @@ public class VariableGapTermsIndexReader
   // start of the field info data
   protected long dirOffset;
 
+  final String segment;
   public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, int codecId)
     throws IOException {
 
     in = dir.openInput(IndexFileNames.segmentFileName(segment, ""+codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
-    
+    this.segment = segment;
     boolean success = false;
 
     try {
@@ -153,15 +158,11 @@ public class VariableGapTermsIndexReader
 
   private final class FieldIndexData {
 
-    private final FieldInfo fieldInfo;
     private final long indexStart;
-
     // Set only if terms index is loaded:
     private volatile FST<Long> fst;
 
     public FieldIndexData(FieldInfo fieldInfo, long indexStart) throws IOException {
-
-      this.fieldInfo = fieldInfo;
       this.indexStart = indexStart;
 
       if (indexDivisor > 0) {
@@ -176,6 +177,14 @@ public class VariableGapTermsIndexReader
         fst = new FST<Long>(clone, fstOutputs);
         clone.close();
 
+        /*
+        final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
+        Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
+        Util.toDot(fst, w, false, false);
+        System.out.println("FST INDEX: SAVED to " + dotFileName);
+        w.close();
+        */
+
         if (indexDivisor > 1) {
           // subsample
           final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java Mon May  2 13:50:57 2011
@@ -18,27 +18,22 @@ package org.apache.lucene.index.codecs.d
  */
 import java.io.IOException;
 import java.util.Comparator;
-import java.util.Iterator;
 import java.util.Set;
-import java.util.Map.Entry;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
-import org.apache.lucene.index.codecs.TermsConsumer;
-import org.apache.lucene.index.values.DocValues;
+import org.apache.lucene.index.codecs.PerDocConsumer;
+import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.values.Writer;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -60,80 +55,46 @@ public class DocValuesCodec extends Code
   }
 
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
+  public PerDocConsumer docsConsumer(final PerDocWriteState state)
       throws IOException {
-    final WrappingFieldsConsumer consumer;
-      consumer = new WrappingFieldsConsumer(other, comparator, state);
-    // nocommit this is a hack and only necessary since
-    // we want to initialized the wrapped
-    // fieldsConsumer lazily with a SegmentWriteState created after the docvalue
-    // ones is. We should fix this in DocumentWriter I guess. See
-    // DocFieldProcessor too!
-    return consumer;
-  }
+    return new PerDocConsumer() {
 
-  private static class WrappingFieldsConsumer extends FieldsConsumer {
-    private final SegmentWriteState state;
-    private FieldsConsumer wrappedConsumer;
-    private final Codec other;
-    private final Comparator<BytesRef> comparator;
-
-    public WrappingFieldsConsumer(Codec other, Comparator<BytesRef> comparator, SegmentWriteState state) {
-      this.other = other;
-      this.comparator = comparator;
-      this.state = state;
-    }
+      @Override
+      public void close() throws IOException {
+      }
 
-    @Override
-    public void close() throws IOException {
-      synchronized (this) {
-        if (wrappedConsumer != null) {
-          wrappedConsumer.close();
-        } 
+      @Override
+      public DocValuesConsumer addValuesField(FieldInfo field)
+          throws IOException {
+        final DocValuesConsumer consumer = Writer.create(field.getDocValues(),
+            docValuesId(state.segmentName, state.codecId, field.number),
+            // TODO can we have a compound file per segment and codec for
+            // docvalues?
+            state.directory, comparator, state.bytesUsed);
+        return consumer;
       }
-    
-    }
+    };
+  }
 
-    @Override
-    public synchronized DocValuesConsumer addValuesField(FieldInfo field)
-        throws IOException {
-      final DocValuesConsumer consumer = Writer.create(field.getDocValues(), docValuesId(state.segmentName, state.codecId, field.number),
-      // TODO can we have a compound file per segment and codec for
-          // docvalues?
-          state.directory, comparator, state.bytesUsed);
-      return consumer;
+  @Override
+  public PerDocValues docsProducer(SegmentReadState state) throws IOException {
+    try {
+    return new DocValuesProducerBase(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
+    }catch (IOException e) {
+      return new DocValuesProducerBase(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
     }
+  }
 
-    @Override
-    public TermsConsumer addField(FieldInfo field) throws IOException {
-      synchronized (this) {
-        if (wrappedConsumer == null)
-          wrappedConsumer = other.fieldsConsumer(state);
-      }
-      return wrappedConsumer.addField(field);
-    }
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
+      throws IOException {
+    return other.fieldsConsumer(state);
   }
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state)
       throws IOException {
-    final FieldInfos fieldInfos = state.fieldInfos;
-    boolean indexed = false;
-    boolean docValues = false;
-    for (FieldInfo fieldInfo : fieldInfos) {
-      if (fieldInfo.getCodecId() == state.codecId) {
-        indexed |= fieldInfo.isIndexed;
-        docValues |= fieldInfo.hasDocValues();
-        if (indexed && docValues)
-          break;
-      }
-    }
-    final FieldsProducer indexedProducer = indexed ? other.fieldsProducer(state) : FieldsProducer.EMPTY;
-    if (docValues) {
-      return new WrappingFielsdProducer(state, indexedProducer);
-    } else {
-      return FieldsProducer.EMPTY;
-    }
+    return other.fieldsProducer(state);
   }
   
   static String docValuesId(String segmentsName, int codecID, int fieldId) {
@@ -187,140 +148,4 @@ public class DocValuesCodec extends Code
     extensions.add(Writer.DATA_EXTENSION);
     extensions.add(Writer.INDEX_EXTENSION);
   }
-
-  static class WrappingFielsdProducer extends DocValuesProducerBase {
-
-    private final FieldsProducer other;
-
-    WrappingFielsdProducer(SegmentReadState state, FieldsProducer other)
-        throws IOException {
-      super(state.segmentInfo, state.dir, state.fieldInfos, state.codecId);
-      this.other = other;
-    }
-
-    @Override
-    public void close() throws IOException {
-      try {
-        other.close();
-      } finally {
-        super.close();
-      }
-    }
-
-    @Override
-    public void loadTermsIndex(int indexDivisor) throws IOException {
-      other.loadTermsIndex(indexDivisor);
-    }
-
-    @Override
-    public FieldsEnum iterator() throws IOException {
-      return new WrappingFieldsEnum(other.iterator(), docValues.entrySet()
-          .iterator());
-    }
-
-    @Override
-    public Terms terms(String field) throws IOException {
-      return other.terms(field);
-    }
-  }
-
-  static abstract class NameValue<V> {
-    String name;
-    V value;
-
-    NameValue<?> smaller(NameValue<?> other) throws IOException {
-      if (other.name == null) {
-        if (this.name == null) {
-          return null;
-        }
-        return this;
-      } else if (this.name == null) {
-        return other;
-      }
-      final int res = this.name.compareTo(other.name);
-      if (res < 0)
-        return this;
-      if (res == 0)
-        other.name = this.name;
-      return other;
-    }
-
-    abstract NameValue<V> next() throws IOException;
-  }
-
-  static class FieldsEnumNameValue extends NameValue<FieldsEnum> {
-    @Override
-    NameValue<FieldsEnum> next() throws IOException {
-      name = value.next();
-      return this;
-    }
-  }
-
-  static class DocValueNameValue extends NameValue<DocValues> {
-    Iterator<Entry<String, DocValues>> iter;
-
-    @Override
-    NameValue<DocValues> next() {
-      if (iter.hasNext()) {
-        Entry<String, DocValues> next = iter.next();
-        value = next.getValue();
-        name = next.getKey();
-      } else {
-        name = null;
-      }
-      return this;
-    }
-  }
-
-  static class WrappingFieldsEnum extends FieldsEnum {
-    private final DocValueNameValue docValues = new DocValueNameValue();
-    private final NameValue<FieldsEnum> fieldsEnum = new FieldsEnumNameValue();
-    private NameValue<?> coordinator;
-
-    @Override
-    public AttributeSource attributes() {
-      return fieldsEnum.value.attributes();
-    }
-
-    public WrappingFieldsEnum(FieldsEnum wrapped,
-        Iterator<Entry<String, DocValues>> docValues) {
-      this.docValues.iter = docValues;
-      this.fieldsEnum.value = wrapped;
-      coordinator = null;
-    }
-
-    @Override
-    public DocValues docValues() throws IOException {
-      if (docValues.name == coordinator.name)
-        return docValues.value;
-      return null;
-    }
-
-    @Override
-    public String next() throws IOException {
-      if (coordinator == null) {
-        coordinator = fieldsEnum.next().smaller(docValues.next());
-      } else {
-        String current = coordinator.name;
-        if (current == docValues.name) {
-          docValues.next();
-        }
-        if (current == fieldsEnum.name) {
-          fieldsEnum.next();
-        }
-        coordinator = docValues.smaller(fieldsEnum);
-
-      }
-      return coordinator == null ? null : coordinator.name;
-    }
-
-    @Override
-    public TermsEnum terms() throws IOException {
-      if (fieldsEnum.name == coordinator.name) {
-        return fieldsEnum.value.terms();
-      }
-      return null;
-    }
-  }
-
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java Mon May  2 13:50:57 2011
@@ -23,7 +23,7 @@ import java.util.TreeMap;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.values.Bytes;
 import org.apache.lucene.index.values.DocValues;
 import org.apache.lucene.index.values.Floats;
@@ -37,7 +37,7 @@ import org.apache.lucene.store.Directory
  * 
  * @lucene.experimental
  */
-public abstract class DocValuesProducerBase extends FieldsProducer {
+public class DocValuesProducerBase extends PerDocValues {
 
   protected final TreeMap<String, DocValues> docValues = new TreeMap<String, DocValues>();
 
@@ -145,4 +145,9 @@ public abstract class DocValuesProducerB
       throw ex;
     }
   }
+
+  @Override
+  public Collection<String> fields() {
+    return docValues.keySet();
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Mon May  2 13:50:57 2011
@@ -37,7 +37,6 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.codecs.FieldsProducer;
-import org.apache.lucene.index.values.DocValues;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
@@ -236,12 +235,6 @@ public class PreFlexFields extends Field
       termsEnum.reset(current);
       return termsEnum;
     }
-
-    @Override
-    public DocValues docValues() throws IOException {
-      //DocValues are not available on PreFlex indices
-      return null;
-    }
   }
   
   private class PreTerms extends Terms {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Mon May  2 13:50:57 2011
@@ -310,7 +310,7 @@ public final class TermInfosReader {
         }
       } else {
         assert sameTermInfo(ti, tiOrd, enumerator);
-        assert (int) enumerator.position == tiOrd.termOrd;
+        assert enumerator.position == tiOrd.termOrd;
       }
     } else {
       ti = null;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java Mon May  2 13:50:57 2011
@@ -233,6 +233,7 @@ public class PulsingPostingsReaderImpl e
     private Bits skipDocs;
     private int docID;
     private int freq;
+    private int payloadLength;
 
     public PulsingDocsEnum(FieldInfo fieldInfo) {
       omitTF = fieldInfo.omitTermFreqAndPositions;
@@ -246,6 +247,7 @@ public class PulsingPostingsReaderImpl e
       System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize);
       postings.reset(bytes);
       docID = 0;
+      payloadLength = 0;
       freq = 1;
       this.skipDocs = skipDocs;
       return this;
@@ -277,7 +279,6 @@ public class PulsingPostingsReaderImpl e
 
           // Skip positions
           if (storePayloads) {
-            int payloadLength = -1;
             for(int pos=0;pos<freq;pos++) {
               final int posCode = postings.readVInt();
               if ((posCode & 1) != 0) {
@@ -352,6 +353,7 @@ public class PulsingPostingsReaderImpl e
       postings.reset(bytes);
       this.skipDocs = skipDocs;
       payloadLength = 0;
+      posPending = 0;
       docID = 0;
       //System.out.println("PR d&p reset storesPayloads=" + storePayloads + " bytes=" + bytes.length + " this=" + this);
       return this;
@@ -359,7 +361,7 @@ public class PulsingPostingsReaderImpl e
 
     @Override
     public int nextDoc() throws IOException {
-      //System.out.println("PR d&p nextDoc this=" + this);
+      //System.out.println("PR.nextDoc this=" + this);
 
       while(true) {
         //System.out.println("  cycle skip posPending=" + posPending);
@@ -367,15 +369,16 @@ public class PulsingPostingsReaderImpl e
         skipPositions();
 
         if (postings.eof()) {
-          //System.out.println("PR   END");
+          //System.out.println("  END");
           return docID = NO_MORE_DOCS;
         }
-
+        //System.out.println("  read doc code");
         final int code = postings.readVInt();
         docID += code >>> 1;            // shift off low bit
         if ((code & 1) != 0) {          // if low bit is set
           freq = 1;                     // freq is one
         } else {
+          //System.out.println("  read freq");
           freq = postings.readVInt();     // else read freq
         }
         posPending = freq;
@@ -400,10 +403,12 @@ public class PulsingPostingsReaderImpl e
 
     @Override
     public int advance(int target) throws IOException {
+      //System.out.println("PR.advance target=" + target);
       int doc;
       while((doc=nextDoc()) != NO_MORE_DOCS) {
+        //System.out.println("  nextDoc got doc=" + doc);
         if (doc >= target) {
-          return doc;
+          return docID = doc;
         }
       }
       return docID = NO_MORE_DOCS;
@@ -411,7 +416,7 @@ public class PulsingPostingsReaderImpl e
 
     @Override
     public int nextPosition() throws IOException {
-      //System.out.println("PR d&p nextPosition posPending=" + posPending + " vs freq=" + freq);
+      //System.out.println("PR.nextPosition posPending=" + posPending + " vs freq=" + freq);
       
       assert posPending > 0;
       posPending--;
@@ -421,6 +426,7 @@ public class PulsingPostingsReaderImpl e
           //System.out.println("PR     skip payload=" + payloadLength);
           postings.skipBytes(payloadLength);
         }
+        //System.out.println("  read pos code");
         final int code = postings.readVInt();
         //System.out.println("PR     code=" + code);
         if ((code & 1) != 0) {
@@ -433,16 +439,17 @@ public class PulsingPostingsReaderImpl e
         position += postings.readVInt();
       }
 
-      //System.out.println("PR d&p nextPos return pos=" + position + " this=" + this);
+      //System.out.println("  return pos=" + position + " hasPayload=" + !payloadRetrieved + " posPending=" + posPending + " this=" + this);
       return position;
     }
 
     private void skipPositions() throws IOException {
+      //System.out.println("PR.skipPositions: posPending=" + posPending);
       while(posPending != 0) {
         nextPosition();
       }
       if (storePayloads && !payloadRetrieved) {
-        //System.out.println("  skip payload len=" + payloadLength);
+        //System.out.println("  skip last payload len=" + payloadLength);
         postings.skipBytes(payloadLength);
         payloadRetrieved = true;
       }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java Mon May  2 13:50:57 2011
@@ -201,6 +201,7 @@ public final class PulsingPostingsWriter
       if (!omitTF) {
         int lastDocID = 0;
         int pendingIDX = 0;
+        int lastPayloadLength = -1;
         while(pendingIDX < pendingCount) {
           final Position doc = pending[pendingIDX];
 
@@ -217,7 +218,6 @@ public final class PulsingPostingsWriter
           }
 
           int lastPos = 0;
-          int lastPayloadLength = -1;
           for(int posIDX=0;posIDX<doc.termFreq;posIDX++) {
             final Position pos = pending[pendingIDX++];
             assert pos.docID == doc.docID;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java Mon May  2 13:50:57 2011
@@ -69,12 +69,13 @@ public final class SepPostingsWriterImpl
    * smaller indexes, greater acceleration, but fewer accelerable cases, while
    * smaller values result in bigger indexes, less acceleration and more
    * accelerable cases. More detailed experiments would be useful here. */
-  final int skipInterval = 16;
+  final int skipInterval;
+  static final int DEFAULT_SKIP_INTERVAL = 16;
   
   /**
    * Expert: minimum docFreq to write any skip data at all
    */
-  final int skipMinimum = skipInterval;
+  final int skipMinimum;
 
   /** Expert: The maximum number of skip levels. Smaller values result in 
    * slightly smaller indexes, but slower skipping in big posting lists.
@@ -102,8 +103,14 @@ public final class SepPostingsWriterImpl
   private final RAMOutputStream indexBytesWriter = new RAMOutputStream();
 
   public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory) throws IOException {
+    this(state, factory, DEFAULT_SKIP_INTERVAL);
+  }
+
+  public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory, int skipInterval) throws IOException {
     super();
     final String codecIdAsString = state.codecIdAsString();
+    this.skipInterval = skipInterval;
+    this.skipMinimum = skipInterval; /* set to the same for now */
     final String docFileName = IndexFileNames.segmentFileName(state.segmentName, codecIdAsString, DOC_EXTENSION);
     docOut = factory.createOutput(state.directory, docFileName);
     docIndex = docOut.index();

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java Mon May  2 13:50:57 2011
@@ -177,6 +177,7 @@ class SepSkipListReader extends MultiLev
   @Override
   protected int readSkipData(int level, IndexInput skipStream) throws IOException {
     int delta;
+    assert !omitTF || !currentFieldStoresPayloads;
     if (currentFieldStoresPayloads) {
       // the current field stores payloads.
       // if the doc delta is odd then we have