lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sim...@apache.org
Subject svn commit: r1102476 - in /lucene/dev/trunk/lucene/src: java/org/apache/lucene/index/ java/org/apache/lucene/index/codecs/sep/ test-framework/org/apache/lucene/index/codecs/mockintblock/ test-framework/org/apache/lucene/index/codecs/mocksep/ test/org/a...
Date Thu, 12 May 2011 21:51:19 GMT
Author: simonw
Date: Thu May 12 21:51:19 2011
New Revision: 1102476

URL: http://svn.apache.org/viewvc?rev=1102476&view=rev
Log:
LUCENE-2984: Move hasVectors() & hasProx() responsibility out of SegmentInfo to FieldInfos

Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
    lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
    lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
    lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Thu May
12 21:51:19 2011
@@ -235,9 +235,10 @@ final class DocFieldProcessor extends Do
     // enabled; we could save [small amount of] CPU
     // here.
     ArrayUtil.quickSort(fields, 0, fieldCount, fieldsComp);
-
-    for(int i=0;i<fieldCount;i++)
-      fields[i].consumer.processFields(fields[i].fields, fields[i].fieldCount);
+    for(int i=0;i<fieldCount;i++) {
+      final DocFieldProcessorPerField perField = fields[i];
+      perField.consumer.processFields(perField.fields, perField.fieldCount);
+    }
 
     if (docState.maxTermPrefix != null && docState.infoStream != null) {
       docState.infoStream.println("WARNING: document contains at least one immense term (whose
UTF8 encoding is longer than the max length " + DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8
+ "), all of which were skipped.  Please correct the analyzer to not produce such terms. 
The prefix of the first immense term is: '" + docState.maxTermPrefix + "...'");

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
(original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
Thu May 12 21:51:19 2011
@@ -235,6 +235,7 @@ public class DocumentsWriterPerThread {
           // mark document as deleted
           deleteDocID(docState.docID);
           numDocsInRAM++;
+          fieldInfos.revertUncommitted();
         } else {
           abort();
         }
@@ -377,15 +378,12 @@ public class DocumentsWriterPerThread {
     boolean success = false;
 
     try {
-
-      SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false,
fieldInfos.hasProx(), flushState.segmentCodecs, false, fieldInfos);
       consumer.flush(flushState);
       pendingDeletes.terms.clear();
-      newSegment.setHasVectors(flushState.hasVectors);
-
+      final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory,
false, flushState.segmentCodecs, fieldInfos.asReadOnly());
       if (infoStream != null) {
         message("new segment has " + (flushState.deletedDocs == null ? 0 : flushState.deletedDocs.count())
+ " deleted docs");
-        message("new segment has " + (flushState.hasVectors ? "vectors" : "no vectors"));
+        message("new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
         message("flushedFiles=" + newSegment.files());
         message("flushed codecs=" + newSegment.getSegmentCodecs());
       }
@@ -435,10 +433,6 @@ public class DocumentsWriterPerThread {
     return bytesUsed.get() + pendingDeletes.bytesUsed.get();
   }
 
-  FieldInfos getFieldInfos() {
-    return fieldInfos;
-  }
-
   void message(String message) {
     writer.message("DWPT: " + message);
   }
@@ -498,4 +492,5 @@ public class DocumentsWriterPerThread {
     this.infoStream = infoStream;
     docState.infoStream = infoStream;
   }
+  
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java Thu May 12 21:51:19
2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 /** @lucene.experimental */
 public final class FieldInfo {
   public static final int UNASSIGNED_CODEC_ID = -1;
-
   public final String name;
   public final int number;
 
@@ -108,4 +107,28 @@ public final class FieldInfo {
     }
     assert !this.omitTermFreqAndPositions || !this.storePayloads;
   }
+  private boolean vectorsCommitted;
+
+  /**
+   * Reverts all uncommitted changes on this {@link FieldInfo}
+   * @see #commitVectors()
+   */
+  void revertUncommitted() {
+    if (storeTermVector && !vectorsCommitted) {
+      storeOffsetWithTermVector = false;
+      storePositionWithTermVector = false;
+      storeTermVector = false;  
+    }
+  }
+
+  /**
+   * Commits term vector modifications. Changes to term-vectors must be
+   * explicitly committed once the necessary files are created. If those changes
+   * are not committed subsequent {@link #revertUncommitted()} will reset the
+   * all term-vector flags before the next document.
+   */
+  void commitVectors() {
+    assert storeTermVector;
+    vectorsCommitted = true;
+  }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java Thu May 12 21:51:19
2011
@@ -216,6 +216,10 @@ public final class FieldInfos implements
   static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
 
   private int format;
+  private boolean hasProx; // only set if readonly
+  private boolean hasVectors; // only set if readonly
+  private long version; // internal use to track changes
+  
 
   /**
    * Creates a new {@link FieldInfos} instance with a private
@@ -263,7 +267,7 @@ public final class FieldInfos implements
    */
   public FieldInfos(Directory d, String name) throws IOException {
     this((FieldNumberBiMap)null, null); // use null here to make this FIs Read-Only
-    IndexInput input = d.openInput(name);
+    final IndexInput input = d.openInput(name);
     try {
       read(input, name);
     } finally {
@@ -299,6 +303,9 @@ public final class FieldInfos implements
   @Override
   synchronized public Object clone() {
     FieldInfos fis = new FieldInfos(globalFieldNumbers, segmentCodecsBuilder);
+    fis.format = format;
+    fis.hasProx = hasProx;
+    fis.hasVectors = hasVectors;
     for (FieldInfo fi : this) {
       FieldInfo clone = (FieldInfo) (fi).clone();
       fis.putInternal(clone);
@@ -308,6 +315,10 @@ public final class FieldInfos implements
 
   /** Returns true if any fields do not omitTermFreqAndPositions */
   public boolean hasProx() {
+    if (isReadOnly()) {
+      return hasProx;
+    }
+    // mutable FIs must check!
     for (FieldInfo fi : this) {
       if (fi.isIndexed && !fi.omitTermFreqAndPositions) {
         return true;
@@ -440,6 +451,7 @@ public final class FieldInfos implements
     if (fi.isIndexed && fi.getCodecId() == FieldInfo.UNASSIGNED_CODEC_ID) {
       segmentCodecsBuilder.tryAddAndSet(fi);
     }
+    version++;
     return fi;
   }
 
@@ -510,6 +522,10 @@ public final class FieldInfos implements
   }
 
   public boolean hasVectors() {
+    if (isReadOnly()) {
+      return hasVectors;
+    }
+    // mutable FIs must check
     for (FieldInfo fi : this) {
       if (fi.storeTermVector) {
         return true;
@@ -562,6 +578,10 @@ public final class FieldInfos implements
   public final boolean isReadOnly() {
     return globalFieldNumbers == null;
   }
+  
+  synchronized final long getVersion() {
+    return version;
+  }
 
   public void write(IndexOutput output) throws IOException {
     output.writeVInt(FORMAT_CURRENT);
@@ -615,7 +635,8 @@ public final class FieldInfos implements
       if (omitTermFreqAndPositions) {
         storePayloads = false;
       }
-
+      hasVectors |= storeTermVector;
+      hasProx |= isIndexed && !omitTermFreqAndPositions;
       final FieldInfo addInternal = addInternal(name, fieldNumber, isIndexed, storeTermVector,
storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
       addInternal.setCodecId(codecId);
     }
@@ -624,5 +645,29 @@ public final class FieldInfos implements
       throw new CorruptIndexException("did not read all bytes from file \"" + fileName +
"\": read " + input.getFilePointer() + " vs size " + input.length());
     }    
   }
+  
+  /**
+   * Reverts all uncommitted changes 
+   * @see FieldInfo#revertUncommitted()
+   */
+  void revertUncommitted() {
+    for (FieldInfo fieldInfo : this) {
+      fieldInfo.revertUncommitted();
+    }
+  }
+  
+  final FieldInfos asReadOnly() {
+    if (isReadOnly()) {
+      return this;
+    }
+    final FieldInfos roFis = new FieldInfos((FieldNumberBiMap)null, null);
+    for (FieldInfo fieldInfo : this) {
+      FieldInfo clone = (FieldInfo) (fieldInfo).clone();
+      roFis.putInternal(clone);
+      roFis.hasVectors |= clone.storeTermVector;
+      roFis.hasProx |= clone.isIndexed && !clone.omitTermFreqAndPositions;
+    }
+    return roFis;
+  }
 
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java Thu May
12 21:51:19 2011
@@ -22,6 +22,7 @@ import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
@@ -196,7 +197,31 @@ final class IndexFileDeleter {
             }
           }
           if (sis != null) {
-            CommitPoint commitPoint = new CommitPoint(commitsToDelete, directory, sis);
+            final SegmentInfos infos = sis;
+            for (SegmentInfo segmentInfo : infos) {
+              try {
+                /*
+                 * Force FI to load for each segment since we could see a
+                 * segments file and load successfully above if the files are
+                 * still referenced when they are deleted and the os doesn't let
+                 * you delete them. Yet its likely that fnm files are removed
+                 * while seg file is still around Since LUCENE-2984 we need FI
+                 * to find out if a seg has vectors and prox so we need those
+                 * files to be opened for a commit point.
+                 */
+                segmentInfo.getFieldInfos();
+              } catch (FileNotFoundException e) {
+                refresh(segmentInfo.name);
+                sis = null;
+                if (infoStream != null) {
+                  message("init: hit FileNotFoundException when loading commit \"" + fileName
+ "\"; skipping this commit point");
+                }
+              }
+            }
+           
+          }
+          if (sis != null) {
+            final CommitPoint commitPoint = new CommitPoint(commitsToDelete, directory, sis);
             if (sis.getGeneration() == segmentInfos.getGeneration()) {
               currentCommitPoint = commitPoint;
             }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Thu May 12 21:51:19
2011
@@ -2355,7 +2355,7 @@ public class IndexWriter implements Clos
 
       String mergedName = newSegmentName();
       SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(),
-                                               mergedName, null, codecs, payloadProcessorProvider,
+                                               mergedName, null, payloadProcessorProvider,
                                                globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)));
 
       for (IndexReader reader : readers)      // add new indexes
@@ -2365,8 +2365,7 @@ public class IndexWriter implements Clos
 
       final FieldInfos fieldInfos = merger.fieldInfos();
       SegmentInfo info = new SegmentInfo(mergedName, docCount, directory,
-                                         false, fieldInfos.hasProx(), merger.getSegmentCodecs(),
-                                         fieldInfos.hasVectors(),
+                                         false, merger.getSegmentCodecs(),
                                          fieldInfos);
       setDiagnostics(info, "addIndexes(IndexReader...)");
 
@@ -3041,7 +3040,16 @@ public class IndexWriter implements Clos
     // is running (while synchronized) to avoid race
     // condition where two conflicting merges from different
     // threads, start
-    message("registerMerge merging=" + mergingSegments);
+    if (infoStream != null) {
+      StringBuilder builder = new StringBuilder("registerMerge merging= [");
+      for (SegmentInfo info : mergingSegments) {
+        builder.append(info.name).append(", ");  
+      }
+      builder.append("]");
+      // don't call mergingSegments.toString() could lead to ConcurrentModException
+      // since merge updates the segments FieldInfos
+      message(builder.toString());  
+    }
     for(SegmentInfo info : merge.segments) {
       message("registerMerge info=" + info);
       mergingSegments.add(info);
@@ -3094,7 +3102,7 @@ public class IndexWriter implements Clos
     // Bind a new segment name here so even with
     // ConcurrentMergePolicy we keep deterministic segment
     // names.
-    merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, false, null, false,
globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)));
+    merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, null, globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)));
 
     // Lock order: IW -> BD
     final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool,
merge.segments);
@@ -3258,16 +3266,14 @@ public class IndexWriter implements Clos
     List<SegmentInfo> sourceSegments = merge.segments;
 
     SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(), mergedName,
merge,
-                                             codecs, payloadProcessorProvider,
-                                             merge.info.getFieldInfos());
+                                             payloadProcessorProvider, merge.info.getFieldInfos());
 
     if (infoStream != null) {
-      message("merging " + merge.segString(directory) + " mergeVectors=" + merger.fieldInfos().hasVectors());
+      message("merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());
     }
 
     merge.readers = new ArrayList<SegmentReader>();
     merge.readerClones = new ArrayList<SegmentReader>();
-
     // This is try/finally to make sure merger's readers are
     // closed:
     boolean success = false;
@@ -3309,8 +3315,6 @@ public class IndexWriter implements Clos
 
       // Record which codec was used to write the segment
       merge.info.setSegmentCodecs(merger.getSegmentCodecs());
-      // Record if we have merged vectors
-      merge.info.setHasVectors(merger.fieldInfos().hasVectors());
 
       if (infoStream != null) {
         message("merge segmentCodecs=" + merger.getSegmentCodecs());
@@ -3324,13 +3328,11 @@ public class IndexWriter implements Clos
       // because codec must know if prox was written for
       // this segment:
       //System.out.println("merger set hasProx=" + merger.hasProx() + " seg=" + merge.info.name);
-      merge.info.setHasProx(merger.fieldInfos().hasProx());
-
       boolean useCompoundFile;
       synchronized (this) { // Guard segmentInfos
         useCompoundFile = mergePolicy.useCompoundFile(segmentInfos, merge.info);
       }
-
+      
       if (useCompoundFile) {
         success = false;
         final String compoundFileName = IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
@@ -3532,6 +3534,7 @@ public class IndexWriter implements Clos
 
   // called only from assert
   private boolean filesExist(SegmentInfos toSync) throws IOException {
+    
     Collection<String> files = toSync.files(directory, false);
     for(final String fileName: files) {
       assert directory.fileExists(fileName): "file " + fileName + " does not exist";

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Thu May 12 21:51:19
2011
@@ -41,7 +41,8 @@ import org.apache.lucene.util.Constants;
  * @lucene.experimental
  */
 public final class SegmentInfo {
-
+  // TODO: remove with hasVector and hasProx
+  private static final int CHECK_FIELDINFO = -2;
   static final int NO = -1;          // e.g. no norms; no deletes;
   static final int YES = 1;          // e.g. have norms; have deletes;
   static final int WITHOUT_GEN = 0;  // a file name that has no GEN in it.
@@ -83,10 +84,12 @@ public final class SegmentInfo {
   private boolean docStoreIsCompoundFile;         // whether doc store files are stored in
compound file (*.cfx)
 
   private int delCount;                           // How many deleted docs in this segment
+  
+  //TODO: remove when we don't have to support old indexes anymore that had this field
+  private int hasVectors = CHECK_FIELDINFO;
+  //TODO: remove when we don't have to support old indexes anymore that had this field
+  private int hasProx = CHECK_FIELDINFO;     // True if this segment has any fields with
omitTermFreqAndPositions==false
 
-  private boolean hasProx;                        // True if this segment has any fields
with omitTermFreqAndPositions==false
-
-  private boolean hasVectors;                     // True if this segment wrote term vectors
   
   private FieldInfos fieldInfos;
 
@@ -104,9 +107,12 @@ public final class SegmentInfo {
   // NOTE: only used in-RAM by IW to track buffered deletes;
   // this is never written to/read from the Directory
   private long bufferedDeletesGen;
-
+  
+  // holds the fieldInfos Version to refresh files() cache if FI has changed
+  private long fieldInfosVersion;
+  
   public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
-                     boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors, FieldInfos
fieldInfos) {
+                     SegmentCodecs segmentCodecs, FieldInfos fieldInfos) {
     this.name = name;
     this.docCount = docCount;
     this.dir = dir;
@@ -114,9 +120,7 @@ public final class SegmentInfo {
     this.isCompoundFile = isCompoundFile;
     this.docStoreOffset = -1;
     this.docStoreSegment = name;
-    this.hasProx = hasProx;
     this.segmentCodecs = segmentCodecs;
-    this.hasVectors = hasVectors;
     delCount = 0;
     version = Constants.LUCENE_MAIN_VERSION;
     this.fieldInfos = fieldInfos;
@@ -211,7 +215,7 @@ public final class SegmentInfo {
     delCount = input.readInt();
     assert delCount <= docCount;
 
-    hasProx = input.readByte() == YES;
+    hasProx = input.readByte();
 
     // System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx
+ " seg=" + name);
     if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
@@ -224,7 +228,7 @@ public final class SegmentInfo {
     diagnostics = input.readStringStringMap();
 
     if (format <= DefaultSegmentInfosWriter.FORMAT_HAS_VECTORS) {
-      hasVectors = input.readByte() == 1;
+      hasVectors = input.readByte();
     } else {
       final String storesSegment;
       final String ext;
@@ -245,7 +249,7 @@ public final class SegmentInfo {
         dirToTest = dir;
       }
       try {
-        hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "",
IndexFileNames.VECTORS_INDEX_EXTENSION));
+        hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "",
IndexFileNames.VECTORS_INDEX_EXTENSION)) ? YES : NO;
       } finally {
         if (isCompoundFile) {
           dirToTest.close();
@@ -309,14 +313,9 @@ public final class SegmentInfo {
   }
 
   public boolean getHasVectors() throws IOException {
-    return hasVectors;
-  }
-
-  public void setHasVectors(boolean v) {
-    hasVectors = v;
-    clearFilesCache();
+    return hasVectors == CHECK_FIELDINFO ? getFieldInfos().hasVectors() : hasVectors == YES;
   }
-
+  
   public FieldInfos getFieldInfos() throws IOException {
     loadFieldInfos(dir, true);
     return fieldInfos;
@@ -347,7 +346,7 @@ public final class SegmentInfo {
 
   @Override
   public Object clone() {
-    final SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, hasProx,
segmentCodecs, hasVectors,
+    final SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, segmentCodecs,
         fieldInfos == null ? null : (FieldInfos) fieldInfos.clone());
     si.docStoreOffset = docStoreOffset;
     si.docStoreSegment = docStoreSegment;
@@ -362,6 +361,8 @@ public final class SegmentInfo {
       }
     }
     si.version = version;
+    si.hasProx = hasProx;
+    si.hasVectors = hasVectors;
     return si;
   }
 
@@ -567,19 +568,14 @@ public final class SegmentInfo {
 
     output.writeByte((byte) (isCompoundFile ? YES : NO));
     output.writeInt(delCount);
-    output.writeByte((byte) (hasProx ? 1:0));
+    output.writeByte((byte) (hasProx));
     segmentCodecs.write(output);
     output.writeStringStringMap(diagnostics);
-    output.writeByte((byte) (hasVectors ? 1 : 0));
-  }
-
-  void setHasProx(boolean hasProx) {
-    this.hasProx = hasProx;
-    clearFilesCache();
+    output.writeByte((byte) (hasVectors));
   }
 
-  public boolean getHasProx() {
-    return hasProx;
+  public boolean getHasProx() throws IOException {
+    return hasProx == CHECK_FIELDINFO ? getFieldInfos().hasProx() : hasProx == YES;
   }
 
   /** Can only be called once. */
@@ -607,13 +603,14 @@ public final class SegmentInfo {
    */
 
   public List<String> files() throws IOException {
-
-    if (files != null) {
+    final long fisVersion = fieldInfosVersion;
+    if (fisVersion != (fieldInfosVersion = getFieldInfos().getVersion())) {
+      clearFilesCache(); // FIS has modifications - need to recompute
+    } else if (files != null) {
       // Already cached:
       return files;
     }
-
-    Set<String> fileSet = new HashSet<String>();
+    final Set<String> fileSet = new HashSet<String>();
 
     boolean useCompoundFile = getUseCompoundFile();
 
@@ -635,7 +632,7 @@ public final class SegmentInfo {
       } else {
         fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
         fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION));
-        if (hasVectors) {
+        if (getHasVectors()) {
           fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
           fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
           fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
@@ -644,7 +641,7 @@ public final class SegmentInfo {
     } else if (!useCompoundFile) {
       fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
       fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_EXTENSION));
-      if (hasVectors) {
+      if (getHasVectors()) {
         fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
         fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
         fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
@@ -707,8 +704,12 @@ public final class SegmentInfo {
     if (this.dir != dir) {
       s.append('x');
     }
-    if (hasVectors) {
-      s.append('v');
+    try {
+      if (getHasVectors()) {
+        s.append('v');
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
     }
     s.append(docCount);
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Thu May 12
21:51:19 2011
@@ -65,16 +65,14 @@ final class SegmentMerger {
       when merging stored fields */
   private final static int MAX_RAW_MERGE_DOCS = 4192;
 
-  private final CodecProvider codecs;
   private Codec codec;
   private SegmentWriteState segmentWriteState;
 
   private PayloadProcessorProvider payloadProcessorProvider;
 
-  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge,
CodecProvider codecs, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos)
{
+  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge,
PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos) {
     this.payloadProcessorProvider = payloadProcessorProvider;
     directory = dir;
-    this.codecs = codecs;
     segment = name;
     this.fieldInfos = fieldInfos;
     if (merge != null) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java Thu May
12 21:51:19 2011
@@ -31,7 +31,6 @@ public class SegmentWriteState {
   public final String segmentName;
   public final FieldInfos fieldInfos;
   public final int numDocs;
-  public boolean hasVectors;
 
   // Deletes to apply while we are flushing the segment.  A
   // Term is enrolled in here if it was deleted at one

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Thu
May 12 21:51:19 2011
@@ -63,7 +63,6 @@ final class TermVectorsTermsWriter exten
       }
 
       lastDocID = 0;
-      state.hasVectors = hasVectors;
       hasVectors = false;
     }
 
@@ -121,8 +120,7 @@ final class TermVectorsTermsWriter exten
     fill(docState.docID);
 
     // Append term vectors to the real outputs:
-    long pointer = tvd.getFilePointer();
-    tvx.writeLong(pointer);
+    tvx.writeLong(tvd.getFilePointer());
     tvx.writeLong(tvf.getFilePointer());
     tvd.writeVInt(numVectorFields);
     if (numVectorFields > 0) {
@@ -136,6 +134,8 @@ final class TermVectorsTermsWriter exten
         tvd.writeVLong(pos-lastPos);
         lastPos = pos;
         perFields[i].finishDocument();
+        // commit the termVectors once successful success - FI will otherwise reset them
+        perFields[i].fieldInfo.commitVectors();
       }
     }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
(original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
Thu May 12 21:51:19 2011
@@ -85,7 +85,7 @@ public class SepPostingsReaderImpl exten
     }
   }
 
-  public static void files(SegmentInfo segmentInfo, String codecId, Collection<String>
files) {
+  public static void files(SegmentInfo segmentInfo, String codecId, Collection<String>
files) throws IOException {
     files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.DOC_EXTENSION));
     files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION));
 

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
(original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
Thu May 12 21:51:19 2011
@@ -186,7 +186,7 @@ public class MockFixedIntBlockCodec exte
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String>
files) {
+  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String>
files) throws IOException {
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
(original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
Thu May 12 21:51:19 2011
@@ -211,7 +211,7 @@ public class MockVariableIntBlockCodec e
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String>
files) {
+  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String>
files) throws IOException {
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);

Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
(original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
Thu May 12 21:51:19 2011
@@ -130,7 +130,7 @@ public class MockSepCodec extends Codec 
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String>
files) {
+  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String>
files) throws IOException {
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java Thu May 12 21:51:19
2011
@@ -240,8 +240,7 @@ public class TestCodecs extends LuceneTe
     final Directory dir = newDirectory();
     FieldInfos clonedFieldInfos = (FieldInfos) fieldInfos.clone();
     this.write(fieldInfos, dir, fields, true);
-    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, clonedFieldInfos.buildSegmentCodecs(false),
clonedFieldInfos.hasVectors(), clonedFieldInfos);
-    si.setHasProx(false);
+    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, clonedFieldInfos.buildSegmentCodecs(false),
clonedFieldInfos);
 
     final FieldsProducer reader = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir,
si, fieldInfos, 64, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
 
@@ -293,7 +292,7 @@ public class TestCodecs extends LuceneTe
 
     FieldInfos clonedFieldInfos = (FieldInfos) fieldInfos.clone();
     this.write(fieldInfos, dir, fields, false);
-    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true,  clonedFieldInfos.buildSegmentCodecs(false),
clonedFieldInfos.hasVectors(), clonedFieldInfos);
+    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, clonedFieldInfos.buildSegmentCodecs(false),
clonedFieldInfos);
 
     if (VERBOSE) {
       System.out.println("TEST: now read postings");

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java Thu May 12 21:51:19
2011
@@ -196,7 +196,7 @@ public class TestDoc extends LuceneTestC
       SegmentReader r1 = SegmentReader.get(true, si1, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
       SegmentReader r2 = SegmentReader.get(true, si2, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
 
-      SegmentMerger merger = new SegmentMerger(si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
merged, null, CodecProvider.getDefault(), null, new FieldInfos());
+      SegmentMerger merger = new SegmentMerger(si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
merged, null, null, new FieldInfos());
 
       merger.add(r1);
       merger.add(r2);
@@ -205,8 +205,7 @@ public class TestDoc extends LuceneTestC
       r2.close();
       final FieldInfos fieldInfos =  merger.fieldInfos();
       final SegmentInfo info = new SegmentInfo(merged, si1.docCount + si2.docCount, si1.dir,
-                                               false, fieldInfos.hasProx(), merger.getSegmentCodecs(),
-                                               fieldInfos.hasVectors(), fieldInfos);
+                                               false, merger.getSegmentCodecs(), fieldInfos);
       
       if (useCompoundFile) {
         Collection<String> filesToDelete = merger.createCompoundFile(merged + ".cfs",
info);

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
(original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
Thu May 12 21:51:19 2011
@@ -33,9 +33,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.MockDirectoryWrapper.Failure;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -1179,4 +1177,105 @@ public class TestIndexWriterExceptions e
       writer.close();
       dir.close();
   }
+
+  public void testTermVectorExceptions() throws IOException {
+    FailOnTermVectors[] failures = new FailOnTermVectors[] {
+        new FailOnTermVectors(FailOnTermVectors.AFTER_INIT_STAGE),
+        new FailOnTermVectors(FailOnTermVectors.INIT_STAGE), };
+    for (int j = 0; j < 3 * RANDOM_MULTIPLIER; j++) {
+      for (FailOnTermVectors failure : failures) {
+        MockDirectoryWrapper dir = newDirectory();
+        IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(
+            TEST_VERSION_CURRENT, new MockAnalyzer(random)));
+        dir.failOn(failure);
+        int numDocs = 10 + random.nextInt(30);
+        for (int i = 0; i < numDocs; i++) {
+          Document doc = new Document();
+          Field field = newField(random, "field", "a field", Field.Store.YES,
+              Field.Index.ANALYZED);
+          doc.add(field);
+          // random TV
+          try {
+            w.addDocument(doc);
+            assertFalse(field.isTermVectorStored());
+          } catch (RuntimeException e) {
+            assertTrue(e.getMessage().startsWith(FailOnTermVectors.EXC_MSG));
+          }
+          if (random.nextInt(20) == 0) {
+            w.commit();
+            _TestUtil.checkIndex(dir);
+          }
+            
+        }
+        Document document = new Document();
+        document.add(new Field("field", "a field", Field.Store.YES,
+            Field.Index.ANALYZED));
+        w.addDocument(document);
+
+        for (int i = 0; i < numDocs; i++) {
+          Document doc = new Document();
+          Field field = newField(random, "field", "a field", Field.Store.YES,
+              Field.Index.ANALYZED);
+          doc.add(field);
+          // random TV
+          try {
+            w.addDocument(doc);
+            assertFalse(field.isTermVectorStored());
+          } catch (RuntimeException e) {
+            assertTrue(e.getMessage().startsWith(FailOnTermVectors.EXC_MSG));
+          }
+          if (random.nextInt(20) == 0) {
+            w.commit();
+            _TestUtil.checkIndex(dir);
+          }
+        }
+        document = new Document();
+        document.add(new Field("field", "a field", Field.Store.YES,
+            Field.Index.ANALYZED));
+        w.addDocument(document);
+        w.close();
+        IndexReader reader = IndexReader.open(dir);
+        assertTrue(reader.numDocs() > 0);
+        reader.close();
+        SegmentInfos sis = new SegmentInfos();
+        sis.read(dir);
+        for (SegmentInfo segmentInfo : sis) {
+          assertFalse(segmentInfo.getHasVectors());
+        }
+        dir.close();
+        
+      }
+    }
+  }
+  
+  private static class FailOnTermVectors extends MockDirectoryWrapper.Failure {
+
+    private static final String INIT_STAGE = "initTermVectorsWriter";
+    private static final String AFTER_INIT_STAGE = "finishDocument";
+    private static final String EXC_MSG = "FOTV";
+    private final String stage;
+    
+    public FailOnTermVectors(String stage) {
+      this.stage = stage;
+    }
+
+    @Override
+    public void eval(MockDirectoryWrapper dir)  throws IOException {
+      StackTraceElement[] trace = new Exception().getStackTrace();
+      boolean failOnInit = false;
+      boolean failOnfinish = false;
+      for (int i = 0; i < trace.length; i++) {
+        if ("org.apache.lucene.index.TermVectorsTermsWriter".equals(trace[i].getClassName())
&& stage.equals(trace[i].getMethodName()))
+          failOnInit = true;
+        if ("org.apache.lucene.index.TermVectorsTermsWriter".equals(trace[i].getClassName())
&& stage.equals(trace[i].getMethodName()))
+          failOnfinish = true;
+      }
+      
+      if (failOnInit) {
+        throw new RuntimeException(EXC_MSG + " fail on init");
+      } else if (failOnfinish) {
+        throw new RuntimeException(EXC_MSG + " fail on finishDoc");
+      }
+    }
+  }
 }

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1102476&r1=1102475&r2=1102476&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java Thu May
12 21:51:19 2011
@@ -73,15 +73,15 @@ public class TestSegmentMerger extends L
   }
 
   public void testMerge() throws IOException {
-    SegmentMerger merger = new SegmentMerger(mergedDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
mergedSegment, null, CodecProvider.getDefault(), null, new FieldInfos());
+    SegmentMerger merger = new SegmentMerger(mergedDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
mergedSegment, null, null, new FieldInfos());
     merger.add(reader1);
     merger.add(reader2);
     int docsMerged = merger.merge();
     assertTrue(docsMerged == 2);
     final FieldInfos fieldInfos = merger.fieldInfos();
     //Should be able to open a new SegmentReader against the new directory
-    SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment,
docsMerged, mergedDir, false, fieldInfos.hasProx(),
-                                                                                     merger.getSegmentCodecs(),
fieldInfos.hasVectors(), fieldInfos),
+    SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment,
docsMerged, mergedDir, false,
+                                                                                     merger.getSegmentCodecs(),
fieldInfos),
                                                    BufferedIndexInput.BUFFER_SIZE, true,
IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
     assertTrue(mergedReader != null);
     assertTrue(mergedReader.numDocs() == 2);



Mime
View raw message