You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2008/01/25 12:32:35 UTC

svn commit: r615183 - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/test/org/apache/lucene/index/

Author: mikemccand
Date: Fri Jan 25 03:32:32 2008
New Revision: 615183

URL: http://svn.apache.org/viewvc?rev=615183&view=rev
Log:
LUCENE-1120: speed up merging of term vectors by bulk-copying the raw bytes

Added:
    lucene/java/trunk/src/test/org/apache/lucene/index/index.19.cfs.zip   (with props)
    lucene/java/trunk/src/test/org/apache/lucene/index/index.19.nocfs.zip   (with props)
    lucene/java/trunk/src/test/org/apache/lucene/index/index.20.cfs.zip   (with props)
    lucene/java/trunk/src/test/org/apache/lucene/index/index.20.nocfs.zip   (with props)
    lucene/java/trunk/src/test/org/apache/lucene/index/index.21.cfs.zip   (with props)
    lucene/java/trunk/src/test/org/apache/lucene/index/index.21.nocfs.zip   (with props)
    lucene/java/trunk/src/test/org/apache/lucene/index/index.22.cfs.zip   (with props)
    lucene/java/trunk/src/test/org/apache/lucene/index/index.22.nocfs.zip   (with props)
Removed:
    lucene/java/trunk/src/test/org/apache/lucene/index/index.prelockless.cfs.zip
    lucene/java/trunk/src/test/org/apache/lucene/index/index.prelockless.nocfs.zip
    lucene/java/trunk/src/test/org/apache/lucene/index/index.presharedstores.cfs.zip
    lucene/java/trunk/src/test/org/apache/lucene/index/index.presharedstores.nocfs.zip
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsWriter.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=615183&r1=615182&r2=615183&view=diff
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Fri Jan 25 03:32:32 2008
@@ -1,4 +1,4 @@
-Lucene Change Log
+Lucene Change Log
 $Id$
 
 ======================= Trunk (not yet released) =======================
@@ -31,6 +31,10 @@
     in how the CFS file is stored, and allows us to detect an upcoming
     disk full situation before actually filling up the disk.  (Mike
     McCandless)
+
+ 2. LUCENE-1120: Speed up merging of term vectors by bulk-copying the
+    raw bytes for each contiguous range of non-deleted documents.
+    (Mike McCandless)
 
 Documentation
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=615183&r1=615182&r2=615183&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DocumentsWriter.java Fri Jan 25 03:32:32 2008
@@ -662,12 +662,12 @@
         // Append term vectors to the real outputs:
         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];
-            tvd.writeVLong(tvf.getFilePointer());
             long lastPos = vectorFieldPointers[0];
             for(int i=1;i<numVectorFields;i++) {
               long pos = vectorFieldPointers[i];
@@ -870,22 +870,23 @@
           // state:
           try {
             tvx = directory.createOutput(docStoreSegment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
-            tvx.writeInt(TermVectorsReader.FORMAT_VERSION);
+            tvx.writeInt(TermVectorsReader.FORMAT_VERSION2);
             tvd = directory.createOutput(docStoreSegment +  "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
-            tvd.writeInt(TermVectorsReader.FORMAT_VERSION);
+            tvd.writeInt(TermVectorsReader.FORMAT_VERSION2);
             tvf = directory.createOutput(docStoreSegment +  "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
-            tvf.writeInt(TermVectorsReader.FORMAT_VERSION);
+            tvf.writeInt(TermVectorsReader.FORMAT_VERSION2);
 
             // We must "catch up" for all docIDs that had no
             // vectors before this one
-            for(int i=0;i<docID;i++)
+            for(int i=0;i<docID;i++) {
               tvx.writeLong(0);
+              tvx.writeLong(0);
+            }
           } catch (Throwable t) {
             throw new AbortException(t, DocumentsWriter.this);
           }
           files = null;
         }
-
         numVectorFields = 0;
       }
     }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java?rev=615183&r1=615182&r2=615183&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java Fri Jan 25 03:32:32 2008
@@ -205,6 +205,38 @@
     }
   }
 
