lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r677865 [3/5] - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/store/ src/java/org/apache/lucene/util/ src/test/org/apache/lucene/ src/test/org/apache/lucene/index/ src/test/org/apache/lucene/search/
Date Fri, 18 Jul 2008 09:20:14 GMT
Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriterThreadState.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriterThreadState.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriterThreadState.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriterThreadState.java Fri Jul 18 02:20:12 2008
@@ -18,13 +18,6 @@
  */
 
 import java.io.IOException;
-import java.util.List;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.util.UnicodeUtil;
 
 /** Used by DocumentsWriter to maintain per-thread state.
  *  We keep a separate Posting hash and other state for each
@@ -32,693 +25,26 @@
  *  when writing the segment. */
 final class DocumentsWriterThreadState {
 
-  Posting[] postingsFreeList;           // Free Posting instances
-  int postingsFreeCount;
-
-  RAMOutputStream tvfLocal = new RAMOutputStream();    // Term vectors for one doc
-  RAMOutputStream fdtLocal = new RAMOutputStream();    // Stored fields for one doc
-  FieldsWriter localFieldsWriter;       // Fields for one doc
-
-  long[] vectorFieldPointers;
-  int[] vectorFieldNumbers;
-
-  boolean isIdle = true;                // Whether we are in use
-  int numThreads = 1;                   // Number of threads that use this instance
-
-  int docID;                            // docID we are now working on
-  int numStoredFields;                  // How many stored fields in current doc
-  float docBoost;                       // Boost for current doc
-
-  DocumentsWriterFieldData[] fieldDataArray;           // Fields touched by current doc
-  int numFieldData;                     // How many fields in current doc
-  int numVectorFields;                  // How many vector fields in current doc
-
-  DocumentsWriterFieldData[] allFieldDataArray = new DocumentsWriterFieldData[10]; // All FieldData instances
-  int numAllFieldData;
-  DocumentsWriterFieldData[] fieldDataHash;            // Hash FieldData instances by field name
-  int fieldDataHashMask;
-  String maxTermPrefix;                 // Non-null prefix of a too-large term if this
-  // doc has one
-
-  boolean doFlushAfter;
+  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;
 
-  final ByteBlockPool postingsPool;
-  final ByteBlockPool vectorsPool;
-  final CharBlockPool charPool;
-
-  public DocumentsWriterThreadState(DocumentsWriter docWriter) {
+  public DocumentsWriterThreadState(DocumentsWriter docWriter) throws IOException {
     this.docWriter = docWriter;
-    fieldDataArray = new DocumentsWriterFieldData[8];
-
-    fieldDataHash = new DocumentsWriterFieldData[16];
-    fieldDataHashMask = 15;
-
-    vectorFieldPointers = new long[10];
-    vectorFieldNumbers = new int[10];
-    postingsFreeList = new Posting[256];
-    postingsFreeCount = 0;
-
-    postingsPool = new ByteBlockPool(docWriter.byteBlockAllocator, true);
-    vectorsPool = new ByteBlockPool(docWriter.byteBlockAllocator, false);
-    charPool = new CharBlockPool(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);
   }
 
-  /** Clear the postings hash and return objects back to
-   *  shared pool */
-  public void resetPostings() throws IOException {
-    fieldGen = 0;
-    maxPostingsVectors = 0;
+  void doAfterFlush() {
+    numThreads = 0;
     doFlushAfter = false;
-    if (localFieldsWriter != null) {
-      localFieldsWriter.close();
-      localFieldsWriter = null;
-    }
-    postingsPool.reset();
-    charPool.reset();
-    docWriter.recyclePostings(postingsFreeList, postingsFreeCount);
-    postingsFreeCount = 0;
-    for(int i=0;i<numAllFieldData;i++) {
-      DocumentsWriterFieldData fp = allFieldDataArray[i];
-      fp.lastGen = -1;
-      if (fp.numPostings > 0)
-        fp.resetPostingArrays();
-    }
-  }
-
-  /** Move all per-document state that was accumulated in
-   *  the ThreadState into the "real" stores. */
-  public void writeDocument() throws IOException, AbortException {
-
-    // If we hit an exception while appending to the
-    // stored fields or term vectors files, we have to
-    // abort all documents since we last flushed because
-    // it means those files are possibly inconsistent.
-    try {
-
-      docWriter.numDocsInStore++;
-
-      // Append stored fields to the real FieldsWriter:
-      docWriter.fieldsWriter.flushDocument(numStoredFields, fdtLocal);
-      fdtLocal.reset();
-
-      // Append term vectors to the real outputs:
-      final IndexOutput tvx = docWriter.tvx;
-      final IndexOutput tvd = docWriter.tvd;
-      final IndexOutput tvf = docWriter.tvf;
-      if (tvx != null) {
-        tvx.writeLong(tvd.getFilePointer());
-        tvx.writeLong(tvf.getFilePointer());
-        tvd.writeVInt(numVectorFields);
-        if (numVectorFields > 0) {
-          for(int i=0;i<numVectorFields;i++)
-            tvd.writeVInt(vectorFieldNumbers[i]);
-          assert 0 == vectorFieldPointers[0];
-          long lastPos = vectorFieldPointers[0];
-          for(int i=1;i<numVectorFields;i++) {
-            long pos = vectorFieldPointers[i];
-            tvd.writeVLong(pos-lastPos);
-            lastPos = pos;
-          }
-          tvfLocal.writeTo(tvf);
-          tvfLocal.reset();
-        }
-      }
-
-      // Append norms for the fields we saw:
-      for(int i=0;i<numFieldData;i++) {
-        DocumentsWriterFieldData fp = fieldDataArray[i];
-        if (fp.doNorms) {
-          BufferedNorms bn = docWriter.norms[fp.fieldInfo.number];
-          assert bn != null;
-          assert bn.upto <= docID;
-          bn.fill(docID);
-          float norm = fp.boost * docWriter.writer.getSimilarity().lengthNorm(fp.fieldInfo.name, fp.length);
-          bn.add(norm);
-        }
-      }
-    } catch (Throwable t) {
-      // Forcefully idle this threadstate -- its state will
-      // be reset by abort()
-      isIdle = true;
-      throw new AbortException(t, docWriter);
-    }
-
-    if (docWriter.bufferIsFull && !docWriter.flushPending) {
-      docWriter.flushPending = true;
-      doFlushAfter = true;
-    }
   }
-
-  int fieldGen;
-
-  /** Initializes shared state for this new document */
-  void init(Document doc, int docID) throws IOException, AbortException {
-
-    assert !isIdle;
-    assert docWriter.writer.testPoint("DocumentsWriter.ThreadState.init start");
-
-    this.docID = docID;
-    docBoost = doc.getBoost();
-    numStoredFields = 0;
-    numFieldData = 0;
-    numVectorFields = 0;
-    maxTermPrefix = null;
-
-    assert 0 == fdtLocal.length();
-    assert 0 == fdtLocal.getFilePointer();
-    assert 0 == tvfLocal.length();
-    assert 0 == tvfLocal.getFilePointer();
-    final int thisFieldGen = fieldGen++;
-
-    List docFields = doc.getFields();
-    final int numDocFields = docFields.size();
-    boolean docHasVectors = false;
-
-    // Absorb any new fields first seen in this document.
-    // Also absorb any changes to fields we had already
-    // seen before (eg suddenly turning on norms or
-    // vectors, etc.):
-
-    for(int i=0;i<numDocFields;i++) {
-      Fieldable field = (Fieldable) docFields.get(i);
-
-      FieldInfo fi = docWriter.fieldInfos.add(field.name(), field.isIndexed(), field.isTermVectorStored(),
-                                              field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(),
-                                              field.getOmitNorms(), false);
-      if (fi.isIndexed && !fi.omitNorms) {
-        // Maybe grow our buffered norms
-        if (docWriter.norms.length <= fi.number) {
-          int newSize = (int) ((1+fi.number)*1.25);
-          BufferedNorms[] newNorms = new BufferedNorms[newSize];
-          System.arraycopy(docWriter.norms, 0, newNorms, 0, docWriter.norms.length);
-          docWriter.norms = newNorms;
-        }
-          
-        if (docWriter.norms[fi.number] == null)
-          docWriter.norms[fi.number] = new BufferedNorms();
-
-        docWriter.hasNorms = true;
-      }
-
-      // Make sure we have a FieldData allocated
-      int hashPos = fi.name.hashCode() & fieldDataHashMask;
-      DocumentsWriterFieldData fp = fieldDataHash[hashPos];
-      while(fp != null && !fp.fieldInfo.name.equals(fi.name))
-        fp = fp.next;
-
-      if (fp == null) {
-
-        fp = new DocumentsWriterFieldData(this, fi);
-        fp.next = fieldDataHash[hashPos];
-        fieldDataHash[hashPos] = fp;
-
-        if (numAllFieldData == allFieldDataArray.length) {
-          int newSize = (int) (allFieldDataArray.length*1.5);
-          int newHashSize = fieldDataHash.length*2;
-
-          DocumentsWriterFieldData newArray[] = new DocumentsWriterFieldData[newSize];
-          DocumentsWriterFieldData newHashArray[] = new DocumentsWriterFieldData[newHashSize];
-          System.arraycopy(allFieldDataArray, 0, newArray, 0, numAllFieldData);
-
-          // Rehash
-          fieldDataHashMask = newSize-1;
-          for(int j=0;j<fieldDataHash.length;j++) {
-            DocumentsWriterFieldData fp0 = fieldDataHash[j];
-            while(fp0 != null) {
-              hashPos = fp0.fieldInfo.name.hashCode() & fieldDataHashMask;
-              DocumentsWriterFieldData nextFP0 = fp0.next;
-              fp0.next = newHashArray[hashPos];
-              newHashArray[hashPos] = fp0;
-              fp0 = nextFP0;
-            }
-          }
-
-          allFieldDataArray = newArray;
-          fieldDataHash = newHashArray;
-        }
-        allFieldDataArray[numAllFieldData++] = fp;
-      } else {
-        assert fp.fieldInfo == fi;
-      }
-
-      if (thisFieldGen != fp.lastGen) {
-
-        // First time we're seeing this field for this doc
-        fp.lastGen = thisFieldGen;
-        fp.fieldCount = 0;
-        fp.doVectors = fp.doVectorPositions = fp.doVectorOffsets = false;
-        fp.doNorms = fi.isIndexed && !fi.omitNorms;
-
-        if (numFieldData == fieldDataArray.length) {
-          int newSize = fieldDataArray.length*2;
-          DocumentsWriterFieldData newArray[] = new DocumentsWriterFieldData[newSize];
-          System.arraycopy(fieldDataArray, 0, newArray, 0, numFieldData);
-          fieldDataArray = newArray;
-
-        }
-        fieldDataArray[numFieldData++] = fp;
-      }
-
-      if (field.isTermVectorStored()) {
-        if (!fp.doVectors && numVectorFields++ == vectorFieldPointers.length) {
-          final int newSize = (int) (numVectorFields*1.5);
-          vectorFieldPointers = new long[newSize];
-          vectorFieldNumbers = new int[newSize];
-        }
-        fp.doVectors = true;
-        docHasVectors = true;
-
-        fp.doVectorPositions |= field.isStorePositionWithTermVector();
-        fp.doVectorOffsets |= field.isStoreOffsetWithTermVector();
-      }
-
-      if (fp.fieldCount == fp.docFields.length) {
-        Fieldable[] newArray = new Fieldable[fp.docFields.length*2];
-        System.arraycopy(fp.docFields, 0, newArray, 0, fp.docFields.length);
-        fp.docFields = newArray;
-      }
-
-      // Lazily allocate arrays for postings:
-      if (field.isIndexed() && fp.postingsHash == null)
-        fp.initPostingArrays();
-
-      fp.docFields[fp.fieldCount++] = field;
-    }
-
-    // Maybe init the local & global fieldsWriter
-    if (localFieldsWriter == null) {
-      if (docWriter.fieldsWriter == null) {
-        assert docWriter.docStoreSegment == null;
-        assert docWriter.segment != null;
-        docWriter.files = null;
-        docWriter.docStoreSegment = docWriter.segment;
-        // If we hit an exception while init'ing the
-        // fieldsWriter, we must abort this segment
-        // because those files will be in an unknown
-        // state:
-        try {
-          docWriter.fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.docStoreSegment, docWriter.fieldInfos);
-        } catch (Throwable t) {
-          throw new AbortException(t, docWriter);
-        }
-      }
-      localFieldsWriter = new FieldsWriter(null, fdtLocal, docWriter.fieldInfos);
-    }
-
-    // First time we see a doc that has field(s) with
-    // stored vectors, we init our tvx writer
-    if (docHasVectors) {
-      if (docWriter.tvx == null) {
-        assert docWriter.docStoreSegment != null;
-        docWriter.files = 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:
-        try {
-          docWriter.tvx = docWriter.directory.createOutput(docWriter.docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
-          docWriter.tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
-          docWriter.tvd = docWriter.directory.createOutput(docWriter.docStoreSegment +  "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
-          docWriter.tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
-          docWriter.tvf = docWriter.directory.createOutput(docWriter.docStoreSegment +  "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
-          docWriter.tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
-
-          // We must "catch up" for all docs before us
-          // that had no vectors:
-          for(int i=0;i<docWriter.numDocsInStore;i++) {
-            docWriter.tvx.writeLong(docWriter.tvd.getFilePointer());
-            docWriter.tvd.writeVInt(0);
-            docWriter.tvx.writeLong(0);
-          }
-        } catch (Throwable t) {
-          throw new AbortException(t, docWriter);
-        }
-      }
-      numVectorFields = 0;
-    }
-  }
-
-  /** Do in-place sort of Posting array */
-  void doPostingSort(Posting[] postings, int numPosting) {
-    quickSort(postings, 0, numPosting-1);
-  }
-
-  void quickSort(Posting[] postings, int lo, int hi) {
-    if (lo >= hi)
-      return;
-    else if (hi == 1+lo) {
-      if (comparePostings(postings[lo], postings[hi]) > 0) {
-        final Posting tmp = postings[lo];
-        postings[lo] = postings[hi];
-        postings[hi] = tmp;
-      }
-      return;
-    }
-
-    int mid = (lo + hi) >>> 1;
-
-    if (comparePostings(postings[lo], postings[mid]) > 0) {
-      Posting tmp = postings[lo];
-      postings[lo] = postings[mid];
-      postings[mid] = tmp;
-    }
-
-    if (comparePostings(postings[mid], postings[hi]) > 0) {
-      Posting tmp = postings[mid];
-      postings[mid] = postings[hi];
-      postings[hi] = tmp;
-
-      if (comparePostings(postings[lo], postings[mid]) > 0) {
-        Posting tmp2 = postings[lo];
-        postings[lo] = postings[mid];
-        postings[mid] = tmp2;
-      }
-    }
-
-    int left = lo + 1;
-    int right = hi - 1;
-
-    if (left >= right)
-      return;
-
-    Posting partition = postings[mid];
-
-    for (; ;) {
-      while (comparePostings(postings[right], partition) > 0)
-        --right;
-
-      while (left < right && comparePostings(postings[left], partition) <= 0)
-        ++left;
-
-      if (left < right) {
-        Posting tmp = postings[left];
-        postings[left] = postings[right];
-        postings[right] = tmp;
-        --right;
-      } else {
-        break;
-      }
-    }
-
-    quickSort(postings, lo, left);
-    quickSort(postings, left + 1, hi);
-  }
-
-  /** Do in-place sort of PostingVector array */
-  void doVectorSort(PostingVector[] postings, int numPosting) {
-    quickSort(postings, 0, numPosting-1);
-  }
-
-  void quickSort(PostingVector[] postings, int lo, int hi) {
-    if (lo >= hi)
-      return;
-    else if (hi == 1+lo) {
-      if (comparePostings(postings[lo].p, postings[hi].p) > 0) {
-        final PostingVector tmp = postings[lo];
-        postings[lo] = postings[hi];
-        postings[hi] = tmp;
-      }
-      return;
-    }
-
-    int mid = (lo + hi) >>> 1;
-
-    if (comparePostings(postings[lo].p, postings[mid].p) > 0) {
-      PostingVector tmp = postings[lo];
-      postings[lo] = postings[mid];
-      postings[mid] = tmp;
-    }
-
-    if (comparePostings(postings[mid].p, postings[hi].p) > 0) {
-      PostingVector tmp = postings[mid];
-      postings[mid] = postings[hi];
-      postings[hi] = tmp;
-
-      if (comparePostings(postings[lo].p, postings[mid].p) > 0) {
-        PostingVector tmp2 = postings[lo];
-        postings[lo] = postings[mid];
-        postings[mid] = tmp2;
-      }
-    }
-
-    int left = lo + 1;
-    int right = hi - 1;
-
-    if (left >= right)
-      return;
-
-    PostingVector partition = postings[mid];
-
-    for (; ;) {
-      while (comparePostings(postings[right].p, partition.p) > 0)
-        --right;
-
-      while (left < right && comparePostings(postings[left].p, partition.p) <= 0)
-        ++left;
-
-      if (left < right) {
-        PostingVector tmp = postings[left];
-        postings[left] = postings[right];
-        postings[right] = tmp;
-        --right;
-      } else {
-        break;
-      }
-    }
-
-    quickSort(postings, lo, left);
-    quickSort(postings, left + 1, hi);
-  }
-
-  void quickSort(DocumentsWriterFieldData[] array, int lo, int hi) {
-    if (lo >= hi)
-      return;
-    else if (hi == 1+lo) {
-      if (array[lo].compareTo(array[hi]) > 0) {
-        final DocumentsWriterFieldData tmp = array[lo];
-        array[lo] = array[hi];
-        array[hi] = tmp;
-      }
-      return;
-    }
-
-    int mid = (lo + hi) >>> 1;
-
-    if (array[lo].compareTo(array[mid]) > 0) {
-      DocumentsWriterFieldData tmp = array[lo];
-      array[lo] = array[mid];
-      array[mid] = tmp;
-    }
-
-    if (array[mid].compareTo(array[hi]) > 0) {
-      DocumentsWriterFieldData tmp = array[mid];
-      array[mid] = array[hi];
-      array[hi] = tmp;
-
-      if (array[lo].compareTo(array[mid]) > 0) {
-        DocumentsWriterFieldData tmp2 = array[lo];
-        array[lo] = array[mid];
-        array[mid] = tmp2;
-      }
-    }
-
-    int left = lo + 1;
-    int right = hi - 1;
-
-    if (left >= right)
-      return;
-
-    DocumentsWriterFieldData partition = array[mid];
-
-    for (; ;) {
-      while (array[right].compareTo(partition) > 0)
-        --right;
-
-      while (left < right && array[left].compareTo(partition) <= 0)
-        ++left;
-
-      if (left < right) {
-        DocumentsWriterFieldData tmp = array[left];
-        array[left] = array[right];
-        array[right] = tmp;
-        --right;
-      } else {
-        break;
-      }
-    }
-
-    quickSort(array, lo, left);
-    quickSort(array, left + 1, hi);
-  }
-
-  /** If there are fields we've seen but did not see again
-   *  in the last run, then free them up.  Also reduce
-   *  postings hash size. */
-  void trimFields() {
-
-    int upto = 0;
-    for(int i=0;i<numAllFieldData;i++) {
-      DocumentsWriterFieldData fp = allFieldDataArray[i];
-      if (fp.lastGen == -1) {
-        // This field was not seen since the previous
-        // flush, so, free up its resources now
-
-        // Unhash
-        final int hashPos = fp.fieldInfo.name.hashCode() & fieldDataHashMask;
-        DocumentsWriterFieldData last = null;
-        DocumentsWriterFieldData fp0 = fieldDataHash[hashPos];
-        while(fp0 != fp) {
-          last = fp0;
-          fp0 = fp0.next;
-        }
-
-        if (last == null)
-          fieldDataHash[hashPos] = fp.next;
-        else
-          last.next = fp.next;
-
-        if (docWriter.infoStream != null)
-          docWriter.infoStream.println("  remove field=" + fp.fieldInfo.name);
-
-      } else {
-        // Reset
-        fp.lastGen = -1;
-        allFieldDataArray[upto++] = fp;
-          
-        if (fp.numPostings > 0 && ((float) fp.numPostings) / fp.postingsHashSize < 0.2) {
-          int hashSize = fp.postingsHashSize;
-
-          // Reduce hash so it's between 25-50% full
-          while (fp.numPostings < (hashSize>>1) && hashSize >= 2)
-            hashSize >>= 1;
-          hashSize <<= 1;
-
-          if (hashSize != fp.postingsHash.length)
-            fp.rehashPostings(hashSize);
-        }
-      }
-    }
-
-    // If we didn't see any norms for this field since
-    // last flush, free it
-    for(int i=0;i<docWriter.norms.length;i++) {
-      BufferedNorms n = docWriter.norms[i];
-      if (n != null && n.upto == 0)
-        docWriter.norms[i] = null;
-    }
-
-    numAllFieldData = upto;
-
-    // Also pare back PostingsVectors if it's excessively
-    // large
-    if (maxPostingsVectors * 1.5 < postingsVectors.length) {
-      final int newSize;
-      if (0 == maxPostingsVectors)
-        newSize = 1;
-      else
-        newSize = (int) (1.5*maxPostingsVectors);
-      PostingVector[] newArray = new PostingVector[newSize];
-      System.arraycopy(postingsVectors, 0, newArray, 0, newSize);
-      postingsVectors = newArray;
-    }
-  }
-
-  /** Tokenizes the fields of a document into Postings */
-  void processDocument(Analyzer analyzer)
-    throws IOException, AbortException {
-
-    final int numFields = numFieldData;
-    assert clearLastVectorFieldName();
-
-    assert 0 == fdtLocal.length();
-
-    if (docWriter.tvx != null)
-      // 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(fieldDataArray, 0, numFields-1);
-
-    // We process the document one field at a time
-    for(int i=0;i<numFields;i++)
-      fieldDataArray[i].processField(analyzer);
-
-    if (docWriter.infoStream != null && maxTermPrefix != null)
-      docWriter.infoStream.println("WARNING: document contains at least one immense term (longer than the max length " + DocumentsWriter.MAX_TERM_LENGTH + "), all of which were skipped.  Please correct the analyzer to not produce such terms.  The prefix of the first immense term is: '" + maxTermPrefix + "...'"); 
-  }
-
-  // USE ONLY FOR DEBUGGING!
-  /*
-    public String getPostingText() {
-    char[] text = charPool.buffers[p.textStart >> CHAR_BLOCK_SHIFT];
-    int upto = p.textStart & CHAR_BLOCK_MASK;
-    while(text[upto] != 0xffff)
-    upto++;
-    return new String(text, p.textStart, upto-(p.textStart & BYTE_BLOCK_MASK));
-    }
-  */
-
-  /** Compares term text for two Posting instance and
-   *  returns -1 if p1 < p2; 1 if p1 > p2; else 0.
-   */
-  int comparePostings(Posting p1, Posting p2) {
-    if (p1 == p2)
-      return 0;
-    final char[] text1 = charPool.buffers[p1.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
-    int pos1 = p1.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
-    final char[] text2 = charPool.buffers[p2.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
-    int pos2 = p2.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
-
-    assert text1 != text2 || pos1 != pos2;
-
-    while(true) {
-      final char c1 = text1[pos1++];
-      final char c2 = text2[pos2++];
-      if (c1 != c2) {
-        if (0xffff == c2)
-          return 1;
-        else if (0xffff == c1)
-          return -1;
-        else
-          return c1-c2;
-      } else
-        // This method should never compare equal postings
-        // unless p1==p2
-        assert c1 != 0xffff;
-    }
-  }
-
-  String lastVectorFieldName;
-
-  // Called only by assert
-  final boolean clearLastVectorFieldName() {
-    lastVectorFieldName = null;
-    return true;
-  }
-
-  // Called only by assert
-  final boolean vectorFieldsInOrder(FieldInfo fi) {
-    try {
-      if (lastVectorFieldName != null)
-        return lastVectorFieldName.compareTo(fi.name) < 0;
-      else
-        return true;
-    } finally {
-      lastVectorFieldName = fi.name;
-    }
-  }
-
-  PostingVector[] postingsVectors = new PostingVector[1];
-  int maxPostingsVectors;
-
-  // Used to read a string value for a field
-  ReusableStringReader stringReader = new ReusableStringReader();
-
-  final UnicodeUtil.UTF8Result utf8Results[] = {new UnicodeUtil.UTF8Result(),
-                                                new UnicodeUtil.UTF8Result()};
 }
-

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfo.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfo.java Fri Jul 18 02:20:12 2008
@@ -48,4 +48,47 @@
     return new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector,
                          storeOffsetWithTermVector, omitNorms, storePayloads);
   }
+
+  void update(boolean isIndexed, boolean storeTermVector, boolean storePositionWithTermVector, 
+              boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads) {
+    if (this.isIndexed != isIndexed) {
+      this.isIndexed = true;                      // once indexed, always index
+    }
+    if (this.storeTermVector != storeTermVector) {
+      this.storeTermVector = true;                // once vector, always vector
+    }
+    if (this.storePositionWithTermVector != storePositionWithTermVector) {
+      this.storePositionWithTermVector = true;                // once vector, always vector
+    }
+    if (this.storeOffsetWithTermVector != storeOffsetWithTermVector) {
+      this.storeOffsetWithTermVector = true;                // once vector, always vector
+    }
+    if (this.omitNorms != omitNorms) {
+      this.omitNorms = false;                // once norms are stored, always store
+    }
+    if (this.storePayloads != storePayloads) {
+      this.storePayloads = true;
+    }
+  }
+
+  void update(FieldInfo other) {
+    if (isIndexed != other.isIndexed) {
+      isIndexed = true;                      // once indexed, always index
+    }
+    if (storeTermVector != other.storeTermVector) {
+      storeTermVector = true;                // once vector, always vector
+    }
+    if (storePositionWithTermVector != other.storePositionWithTermVector) {
+      storePositionWithTermVector = true;                // once vector, always vector
+    }
+    if (storeOffsetWithTermVector != other.storeOffsetWithTermVector) {
+      storeOffsetWithTermVector = true;                // once vector, always vector
+    }
+    if (omitNorms != other.omitNorms) {
+      omitNorms = false;                // once norms are stored, always store
+    }
+    if (storePayloads != other.storePayloads) {
+      storePayloads = true;
+    }
+  }
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FieldInfos.java Fri Jul 18 02:20:12 2008
@@ -65,7 +65,7 @@
   /**
    * Returns a deep clone of this FieldInfos instance.
    */
-  public Object clone() {
+  synchronized public Object clone() {
     FieldInfos fis = new FieldInfos();
     final int numField = byNumber.size();
     for(int i=0;i<numField;i++) {
@@ -77,7 +77,7 @@
   }
 
   /** Adds field info for a Document. */
-  public void add(Document doc) {
+  synchronized public void add(Document doc) {
     List fields = doc.getFields();
     Iterator fieldIterator = fields.iterator();
     while (fieldIterator.hasNext()) {
@@ -95,7 +95,7 @@
    * @param storePositionWithTermVector treu if positions should be stored.
    * @param storeOffsetWithTermVector true if offsets should be stored
    */
-  public void addIndexed(Collection names, boolean storeTermVectors, boolean storePositionWithTermVector, 
+  synchronized public void addIndexed(Collection names, boolean storeTermVectors, boolean storePositionWithTermVector, 
                          boolean storeOffsetWithTermVector) {
     Iterator i = names.iterator();
     while (i.hasNext()) {
@@ -111,7 +111,7 @@
    * 
    * @see #add(String, boolean)
    */
-  public void add(Collection names, boolean isIndexed) {
+  synchronized public void add(Collection names, boolean isIndexed) {
     Iterator i = names.iterator();
     while (i.hasNext()) {
       add((String)i.next(), isIndexed);
@@ -125,7 +125,7 @@
    * @param isIndexed true if the field is indexed
    * @see #add(String, boolean, boolean, boolean, boolean)
    */
-  public void add(String name, boolean isIndexed) {
+  synchronized public void add(String name, boolean isIndexed) {
     add(name, isIndexed, false, false, false, false);
   }
 
@@ -136,7 +136,7 @@
    * @param isIndexed  true if the field is indexed
    * @param storeTermVector true if the term vector should be stored
    */
-  public void add(String name, boolean isIndexed, boolean storeTermVector){
+  synchronized public void add(String name, boolean isIndexed, boolean storeTermVector){
     add(name, isIndexed, storeTermVector, false, false, false);
   }
   
@@ -151,7 +151,7 @@
    * @param storePositionWithTermVector true if the term vector with positions should be stored
    * @param storeOffsetWithTermVector true if the term vector with offsets should be stored
    */
-  public void add(String name, boolean isIndexed, boolean storeTermVector,
+  synchronized public void add(String name, boolean isIndexed, boolean storeTermVector,
                   boolean storePositionWithTermVector, boolean storeOffsetWithTermVector) {
 
     add(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, false);
@@ -169,7 +169,7 @@
    * @param storeOffsetWithTermVector true if the term vector with offsets should be stored
    * @param omitNorms true if the norms for the indexed field should be omitted
    */
-  public void add(String name, boolean isIndexed, boolean storeTermVector,
+  synchronized public void add(String name, boolean isIndexed, boolean storeTermVector,
                   boolean storePositionWithTermVector, boolean storeOffsetWithTermVector, boolean omitNorms) {
     add(name, isIndexed, storeTermVector, storePositionWithTermVector,
         storeOffsetWithTermVector, omitNorms, false);
@@ -188,32 +188,26 @@
    * @param omitNorms true if the norms for the indexed field should be omitted
    * @param storePayloads true if payloads should be stored for this field
    */
-  public FieldInfo add(String name, boolean isIndexed, boolean storeTermVector,
+  synchronized public FieldInfo add(String name, boolean isIndexed, boolean storeTermVector,
                        boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
                        boolean omitNorms, boolean storePayloads) {
     FieldInfo fi = fieldInfo(name);
     if (fi == null) {
       return addInternal(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads);
     } else {
-      if (fi.isIndexed != isIndexed) {
-        fi.isIndexed = true;                      // once indexed, always index
-      }
-      if (fi.storeTermVector != storeTermVector) {
-        fi.storeTermVector = true;                // once vector, always vector
-      }
-      if (fi.storePositionWithTermVector != storePositionWithTermVector) {
-        fi.storePositionWithTermVector = true;                // once vector, always vector
-      }
-      if (fi.storeOffsetWithTermVector != storeOffsetWithTermVector) {
-        fi.storeOffsetWithTermVector = true;                // once vector, always vector
-      }
-      if (fi.omitNorms != omitNorms) {
-        fi.omitNorms = false;                // once norms are stored, always store
-      }
-      if (fi.storePayloads != storePayloads) {
-        fi.storePayloads = true;
-      }
+      fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads);
+    }
+    return fi;
+  }
 
+  synchronized public FieldInfo add(FieldInfo fieldInfo) {
+    FieldInfo fi = fieldInfo(fieldInfo.name);
+    if (fi == null) {
+      return addInternal(fieldInfo.name, fieldInfo.isIndexed, fieldInfo.storeTermVector,
+                         fieldInfo.storePositionWithTermVector, fieldInfo.storeOffsetWithTermVector,
+                         fieldInfo.omitNorms, fieldInfo.storePayloads);
+    } else {
+      fi.update(fieldInfo);
     }
     return fi;
   }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FieldsReader.java Fri Jul 18 02:20:12 2008
@@ -109,7 +109,7 @@
 
         // Verify the file is long enough to hold all of our
         // docs
-        assert ((int) (indexSize / 8)) >= size + this.docStoreOffset;
+        assert ((int) (indexSize / 8)) >= size + this.docStoreOffset: "indexSize=" + indexSize + " size=" + size + " docStoreOffset=" + docStoreOffset;
       } else {
         this.docStoreOffset = 0;
         this.size = (int) (indexSize >> 3);

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FieldsWriter.java Fri Jul 18 02:20:12 2008
@@ -55,7 +55,7 @@
 
     FieldsWriter(Directory d, String segment, FieldInfos fn) throws IOException {
         fieldInfos = fn;
-        
+
         boolean success = false;
         final String fieldsName = segment + "." + IndexFileNames.FIELDS_EXTENSION;
         try {
@@ -112,6 +112,10 @@
         doClose = false;
     }
 
+    void setFieldsStream(IndexOutput stream) {
+      this.fieldsStream = stream;
+    }
+
     // Writes the contents of buffer into the fields stream
     // and adds a new entry for this document into the index
     // stream.  This assumes the buffer was already written
@@ -122,6 +126,11 @@
       buffer.writeTo(fieldsStream);
     }
 
+    void skipDocument() throws IOException {
+      indexStream.writeLong(fieldsStream.getFilePointer());
+      fieldsStream.writeVInt(0);
+    }
+
     void flush() throws IOException {
       indexStream.flush();
       fieldsStream.flush();
@@ -129,10 +138,37 @@
 
     final void close() throws IOException {
       if (doClose) {
-        if (fieldsStream != null)
-          fieldsStream.close();
-        if (indexStream != null)
-          indexStream.close();
+
+        try {
+          if (fieldsStream != null) {
+            try {
+              fieldsStream.close();
+            } finally {
+              fieldsStream = null;
+            }
+          }
+        } catch (IOException ioe) {
+          try {
+            if (indexStream != null) {
+              try {
+                indexStream.close();
+              } finally {
+                indexStream = null;
+              }
+            }
+          } catch (IOException ioe2) {
+            // Ignore so we throw only first IOException hit
+          }
+          throw ioe;
+        } finally {
+          if (indexStream != null) {
+            try {
+              indexStream.close();
+            } finally {
+              indexStream = null;
+            }
+          }
+        }
       }
     }
 

Added: lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxFieldMergeState.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxFieldMergeState.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxFieldMergeState.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxFieldMergeState.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,97 @@
+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;
+
+// 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;
+  final CharBlockPool charPool;
+  final RawPostingList[] postings;
+
+  private FreqProxTermsWriter.PostingList p;
+  char[] text;
+  int textOffset;
+
+  private int postingUpto = -1;
+
+  final ByteSliceReader freq = new ByteSliceReader();
+  final ByteSliceReader prox = new ByteSliceReader();
+
+  int docID;
+  int termFreq;
+
+  public FreqProxFieldMergeState(FreqProxTermsWriterPerField field) {
+    this.field = field;
+    this.charPool = field.perThread.termsHashPerThread.charPool;
+    this.numPostings = field.termsHashPerField.numPostings;
+    this.postings = field.termsHashPerField.sortPostings();
+  }
+
+  boolean nextTerm() throws IOException {
+    postingUpto++;
+    if (postingUpto == numPostings)
+      return false;
+
+    p = (FreqProxTermsWriter.PostingList) postings[postingUpto];
+    docID = 0;
+
+    text = charPool.buffers[p.textStart >> DocumentsWriter.CHAR_BLOCK_SHIFT];
+    textOffset = p.textStart & DocumentsWriter.CHAR_BLOCK_MASK;
+
+    field.termsHashPerField.initReader(freq, p, 0);
+    field.termsHashPerField.initReader(prox, p, 1);
+
+    // Should always be true
+    boolean result = nextDoc();
+    assert result;
+
+    return true;
+  }
+
+  public boolean nextDoc() throws IOException {
+    if (freq.eof()) {
+      if (p.lastDocCode != -1) {
+        // Return last doc
+        docID = p.lastDocID;
+        termFreq = p.docFreq;
+        p.lastDocCode = -1;
+        return true;
+      } else 
+        // EOF
+        return false;
+    }
+
+    final int code = freq.readVInt();
+    docID += code >>> 1;
+    if ((code & 1) != 0)
+      termFreq = 1;
+    else
+      termFreq = freq.readVInt();
+
+    return true;
+  }
+}

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

Added: lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,366 @@
+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.store.IndexOutput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.UnicodeUtil;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Iterator;
+
+final class FreqProxTermsWriter extends TermsHashConsumer {
+
+  FreqProxTermsWriter() {
+    streamCount = 2;
+  }
+
+  public TermsHashConsumerPerThread addThread(TermsHashPerThread perThread) {
+    return new FreqProxTermsWriterPerThread(perThread);
+  }
+
+  void createPostings(RawPostingList[] postings, int start, int count) {
+    final int end = start + count;
+    for(int i=start;i<end;i++)
+      postings[i] = new PostingList();
+  }
+
+  private static int compareText(final char[] text1, int pos1, final char[] text2, int pos2) {
+    while(true) {
+      final char c1 = text1[pos1++];
+      final char c2 = text2[pos2++];
+      if (c1 != c2) {
+        if (0xffff == c2)
+          return 1;
+        else if (0xffff == c1)
+          return -1;
+        else
+          return c1-c2;
+      } else if (0xffff == c1)
+        return 0;
+    }
+  }
+
+  void closeDocStore(DocumentsWriter.FlushState state) {}
+  void abort() {}
+
+
+  // TODO: would be nice to factor out morme 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...
+
+  public void flush(Map threadsAndFields, final DocumentsWriter.FlushState state) throws IOException {
+
+    // Gather all FieldData's that have postings, across all
+    // ThreadStates
+    List allFields = new ArrayList();
+
+    Iterator it = threadsAndFields.entrySet().iterator();
+    while(it.hasNext()) {
+
+      Map.Entry entry = (Map.Entry) it.next();
+
+      Collection fields = (Collection) entry.getValue();
+
+      Iterator fieldsIt = fields.iterator();
+
+      while(fieldsIt.hasNext()) {
+        FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) fieldsIt.next();
+        if (perField.termsHashPerField.numPostings > 0)
+          allFields.add(perField);
+      }
+    }
+
+    // Sort by field name
+    Collections.sort(allFields);
+    final int numAllFields = allFields.size();
+
+    final TermInfosWriter termsOut = new TermInfosWriter(state.directory,
+                                                         state.segmentName,
+                                                         fieldInfos,
+                                                         state.docWriter.writer.getTermIndexInterval());
+
+    final IndexOutput freqOut = state.directory.createOutput(state.segmentFileName(IndexFileNames.FREQ_EXTENSION));
+    final IndexOutput proxOut = state.directory.createOutput(state.segmentFileName(IndexFileNames.PROX_EXTENSION));
+
+    final DefaultSkipListWriter skipListWriter = new DefaultSkipListWriter(termsOut.skipInterval,
+                                                                           termsOut.maxSkipLevels,
+                                                                           state.numDocsInRAM, freqOut, proxOut);
+
+    int start = 0;
+    while(start < numAllFields) {
+      final FieldInfo fieldInfo = ((FreqProxTermsWriterPerField) allFields.get(start)).fieldInfo;
+      final String fieldName = fieldInfo.name;
+
+      int end = start+1;
+      while(end < numAllFields && ((FreqProxTermsWriterPerField) allFields.get(end)).fieldInfo.name.equals(fieldName))
+        end++;
+      
+      FreqProxTermsWriterPerField[] fields = new FreqProxTermsWriterPerField[end-start];
+      for(int i=start;i<end;i++) {
+        fields[i-start] = (FreqProxTermsWriterPerField) allFields.get(i);
+
+        // 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(state, fields, termsOut, freqOut, proxOut, skipListWriter);
+
+      for(int i=0;i<fields.length;i++) {
+        TermsHashPerField perField = fields[i].termsHashPerField;
+        int numPostings = perField.numPostings;
+        perField.reset();
+        perField.shrinkHash(numPostings);
+      }
+
+      start = end;
+    }
+
+    it = threadsAndFields.entrySet().iterator();
+    while(it.hasNext()) {
+      Map.Entry entry = (Map.Entry) it.next();
+      FreqProxTermsWriterPerThread perThread = (FreqProxTermsWriterPerThread) entry.getKey();
+      perThread.termsHashPerThread.reset(true);
+    }
+
+    freqOut.close();
+    proxOut.close();
+    termsOut.close();
+    
+    // Record all files we have flushed
+    state.flushedFiles.add(state.segmentFileName(IndexFileNames.FIELD_INFOS_EXTENSION));
+    state.flushedFiles.add(state.segmentFileName(IndexFileNames.FREQ_EXTENSION));
+    state.flushedFiles.add(state.segmentFileName(IndexFileNames.PROX_EXTENSION));
+    state.flushedFiles.add(state.segmentFileName(IndexFileNames.TERMS_EXTENSION));
+    state.flushedFiles.add(state.segmentFileName(IndexFileNames.TERMS_INDEX_EXTENSION));
+  }
+
+  final byte[] copyByteBuffer = new byte[4096];
+
+  /** Copy numBytes from srcIn to destIn */
+  void copyBytes(IndexInput srcIn, IndexOutput destIn, long numBytes) throws IOException {
+    // TODO: we could do this more efficiently (save a copy)
+    // because it's always from a ByteSliceReader ->
+    // IndexOutput
+    while(numBytes > 0) {
+      final int chunk;
+      if (numBytes > 4096)
+        chunk = 4096;
+      else
+        chunk = (int) numBytes;
+      srcIn.readBytes(copyByteBuffer, 0, chunk);
+      destIn.writeBytes(copyByteBuffer, 0, chunk);
+      numBytes -= chunk;
+    }
+  }
+
+  /* Walk through all unique text tokens (Posting
+   * instances) found in this field and serialize them
+   * into a single RAM segment. */
+  void appendPostings(final DocumentsWriter.FlushState flushState,
+                      FreqProxTermsWriterPerField[] fields,
+                      TermInfosWriter termsOut,
+                      IndexOutput freqOut,
+                      IndexOutput proxOut,
+                      DefaultSkipListWriter skipListWriter)
+    throws CorruptIndexException, IOException {
+
+    final int fieldNumber = fields[0].fieldInfo.number;
+    int numFields = fields.length;
+
+    final FreqProxFieldMergeState[] mergeStates = new FreqProxFieldMergeState[numFields];
+
+    for(int i=0;i<numFields;i++) {
+      FreqProxFieldMergeState fms = mergeStates[i] = new FreqProxFieldMergeState(fields[i]);
+
+      assert fms.field.fieldInfo == fields[0].fieldInfo;
+
+      // Should always be true
+      boolean result = fms.nextTerm();
+      assert result;
+    }
+
+    final int skipInterval = termsOut.skipInterval;
+    final boolean currentFieldStorePayloads = fields[0].fieldInfo.storePayloads;
+
+    FreqProxFieldMergeState[] termStates = new FreqProxFieldMergeState[numFields];
+
+    while(numFields > 0) {
+
+      // Get the next term to merge
+      termStates[0] = mergeStates[0];
+      int numToMerge = 1;
+
+      for(int i=1;i<numFields;i++) {
+        final char[] text = mergeStates[i].text;
+        final int textOffset = mergeStates[i].textOffset;
+        final int cmp = compareText(text, textOffset, termStates[0].text, termStates[0].textOffset);
+
+        if (cmp < 0) {
+          termStates[0] = mergeStates[i];
+          numToMerge = 1;
+        } else if (cmp == 0)
+          termStates[numToMerge++] = mergeStates[i];
+      }
+
+      int df = 0;
+      int lastPayloadLength = -1;
+
+      int lastDoc = 0;
+
+      final char[] text = termStates[0].text;
+      final int start = termStates[0].textOffset;
+
+      long freqPointer = freqOut.getFilePointer();
+      long proxPointer = proxOut.getFilePointer();
+
+      skipListWriter.resetSkip();
+
+      // Now termStates has numToMerge FieldMergeStates
+      // which all share the same term.  Now we must
+      // interleave the docID streams.
+      while(numToMerge > 0) {
+        
+        if ((++df % skipInterval) == 0) {
+          skipListWriter.setSkipData(lastDoc, currentFieldStorePayloads, lastPayloadLength);
+          skipListWriter.bufferSkip(df);
+        }
+
+        FreqProxFieldMergeState minState = termStates[0];
+        for(int i=1;i<numToMerge;i++)
+          if (termStates[i].docID < minState.docID)
+            minState = termStates[i];
+
+        final int doc = minState.docID;
+        final int termDocFreq = minState.termFreq;
+
+        assert doc < flushState.numDocsInRAM;
+        assert doc > lastDoc || df == 1;
+
+        final int newDocCode = (doc-lastDoc)<<1;
+
+        lastDoc = doc;
+
+        final ByteSliceReader prox = minState.prox;
+
+        // Carefully copy over the prox + payload info,
+        // changing the format to match Lucene's segment
+        // format.
+        for(int j=0;j<termDocFreq;j++) {
+          final int code = prox.readVInt();
+          if (currentFieldStorePayloads) {
+            final int payloadLength;
+            if ((code & 1) != 0) {
+              // This position has a payload
+              payloadLength = prox.readVInt();
+            } else
+              payloadLength = 0;
+            if (payloadLength != lastPayloadLength) {
+              proxOut.writeVInt(code|1);
+              proxOut.writeVInt(payloadLength);
+              lastPayloadLength = payloadLength;
+            } else
+              proxOut.writeVInt(code & (~1));
+            if (payloadLength > 0)
+              copyBytes(prox, proxOut, payloadLength);
+          } else {
+            assert 0 == (code & 1);
+            proxOut.writeVInt(code>>1);
+          }
+        }
+
+        if (1 == termDocFreq) {
+          freqOut.writeVInt(newDocCode|1);
+        } else {
+          freqOut.writeVInt(newDocCode);
+          freqOut.writeVInt(termDocFreq);
+        }
+
+        if (!minState.nextDoc()) {
+
+          // Remove from termStates
+          int upto = 0;
+          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 df > 0;
+
+      // Done merging this term
+
+      long skipPointer = skipListWriter.writeSkip(freqOut);
+
+      // Write term
+      termInfo.set(df, freqPointer, proxPointer, (int) (skipPointer - freqPointer));
+
+      // TODO: we could do this incrementally
+      UnicodeUtil.UTF16toUTF8(text, start, termsUTF8);
+
+      // TODO: we could save O(n) re-scan of the term by
+      // computing the shared prefix with the last term
+      // while during the UTF8 encoding
+      termsOut.add(fieldNumber,
+                   termsUTF8.result,
+                   termsUTF8.length,
+                   termInfo);
+    }
+  }
+
+  private final TermInfo termInfo = new TermInfo(); // minimize consing
+
+  final UnicodeUtil.UTF8Result termsUTF8 = new UnicodeUtil.UTF8Result();
+
+  void files(Collection files) {}
+
+  static final class PostingList extends RawPostingList {
+    int docFreq;                                    // # times this term occurs in the current doc
+    int lastDocID;                                  // Last docID where this term occurred
+    int lastDocCode;                                // Code for prior doc
+    int lastPosition;                               // Last position where this term occurred
+  }
+
+  int bytesPerPosting() {
+    return RawPostingList.BYTES_SIZE + 4 * DocumentsWriter.INT_NUM_BYTE;
+  }
+}

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

Added: lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,117 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.analysis.Token;
+
+// 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 {
+
+  final FreqProxTermsWriterPerThread perThread;
+  final TermsHashPerField termsHashPerField;
+  final FieldInfo fieldInfo;
+  final DocumentsWriter.DocState docState;
+  final DocInverter.FieldInvertState fieldState;
+
+  public FreqProxTermsWriterPerField(TermsHashPerField termsHashPerField, FreqProxTermsWriterPerThread perThread, FieldInfo fieldInfo) {
+    this.termsHashPerField = termsHashPerField;
+    this.perThread = perThread;
+    this.fieldInfo = fieldInfo;
+    docState = termsHashPerField.docState;
+    fieldState = termsHashPerField.fieldState;
+  }
+
+  void finish() {}
+
+  //boolean doNext;
+  boolean hasPayloads;
+
+  void skippingLongTerm(Token t) throws IOException {}
+
+  public int compareTo(Object other0) {
+    FreqProxTermsWriterPerField other = (FreqProxTermsWriterPerField) other0;
+    return fieldInfo.name.compareTo(other.fieldInfo.name);
+  }
+
+  boolean start(Fieldable[] fields, int count) {
+    for(int i=0;i<count;i++)
+      if (fields[i].isIndexed())
+        return true;
+    return false;
+  }     
+
+  final void writeProx(Token t, FreqProxTermsWriter.PostingList p, int proxCode) {
+    final Payload payload = t.getPayload();    
+    if (payload != null && payload.length > 0) {
+      termsHashPerField.writeVInt(1, (proxCode<<1)|1);
+      termsHashPerField.writeVInt(1, payload.length);
+      termsHashPerField.writeBytes(1, payload.data, payload.offset, payload.length);
+      hasPayloads = true;      
+    } else
+      termsHashPerField.writeVInt(1, proxCode<<1);
+    p.lastPosition = fieldState.position;
+  }
+
+  final void newTerm(Token t, RawPostingList p0) {
+    // First time we're seeing this term since the last
+    // flush
+    assert docState.testPoint("FreqProxTermsWriterPerField.newTerm start");
+    FreqProxTermsWriter.PostingList p = (FreqProxTermsWriter.PostingList) p0;
+    p.lastDocCode = docState.docID << 1;
+    p.lastDocID = docState.docID;
+    p.docFreq = 1;
+    writeProx(t, p, fieldState.position);
+  }
+
+  final void addTerm(Token t, RawPostingList p0) {
+
+    assert docState.testPoint("FreqProxTermsWriterPerField.addTerm start");
+
+    FreqProxTermsWriter.PostingList p = (FreqProxTermsWriter.PostingList) p0;
+
+    assert p.docFreq > 0;
+
+    if (docState.docID != p.lastDocID) {
+      // Term not yet seen in the current doc but previously
+      // seen in other doc(s) since the last flush
+
+      // Now that we know doc freq for previous doc,
+      // write it & lastDocCode
+      if (1 == p.docFreq)
+        termsHashPerField.writeVInt(0, p.lastDocCode|1);
+      else {
+        termsHashPerField.writeVInt(0, p.lastDocCode);
+        termsHashPerField.writeVInt(0, p.docFreq);
+      }
+      p.docFreq = 1;
+      p.lastDocCode = (docState.docID - p.lastDocID) << 1;
+      p.lastDocID = docState.docID;
+      writeProx(t, p, fieldState.position);
+    } else {
+      p.docFreq++;
+      writeProx(t, p, fieldState.position-p.lastPosition);
+    }
+  }
+
+  public void abort() {}
+}
+

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

Added: lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/FreqProxTermsWriterPerThread.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,41 @@
+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;
+  }
+  
+  public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
+    return new FreqProxTermsWriterPerField(termsHashPerField, this, fieldInfo);
+  }
+
+  void startDocument() {
+  }
+
+  DocumentsWriter.DocWriter finishDocument() {
+    return null;
+  }
+
+  public void abort() {}
+}

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

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java Fri Jul 18 02:20:12 2008
@@ -403,9 +403,9 @@
 
       final List docWriterFiles;
       if (docWriter != null) {
-        docWriterFiles = docWriter.files();
+        docWriterFiles = docWriter.openFiles();
         if (docWriterFiles != null)
-          // We must incRef thes files before decRef'ing
+          // We must incRef these files before decRef'ing
           // last files to make sure we don't accidentally
           // delete them:
           incRef(docWriterFiles);
@@ -510,11 +510,13 @@
 
   /** Delets the specified files, but only if they are new
    *  (have not yet been incref'd). */
-  void deleteNewFiles(List files) throws IOException {
-    final int size = files.size();
-    for(int i=0;i<size;i++)
-      if (!refCounts.containsKey(files.get(i)))
-        deleteFile((String) files.get(i));
+  void deleteNewFiles(Collection files) throws IOException {
+    final Iterator it = files.iterator();
+    while(it.hasNext()) {
+      final String fileName = (String) it.next();
+      if (!refCounts.containsKey(fileName))
+        deleteFile(fileName);
+    }
   }
 
   void deleteFile(String fileName)

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java?rev=677865&r1=677864&r2=677865&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java Fri Jul 18 02:20:12 2008
@@ -426,6 +426,7 @@
   public void setSimilarity(Similarity similarity) {
     ensureOpen();
     this.similarity = similarity;
+    docWriter.setSimilarity(similarity);
   }
 
   /** Expert: Return the Similarity implementation used by this IndexWriter.
@@ -1130,6 +1131,7 @@
 
       docWriter = new DocumentsWriter(directory, this);
       docWriter.setInfoStream(infoStream);
+      docWriter.setMaxFieldLength(maxFieldLength);
 
       // Default deleter (for backwards compatibility) is
       // KeepOnlyLastCommitDeleter:
@@ -1267,6 +1269,7 @@
   public void setMaxFieldLength(int maxFieldLength) {
     ensureOpen();
     this.maxFieldLength = maxFieldLength;
+    docWriter.setMaxFieldLength(maxFieldLength);
     if (infoStream != null)
       message("setMaxFieldLength " + maxFieldLength);
   }
@@ -1715,62 +1718,61 @@
    */
   private synchronized boolean flushDocStores() throws IOException {
 
-    List files = docWriter.files();
-
     boolean useCompoundDocStore = false;
 
-    if (files.size() > 0) {
-      String docStoreSegment;
+    String docStoreSegment;
 
-      boolean success = false;
-      try {
-        docStoreSegment = docWriter.closeDocStore();
-        success = true;
-      } finally {
-        if (!success) {
-          if (infoStream != null)
-            message("hit exception closing doc store segment");
-          docWriter.abort(null);
-        }
+    boolean success = false;
+    try {
+      docStoreSegment = docWriter.closeDocStore();
+      success = true;
+    } finally {
+      if (!success) {
+        if (infoStream != null)
+          message("hit exception closing doc store segment");
       }
+    }
 
-      useCompoundDocStore = mergePolicy.useCompoundDocStore(segmentInfos);
+    useCompoundDocStore = mergePolicy.useCompoundDocStore(segmentInfos);
       
-      if (useCompoundDocStore && docStoreSegment != null) {
-        // Now build compound doc store file
+    if (useCompoundDocStore && docStoreSegment != null && docWriter.closedFiles().size() != 0) {
+      // Now build compound doc store file
 
-        success = false;
+      success = false;
 
-        final int numSegments = segmentInfos.size();
-        final String compoundFileName = docStoreSegment + "." + IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
+      final int numSegments = segmentInfos.size();
+      final String compoundFileName = docStoreSegment + "." + IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
 
-        try {
-          CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
-          final int size = files.size();
-          for(int i=0;i<size;i++)
-            cfsWriter.addFile((String) files.get(i));
+      try {
+        CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
+        final Iterator it = docWriter.closedFiles().iterator();
+        while(it.hasNext())
+          cfsWriter.addFile((String) it.next());
       
-          // Perform the merge
-          cfsWriter.close();
-          success = true;
-
-        } finally {
-          if (!success) {
-            if (infoStream != null)
-              message("hit exception building compound file doc store for segment " + docStoreSegment);
-            deleter.deleteFile(compoundFileName);
-          }
-        }
+        // Perform the merge
+        cfsWriter.close();
+        success = true;
 
-        for(int i=0;i<numSegments;i++) {
-          SegmentInfo si = segmentInfos.info(i);
-          if (si.getDocStoreOffset() != -1 &&
-              si.getDocStoreSegment().equals(docStoreSegment))
-            si.setDocStoreIsCompoundFile(true);
+      } finally {
+        if (!success) {
+          if (infoStream != null)
+            message("hit exception building compound file doc store for segment " + docStoreSegment);
+          deleter.deleteFile(compoundFileName);
         }
+      }
 
-        checkpoint();
+      for(int i=0;i<numSegments;i++) {
+        SegmentInfo si = segmentInfos.info(i);
+        if (si.getDocStoreOffset() != -1 &&
+            si.getDocStoreSegment().equals(docStoreSegment))
+          si.setDocStoreIsCompoundFile(true);
       }
+
+      checkpoint();
+
+      // In case the files we just merged into a CFS were
+      // not previously checkpointed:
+      deleter.deleteNewFiles(docWriter.closedFiles());
     }
 
     return useCompoundDocStore;
@@ -1947,7 +1949,7 @@
             // If docWriter has some aborted files that were
             // never incref'd, then we clean them up here
             if (docWriter != null) {
-              final List files = docWriter.abortedFiles();
+              final Collection files = docWriter.abortedFiles();
               if (files != null)
                 deleter.deleteNewFiles(files);
             }
@@ -2076,7 +2078,7 @@
           synchronized (this) {
             // If docWriter has some aborted files that were
             // never incref'd, then we clean them up here
-            final List files = docWriter.abortedFiles();
+            final Collection files = docWriter.abortedFiles();
             if (files != null)
               deleter.deleteNewFiles(files);
           }
@@ -2650,8 +2652,8 @@
         // once").
         segmentInfos.clear();
         segmentInfos.addAll(rollbackSegmentInfos);
-
-        docWriter.abort(null);
+        
+        docWriter.abort();
 
         // Ask deleter to locate unreferenced files & remove
         // them:
@@ -3338,7 +3340,6 @@
           if (!success) {
             if (infoStream != null)
               message("hit exception flushing segment " + segment);
-            docWriter.abort(null);
             deleter.refresh(segment);
           }
         }
@@ -3830,8 +3831,9 @@
 
       // If the segment is referencing the current "live"
       // doc store outputs then we must merge
-      if (si.getDocStoreOffset() != -1 && currentDocStoreSegment != null && si.getDocStoreSegment().equals(currentDocStoreSegment))
+      if (si.getDocStoreOffset() != -1 && currentDocStoreSegment != null && si.getDocStoreSegment().equals(currentDocStoreSegment)) {
         doFlushDocStore = true;
+      }
     }
 
     final int docStoreOffset;
@@ -3859,8 +3861,9 @@
       // newly flushed doc store files then we should not
       // make compound file out of them...
       if (infoStream != null)
-        message("flush at merge");
-      flush(false, true, false);
+        message("now flush at merge");
+      doFlush(true, false);
+      //flush(false, true, false);
     }
 
     // We must take a full copy at this point so that we can

Added: lucene/java/trunk/src/java/org/apache/lucene/index/IntBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IntBlockPool.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IntBlockPool.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IntBlockPool.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,65 @@
+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 IntBlockPool {
+
+  public int[][] buffers = new int[10][];
+
+  int bufferUpto = -1;                        // Which buffer we are upto
+  public int intUpto = DocumentsWriter.INT_BLOCK_SIZE;             // Where we are in head buffer
+
+  public int[] buffer;                              // Current head buffer
+  public int intOffset = -DocumentsWriter.INT_BLOCK_SIZE;          // Current head offset
+
+  final private DocumentsWriter docWriter;
+  final boolean trackAllocations;
+
+  public IntBlockPool(DocumentsWriter docWriter, boolean trackAllocations) {
+    this.docWriter = docWriter;
+    this.trackAllocations = trackAllocations;
+  }
+
+  public void reset() {
+    if (bufferUpto != -1) {
+      if (bufferUpto > 0)
+        // Recycle all but the first buffer
+        docWriter.recycleIntBlocks(buffers, 1, 1+bufferUpto);
+
+      // Reuse first buffer
+      bufferUpto = 0;
+      intUpto = 0;
+      intOffset = 0;
+      buffer = buffers[0];
+    }
+  }
+
+  public void nextBuffer() {
+    if (1+bufferUpto == buffers.length) {
+      int[][] newBuffers = new int[(int) (buffers.length*1.5)][];
+      System.arraycopy(buffers, 0, newBuffers, 0, buffers.length);
+      buffers = newBuffers;
+    }
+    buffer = buffers[1+bufferUpto] = docWriter.getIntBlock(trackAllocations);
+    bufferUpto++;
+
+    intUpto = 0;
+    intOffset += DocumentsWriter.INT_BLOCK_SIZE;
+  }
+}
+

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

Added: lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumer.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumer.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumer.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,46 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Map;
+import java.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 threadsAndFields, DocumentsWriter.FlushState state) throws IOException;
+
+  /** Close doc stores */
+  abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
+
+  /** Attempt to free RAM, returning true if any RAM was
+   *  freed */
+  abstract boolean freeRAM();
+
+  FieldInfos fieldInfos;
+
+  void setFieldInfos(FieldInfos fieldInfos) {
+    this.fieldInfos = fieldInfos;
+  }
+}

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

Added: lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,41 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.analysis.Token;
+import java.io.IOException;
+
+abstract class InvertedDocConsumerPerField {
+
+  // Called once per field, and is given all Fieldable
+  // occurrences for this field in the document.  Return
+  // true if you wish to see inverted tokens for these
+  // fields:
+  abstract boolean start(Fieldable[] fields, int count) throws IOException;
+
+  // Called once per inverted token
+  abstract void add(Token token) throws IOException;
+
+  // Called once per field per document, after all Fieldable
+  // occurrences are inverted
+  abstract void finish() throws IOException;
+
+  // Called on hitting an aborting exception
+  abstract void abort();
+}

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

Added: lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,27 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+abstract class InvertedDocConsumerPerThread {
+  abstract void startDocument() throws IOException;
+  abstract InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
+  abstract DocumentsWriter.DocWriter finishDocument() throws IOException;
+  abstract void abort();
+}

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

Added: lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,29 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Map;
+import java.io.IOException;
+
+abstract class InvertedDocEndConsumer {
+  abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
+  abstract void flush(Map threadsAndFields, DocumentsWriter.FlushState state) throws IOException;
+  abstract void closeDocStore(DocumentsWriter.FlushState state) throws IOException;
+  abstract void abort();
+  abstract void setFieldInfos(FieldInfos fieldInfos);
+}

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

Added: lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerField.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,23 @@
+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 InvertedDocEndConsumerPerField {
+  abstract void finish();
+  abstract void abort();
+}

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

Added: lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java?rev=677865&view=auto
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java Fri Jul 18 02:20:12 2008
@@ -0,0 +1,25 @@
+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();
+}

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



Mime
View raw message