You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/01/18 00:47:15 UTC

svn commit: r1232652 - in /lucene/dev/trunk/lucene/src: java/org/apache/lucene/codecs/lucene3x/ java/org/apache/lucene/codecs/lucene40/ test-framework/java/org/apache/lucene/codecs/preflexrw/

Author: rmuir
Date: Tue Jan 17 23:47:14 2012
New Revision: 1232652

URL: http://svn.apache.org/viewvc?rev=1232652&view=rev
Log:
LUCENE-3613: split out 4.0/3.x term vectors implementations

Added:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsFormat.java   (with props)
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java
      - copied, changed from r1232535, lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
    lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsFormat.java   (with props)
    lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsWriter.java
      - copied, changed from r1232535, lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java
Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
    lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java?rev=1232652&r1=1232651&r2=1232652&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java Tue Jan 17 23:47:14 2012
@@ -51,8 +51,7 @@ public class Lucene3xCodec extends Codec
   // TODO: this should really be a different impl
   private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
   
-  // TODO: this should really be a different impl
-  private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
+  private final TermVectorsFormat vectorsFormat = new Lucene3xTermVectorsFormat();
   
   private final FieldInfosFormat fieldInfosFormat = new Lucene3xFieldInfosFormat();
 

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsFormat.java?rev=1232652&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsFormat.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsFormat.java Tue Jan 17 23:47:14 2012
@@ -0,0 +1,49 @@
+package org.apache.lucene.codecs.lucene3x;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+public class Lucene3xTermVectorsFormat extends TermVectorsFormat {
+
+  @Override
+  public TermVectorsReader vectorsReader(Directory directory,SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
+    return new Lucene3xTermVectorsReader(directory, segmentInfo, fieldInfos, context);
+  }
+
+  @Override
+  public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
+    // TODO all these IAEs in preflex should be UOEs?
+    throw new IllegalArgumentException("this codec can only be used for reading");
+  }
+
+  @Override
+  public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+    Lucene3xTermVectorsReader.files(dir, info, files);
+  }
+  
+}