+  private SegmentReader[] matchingSegmentReaders;
+  private int[] rawDocLengths;
+  private int[] rawDocLengths2;
+
+  private void setMatchingSegmentReaders() {
+    // If the i'th reader is a SegmentReader and has
+    // identical fieldName -> number mapping, then this
+    // array will be non-null at position i:
+    matchingSegmentReaders = new SegmentReader[readers.size()];
+
+    // If this reader is a SegmentReader, and all of its
+    // field name -> number mappings match the "merged"
+    // FieldInfos, then we can do a bulk copy of the
+    // stored fields:
+    for (int i = 0; i < readers.size(); i++) {
+      IndexReader reader = (IndexReader) readers.elementAt(i);
+      if (reader instanceof SegmentReader) {
+        SegmentReader segmentReader = (SegmentReader) reader;
+        boolean same = true;
+        FieldInfos segmentFieldInfos = segmentReader.getFieldInfos();
+        for (int j = 0; same && j < segmentFieldInfos.size(); j++)
+          same = fieldInfos.fieldName(j).equals(segmentFieldInfos.fieldName(j));
+        if (same)
+          matchingSegmentReaders[i] = segmentReader;
+      }
+    }
+
+    // Used for bulk-reading raw bytes for stored fields
+    rawDocLengths = new int[MAX_RAW_MERGE_DOCS];
+    rawDocLengths2 = new int[MAX_RAW_MERGE_DOCS];
+  }
+
   /**
    * 
    * @return The number of documents in all of the readers
@@ -248,33 +280,9 @@
 
     int docCount = 0;
 
-    if (mergeDocStores) {
+    setMatchingSegmentReaders();
 
-      // If the i'th reader is a SegmentReader and has
-      // identical fieldName -> number mapping, then this
-      // array will be non-null at position i:
-      SegmentReader[] matchingSegmentReaders = new SegmentReader[readers.size()];
-
-      // If this reader is a SegmentReader, and all of its
-      // field name -> number mappings match the "merged"
-      // FieldInfos, then we can do a bulk copy of the
-      // stored fields:
-      for (int i = 0; i < readers.size(); i++) {
-        IndexReader reader = (IndexReader) readers.elementAt(i);
-        if (reader instanceof SegmentReader) {
-          SegmentReader segmentReader = (SegmentReader) reader;
-          boolean same = true;
-          FieldInfos segmentFieldInfos = segmentReader.getFieldInfos();
-          for (int j = 0; same && j < segmentFieldInfos.size(); j++)
-            same = fieldInfos.fieldName(j).equals(segmentFieldInfos.fieldName(j));
-          if (same) {
-            matchingSegmentReaders[i] = segmentReader;
-          }
-        }
-      }
-	
-      // Used for bulk-reading raw bytes for stored fields
-      final int[] rawDocLengths = new int[MAX_RAW_MERGE_DOCS];
+    if (mergeDocStores) {
 
       // for merging we don't want to compress/uncompress the data, so to tell the FieldsReader that we're
       // in  merge mode, we use this FieldSelector
@@ -350,15 +358,45 @@
 
     try {
       for (int r = 0; r < readers.size(); r++) {
+        final SegmentReader matchingSegmentReader = matchingSegmentReaders[r];
+        TermVectorsReader matchingVectorsReader;
+        if (matchingSegmentReader != null) {
+          matchingVectorsReader = matchingSegmentReader.termVectorsReaderOrig;
+
+          // If the TV* files are an older format then they
+          // cannot read raw docs:
+          if (matchingVectorsReader != null && !matchingVectorsReader.canReadRawDocs())
+            matchingVectorsReader = null;
+        } else
+          matchingVectorsReader = null;
         IndexReader reader = (IndexReader) readers.elementAt(r);
         int maxDoc = reader.maxDoc();
-        for (int docNum = 0; docNum < maxDoc; docNum++) {
+        for (int docNum = 0; docNum < maxDoc;) {
           // skip deleted docs
-          if (reader.isDeleted(docNum)) 
-            continue;
-          termVectorsWriter.addAllDocVectors(reader.getTermFreqVectors(docNum));
-          if (checkAbort != null)
-            checkAbort.work(300);
+          if (!reader.isDeleted(docNum)) {
+            if (matchingVectorsReader != null) {
+              // We can optimize this case (doing a bulk
+              // byte copy) since the field numbers are
+              // identical
+              int start = docNum;
+              int numDocs = 0;
+              do {
+                docNum++;
+                numDocs++;
+              } while(docNum < maxDoc && !matchingSegmentReader.isDeleted(docNum) && numDocs < MAX_RAW_MERGE_DOCS);
+
+              matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, start, numDocs);
+              termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, numDocs);
+              if (checkAbort != null)
+                checkAbort.work(300*numDocs);
+            } else {
+              termVectorsWriter.addAllDocVectors(reader.getTermFreqVectors(docNum));
+              docNum++;
+              if (checkAbort != null)
+                checkAbort.work(300);
+            }
+          } else
+            docNum++;
         }
       }
     } finally {

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsReader.java?rev=615183&r1=615182&r2=615183&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsReader.java Fri Jan 25 03:32:32 2008
@@ -22,13 +22,18 @@
 import org.apache.lucene.store.IndexInput;
 
 import java.io.IOException;
+import java.util.Arrays;
 
 /**
  * @version $Id$
  */
 class TermVectorsReader implements Cloneable {
 
+  // NOTE: if you make a new format, it must be larger than
+  // the current format
   static final int FORMAT_VERSION = 2;
+  static final int FORMAT_VERSION2 = 3;
+
   //The size in bytes that the FORMAT_VERSION will take up at the beginning of each file 
   static final int FORMAT_SIZE = 4;
 
@@ -41,13 +46,13 @@
   private IndexInput tvd;
   private IndexInput tvf;
   private int size;
+  private int numTotalDocs;
 
   // The docID offset where our docs begin in the index
   // file.  This will be 0 if we have our own private file.
   private int docStoreOffset;
   
-  private int tvdFormat;
-  private int tvfFormat;
+  private final int format;
 
   TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos)
     throws CorruptIndexException, IOException {
@@ -56,7 +61,7 @@
 
   TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, int readBufferSize)
     throws CorruptIndexException, IOException {
-    this(d, segment, fieldInfos, BufferedIndexInput.BUFFER_SIZE, -1, 0);
+    this(d, segment, fieldInfos, readBufferSize, -1, 0);
   }
     
   TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, int readBufferSize, int docStoreOffset, int size)
@@ -66,22 +71,35 @@
     try {
       if (d.fileExists(segment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION)) {
         tvx = d.openInput(segment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION, readBufferSize);
-        checkValidFormat(tvx);
+        format = checkValidFormat(tvx);
         tvd = d.openInput(segment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION, readBufferSize);
-        tvdFormat = checkValidFormat(tvd);
+        final int tvdFormat = checkValidFormat(tvd);
         tvf = d.openInput(segment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION, readBufferSize);
-        tvfFormat = checkValidFormat(tvf);
+        final int tvfFormat = checkValidFormat(tvf);
+
+        assert format == tvdFormat;
+        assert format == tvfFormat;
+
+        if (format >= FORMAT_VERSION2) {
+          assert (tvx.length()-FORMAT_SIZE) % 16 == 0;
+          numTotalDocs = (int) (tvx.length() >> 4);
+        } else {
+          assert (tvx.length()-FORMAT_SIZE) % 8 == 0;
+          numTotalDocs = (int) (tvx.length() >> 3);
+        }
+
         if (-1 == docStoreOffset) {
           this.docStoreOffset = 0;
-          this.size = (int) (tvx.length() >> 3);
+          this.size = numTotalDocs;
         } else {
           this.docStoreOffset = docStoreOffset;
           this.size = size;
           // Verify the file is long enough to hold all of our
           // docs
-          assert ((int) (tvx.length() / 8)) >= size + docStoreOffset;
+          assert numTotalDocs >= size + docStoreOffset;
         }
-      }
+      } else
+        format = 0;
 
       this.fieldInfos = fieldInfos;
       success = true;
@@ -96,26 +114,94 @@
       }
     }
   }