Copied: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java (from r1232535, lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java?p2=lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java&r1=1232535&r2=1232652&rev=1232652&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java Tue Jan 17 23:47:14 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene40;
+package org.apache.lucene.codecs.lucene3x;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
@@ -45,7 +44,7 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
-public class Lucene40TermVectorsReader extends TermVectorsReader {
+public class Lucene3xTermVectorsReader extends TermVectorsReader {
 
   // NOTE: if you make a new format, it must be larger than
   // the current format
@@ -55,26 +54,25 @@ public class Lucene40TermVectorsReader e
 
   // NOTE: always change this if you switch to a new format!
   // whenever you add a new format, make it 1 larger (positive version logic)!
-  static final int FORMAT_CURRENT = FORMAT_UTF8_LENGTH_IN_BYTES;
+  public static final int FORMAT_CURRENT = FORMAT_UTF8_LENGTH_IN_BYTES;
   
   // when removing support for old versions, leave the last supported version here
-  static final int FORMAT_MINIMUM = FORMAT_UTF8_LENGTH_IN_BYTES;
+  public static final int FORMAT_MINIMUM = FORMAT_UTF8_LENGTH_IN_BYTES;
 
   //The size in bytes that the FORMAT_VERSION will take up at the beginning of each file 
   static final int FORMAT_SIZE = 4;
 
-  static final byte STORE_POSITIONS_WITH_TERMVECTOR = 0x1;
+  public static final byte STORE_POSITIONS_WITH_TERMVECTOR = 0x1;
 
-  static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x2;
+  public static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x2;
   
   /** Extension of vectors fields file */
-  static final String VECTORS_FIELDS_EXTENSION = "tvf";
+  public static final String VECTORS_FIELDS_EXTENSION = "tvf";
 
   /** Extension of vectors documents file */
-  static final String VECTORS_DOCUMENTS_EXTENSION = "tvd";
+  public static final String VECTORS_DOCUMENTS_EXTENSION = "tvd";
 
   /** Extension of vectors index file */
-  // TODO: shouldnt be visible to segments reader, preflex should do this itself somehow
   public static final String VECTORS_INDEX_EXTENSION = "tvx";
 
   private FieldInfos fieldInfos;
@@ -92,7 +90,7 @@ public class Lucene40TermVectorsReader e
   private final int format;
 
   // used by clone
-  Lucene40TermVectorsReader(FieldInfos fieldInfos, IndexInput tvx, IndexInput tvd, IndexInput tvf, int size, int numTotalDocs, int docStoreOffset, int format) {
+  Lucene3xTermVectorsReader(FieldInfos fieldInfos, IndexInput tvx, IndexInput tvd, IndexInput tvf, int size, int numTotalDocs, int docStoreOffset, int format) {
     this.fieldInfos = fieldInfos;
     this.tvx = tvx;
     this.tvd = tvd;
@@ -103,7 +101,7 @@ public class Lucene40TermVectorsReader e
     this.format = format;
   }
     
-  public Lucene40TermVectorsReader(Directory d, SegmentInfo si, FieldInfos fieldInfos, IOContext context)
+  public Lucene3xTermVectorsReader(Directory d, SegmentInfo si, FieldInfos fieldInfos, IOContext context)
     throws CorruptIndexException, IOException {
     final String segment = si.getDocStoreSegment();
     final int docStoreOffset = si.getDocStoreOffset();
@@ -153,72 +151,11 @@ public class Lucene40TermVectorsReader e
     }
   }
 
-  // Used for bulk copy when merging
-  IndexInput getTvdStream() {
-    return tvd;
-  }
-
-  // Used for bulk copy when merging
-  IndexInput getTvfStream() {
-    return tvf;
-  }
-
   // Not private to avoid synthetic access$NNN methods
   void seekTvx(final int docNum) throws IOException {
     tvx.seek((docNum + docStoreOffset) * 16L + FORMAT_SIZE);
   }
 
-  boolean canReadRawDocs() {
-    // we can always read raw docs, unless the term vectors
-    // didn't exist
-    return format != 0;
-  }
-
-  /** 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;
-    }
-
-    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 = docStoreOffset + startDocID + count + 1;
-      assert docID <= numTotalDocs;
-      if (docID < numTotalDocs)  {
-        tvdPosition = tvx.readLong();
-        tvfPosition = tvx.readLong();
-      } else {
-        tvdPosition = tvd.length();
-        tvfPosition = tvf.length();
-        assert count == numDocs-1;
-      }
-      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();
@@ -397,7 +334,7 @@ public class Lucene40TermVectorsReader e
 
     // NOTE: tvf is pre-positioned by caller
     public TVTermsEnum() throws IOException {
-      this.origTVF = Lucene40TermVectorsReader.this.tvf;
+      this.origTVF = Lucene3xTermVectorsReader.this.tvf;
       tvf = (IndexInput) origTVF.clone();
     }
 
@@ -712,7 +649,7 @@ public class Lucene40TermVectorsReader e
       cloneTvf = (IndexInput) tvf.clone();
     }
     
-    return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, docStoreOffset, format);
+    return new Lucene3xTermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, docStoreOffset, format);
   }
   
   public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1232652&r1=1232651&r2=1232652&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Tue Jan 17 23:47:14 2012
@@ -74,8 +74,7 @@ public class Lucene40TermVectorsReader e
   static final String VECTORS_DOCUMENTS_EXTENSION = "tvd";
 
   /** Extension of vectors index file */
-  // TODO: shouldnt be visible to segments reader, preflex should do this itself somehow
-  public static final String VECTORS_INDEX_EXTENSION = "tvx";
+  static final String VECTORS_INDEX_EXTENSION = "tvx";
 
   private FieldInfos fieldInfos;
 
@@ -84,29 +83,23 @@ public class Lucene40TermVectorsReader e
   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 final int format;
 
   // used by clone
-  Lucene40TermVectorsReader(FieldInfos fieldInfos, IndexInput tvx, IndexInput tvd, IndexInput tvf, int size, int numTotalDocs, int docStoreOffset, int format) {
+  Lucene40TermVectorsReader(FieldInfos fieldInfos, IndexInput tvx, IndexInput tvd, IndexInput tvf, int size, int numTotalDocs, int format) {
     this.fieldInfos = fieldInfos;
     this.tvx = tvx;
     this.tvd = tvd;
     this.tvf = tvf;
     this.size = size;
     this.numTotalDocs = numTotalDocs;
-    this.docStoreOffset = docStoreOffset;
     this.format = format;
   }
     
   public Lucene40TermVectorsReader(Directory d, SegmentInfo si, FieldInfos fieldInfos, IOContext context)
     throws CorruptIndexException, IOException {
-    final String segment = si.getDocStoreSegment();
-    final int docStoreOffset = si.getDocStoreOffset();
+    final String segment = si.name;
     final int size = si.docCount;
     
     boolean success = false;
@@ -127,17 +120,8 @@ public class Lucene40TermVectorsReader e
 
       numTotalDocs = (int) (tvx.length() >> 4);
 
-      if (-1 == docStoreOffset) {
-        this.docStoreOffset = 0;
-        this.size = numTotalDocs;
-        assert size == 0 || numTotalDocs == size;
-      } else {
-        this.docStoreOffset = docStoreOffset;
-        this.size = size;
-        // Verify the file is long enough to hold all of our
-        // docs
-        assert numTotalDocs >= size + docStoreOffset: "numTotalDocs=" + numTotalDocs + " size=" + size + " docStoreOffset=" + docStoreOffset;
-      }
+      this.size = numTotalDocs;
+      assert size == 0 || numTotalDocs == size;
 
       this.fieldInfos = fieldInfos;
       success = true;
@@ -165,7 +149,7 @@ public class Lucene40TermVectorsReader e
 
   // Not private to avoid synthetic access$NNN methods
   void seekTvx(final int docNum) throws IOException {
-    tvx.seek((docNum + docStoreOffset) * 16L + FORMAT_SIZE);
+    tvx.seek(docNum * 16L + FORMAT_SIZE);
   }
 
   boolean canReadRawDocs() {
@@ -201,7 +185,7 @@ public class Lucene40TermVectorsReader e
 
     int count = 0;
     while (count < numDocs) {
-      final int docID = docStoreOffset + startDocID + count + 1;
+      final int docID = startDocID + count + 1;
       assert docID <= numTotalDocs;
       if (docID < numTotalDocs)  {
         tvdPosition = tvx.readLong();
@@ -712,23 +696,14 @@ public class Lucene40TermVectorsReader e
       cloneTvf = (IndexInput) tvf.clone();
     }
     
-    return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, docStoreOffset, format);
+    return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs, format);
   }
   
   public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
     if (info.getHasVectors()) {
-      if (info.getDocStoreOffset() != -1) {
-        assert info.getDocStoreSegment() != null;
-        if (!info.getDocStoreIsCompoundFile()) {
-          files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_INDEX_EXTENSION));
-          files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_FIELDS_EXTENSION));
-          files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_DOCUMENTS_EXTENSION));
-        }
-      } else {
-        files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_INDEX_EXTENSION));
-        files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_FIELDS_EXTENSION));
-        files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
-      }
+      files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_INDEX_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_FIELDS_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
     }
   }
 }

Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java?rev=1232652&r1=1232651&r2=1232652&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java Tue Jan 17 23:47:14 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.preflex
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.lucene3x.Lucene3xCodec;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -31,6 +32,7 @@ public class PreFlexRWCodec extends Luce
   private final PostingsFormat postings = new PreFlexRWPostingsFormat();
   private final NormsFormat norms = new PreFlexRWNormsFormat();
   private final FieldInfosFormat fieldInfos = new PreFlexRWFieldInfosFormat();
+  private final TermVectorsFormat termVectors = new PreFlexRWTermVectorsFormat();
   
   @Override
   public PostingsFormat postingsFormat() {
@@ -58,4 +60,13 @@ public class PreFlexRWCodec extends Luce
       return super.fieldInfosFormat();
     }
   }
+
+  @Override
+  public TermVectorsFormat termVectorsFormat() {
+    if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
+      return termVectors;
+    } else {
+      return super.termVectorsFormat();
+    }
+  }
 }

Added: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsFormat.java?rev=1232652&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsFormat.java (added)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsFormat.java Tue Jan 17 23:47:14 2012
@@ -0,0 +1,33 @@
+package org.apache.lucene.codecs.preflexrw;
+
+/**
+ * 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.codecs.TermVectorsWriter;
+import org.apache.lucene.codecs.lucene3x.Lucene3xTermVectorsFormat;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+public class PreFlexRWTermVectorsFormat extends Lucene3xTermVectorsFormat {
+
+  @Override
+  public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
+    return new PreFlexRWTermVectorsWriter(directory, segment, context);
+  }
+}

Copied: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsWriter.java (from r1232535, lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsWriter.java?p2=lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsWriter.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java&r1=1232535&r2=1232652&rev=1232652&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWTermVectorsWriter.java Tue Jan 17 23:47:14 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene40;
+package org.apache.lucene.codecs.preflexrw;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,50 +19,37 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.codecs.lucene3x.Lucene3xTermVectorsReader;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.MergePolicy.MergeAbortedException;
-import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.SegmentReader;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 
-// TODO: make a new 4.0 TV format that encodes better
-//   - use startOffset (not endOffset) as base for delta on
-//     next startOffset because today for syns or ngrams or
-//     WDF or shingles etc. we are encoding negative vints
-//     (= slow, 5 bytes per)
-//   - if doc has no term vectors, write 0 into the tvx
-//     file; saves a seek to tvd only to read a 0 vint (and
-//     saves a byte in tvd)
-
-public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
+// TODO: surrogates dance!
+public final class PreFlexRWTermVectorsWriter extends TermVectorsWriter {
   private final Directory directory;
   private final String segment;
   private IndexOutput tvx = null, tvd = null, tvf = null;
 
-  public Lucene40TermVectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
+  public PreFlexRWTermVectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
     this.directory = directory;
     this.segment = segment;
     boolean success = false;
     try {
       // Open files for TermVector storage
-      tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_INDEX_EXTENSION), context);
-      tvx.writeInt(Lucene40TermVectorsReader.FORMAT_CURRENT);
-      tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_DOCUMENTS_EXTENSION), context);
-      tvd.writeInt(Lucene40TermVectorsReader.FORMAT_CURRENT);
-      tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_FIELDS_EXTENSION), context);
-      tvf.writeInt(Lucene40TermVectorsReader.FORMAT_CURRENT);
+      tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION), context);
+      tvx.writeInt(Lucene3xTermVectorsReader.FORMAT_CURRENT);
+      tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION), context);
+      tvd.writeInt(Lucene3xTermVectorsReader.FORMAT_CURRENT);
+      tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION), context);
+      tvf.writeInt(Lucene3xTermVectorsReader.FORMAT_CURRENT);
       success = true;
     } finally {
       if (!success) {
@@ -99,9 +86,9 @@ public final class Lucene40TermVectorsWr
     tvf.writeVInt(numTerms);
     byte bits = 0x0;
     if (positions)
-      bits |= Lucene40TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
+      bits |= Lucene3xTermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
     if (offsets)
-      bits |= Lucene40TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
+      bits |= Lucene3xTermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
     tvf.writeByte(bits);
     
     assert fieldCount <= numVectorFields;
@@ -202,148 +189,9 @@ public final class Lucene40TermVectorsWr
     try {
       close();
     } catch (IOException ignored) {}
-    IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_INDEX_EXTENSION),
-        IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_DOCUMENTS_EXTENSION),
-        IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_FIELDS_EXTENSION));
-  }
-
-  /**
-   * Do a bulk copy of numDocs documents from reader to our
-   * streams.  This is used to expedite merging, if the
-   * field numbers are congruent.
-   */
-  private void addRawDocuments(Lucene40TermVectorsReader 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;
-  }
-
-  @Override
-  public final int merge(MergeState mergeState) throws IOException {
-    // Used for bulk-reading raw bytes for term vectors
-    int rawDocLengths[] = new int[MAX_RAW_MERGE_DOCS];
-    int rawDocLengths2[] = new int[MAX_RAW_MERGE_DOCS];
-
-    int idx = 0;
-    int numDocs = 0;
-    for (final MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
-      final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
-      Lucene40TermVectorsReader matchingVectorsReader = null;
-      if (matchingSegmentReader != null) {
-        TermVectorsReader vectorsReader = matchingSegmentReader.getTermVectorsReader();
-
-        if (vectorsReader != null && vectorsReader instanceof Lucene40TermVectorsReader) {
-          // If the TV* files are an older format then they cannot read raw docs:
-          if (((Lucene40TermVectorsReader)vectorsReader).canReadRawDocs()) {
-            matchingVectorsReader = (Lucene40TermVectorsReader) vectorsReader;
-          }
-        }
-      }
-      if (reader.liveDocs != null) {
-        numDocs += copyVectorsWithDeletions(mergeState, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
-      } else {
-        numDocs += copyVectorsNoDeletions(mergeState, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
-      }
-    }
-    finish(numDocs);
-    return numDocs;
-  }
-
-  /** Maximum number of contiguous documents to bulk-copy
-      when merging term vectors */
-  private final static int MAX_RAW_MERGE_DOCS = 4192;
-
-  private int copyVectorsWithDeletions(MergeState mergeState,
-                                        final Lucene40TermVectorsReader matchingVectorsReader,
-                                        final MergeState.IndexReaderAndLiveDocs reader,
-                                        int rawDocLengths[],
-                                        int rawDocLengths2[])
-          throws IOException, MergeAbortedException {
-    final int maxDoc = reader.reader.maxDoc();
-    final Bits liveDocs = reader.liveDocs;
-    int totalNumDocs = 0;
-    if (matchingVectorsReader != null) {
-      // We can bulk-copy because the fieldInfos are "congruent"
-      for (int docNum = 0; docNum < maxDoc;) {
-        if (!liveDocs.get(docNum)) {
-          // skip deleted docs
-          ++docNum;
-          continue;
-        }
-        // We can optimize this case (doing a bulk byte copy) since the field
-        // numbers are identical
-        int start = docNum, numDocs = 0;
-        do {
-          docNum++;
-          numDocs++;
-          if (docNum >= maxDoc) break;
-          if (!liveDocs.get(docNum)) {
-            docNum++;
-            break;
-          }
-        } while(numDocs < MAX_RAW_MERGE_DOCS);
-        
-        matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, start, numDocs);
-        addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, numDocs);
-        totalNumDocs += numDocs;
-        mergeState.checkAbort.work(300 * numDocs);
-      }
-    } else {
-      for (int docNum = 0; docNum < maxDoc; docNum++) {
-        if (!liveDocs.get(docNum)) {
-          // skip deleted docs
-          continue;
-        }
-        
-        // NOTE: it's very important to first assign to vectors then pass it to
-        // termVectorsWriter.addAllDocVectors; see LUCENE-1282
-        Fields vectors = reader.reader.getTermVectors(docNum);
-        addAllDocVectors(vectors, mergeState.fieldInfos);
-        totalNumDocs++;
-        mergeState.checkAbort.work(300);
-      }
-    }
-    return totalNumDocs;
-  }
-  
-  private int copyVectorsNoDeletions(MergeState mergeState,
-                                      final Lucene40TermVectorsReader matchingVectorsReader,
-                                      final MergeState.IndexReaderAndLiveDocs reader,
-                                      int rawDocLengths[],
-                                      int rawDocLengths2[])
-          throws IOException, MergeAbortedException {
-    final int maxDoc = reader.reader.maxDoc();
-    if (matchingVectorsReader != null) {
-      // We can bulk-copy because the fieldInfos are "congruent"
-      int docCount = 0;
-      while (docCount < maxDoc) {
-        int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
-        matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, docCount, len);
-        addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, len);
-        docCount += len;
-        mergeState.checkAbort.work(300 * len);
-      }
-    } else {
-      for (int docNum = 0; docNum < maxDoc; docNum++) {
-        // NOTE: it's very important to first assign to vectors then pass it to
-        // termVectorsWriter.addAllDocVectors; see LUCENE-1282
-        Fields vectors = reader.reader.getTermVectors(docNum);
-        addAllDocVectors(vectors, mergeState.fieldInfos);
-        mergeState.checkAbort.work(300);
-      }
-    }
-    return maxDoc;
+    IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION),
+        IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_DOCUMENTS_EXTENSION),
+        IndexFileNames.segmentFileName(segment, "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION));
   }
   
   @Override