-  
+
+  // Used for bulk copy when merging
+  IndexInput getTvdStream() {
+    return tvd;
+  }
+
+  // Used for bulk copy when merging
+  IndexInput getTvfStream() {
+    return tvf;
+  }
+
+  final private void seekTvx(final int docNum) throws IOException {
+    if (format < FORMAT_VERSION2)
+      tvx.seek((docNum + docStoreOffset) * 8L + FORMAT_SIZE);
+    else
+      tvx.seek((docNum + docStoreOffset) * 16L + FORMAT_SIZE);
+  }
+
+  boolean canReadRawDocs() {
+    return format >= FORMAT_VERSION2;
+  }
+
+  /** Retrieve the length (in bytes) of the tvd and tvf
+   *  entries for the next numDocs starting with
+   *  startDocID.  This is used for bulk copying when
+   *  merging segments, if the field numbers are
+   *  congruent.  Once this returns, the tvf & tvd streams
+   *  are seeked to the startDocID. */
+  final void rawDocs(int[] tvdLengths, int[] tvfLengths, int startDocID, int numDocs) throws IOException {
+
+    if (tvx == null) {
+      Arrays.fill(tvdLengths, 0);
+      Arrays.fill(tvfLengths, 0);
+      return;
+    }
+
+    // SegmentMerger calls canReadRawDocs() first and should
+    // not call us if that returns false.
+    if (format < FORMAT_VERSION2)
+      throw new IllegalStateException("cannot read raw docs with older term vector formats");
+
+    seekTvx(startDocID);
+
+    long tvdPosition = tvx.readLong();
+    tvd.seek(tvdPosition);
+
+    long tvfPosition = tvx.readLong();
+    tvf.seek(tvfPosition);
+
+    long lastTvdPosition = tvdPosition;
+    long lastTvfPosition = tvfPosition;
+
+    int count = 0;
+    while (count < numDocs) {
+      final int docID = startDocID + count + 1;
+      if (docID < numTotalDocs)  {
+        tvdPosition = tvx.readLong();
+        tvfPosition = tvx.readLong();
+      } else {
+        tvdPosition = tvd.length();
+        tvfPosition = tvf.length();
+      }
+      tvdLengths[count] = (int) (tvdPosition-lastTvdPosition);
+      tvfLengths[count] = (int) (tvfPosition-lastTvfPosition);
+      count++;
+      lastTvdPosition = tvdPosition;
+      lastTvfPosition = tvfPosition;
+    }
+  }
+
   private int checkValidFormat(IndexInput in) throws CorruptIndexException, IOException
   {
     int format = in.readInt();
-    if (format > FORMAT_VERSION)
-    {
+    if (format > FORMAT_VERSION2) {
       throw new CorruptIndexException("Incompatible format version: " + format + " expected " 
-                                      + FORMAT_VERSION + " or less");
+                                      + FORMAT_VERSION2 + " or less");
     }
     return format;
   }
 
   void close() throws IOException {
-  	// make all effort to close up. Keep the first exception
-  	// and throw it as a new one.
-  	IOException keep = null;
-  	if (tvx != null) try { tvx.close(); } catch (IOException e) { if (keep == null) 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();
+    // make all effort to close up. Keep the first exception
+    // and throw it as a new one.
+    IOException keep = null;
+    if (tvx != null) try { tvx.close(); } catch (IOException e) { if (keep == null) 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();
   }
 
   /**
@@ -133,11 +219,11 @@
       //We don't need to do this in other seeks because we already have the
       // file pointer
       //that was written in another file
-      tvx.seek(((docNum + docStoreOffset) * 8L) + FORMAT_SIZE);
+      seekTvx(docNum);
       //System.out.println("TVX Pointer: " + tvx.getFilePointer());
-      long position = tvx.readLong();
+      long tvdPosition = tvx.readLong();
 
-      tvd.seek(position);
+      tvd.seek(tvdPosition);
       int fieldCount = tvd.readVInt();
       //System.out.println("Num Fields: " + fieldCount);
       // There are only a few fields per document. We opt for a full scan
@@ -146,7 +232,7 @@
       int number = 0;
       int found = -1;
       for (int i = 0; i < fieldCount; i++) {
-        if(tvdFormat == FORMAT_VERSION)
+        if (format >= FORMAT_VERSION)
           number = tvd.readVInt();
         else
           number += tvd.readVInt();
@@ -159,8 +245,12 @@
       // document
       if (found != -1) {
         // Compute position in the tvf file
-        position = 0;
-        for (int i = 0; i <= found; i++)
+        long position;
+        if (format >= FORMAT_VERSION2)
+          position = tvx.readLong();
+        else
+          position = tvd.readVLong();
+        for (int i = 1; i <= found; i++)
           position += tvd.readVLong();
 
         mapper.setDocumentNumber(docNum);
@@ -190,6 +280,45 @@
     return mapper.materializeVector();
   }
 
+  // Reads the String[] fields; you have to pre-seek tvd to
+  // the right point
+  final private String[] readFields(int fieldCount) throws IOException {
+    int number = 0;
+    String[] fields = new String[fieldCount];
+
+    for (int i = 0; i < fieldCount; i++) {
+      if (format >= FORMAT_VERSION)
+        number = tvd.readVInt();
+      else
+        number += tvd.readVInt();
+
+      fields[i] = fieldInfos.fieldName(number);
+    }
+
+    return fields;
+  }
+
+  // Reads the long[] offsets into TVF; you have to pre-seek
+  // tvx/tvd to the right point
+  final private long[] readTvfPointers(int fieldCount) throws IOException {
+    // Compute position in the tvf file
+    long position;
+    if (format >= FORMAT_VERSION2)
+      position = tvx.readLong();
+    else
+      position = tvd.readVLong();
+
+    long[] tvfPointers = new long[fieldCount];
+    tvfPointers[0] = position;
+
+    for (int i = 1; i < fieldCount; i++) {
+      position += tvd.readVLong();
+      tvfPointers[i] = position;
+    }
+
+    return tvfPointers;
+  }
+
   /**
    * Return all term vectors stored for this document or null if the could not be read in.
    * 
@@ -201,34 +330,16 @@
     TermFreqVector[] result = null;
     if (tvx != null) {
       //We need to offset by
-      tvx.seek(((docNum + docStoreOffset) * 8L) + FORMAT_SIZE);
-      long position = tvx.readLong();
+      seekTvx(docNum);
+      long tvdPosition = tvx.readLong();
 
-      tvd.seek(position);
+      tvd.seek(tvdPosition);
       int fieldCount = tvd.readVInt();
 
       // No fields are vectorized for this document
       if (fieldCount != 0) {
-        int number = 0;
-        String[] fields = new String[fieldCount];
-
-        for (int i = 0; i < fieldCount; i++) {
-          if(tvdFormat == FORMAT_VERSION)
-            number = tvd.readVInt();
-          else
-            number += tvd.readVInt();
-
-          fields[i] = fieldInfos.fieldName(number);
-        }
-
-        // Compute position in the tvf file
-        position = 0;
-        long[] tvfPointers = new long[fieldCount];
-        for (int i = 0; i < fieldCount; i++) {
-          position += tvd.readVLong();
-          tvfPointers[i] = position;
-        }
-
+        final String[] fields = readFields(fieldCount);
+        final long[] tvfPointers = readTvfPointers(fieldCount);
         result = readTermVectors(docNum, fields, tvfPointers);
       }
     } else {
@@ -241,34 +352,17 @@
     // Check if no term vectors are available for this segment at all
     if (tvx != null) {
       //We need to offset by
-      tvx.seek((docNumber * 8L) + FORMAT_SIZE);
-      long position = tvx.readLong();
 
-      tvd.seek(position);
+      seekTvx(docNumber);
+      long tvdPosition = tvx.readLong();
+
+      tvd.seek(tvdPosition);
       int fieldCount = tvd.readVInt();
 
       // No fields are vectorized for this document
       if (fieldCount != 0) {
-        int number = 0;
-        String[] fields = new String[fieldCount];
-
-        for (int i = 0; i < fieldCount; i++) {
-          if(tvdFormat == FORMAT_VERSION)
-            number = tvd.readVInt();
-          else
-            number += tvd.readVInt();
-
-          fields[i] = fieldInfos.fieldName(number);
-        }
-
-        // Compute position in the tvf file
-        position = 0;
-        long[] tvfPointers = new long[fieldCount];
-        for (int i = 0; i < fieldCount; i++) {
-          position += tvd.readVLong();
-          tvfPointers[i] = position;
-        }
-
+        final String[] fields = readFields(fieldCount);
+        final long[] tvfPointers = readTvfPointers(fieldCount);
         mapper.setDocumentNumber(docNumber);
         readTermVectors(fields, tvfPointers, mapper);
       }
@@ -293,9 +387,8 @@
   private void readTermVectors(String fields[], long tvfPointers[], TermVectorMapper mapper)
           throws IOException {
     for (int i = 0; i < fields.length; i++) {
-       readTermVector(fields[i], tvfPointers[i], mapper);
+      readTermVector(fields[i], tvfPointers[i], mapper);
     }
-
   }
 
 
@@ -324,7 +417,7 @@
     boolean storePositions;
     boolean storeOffsets;
     
-    if(tvfFormat == FORMAT_VERSION){
+    if (format >= FORMAT_VERSION){
       byte bits = tvf.readByte();
       storePositions = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
       storeOffsets = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
@@ -400,8 +493,6 @@
     }
   }
 
-
-
   protected Object clone() {
     
     if (tvx == null || tvd == null || tvf == null)
@@ -418,10 +509,8 @@
     
     return clone;
   }
-
-
-
 }
+
 
 /**
  * Models the existing parallel array structure

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsWriter.java?rev=615183&r1=615182&r2=615183&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermVectorsWriter.java Fri Jan 25 03:32:32 2008
@@ -33,11 +33,11 @@
     throws IOException {
     // Open files for TermVector storage
     tvx = directory.createOutput(segment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
-    tvx.writeInt(TermVectorsReader.FORMAT_VERSION);
+    tvx.writeInt(TermVectorsReader.FORMAT_VERSION2);
     tvd = directory.createOutput(segment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
-    tvd.writeInt(TermVectorsReader.FORMAT_VERSION);
+    tvd.writeInt(TermVectorsReader.FORMAT_VERSION2);
     tvf = directory.createOutput(segment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
-    tvf.writeInt(TermVectorsReader.FORMAT_VERSION);
+    tvf.writeInt(TermVectorsReader.FORMAT_VERSION2);
 
     this.fieldInfos = fieldInfos;
   }
@@ -53,6 +53,7 @@
       throws IOException {
 
     tvx.writeLong(tvd.getFilePointer());
+    tvx.writeLong(tvf.getFilePointer());
 
     if (vectors != null) {
       final int numFields = vectors.length;
@@ -145,14 +146,36 @@
       }
 
       // 2nd pass: write field pointers to tvd
-      long lastFieldPointer = 0;
-      for (int i=0; i<numFields; i++) {
+      long lastFieldPointer = fieldPointers[0];
+      for (int i=1; i<numFields; i++) {
         final long fieldPointer = fieldPointers[i];
         tvd.writeVLong(fieldPointer-lastFieldPointer);
         lastFieldPointer = fieldPointer;
       }
     } else
       tvd.writeVInt(0);
+  }
+
+  /**
+   * Do a bulk copy of numDocs documents from reader to our
+   * streams.  This is used to expedite merging, if the
+   * field numbers are congruent.
+   */
+  final void addRawDocuments(TermVectorsReader reader, int[] tvdLengths, int[] tvfLengths, int numDocs) throws IOException {
+    long tvdPosition = tvd.getFilePointer();
+    long tvfPosition = tvf.getFilePointer();
+    long tvdStart = tvdPosition;
+    long tvfStart = tvfPosition;
+    for(int i=0;i<numDocs;i++) {
+      tvx.writeLong(tvdPosition);
+      tvdPosition += tvdLengths[i];
+      tvx.writeLong(tvfPosition);
+      tvfPosition += tvfLengths[i];
+    }
+    tvd.copyBytes(reader.getTvdStream(), tvdPosition-tvdStart);
+    tvf.copyBytes(reader.getTvfStream(), tvfPosition-tvfStart);
+    assert tvd.getFilePointer() == tvdPosition;
+    assert tvf.getFilePointer() == tvfPosition;
   }
   
   /** Close all streams. */

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=615183&r1=615182&r2=615183&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Fri Jan 25 03:32:32 2008
@@ -18,10 +18,16 @@
  */
 
 import org.apache.lucene.util.LuceneTestCase;
-import java.util.Vector;
+
 import java.util.Arrays;
-import java.io.ByteArrayOutputStream;
-import java.io.ObjectOutputStream;
+import java.util.Enumeration;
+import java.util.zip.ZipFile;
+import java.util.zip.ZipEntry;
+
+import java.io.OutputStream;
+import java.io.InputStream;
+import java.io.FileOutputStream;
+import java.io.BufferedOutputStream;
 import java.io.IOException;
 import java.io.File;
 
@@ -33,9 +39,6 @@
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import java.io.*;
-import java.util.*;
-import java.util.zip.*;
 
 /*
   Verify we can read the pre-2.1 file format, do searches
@@ -50,11 +53,11 @@
 
   /*
   public void testCreatePreLocklessCFS() throws IOException {
-    createIndex("src/test/org/apache/lucene/index/index.prelockless.cfs", true);
+    createIndex("src/test/org/apache/lucene/index/index.cfs", true);
   }
 
   public void testCreatePreLocklessNoCFS() throws IOException {
-    createIndex("src/test/org/apache/lucene/index/index.prelockless.nocfs", false);
+    createIndex("src/test/org/apache/lucene/index/index.nocfs", false);
   }
   */
 
@@ -106,10 +109,14 @@
     rmDir(dirName);
   }
 
-  final String[] oldNames = {"prelockless.cfs",
-                             "prelockless.nocfs",
-                             "presharedstores.cfs",
-                             "presharedstores.nocfs"};
+  final String[] oldNames = {"19.cfs",
+                             "19.nocfs",
+                             "20.cfs",
+                             "20.nocfs",
+                             "21.cfs",
+                             "21.nocfs",
+                             "22.cfs",
+                             "22.nocfs"};
 
   public void testSearchOldIndex() throws IOException {
     for(int i=0;i<oldNames.length;i++) {
@@ -146,6 +153,15 @@
     }
   }
 
+  private void testHits(Hits hits, int expectedCount, IndexReader reader) throws IOException {
+    final int hitCount = hits.length();
+    assertEquals("wrong number of hits", expectedCount, hitCount);
+    for(int i=0;i<hitCount;i++) {
+      hits.doc(i);
+      reader.getTermFreqVectors(hits.id(i));
+    }
+  }
+
   public void searchIndex(String dirName) throws IOException {
     //QueryParser parser = new QueryParser("contents", new WhitespaceAnalyzer());
     //Query query = parser.parse("handle:1");
@@ -156,12 +172,14 @@
     IndexSearcher searcher = new IndexSearcher(dir);
     
     Hits hits = searcher.search(new TermQuery(new Term("content", "aaa")));
-    assertEquals(34, hits.length());
-    Document d = hits.doc(0);
 
-    // First document should be #21 since it's norm was increased:
+    // First document should be #21 since it's norm was
+    // increased:
+    Document d = hits.doc(0);
     assertEquals("didn't get the right document first", "21", d.get("id"));
 
+    testHits(hits, 34, searcher.getIndexReader());
+
     searcher.close();
     dir.close();
   }
@@ -189,9 +207,9 @@
     // make sure searching sees right # hits
     IndexSearcher searcher = new IndexSearcher(dir);
     Hits hits = searcher.search(new TermQuery(new Term("content", "aaa")));
-    assertEquals("wrong number of hits", 44, hits.length());
     Document d = hits.doc(0);
     assertEquals("wrong first document", "21", d.get("id"));
+    testHits(hits, 44, searcher.getIndexReader());
     searcher.close();
 
     // make sure we can do delete & setNorm against this
@@ -209,6 +227,7 @@
     assertEquals("wrong number of hits", 43, hits.length());
     d = hits.doc(0);
     assertEquals("wrong first document", "22", d.get("id"));
+    testHits(hits, 43, searcher.getIndexReader());
     searcher.close();
 
     // optimize
@@ -220,6 +239,7 @@
     hits = searcher.search(new TermQuery(new Term("content", "aaa")));
     assertEquals("wrong number of hits", 43, hits.length());
     d = hits.doc(0);
+    testHits(hits, 43, searcher.getIndexReader());
     assertEquals("wrong first document", "22", d.get("id"));
     searcher.close();
 
@@ -257,6 +277,7 @@
     assertEquals("wrong number of hits", 33, hits.length());
     d = hits.doc(0);
     assertEquals("wrong first document", "22", d.get("id"));
+    testHits(hits, 33, searcher.getIndexReader());
     searcher.close();
 
     // optimize
@@ -269,6 +290,7 @@
     assertEquals("wrong number of hits", 33, hits.length());
     d = hits.doc(0);
     assertEquals("wrong first document", "22", d.get("id"));
+    testHits(hits, 33, searcher.getIndexReader());
     searcher.close();
 
     dir.close();
@@ -283,6 +305,7 @@
     Directory dir = FSDirectory.getDirectory(dirName);
     IndexWriter writer = new IndexWriter(dir, new WhitespaceAnalyzer(), true);
     writer.setUseCompoundFile(doCFS);
+    writer.setMaxBufferedDocs(10);
     
     for(int i=0;i<35;i++) {
       addDoc(writer, i);
@@ -393,6 +416,7 @@
     Document doc = new Document();
     doc.add(new Field("content", "aaa", Field.Store.NO, Field.Index.TOKENIZED));
     doc.add(new Field("id", Integer.toString(id), Field.Store.YES, Field.Index.UN_TOKENIZED));
+    doc.add(new Field("content2", "here is more content with aaa aaa aaa", Field.Store.YES, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
     writer.addDocument(doc);
   }
 

Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=615183&r1=615182&r2=615183&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java Fri Jan 25 03:32:32 2008
@@ -1937,9 +1937,10 @@
       for(int j=0;j<reader.maxDoc();j++) {
         if (reader.isDeleted(j))
           numDel++;
-        else
+        else {
           reader.document(j);
-        reader.getTermFreqVectors(j);
+          reader.getTermFreqVectors(j);
+        }
       }
       reader.close();
 
@@ -1963,9 +1964,10 @@
       for(int j=0;j<reader.maxDoc();j++) {
         if (reader.isDeleted(j))
           numDel++;
-        else
+        else {
           reader.document(j);
-        reader.getTermFreqVectors(j);
+          reader.getTermFreqVectors(j);
+        }
       }
       reader.close();
       assertEquals(0, numDel);
@@ -2053,9 +2055,10 @@
       for(int j=0;j<reader.maxDoc();j++) {
         if (reader.isDeleted(j))
           numDel++;
-        else
+        else {
           reader.document(j);
-        reader.getTermFreqVectors(j);
+          reader.getTermFreqVectors(j);
+        }
       }
       reader.close();
 
@@ -2079,9 +2082,10 @@
       for(int j=0;j<reader.maxDoc();j++) {
         if (reader.isDeleted(j))
           numDel++;
-        else
+        else {
           reader.document(j);
-        reader.getTermFreqVectors(j);
+          reader.getTermFreqVectors(j);
+        }
       }
       reader.close();
       assertEquals(0, numDel);

Added: lucene/java/trunk/src/test/org/apache/lucene/index/index.19.cfs.zip
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/index.19.cfs.zip?rev=615183&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/index.19.cfs.zip
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/java/trunk/src/test/org/apache/lucene/index/index.19.nocfs.zip
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/index.19.nocfs.zip?rev=615183&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/index.19.nocfs.zip
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/java/trunk/src/test/org/apache/lucene/index/index.20.cfs.zip
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/index.20.cfs.zip?rev=615183&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/index.20.cfs.zip
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/java/trunk/src/test/org/apache/lucene/index/index.20.nocfs.zip
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/index.20.nocfs.zip?rev=615183&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/index.20.nocfs.zip
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/java/trunk/src/test/org/apache/lucene/index/index.21.cfs.zip
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/index.21.cfs.zip?rev=615183&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/index.21.cfs.zip
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/java/trunk/src/test/org/apache/lucene/index/index.21.nocfs.zip
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/index.21.nocfs.zip?rev=615183&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/index.21.nocfs.zip
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/java/trunk/src/test/org/apache/lucene/index/index.22.cfs.zip
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/index.22.cfs.zip?rev=615183&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/index.22.cfs.zip
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/java/trunk/src/test/org/apache/lucene/index/index.22.nocfs.zip
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/index.22.nocfs.zip?rev=615183&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/java/trunk/src/test/org/apache/lucene/index/index.22.nocfs.zip
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream