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 2010/12/19 01:24:06 UTC

svn commit: r1050738 [2/3] - in /lucene/dev/branches/bulkpostings: ./ lucene/ lucene/contrib/ant/src/java/org/apache/lucene/ant/ lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/ lucene/contrib/wordnet/src/java/org/apache/lucene/wo...

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/IndexWriter.java Sun Dec 19 00:24:04 2010
@@ -17,35 +17,36 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.BufferedIndexInput;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.Constants;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.ThreadInterruptedException;
-import org.apache.lucene.util.Bits;
-
-import java.io.IOException;
-import java.io.Closeable;
-import java.io.PrintStream;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.List;
-import java.util.Collection;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Date;
 
 /**
   An <code>IndexWriter</code> creates and maintains an index.
@@ -346,7 +347,7 @@ public class IndexWriter implements Clos
     // just like we do when loading segments_N
     IndexReader r;
     synchronized(this) {
-      flush(false, true, true);
+      flush(false, true);
       r = new DirectoryReader(this, segmentInfos, config.getReaderTermsIndexDivisor(), codecs);
       if (infoStream != null) {
         message("return reader version=" + r.getVersion() + " reader=" + r);
@@ -773,7 +774,7 @@ public class IndexWriter implements Clos
       // KeepOnlyLastCommitDeleter:
       deleter = new IndexFileDeleter(directory,
                                      conf.getIndexDeletionPolicy(),
-                                     segmentInfos, infoStream, docWriter, codecs);
+                                     segmentInfos, infoStream, codecs);
 
       if (deleter.startingCommitDeleted) {
         // Deletion policy deleted the "head" commit point.
@@ -810,20 +811,41 @@ public class IndexWriter implements Clos
     }
   }
 
-  private FieldInfos getCurrentFieldInfos() throws IOException {
-    final FieldInfos fieldInfos;
-    if (segmentInfos.size() > 0) {
-      SegmentInfo info = segmentInfos.info(segmentInfos.size()-1);
-      Directory cfsDir;
+  private FieldInfos getFieldInfos(SegmentInfo info) throws IOException {
+    Directory cfsDir = null;
+    try {
       if (info.getUseCompoundFile()) {
         cfsDir = new CompoundFileReader(directory, IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
       } else {
         cfsDir = directory;
       }
-      fieldInfos = new FieldInfos(cfsDir, IndexFileNames.segmentFileName(info.name, "", IndexFileNames.FIELD_INFOS_EXTENSION));
-      if (info.getUseCompoundFile()) {
+      return new FieldInfos(cfsDir, IndexFileNames.segmentFileName(info.name, "", IndexFileNames.FIELD_INFOS_EXTENSION));
+    } finally {
+      if (info.getUseCompoundFile() && cfsDir != null) {
         cfsDir.close();
       }
+    }
+  }
+
+  private FieldInfos getCurrentFieldInfos() throws IOException {
+    final FieldInfos fieldInfos;
+    if (segmentInfos.size() > 0) {
+      if (segmentInfos.getFormat() > DefaultSegmentInfosWriter.FORMAT_4_0) {
+        // Pre-4.0 index.  In this case we sweep all
+        // segments, merging their FieldInfos:
+        fieldInfos = new FieldInfos();
+        for(SegmentInfo info : segmentInfos) {
+          final FieldInfos segFieldInfos = getFieldInfos(info);
+          final int fieldCount = segFieldInfos.size();
+          for(int fieldNumber=0;fieldNumber<fieldCount;fieldNumber++) {
+            fieldInfos.add(segFieldInfos.fieldInfo(fieldNumber));
+          }
+        }
+      } else {
+        // Already a 4.0 index; just seed the FieldInfos
+        // from the last segment
+        fieldInfos = getFieldInfos(segmentInfos.info(segmentInfos.size()-1));
+      }
     } else {
       fieldInfos = new FieldInfos();
     }
@@ -1037,7 +1059,7 @@ public class IndexWriter implements Clos
       // Only allow a new merge to be triggered if we are
       // going to wait for merges:
       if (!hitOOM) {
-        flush(waitForMerges, true, true);
+        flush(waitForMerges, true);
       }
 
       if (waitForMerges)
@@ -1111,7 +1133,7 @@ public class IndexWriter implements Clos
   public synchronized int maxDoc() {
     int count;
     if (docWriter != null)
-      count = docWriter.getNumDocsInRAM();
+      count = docWriter.getNumDocs();
     else
       count = 0;
 
@@ -1129,7 +1151,7 @@ public class IndexWriter implements Clos
   public synchronized int numDocs() throws IOException {
     int count;
     if (docWriter != null)
-      count = docWriter.getNumDocsInRAM();
+      count = docWriter.getNumDocs();
     else
       count = 0;
 
@@ -1244,27 +1266,11 @@ public class IndexWriter implements Clos
         doFlush = docWriter.updateDocument(doc, analyzer, null);
         success = true;
       } finally {
-        if (!success) {
-
-          if (infoStream != null) {
-            message("hit exception adding document");
-          }
-
-          synchronized (this) {
-            // If docWriter has some aborted files that were
-            // never incref'd, then we clean them up here
-            deleter.checkpoint(segmentInfos, false);
-            if (docWriter != null) {
-              final Collection<String> files = docWriter.abortedFiles();
-              if (files != null) {
-                deleter.deleteNewFiles(files);
-              }
-            }
-          }
-        }
+        if (!success && infoStream != null)
+          message("hit exception adding document");
       }
       if (doFlush)
-        flush(true, false, false);
+        flush(true, false);
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "addDocument");
     }
@@ -1285,7 +1291,7 @@ public class IndexWriter implements Clos
     ensureOpen();
     try {
       if (docWriter.deleteTerm(term, false)) {
-        flush(true, false, false);
+        flush(true, false);
       }
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Term)");
@@ -1309,7 +1315,7 @@ public class IndexWriter implements Clos
     ensureOpen();
     try {
       if (docWriter.deleteTerms(terms)) {
-        flush(true, false, false);
+        flush(true, false);
       }
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Term..)");
@@ -1331,7 +1337,7 @@ public class IndexWriter implements Clos
     ensureOpen();
     try {
       if (docWriter.deleteQuery(query)) {
-        flush(true, false, false);
+        flush(true, false);
       }
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Query)");
@@ -1355,7 +1361,7 @@ public class IndexWriter implements Clos
     ensureOpen();
     try {
       if (docWriter.deleteQueries(queries)) {
-        flush(true, false, false);
+        flush(true, false);
       }
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Query..)");
@@ -1412,26 +1418,11 @@ public class IndexWriter implements Clos
         doFlush = docWriter.updateDocument(doc, analyzer, term);
         success = true;
       } finally {
-        if (!success) {
-
-          if (infoStream != null) {
-            message("hit exception updating document");
-          }
-
-          synchronized (this) {
-            // If docWriter has some aborted files that were
-            // never incref'd, then we clean them up here
-            if (docWriter != null) {
-              final Collection<String> files = docWriter.abortedFiles();
-              if (files != null) {
-                deleter.deleteNewFiles(files);
-              }
-            }
-          }
-        }
+        if (!success && infoStream != null)
+          message("hit exception updating document");
       }
       if (doFlush) {
-        flush(true, false, false);
+        flush(true, false);
       }
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "updateDocument");
@@ -1445,7 +1436,7 @@ public class IndexWriter implements Clos
 
   // for test purpose
   final synchronized int getNumBufferedDocuments(){
-    return docWriter.getNumDocsInRAM();
+    return docWriter.getNumDocs();
   }
 
   // for test purpose
@@ -1601,7 +1592,7 @@ public class IndexWriter implements Clos
       message("now flush at optimize");
     }
 
-    flush(true, false, true);
+    flush(true, true);
 
     synchronized(this) {
       resetMergeExceptions();
@@ -2141,7 +2132,7 @@ public class IndexWriter implements Clos
     try {
       if (infoStream != null)
         message("flush at addIndexes(Directory...)");
-      flush(false, false, true);
+      flush(false, true);
 
       int docCount = 0;
       List<SegmentInfo> infos = new ArrayList<SegmentInfo>();
@@ -2151,7 +2142,8 @@ public class IndexWriter implements Clos
         }
         SegmentInfos sis = new SegmentInfos(codecs); // read infos from dir
         sis.read(dir, codecs);
-        Map<String, String> dsNames = new HashMap<String, String>();
+        final Set<String> dsFilesCopied = new HashSet<String>();
+        final Map<String, String> dsNames = new HashMap<String, String>();
         for (SegmentInfo info : sis) {
           assert !infos.contains(info): "dup info dir=" + info.dir + " name=" + info.name;
 
@@ -2160,7 +2152,7 @@ public class IndexWriter implements Clos
           String dsName = info.getDocStoreSegment();
 
           if (infoStream != null) {
-            message("addIndexes: process segment origName=" + info.name + " newName=" + newSegName + " dsName=" + dsName);
+            message("addIndexes: process segment origName=" + info.name + " newName=" + newSegName + " dsName=" + dsName + " info=" + info);
           }
 
           // Determine if the doc store of this segment needs to be copied. It's
@@ -2170,22 +2162,32 @@ public class IndexWriter implements Clos
           // NOTE: pre-3x segments include a null DSName if they don't share doc
           // store. So the following code ensures we don't accidentally insert
           // 'null' to the map.
-          String newDsName = newSegName;
-          boolean docStoreCopied = false;
-          if (dsNames.containsKey(dsName)) {
-            newDsName = dsNames.get(dsName);
-            docStoreCopied = true;
-          } else if (dsName != null) {
-            dsNames.put(dsName, newSegName);
-            docStoreCopied = false;
+          final String newDsName;
+          if (dsName != null) {
+            if (dsNames.containsKey(dsName)) {
+              newDsName = dsNames.get(dsName);
+            } else {
+              dsNames.put(dsName, newSegName);
+              newDsName = newSegName;
+            }
+          } else {
+            newDsName = newSegName;
           }
 
           // Copy the segment files
-          for (String file : info.files()) {
-            if (docStoreCopied && IndexFileNames.isDocStoreFile(file)) {
-              continue;
-            } 
-            dir.copy(directory, file, newSegName + IndexFileNames.stripSegmentName(file));
+          for (String file: info.files()) {
+            final String newFileName;
+            if (IndexFileNames.isDocStoreFile(file)) {
+              newFileName = newDsName + IndexFileNames.stripSegmentName(file);
+              if (dsFilesCopied.contains(newFileName)) {
+                continue;
+              }
+              dsFilesCopied.add(newFileName);
+            } else {
+              newFileName = newSegName + IndexFileNames.stripSegmentName(file);
+            }
+            assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
+            dir.copy(directory, file, newFileName);
           }
 
           // Update SI appropriately
@@ -2233,7 +2235,8 @@ public class IndexWriter implements Clos
     try {
       String mergedName = newSegmentName();
       SegmentMerger merger = new SegmentMerger(directory, termIndexInterval,
-          mergedName, null, codecs, payloadProcessorProvider);
+                                               mergedName, null, codecs, payloadProcessorProvider,
+                                               ((FieldInfos) docWriter.getFieldInfos().clone()));
       
       for (IndexReader reader : readers)      // add new indexes
         merger.add(reader);
@@ -2241,8 +2244,8 @@ public class IndexWriter implements Clos
       int docCount = merger.merge();                // merge 'em
       
       SegmentInfo info = new SegmentInfo(mergedName, docCount, directory,
-                                         false, -1, null, false, merger.hasProx(), merger.getSegmentCodecs(),
-                                         merger.hasVectors());
+                                         false, merger.fieldInfos().hasProx(), merger.getSegmentCodecs(),
+                                         merger.fieldInfos().hasVectors());
       setDiagnostics(info, "addIndexes(IndexReader...)");
 
       boolean useCompoundFile;
@@ -2337,7 +2340,7 @@ public class IndexWriter implements Clos
     if (infoStream != null)
       message("prepareCommit: flush");
 
-    flush(true, true, true);
+    flush(true, true);
 
     startCommit(commitUserData);
   }
@@ -2454,12 +2457,9 @@ public class IndexWriter implements Clos
    * to the Directory.
    * @param triggerMerge if true, we may merge segments (if
    *  deletes or docs were flushed) if necessary
-   * @param flushDocStores if false we are allowed to keep
-   *  doc stores open to share with the next segment
    * @param flushDeletes whether pending deletes should also
-   *  be flushed
    */
-  protected final void flush(boolean triggerMerge, boolean flushDocStores, boolean flushDeletes) throws CorruptIndexException, IOException {
+  protected final void flush(boolean triggerMerge, boolean flushDeletes) throws CorruptIndexException, IOException {
 
     // NOTE: this method cannot be sync'd because
     // maybeMerge() in turn calls mergeScheduler.merge which
@@ -2470,7 +2470,7 @@ public class IndexWriter implements Clos
 
     // We can be called during close, when closing==true, so we must pass false to ensureOpen:
     ensureOpen(false);
-    if (doFlush(flushDocStores, flushDeletes) && triggerMerge) {
+    if (doFlush(flushDeletes) && triggerMerge) {
       maybeMerge();
     }
   }
@@ -2478,7 +2478,7 @@ public class IndexWriter implements Clos
   // TODO: this method should not have to be entirely
   // synchronized, ie, merges should be allowed to commit
   // even while a flush is happening
-  private synchronized final boolean doFlush(boolean closeDocStores, boolean applyAllDeletes) throws CorruptIndexException, IOException {
+  private synchronized boolean doFlush(boolean applyAllDeletes) throws CorruptIndexException, IOException {
 
     if (hitOOM) {
       throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot flush");
@@ -2501,11 +2501,11 @@ public class IndexWriter implements Clos
     try {
 
       if (infoStream != null) {
-        message("  start flush: applyAllDeletes=" + applyAllDeletes + " closeDocStores=" + closeDocStores);
+        message("  start flush: applyAllDeletes=" + applyAllDeletes);
         message("  index before flush " + segString());
       }
     
-      final SegmentInfo newSegment = docWriter.flush(this, closeDocStores, deleter, mergePolicy, segmentInfos);
+      final SegmentInfo newSegment = docWriter.flush(this, deleter, mergePolicy, segmentInfos);
       if (newSegment != null) {
         setDiagnostics(newSegment, "flush");
         segmentInfos.add(newSegment);
@@ -2553,17 +2553,8 @@ public class IndexWriter implements Clos
       return false;
     } finally {
       flushControl.clearFlushPending();
-      if (!success) {
-        if (infoStream != null) {
-          message("hit exception during flush");
-        }
-        if (docWriter != null) {
-          final Collection<String> files = docWriter.abortedFiles();
-          if (files != null) {
-            deleter.deleteNewFiles(files);
-          }
-        }
-      }
+      if (!success && infoStream != null)
+        message("hit exception during flush");
     }
   }
 
@@ -2579,7 +2570,7 @@ public class IndexWriter implements Clos
    *  buffered in RAM. */
   public final synchronized int numRamDocs() {
     ensureOpen();
-    return docWriter.getNumDocsInRAM();
+    return docWriter.getNumDocs();
   }
 
   private int ensureContiguousMerge(MergePolicy.OneMerge merge) {
@@ -2684,7 +2675,7 @@ public class IndexWriter implements Clos
   }
 
   /* FIXME if we want to support non-contiguous segment merges */
-  synchronized private boolean commitMerge(MergePolicy.OneMerge merge, SegmentMerger merger, SegmentReader mergedReader) throws IOException {
+  synchronized private boolean commitMerge(MergePolicy.OneMerge merge, SegmentReader mergedReader) throws IOException {
 
     assert testPoint("startCommitMerge");
 
@@ -2719,8 +2710,6 @@ public class IndexWriter implements Clos
     // format as well:
     setMergeDocStoreIsCompoundFile(merge);
 
-    merge.info.setHasProx(merger.hasProx());
-
     segmentInfos.subList(start, start + merge.segments.size()).clear();
     assert !segmentInfos.contains(merge.info);
     segmentInfos.add(start, merge.info);
@@ -2916,7 +2905,7 @@ public class IndexWriter implements Clos
     }
   }
 
-  final synchronized private void _mergeInit(MergePolicy.OneMerge merge) throws IOException {
+  synchronized private void _mergeInit(MergePolicy.OneMerge merge) throws IOException {
 
     assert testPoint("startMergeInit");
 
@@ -2933,134 +2922,26 @@ public class IndexWriter implements Clos
 
     if (merge.isAborted())
       return;
-    
-    final SegmentInfos sourceSegments = merge.segments;
-    final int end = sourceSegments.size();
-
-    // Check whether this merge will allow us to skip
-    // merging the doc stores (stored field & vectors).
-    // This is a very substantial optimization (saves tons
-    // of IO).
-
-    Directory lastDir = directory;
-    String lastDocStoreSegment = null;
-    int next = -1;
 
-    boolean mergeDocStores = false;
-    boolean doFlushDocStore = false;
     boolean hasVectors = false;
-    final String currentDocStoreSegment = docWriter.getDocStoreSegment();
-
-    // Test each segment to be merged: check if we need to
-    // flush/merge doc stores
-    for (int i = 0; i < end; i++) {
-      SegmentInfo si = sourceSegments.info(i);
-
-      // If it has deletions we must merge the doc stores
-      if (si.hasDeletions())
-        mergeDocStores = true;
-
-      if (si.getHasVectors()) {
+    for (SegmentInfo sourceSegment : merge.segments) {
+      if (sourceSegment.getHasVectors()) {
         hasVectors = true;
       }
-
-      // If it has its own (private) doc stores we must
-      // merge the doc stores
-      if (-1 == si.getDocStoreOffset())
-        mergeDocStores = true;
-
-      // If it has a different doc store segment than
-      // previous segments, we must merge the doc stores
-      String docStoreSegment = si.getDocStoreSegment();
-      if (docStoreSegment == null)
-        mergeDocStores = true;
-      else if (lastDocStoreSegment == null)
-        lastDocStoreSegment = docStoreSegment;
-      else if (!lastDocStoreSegment.equals(docStoreSegment))
-        mergeDocStores = true;
-
-      // Segments' docScoreOffsets must be in-order,
-      // contiguous.  For the default merge policy now
-      // this will always be the case but for an arbitrary
-      // merge policy this may not be the case
-      if (-1 == next)
-        next = si.getDocStoreOffset() + si.docCount;
-      else if (next != si.getDocStoreOffset())
-        mergeDocStores = true;
-      else
-        next = si.getDocStoreOffset() + si.docCount;
-      
-      // If the segment comes from a different directory
-      // we must merge
-      if (lastDir != si.dir)
-        mergeDocStores = true;
-
-      // If the segment is referencing the current "live"
-      // doc store outputs then we must merge
-      if (si.getDocStoreOffset() != -1 && currentDocStoreSegment != null && si.getDocStoreSegment().equals(currentDocStoreSegment)) {
-        doFlushDocStore = true;
-      }
-    }
-
-    // if a mergedSegmentWarmer is installed, we must merge
-    // the doc stores because we will open a full
-    // SegmentReader on the merged segment:
-    if (!mergeDocStores && mergedSegmentWarmer != null && currentDocStoreSegment != null && lastDocStoreSegment != null && lastDocStoreSegment.equals(currentDocStoreSegment)) {
-      mergeDocStores = true;
-    }
-
-    final int docStoreOffset;
-    final String docStoreSegment;
-    final boolean docStoreIsCompoundFile;
-
-    if (mergeDocStores) {
-      docStoreOffset = -1;
-      docStoreSegment = null;
-      docStoreIsCompoundFile = false;
-    } else {
-      SegmentInfo si = sourceSegments.info(0);        
-      docStoreOffset = si.getDocStoreOffset();
-      docStoreSegment = si.getDocStoreSegment();
-      docStoreIsCompoundFile = si.getDocStoreIsCompoundFile();
-    }
-
-    if (mergeDocStores && doFlushDocStore) {
-      // SegmentMerger intends to merge the doc stores
-      // (stored fields, vectors), and at least one of the
-      // segments to be merged refers to the currently
-      // live doc stores.
-
-      // TODO: if we know we are about to merge away these
-      // newly flushed doc store files then we should not
-      // make compound file out of them...
-      if (infoStream != null)
-        message("now flush at merge");
-      doFlush(true, false);
-      updatePendingMerges(1, false);
     }
 
-    merge.hasVectors = hasVectors;
-    merge.mergeDocStores = mergeDocStores;
-
     // Bind a new segment name here so even with
     // ConcurrentMergePolicy we keep deterministic segment
     // names.
-    merge.info = new SegmentInfo(newSegmentName(), 0,
-                                 directory, false, docStoreOffset,
-                                 docStoreSegment,
-                                 docStoreIsCompoundFile,
-                                 false,
-                                 null,
-                                 false);
+    merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, false, null, hasVectors);
 
     Map<String,String> details = new HashMap<String,String>();
     details.put("optimize", Boolean.toString(merge.optimize));
-    details.put("mergeFactor", Integer.toString(end));
-    details.put("mergeDocStores", Boolean.toString(mergeDocStores));
+    details.put("mergeFactor", Integer.toString(merge.segments.size()));
     setDiagnostics(merge.info, "merge", details);
 
     if (infoStream != null) {
-      message("merge seg=" + merge.info.name + " mergeDocStores=" + mergeDocStores);
+      message("merge seg=" + merge.info.name);
     }
 
     // Also enroll the merged segment into mergingSegments;
@@ -3184,8 +3065,6 @@ public class IndexWriter implements Clos
 
     final String mergedName = merge.info.name;
     
-    SegmentMerger merger = null;
-
     int mergedDocCount = 0;
 
     SegmentInfos sourceSegments = merge.segments;
@@ -3194,20 +3073,13 @@ public class IndexWriter implements Clos
     if (infoStream != null)
       message("merging " + merge.segString(directory));
 
-    merger = new SegmentMerger(directory, termIndexInterval, mergedName, merge, codecs, payloadProcessorProvider);
+    SegmentMerger merger = new SegmentMerger(directory, termIndexInterval, mergedName, merge,
+                                             codecs, payloadProcessorProvider,
+                                             ((FieldInfos) docWriter.getFieldInfos().clone()));
 
     merge.readers = new SegmentReader[numSegments];
     merge.readersClone = new SegmentReader[numSegments];
 
-    boolean mergeDocStores = false;
-
-    final String currentDocStoreSegment;
-    synchronized(this) {
-      currentDocStoreSegment = docWriter.getDocStoreSegment();
-    }
-
-    boolean currentDSSMerged = false;
-      
     // This is try/finally to make sure merger's readers are
     // closed:
     boolean success = false;
@@ -3219,7 +3091,7 @@ public class IndexWriter implements Clos
 
         // Hold onto the "live" reader; we will use this to
         // commit merged deletes
-        SegmentReader reader = merge.readers[i] = readerPool.get(info, merge.mergeDocStores,
+        SegmentReader reader = merge.readers[i] = readerPool.get(info, true,
                                                                  MERGE_READ_BUFFER_SIZE,
                                                                  -config.getReaderTermsIndexDivisor());
 
@@ -3229,14 +3101,6 @@ public class IndexWriter implements Clos
         SegmentReader clone = merge.readersClone[i] = (SegmentReader) reader.clone(true);
         merger.add(clone);
 
-        if (clone.hasDeletions()) {
-          mergeDocStores = true;
-        }
-        
-        if (info.getDocStoreOffset() != -1 && currentDocStoreSegment != null) {
-          currentDSSMerged |= currentDocStoreSegment.equals(info.getDocStoreSegment());
-        }
-
         totDocCount += clone.numDocs();
       }
 
@@ -3246,46 +3110,15 @@ public class IndexWriter implements Clos
 
       merge.checkAborted(directory);
 
-      // If deletions have arrived and it has now become
-      // necessary to merge doc stores, go and open them:
-      if (mergeDocStores && !merge.mergeDocStores) {
-        merge.mergeDocStores = true;
-        synchronized(this) {
-
-          // If 1) we must now merge doc stores, and 2) at
-          // least one of the segments we are merging uses
-          // the doc store we are now writing to, we must at
-          // this point force this doc store closed (by
-          // calling flush).  If we didn't do this then the
-          // readers will attempt to open an IndexInput
-          // on files that have still-open IndexOutputs
-          // against them:
-          if (currentDSSMerged) {
-            if (infoStream != null) {
-              message("now flush at mergeMiddle");
-            }
-            doFlush(true, false);
-            updatePendingMerges(1, false);
-          }
-        }
-
-        for(int i=0;i<numSegments;i++) {
-          merge.readersClone[i].openDocStores();
-        }
-
-        // Clear DSS
-        merge.info.setDocStore(-1, null, false);
-      }
-
       // This is where all the work happens:
-      mergedDocCount = merge.info.docCount = merger.merge(merge.mergeDocStores);
+      mergedDocCount = merge.info.docCount = merger.merge();
 
       // Record which codec was used to write the segment
       merge.info.setSegmentCodecs(merger.getSegmentCodecs());
-      merge.info.setHasVectors(merger.hasVectors() || merge.hasVectors);
 
       if (infoStream != null) {
         message("merge segmentCodecs=" + merger.getSegmentCodecs());
+        message("merge store matchedCount=" + merger.getMatchedSubReaderCount() + " vs " + numSegments);
       }
       
       assert mergedDocCount == totDocCount;
@@ -3294,7 +3127,7 @@ 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.hasProx());
+      merge.info.setHasProx(merger.fieldInfos().hasProx());
 
       boolean useCompoundFile;
       synchronized (this) { // Guard segmentInfos
@@ -3359,12 +3192,7 @@ public class IndexWriter implements Clos
       final int termsIndexDivisor;
       final boolean loadDocStores;
 
-      // if the merged segment warmer was not installed when
-      // this merge was started, causing us to not force
-      // the docStores to close, we can't warm it now
-      final boolean canWarm = merge.info.getDocStoreSegment() == null || currentDocStoreSegment == null || !merge.info.getDocStoreSegment().equals(currentDocStoreSegment);
-
-      if (poolReaders && mergedSegmentWarmer != null && canWarm) {
+      if (poolReaders && mergedSegmentWarmer != null) {
         // Load terms index & doc stores so the segment
         // warmer can run searches, load documents/term
         // vectors
@@ -3385,7 +3213,7 @@ public class IndexWriter implements Clos
           mergedSegmentWarmer.warm(mergedReader);
         }
 
-        if (!commitMerge(merge, merger, mergedReader)) {
+        if (!commitMerge(merge, mergedReader)) {
           // commitMerge will return false if this merge was aborted
           return 0;
         }
@@ -3516,7 +3344,7 @@ public class IndexWriter implements Clos
         }
         
         // First, we clone & incref the segmentInfos we intend
-        // to sync, then, without locking, we sync() each file
+        // to sync, then, without locking, we sync() all files
         // referenced by toSync, in the background.
         
         if (infoStream != null)
@@ -3524,25 +3352,7 @@ public class IndexWriter implements Clos
 
         readerPool.commit();
         
-        // It's possible another flush (that did not close
-        // the open do stores) snuck in after the flush we
-        // just did, so we remove any tail segments
-        // referencing the open doc store from the
-        // SegmentInfos we are about to sync (the main
-        // SegmentInfos will keep them):
         toSync = (SegmentInfos) segmentInfos.clone();
-        final String dss = docWriter.getDocStoreSegment();
-        if (dss != null) {
-          while(true) {
-            final String dss2 = toSync.info(toSync.size()-1).getDocStoreSegment();
-            if (dss2 == null || !dss2.equals(dss)) {
-              break;
-            }
-            toSync.remove(toSync.size()-1);
-            changeCount++;
-            segmentInfos.changed();
-          }
-        }
         assert filesExist(toSync);
         
         if (commitUserData != null)

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java Sun Dec 19 00:24:04 2010
@@ -32,9 +32,6 @@ abstract class InvertedDocConsumer {
   /** Flush a new segment */
   abstract void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
 
-  /** Close doc stores */
-  abstract void closeDocStore(SegmentWriteState state) throws IOException;
-
   /** Attempt to free RAM, returning true if any RAM was
    *  freed */
   abstract boolean freeRAM();

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java Sun Dec 19 00:24:04 2010
@@ -24,7 +24,6 @@ import java.io.IOException;
 abstract class InvertedDocEndConsumer {
   abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
   abstract void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
-  abstract void closeDocStore(SegmentWriteState state) throws IOException;
   abstract void abort();
   abstract void setFieldInfos(FieldInfos fieldInfos);
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java Sun Dec 19 00:24:04 2010
@@ -70,7 +70,6 @@ public abstract class LogMergePolicy ext
   protected boolean calibrateSizeByDeletes = true;
   
   protected boolean useCompoundFile = true;
-  protected boolean useCompoundDocStore = true;
 
   public LogMergePolicy() {
     super();
@@ -157,27 +156,6 @@ public abstract class LogMergePolicy ext
     return useCompoundFile;
   }
 
-  // Javadoc inherited
-  @Override
-  public boolean useCompoundDocStore(SegmentInfos infos) {
-    return useCompoundDocStore;
-  }
-
-  /** Sets whether compound file format should be used for
-   *  newly flushed and newly merged doc store
-   *  segment files (term vectors and stored fields). */
-  public void setUseCompoundDocStore(boolean useCompoundDocStore) {
-    this.useCompoundDocStore = useCompoundDocStore;
-  }
-
-  /** Returns true if newly flushed and newly merge doc
-   *  store segment files (term vectors and stored fields)
-   *  are written in compound file format. @see
-   *  #setUseCompoundDocStore */
-  public boolean getUseCompoundDocStore() {
-    return useCompoundDocStore;
-  }
-
   /** Sets whether the segment size should be calibrated by
    *  the number of deletes when choosing segments for merge. */
   public void setCalibrateSizeByDeletes(boolean calibrateSizeByDeletes) {
@@ -591,8 +569,7 @@ public abstract class LogMergePolicy ext
     sb.append("maxMergeSize=").append(maxMergeSize).append(", ");
     sb.append("calibrateSizeByDeletes=").append(calibrateSizeByDeletes).append(", ");
     sb.append("maxMergeDocs=").append(maxMergeDocs).append(", ");
-    sb.append("useCompoundFile=").append(useCompoundFile).append(", ");
-    sb.append("useCompoundDocStore=").append(useCompoundDocStore);
+    sb.append("useCompoundFile=").append(useCompoundFile);
     sb.append("]");
     return sb.toString();
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MergePolicy.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MergePolicy.java Sun Dec 19 00:24:04 2010
@@ -67,8 +67,6 @@ public abstract class MergePolicy implem
   public static class OneMerge {
 
     SegmentInfo info;               // used by IndexWriter
-    boolean mergeDocStores;         // used by IndexWriter
-    boolean hasVectors;             // used by IndexWriter
     boolean optimize;               // used by IndexWriter
     boolean registerDone;           // used by IndexWriter
     long mergeGen;                  // used by IndexWriter
@@ -154,9 +152,6 @@ public abstract class MergePolicy implem
         b.append(" into ").append(info.name);
       if (optimize)
         b.append(" [optimize]");
-      if (mergeDocStores) {
-        b.append(" [mergeDocStores]");
-      }
       if (aborted) {
         b.append(" [ABORTED]");
       }
@@ -319,10 +314,4 @@ public abstract class MergePolicy implem
    * Returns true if a new segment (regardless of its origin) should use the compound file format.
    */
   public abstract boolean useCompoundFile(SegmentInfos segments, SegmentInfo newSegment) throws IOException;
-
-  /**
-   * Returns true if the doc store files should use the
-   * compound file format.
-   */
-  public abstract boolean useCompoundDocStore(SegmentInfos segments);
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFields.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFields.java Sun Dec 19 00:24:04 2010
@@ -19,9 +19,10 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.HashMap;
 import java.util.List;
 import java.util.ArrayList;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.ReaderUtil.Gather;  // for javadocs
 import org.apache.lucene.util.Bits;
@@ -45,7 +46,7 @@ import org.apache.lucene.util.BytesRef;
 public final class MultiFields extends Fields {
   private final Fields[] subs;
   private final ReaderUtil.Slice[] subSlices;
-  private final Map<String,Terms> terms = new HashMap<String,Terms>();
+  private final Map<String,Terms> terms = new ConcurrentHashMap<String,Terms>();
 
   /** Returns a single {@link Fields} instance for this
    *  reader, merging fields/terms/docs/positions on the
@@ -253,32 +254,32 @@ public final class MultiFields extends F
   @Override
   public Terms terms(String field) throws IOException {
 
-    final Terms result;
+    Terms result = terms.get(field);
+    if (result != null)
+      return result;
 
-    if (!terms.containsKey(field)) {
 
-      // Lazy init: first time this field is requested, we
-      // create & add to terms:
-      final List<Terms> subs2 = new ArrayList<Terms>();
-      final List<ReaderUtil.Slice> slices2 = new ArrayList<ReaderUtil.Slice>();
-
-      // Gather all sub-readers that share this field
-      for(int i=0;i<subs.length;i++) {
-        final Terms terms = subs[i].terms(field);
-        if (terms != null) {
-          subs2.add(terms);
-          slices2.add(subSlices[i]);
-        }
-      }
-      if (subs2.size() == 0) {
-        result = null;
-      } else {
-        result = new MultiTerms(subs2.toArray(Terms.EMPTY_ARRAY),
-                                slices2.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
+    // Lazy init: first time this field is requested, we
+    // create & add to terms:
+    final List<Terms> subs2 = new ArrayList<Terms>();
+    final List<ReaderUtil.Slice> slices2 = new ArrayList<ReaderUtil.Slice>();
+
+    // Gather all sub-readers that share this field
+    for(int i=0;i<subs.length;i++) {
+      final Terms terms = subs[i].terms(field);
+      if (terms != null) {
+        subs2.add(terms);
+        slices2.add(subSlices[i]);
       }
-      terms.put(field, result);
+    }
+    if (subs2.size() == 0) {
+      result = null;
+      // don't cache this case with an unbounded cache, since the number of fields that don't exist
+      // is unbounded.
     } else {
-      result = terms.get(field);
+      result = new MultiTerms(subs2.toArray(Terms.EMPTY_ARRAY),
+          slices2.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
+      terms.put(field, result);
     }
 
     return result;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/NoMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/NoMergePolicy.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/NoMergePolicy.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/NoMergePolicy.java Sun Dec 19 00:24:04 2010
@@ -67,9 +67,6 @@ public final class NoMergePolicy extends
       throws CorruptIndexException, IOException { return null; }
 
   @Override
-  public boolean useCompoundDocStore(SegmentInfos segments) { return useCompoundFile; }
-
-  @Override
   public boolean useCompoundFile(SegmentInfos segments, SegmentInfo newSegment) { return useCompoundFile; }
 
   @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/NormsWriter.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/NormsWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/NormsWriter.java Sun Dec 19 00:24:04 2010
@@ -171,7 +171,4 @@ final class NormsWriter extends Inverted
       normsOut.close();
     }
   }
-
-  @Override
-  void closeDocStore(SegmentWriteState state) {}
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Sun Dec 19 00:24:04 2010
@@ -80,28 +80,24 @@ public final class SegmentInfo {
 
   private boolean hasProx;                        // True if this segment has any fields with omitTermFreqAndPositions==false
 
-  private byte hasVectors;                        // 0 if no; 1 if yes; 2 if must-check-filesystem (old index)
+  private boolean hasVectors;                     // True if this segment wrote term vectors
 
   private SegmentCodecs segmentCodecs;
 
   private Map<String,String> diagnostics;
 
-  public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, int docStoreOffset, 
-                     String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx, SegmentCodecs segmentCodecs,
-                     boolean hasVectors) { 
+  public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
+                     boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors) {
     this.name = name;
     this.docCount = docCount;
     this.dir = dir;
     delGen = NO;
     this.isCompoundFile = isCompoundFile;
-    this.docStoreOffset = docStoreOffset;
-    this.docStoreSegment = docStoreSegment;
-    this.docStoreIsCompoundFile = docStoreIsCompoundFile;
+    this.docStoreOffset = -1;
     this.hasProx = hasProx;
     this.segmentCodecs = segmentCodecs;
-    this.hasVectors = (byte) (hasVectors ? 1 : 0);
+    this.hasVectors = hasVectors;
     delCount = 0;
-    assert docStoreOffset == -1 || docStoreSegment != null: "dso=" + docStoreOffset + " dss=" + docStoreSegment + " docCount=" + docCount;
   }
 
   /**
@@ -116,6 +112,7 @@ public final class SegmentInfo {
     docStoreOffset = src.docStoreOffset;
     docStoreIsCompoundFile = src.docStoreIsCompoundFile;
     hasVectors = src.hasVectors;
+    hasProx = src.hasProx;
     if (src.normGen == null) {
       normGen = null;
     } else {
@@ -191,9 +188,33 @@ public final class SegmentInfo {
     diagnostics = input.readStringStringMap();
     
     if (format <= DefaultSegmentInfosWriter.FORMAT_HAS_VECTORS) {
-      hasVectors = input.readByte();
+      hasVectors = input.readByte() == 1;
     } else {
-      hasVectors = 2;
+      final String storesSegment;
+      final String ext;
+      final boolean isCompoundFile;
+      if (docStoreOffset != -1) {
+        storesSegment = docStoreSegment;
+        isCompoundFile = docStoreIsCompoundFile;
+        ext = IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
+      } else {
+        storesSegment = name;
+        isCompoundFile = getUseCompoundFile();
+        ext = IndexFileNames.COMPOUND_FILE_EXTENSION;
+      }
+      final Directory dirToTest;
+      if (isCompoundFile) {
+        dirToTest = new CompoundFileReader(dir, IndexFileNames.segmentFileName(storesSegment, "", ext));
+      } else {
+        dirToTest = dir;
+      }
+      try {
+        hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+      } finally {
+        if (isCompoundFile) {
+          dirToTest.close();
+        }
+      }
     }
   }
   
@@ -216,23 +237,11 @@ public final class SegmentInfo {
   }
 
   public boolean getHasVectors() throws IOException {
-    if (hasVectors == 1) {
-      return true;
-    } else if (hasVectors == 0) {
-      return false;
-    } else {
-      final String storesSegment;
-      if (getDocStoreOffset() != -1) {
-        storesSegment = getDocStoreSegment();
-      } else {
-        storesSegment = name;
-      }
-      return dir.fileExists(IndexFileNames.segmentFileName(storesSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
-    }
+    return hasVectors;
   }
 
   public void setHasVectors(boolean v) {
-    hasVectors = (byte) (v ? 1 : 0);
+    hasVectors = v;
     clearFiles();
   }
 
@@ -261,7 +270,10 @@ public final class SegmentInfo {
 
   @Override
   public Object clone() {
-    SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, hasProx, segmentCodecs, false);
+    SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, hasProx, segmentCodecs, false);
+    si.docStoreOffset = docStoreOffset;
+    si.docStoreSegment = docStoreSegment;
+    si.docStoreIsCompoundFile = docStoreIsCompoundFile;
     si.delGen = delGen;
     si.delCount = delCount;
     si.diagnostics = new HashMap<String, String>(diagnostics);
@@ -432,7 +444,7 @@ public final class SegmentInfo {
     output.writeByte((byte) (hasProx ? 1:0));
     segmentCodecs.write(output);
     output.writeStringStringMap(diagnostics);
-    output.writeByte(hasVectors);
+    output.writeByte((byte) (hasVectors ? 1 : 0));
   }
 
   void setHasProx(boolean hasProx) {
@@ -497,27 +509,19 @@ public final class SegmentInfo {
       } else {
         fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
         fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION));
-        if (hasVectors == 1) {
+        if (hasVectors) {
           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));
-        } else if (hasVectors == 2) {
-          addIfExists(fileSet, IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
-          addIfExists(fileSet, IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
-          addIfExists(fileSet, IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
-        }      
+        }
       }
     } else if (!useCompoundFile) {
       fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
       fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_EXTENSION));
-      if (hasVectors == 1) {
+      if (hasVectors) {
         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));
-      } else if (hasVectors == 2) {
-        addIfExists(fileSet, IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
-        addIfExists(fileSet, IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
-        addIfExists(fileSet, IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
       }      
     }
 
@@ -576,6 +580,9 @@ public final class SegmentInfo {
     if (this.dir != dir) {
       s.append('x');
     }
+    if (hasVectors) {
+      s.append('v');
+    }
     s.append(docCount);
 
     int delCount = getDelCount() + pendingDelCount;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Sun Dec 19 00:24:04 2010
@@ -74,6 +74,8 @@ public final class SegmentInfos extends 
   
   private CodecProvider codecs;
 
+  private int format;
+
   /**
    * If non-null, information about loading segments_N files
    * will be printed here.  @see #setInfoStream.
@@ -88,6 +90,14 @@ public final class SegmentInfos extends 
     this.codecs = codecs;
   }
 
+  public void setFormat(int format) {
+    this.format = format;
+  }
+
+  public int getFormat() {
+    return format;
+  }
+
   public final SegmentInfo info(int i) {
     return get(i);
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Sun Dec 19 00:24:04 2010
@@ -56,18 +56,12 @@ final class SegmentMerger {
   private int termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
 
   private List<IndexReader> readers = new ArrayList<IndexReader>();
-  private FieldInfos fieldInfos;
+  private final FieldInfos fieldInfos;
   
   private int mergedDocs;
 
   private final CheckAbort checkAbort;
 
-  // Whether we should merge doc stores (stored fields and
-  // vectors files).  When all segments we are merging
-  // already share the same doc store files, we don't need
-  // to merge the doc stores.
-  private boolean mergeDocStores;
-
   /** Maximum number of contiguous documents to bulk-copy
       when merging stored fields */
   private final static int MAX_RAW_MERGE_DOCS = 4192;
@@ -75,14 +69,14 @@ final class SegmentMerger {
   private final CodecProvider codecs;
   private Codec codec;
   private SegmentWriteState segmentWriteState;
-  private boolean hasVectors;
 
   private PayloadProcessorProvider payloadProcessorProvider;
   
-  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, CodecProvider codecs, PayloadProcessorProvider payloadProcessorProvider) {
+  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, CodecProvider codecs, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos) {
     this.payloadProcessorProvider = payloadProcessorProvider;
     directory = dir;
     this.codecs = codecs;
+    this.fieldInfos = fieldInfos;
     segment = name;
     if (merge != null) {
       checkAbort = new CheckAbort(merge, directory);
@@ -96,13 +90,9 @@ final class SegmentMerger {
     }
     this.termIndexInterval = termIndexInterval;
   }
-  
-  boolean hasProx() {
-    return fieldInfos.hasProx();
-  }
 
-  boolean hasVectors() {
-    return hasVectors;
+  public FieldInfos fieldInfos() {
+    return fieldInfos;
   }
 
   /**
@@ -120,22 +110,6 @@ final class SegmentMerger {
    * @throws IOException if there is a low-level IO error
    */
   final int merge() throws CorruptIndexException, IOException {
-    return merge(true);
-  }
-
-  /**
-   * Merges the readers specified by the {@link #add} method
-   * into the directory passed to the constructor.
-   * @param mergeDocStores if false, we will not merge the
-   * stored fields nor vectors files
-   * @return The number of documents that were merged
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException if there is a low-level IO error
-   */
-  final int merge(boolean mergeDocStores) throws CorruptIndexException, IOException {
-
-    this.mergeDocStores = mergeDocStores;
-    
     // NOTE: it's important to add calls to
     // checkAbort.work(...) if you make any changes to this
     // method that will spend alot of time.  The frequency
@@ -147,7 +121,7 @@ final class SegmentMerger {
     mergeTerms();
     mergeNorms();
 
-    if (mergeDocStores && fieldInfos.hasVectors())
+    if (fieldInfos.hasVectors())
       mergeVectors();
 
     return mergedDocs;
@@ -158,9 +132,7 @@ final class SegmentMerger {
 
     // Basic files
     for (String ext : IndexFileNames.COMPOUND_EXTENSIONS_NOT_CODEC) {
-      if (mergeDocStores || (!ext.equals(IndexFileNames.FIELDS_EXTENSION) &&
-                             !ext.equals(IndexFileNames.FIELDS_INDEX_EXTENSION)))
-        fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
+      fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
     }
 
     segmentWriteState.segmentCodecs.files(directory, info, fileSet);
@@ -176,7 +148,7 @@ final class SegmentMerger {
     }
 
     // Vector files
-    if (fieldInfos.hasVectors() && mergeDocStores) {
+    if (fieldInfos.hasVectors()) {
       for (String ext : IndexFileNames.VECTOR_EXTENSIONS) {
         fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
       }
@@ -216,6 +188,11 @@ final class SegmentMerger {
   private SegmentReader[] matchingSegmentReaders;
   private int[] rawDocLengths;
   private int[] rawDocLengths2;
+  private int matchedCount;
+
+  public int getMatchedSubReaderCount() {
+    return matchedCount;
+  }
 
   private void setMatchingSegmentReaders() {
     // If the i'th reader is a SegmentReader and has
@@ -240,6 +217,7 @@ final class SegmentMerger {
         }
         if (same) {
           matchingSegmentReaders[i] = segmentReader;
+          matchedCount++;
         }
       }
     }
@@ -255,18 +233,7 @@ final class SegmentMerger {
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  private final int mergeFields() throws CorruptIndexException, IOException {
-
-    if (!mergeDocStores) {
-      // When we are not merging by doc stores, their field
-      // name -> number mapping are the same.  So, we start
-      // with the fieldInfos of the last segment in this
-      // case, to keep that numbering.
-      final SegmentReader sr = (SegmentReader) readers.get(readers.size()-1);
-      fieldInfos = (FieldInfos) sr.core.fieldInfos.clone();
-    } else {
-      fieldInfos = new FieldInfos();// merge field names
-    }
+  private int mergeFields() throws CorruptIndexException, IOException {
 
     for (IndexReader reader : readers) {
       if (reader instanceof SegmentReader) {
@@ -274,11 +241,7 @@ final class SegmentMerger {
         FieldInfos readerFieldInfos = segmentReader.fieldInfos();
         int numReaderFieldInfos = readerFieldInfos.size();
         for (int j = 0; j < numReaderFieldInfos; j++) {
-          FieldInfo fi = readerFieldInfos.fieldInfo(j);
-          fieldInfos.add(fi.name, fi.isIndexed, fi.storeTermVector,
-              fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
-              !reader.hasNorms(fi.name), fi.storePayloads,
-              fi.omitTermFreqAndPositions);
+          fieldInfos.add(readerFieldInfos.fieldInfo(j));
         }
       } else {
         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, false);
@@ -298,54 +261,43 @@ final class SegmentMerger {
 
     setMatchingSegmentReaders();
 
-    if (mergeDocStores) {
-      // merge field values
-      final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
-
-      try {
-        int idx = 0;
-        for (IndexReader reader : readers) {
-          final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
-          FieldsReader matchingFieldsReader = null;
-          if (matchingSegmentReader != null) {
-            final FieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
-            if (fieldsReader != null) {
-              matchingFieldsReader = fieldsReader;
-            }
+    final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
+
+    try {
+      int idx = 0;
+      for (IndexReader reader : readers) {
+        final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
+        FieldsReader matchingFieldsReader = null;
+        if (matchingSegmentReader != null) {
+          final FieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
+          if (fieldsReader != null) {
+            matchingFieldsReader = fieldsReader;
           }
-          if (reader.hasDeletions()) {
-            docCount += copyFieldsWithDeletions(fieldsWriter,
-                                                reader, matchingFieldsReader);
-          } else {
-            docCount += copyFieldsNoDeletions(fieldsWriter,
+        }
+        if (reader.hasDeletions()) {
+          docCount += copyFieldsWithDeletions(fieldsWriter,
                                               reader, matchingFieldsReader);
-          }
+        } else {
+          docCount += copyFieldsNoDeletions(fieldsWriter,
+                                            reader, matchingFieldsReader);
         }
-      } finally {
-        fieldsWriter.close();
       }
+    } finally {
+      fieldsWriter.close();
+    }
 
-      final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
-      final long fdxFileLength = directory.fileLength(fileName);
-
-      if (4+((long) docCount)*8 != fdxFileLength)
-        // This is most likely a bug in Sun JRE 1.6.0_04/_05;
-        // we detect that the bug has struck, here, and
-        // throw an exception to prevent the corruption from
-        // entering the index.  See LUCENE-1282 for
-        // details.
-        throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
+    final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
+    final long fdxFileLength = directory.fileLength(fileName);
 
-    } else {
-      // If we are skipping the doc stores, that means there
-      // are no deletions in any of these segments, so we
-      // just sum numDocs() of each segment to get total docCount
-      for (final IndexReader reader : readers) {
-        docCount += reader.numDocs();
-      }
-    }
+    if (4+((long) docCount)*8 != fdxFileLength)
+      // This is most likely a bug in Sun JRE 1.6.0_04/_05;
+      // we detect that the bug has struck, here, and
+      // throw an exception to prevent the corruption from
+      // entering the index.  See LUCENE-1282 for
+      // details.
+      throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
 
-    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval, codecInfo);
+    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo);
     
     return docCount;
   }
@@ -432,7 +384,7 @@ final class SegmentMerger {
   private final void mergeVectors() throws IOException {
     TermVectorsWriter termVectorsWriter = 
       new TermVectorsWriter(directory, segment, fieldInfos);
-    hasVectors = true;
+
     try {
       int idx = 0;
       for (final IndexReader reader : readers) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java Sun Dec 19 00:24:04 2010
@@ -31,9 +31,7 @@ public class SegmentWriteState {
   public final Directory directory;
   public final String segmentName;
   public final FieldInfos fieldInfos;
-  public final String docStoreSegmentName;
   public final int numDocs;
-  public int numDocsInStore;
   public boolean hasVectors;
   public final Collection<String> flushedFiles;
 
@@ -62,15 +60,12 @@ public class SegmentWriteState {
 
 
   public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
-                           String docStoreSegmentName, int numDocs,
-                           int numDocsInStore, int termIndexInterval, SegmentCodecs segmentCodecs) {
+                           int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs) {
     this.infoStream = infoStream;
     this.directory = directory;
     this.segmentName = segmentName;
     this.fieldInfos = fieldInfos;
-    this.docStoreSegmentName = docStoreSegmentName;
     this.numDocs = numDocs;
-    this.numDocsInStore = numDocsInStore;
     this.termIndexInterval = termIndexInterval;
     this.segmentCodecs = segmentCodecs;
     flushedFiles = new HashSet<String>();
@@ -85,9 +80,7 @@ public class SegmentWriteState {
     directory = state.directory;
     segmentName = state.segmentName;
     fieldInfos = state.fieldInfos;
-    docStoreSegmentName = state.docStoreSegmentName;
     numDocs = state.numDocs;
-    numDocsInStore = state.numDocsInStore;
     termIndexInterval = state.termIndexInterval;
     segmentCodecs = state.segmentCodecs;
     flushedFiles = state.flushedFiles;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Sun Dec 19 00:24:04 2010
@@ -29,7 +29,6 @@ final class StoredFieldsWriter {
   final DocumentsWriter docWriter;
   final FieldInfos fieldInfos;
   int lastDocID;
-  private String docStoreSegment;
 
   PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;
@@ -44,60 +43,31 @@ final class StoredFieldsWriter {
   }
 
   synchronized public void flush(SegmentWriteState state) throws IOException {
-
-    if (state.numDocsInStore > 0) {
-      // It's possible that all documents seen in this segment
-      // hit non-aborting exceptions, in which case we will
-      // not have yet init'd the FieldsWriter:
-      initFieldsWriter();
-
-      // Fill fdx file to include any final docs that we
-      // skipped because they hit non-aborting exceptions
-      fill(state.numDocsInStore - docWriter.getDocStoreOffset());
-    }
-
-    if (fieldsWriter != null)
-      fieldsWriter.flush();
-  }
-
-  private synchronized void initFieldsWriter() throws IOException {
-    if (fieldsWriter == null) {
-      docStoreSegment = docWriter.getDocStoreSegment();
-      if (docStoreSegment != null) {
-        fieldsWriter = new FieldsWriter(docWriter.directory,
-                                        docStoreSegment,
-                                        fieldInfos);
-        docWriter.addOpenFile(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION));
-        docWriter.addOpenFile(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
-        lastDocID = 0;
-      }
-    }
-  }
-
-  synchronized public void closeDocStore(SegmentWriteState state) throws IOException {
-    final int inc = state.numDocsInStore - lastDocID;
-    if (inc > 0) {
+    if (state.numDocs > lastDocID) {
       initFieldsWriter();
-      fill(state.numDocsInStore - docWriter.getDocStoreOffset());
+      fill(state.numDocs);
     }
 
     if (fieldsWriter != null) {
       fieldsWriter.close();
       fieldsWriter = null;
-      assert docStoreSegment != null;
-      assert state.docStoreSegmentName != null;
-      assert docStoreSegment.equals(state.docStoreSegmentName): "fieldsWriter wrote to segment=" + docStoreSegment + " vs SegmentWriteState segment=" + state.docStoreSegmentName;
       lastDocID = 0;
-      String fieldsName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.FIELDS_EXTENSION);
-      String fieldsIdxName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
+
+      String fieldsName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELDS_EXTENSION);
+      String fieldsIdxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
       state.flushedFiles.add(fieldsName);
       state.flushedFiles.add(fieldsIdxName);
 
-      docWriter.removeOpenFile(fieldsName);
-      docWriter.removeOpenFile(fieldsIdxName);
+      if (4 + ((long) state.numDocs) * 8 != state.directory.fileLength(fieldsIdxName)) {
+        throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(fieldsIdxName) + " length in bytes of " + fieldsIdxName + " file exists?=" + state.directory.fileExists(fieldsIdxName));
+      }
+    }
+  }
 
-      if (4+((long) state.numDocsInStore)*8 != state.directory.fileLength(fieldsIdxName))
-        throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(fieldsIdxName) + " length in bytes of " + fieldsIdxName + " file exists?=" + state.directory.fileExists(fieldsIdxName));
+  private synchronized void initFieldsWriter() throws IOException {
+    if (fieldsWriter == null) {
+      fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), fieldInfos);
+      lastDocID = 0;
     }
   }
 
@@ -114,16 +84,14 @@ final class StoredFieldsWriter {
         docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       }
       return new PerDoc();
-    } else
+    } else {
       return docFreeList[--freeCount];
+    }
   }
 
   synchronized void abort() {
     if (fieldsWriter != null) {
-      try {
-        fieldsWriter.close();
-      } catch (Throwable t) {
-      }
+      fieldsWriter.abort();
       fieldsWriter = null;
       lastDocID = 0;
     }
@@ -131,12 +99,9 @@ final class StoredFieldsWriter {
 
   /** Fills in any hole in the docIDs */
   void fill(int docID) throws IOException {
-    final int docStoreOffset = docWriter.getDocStoreOffset();
-
     // We must "catch up" for all docs before us
     // that had no stored fields:
-    final int end = docID+docStoreOffset;
-    while(lastDocID < end) {
+    while(lastDocID < docID) {
       fieldsWriter.skipDocument();
       lastDocID++;
     }
@@ -156,10 +121,6 @@ final class StoredFieldsWriter {
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
   }
 
-  public boolean freeRAM() {
-    return false;
-  }
-
   synchronized void free(PerDoc perDoc) {
     assert freeCount < docFreeList.length;
     assert 0 == perDoc.numStoredFields;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Sun Dec 19 00:24:04 2010
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
@@ -30,7 +31,6 @@ import java.util.Map;
 final class TermVectorsTermsWriter extends TermsHashConsumer {
 
   final DocumentsWriter docWriter;
-  TermVectorsWriter termVectorsWriter;
   PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;
   IndexOutput tvx;
@@ -50,26 +50,29 @@ final class TermVectorsTermsWriter exten
 
   @Override
   synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
+    if (tvx != null) {
+      // At least one doc in this run had term vectors enabled
+      fill(state.numDocs);
+      tvx.close();
+      tvf.close();
+      tvd.close();
+      tvx = tvd = tvf = null;
+      assert state.segmentName != null;
+      String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
+      String fldName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
+      String docName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
 
-    // NOTE: it's possible that all documents seen in this segment
-    // hit non-aborting exceptions, in which case we will
-    // not have yet init'd the TermVectorsWriter.  This is
-    // actually OK (unlike in the stored fields case)
-    // because, although FieldInfos.hasVectors() will return
-    // true, the TermVectorsReader gracefully handles
-    // non-existence of the term vectors files.
-    state.hasVectors = hasVectors;
+      if (4 + ((long) state.numDocs) * 16 != state.directory.fileLength(idxName)) {
+        throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
+      }
 
-    if (tvx != null) {
+      state.flushedFiles.add(idxName);
+      state.flushedFiles.add(fldName);
+      state.flushedFiles.add(docName);
 
-      if (state.numDocsInStore > 0)
-        // In case there are some final documents that we
-        // didn't see (because they hit a non-aborting exception):
-        fill(state.numDocsInStore - docWriter.getDocStoreOffset());
-
-      tvx.flush();
-      tvd.flush();
-      tvf.flush();
+      lastDocID = 0;
+      state.hasVectors = hasVectors;
+      hasVectors = false;
     }
 
     for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
@@ -84,37 +87,6 @@ final class TermVectorsTermsWriter exten
     }
   }
 
-  @Override
-  synchronized void closeDocStore(final SegmentWriteState state) throws IOException {
-    if (tvx != null) {
-      // At least one doc in this run had term vectors
-      // enabled
-      fill(state.numDocsInStore - docWriter.getDocStoreOffset());
-      tvx.close();
-      tvf.close();
-      tvd.close();
-      tvx = null;
-      assert state.docStoreSegmentName != null;
-      String idxName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
-      if (4+((long) state.numDocsInStore)*16 != state.directory.fileLength(idxName))
-        throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
-
-      String fldName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
-      String docName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
-      state.flushedFiles.add(idxName);
-      state.flushedFiles.add(fldName);
-      state.flushedFiles.add(docName);
-
-      docWriter.removeOpenFile(idxName);
-      docWriter.removeOpenFile(fldName);
-      docWriter.removeOpenFile(docName);
-
-      lastDocID = 0;
-      state.hasVectors = hasVectors;
-      hasVectors = false;
-    }    
-  }
-
   int allocCount;
 
   synchronized PerDoc getPerDoc() {
@@ -128,18 +100,17 @@ final class TermVectorsTermsWriter exten
         docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       }
       return new PerDoc();
-    } else
+    } else {
       return docFreeList[--freeCount];
+    }
   }
 
   /** Fills in no-term-vectors for all docs we haven't seen
    *  since the last doc that had term vectors. */
   void fill(int docID) throws IOException {
-    final int docStoreOffset = docWriter.getDocStoreOffset();
-    final int end = docID+docStoreOffset;
-    if (lastDocID < end) {
+    if (lastDocID < docID) {
       final long tvfPosition = tvf.getFilePointer();
-      while(lastDocID < end) {
+      while(lastDocID < docID) {
         tvx.writeLong(tvd.getFilePointer());
         tvd.writeVInt(0);
         tvx.writeLong(tvfPosition);
@@ -151,31 +122,19 @@ final class TermVectorsTermsWriter exten
   synchronized void initTermVectorsWriter() throws IOException {        
     if (tvx == null) {
 
-      final String docStoreSegment = docWriter.getDocStoreSegment();
-
-      if (docStoreSegment == null)
-        return;
-
       // If we hit an exception while init'ing the term
       // vector output files, we must abort this segment
       // because those files will be in an unknown
       // state:
-      String idxName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
-      String docName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
-      String fldName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
       hasVectors = true;
-      tvx = docWriter.directory.createOutput(idxName);
-      tvd = docWriter.directory.createOutput(docName);
-      tvf = docWriter.directory.createOutput(fldName);
+      tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+      tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
+      tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
       
       tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
       tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
       tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
 
-      docWriter.addOpenFile(idxName);
-      docWriter.addOpenFile(fldName);
-      docWriter.addOpenFile(docName);
-
       lastDocID = 0;
     }
   }
@@ -193,8 +152,9 @@ final class TermVectorsTermsWriter exten
     tvx.writeLong(tvf.getFilePointer());
     tvd.writeVInt(perDoc.numVectorFields);
     if (perDoc.numVectorFields > 0) {
-      for(int i=0;i<perDoc.numVectorFields;i++)
+      for(int i=0;i<perDoc.numVectorFields;i++) {
         tvd.writeVInt(perDoc.fieldNumbers[i]);
+      }
       assert 0 == perDoc.fieldPointers[0];
       long lastPos = perDoc.fieldPointers[0];
       for(int i=1;i<perDoc.numVectorFields;i++) {
@@ -206,7 +166,7 @@ final class TermVectorsTermsWriter exten
       perDoc.numVectorFields = 0;
     }
 
-    assert lastDocID == perDoc.docID + docWriter.getDocStoreOffset();
+    assert lastDocID == perDoc.docID;
 
     lastDocID++;
 
@@ -215,36 +175,26 @@ final class TermVectorsTermsWriter exten
     assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
   }
 
-  public boolean freeRAM() {
-    // We don't hold any state beyond one doc, so we don't
-    // free persistent RAM here
-    return false;
-  }
-
   @Override
   public void abort() {
     hasVectors = false;
-    if (tvx != null) {
-      try {
-        tvx.close();
-      } catch (Throwable t) {
-      }
-      tvx = null;
-    }
-    if (tvd != null) {
-      try {
-        tvd.close();
-      } catch (Throwable t) {
-      }
-      tvd = null;
-    }
-    if (tvf != null) {
-      try {
-        tvf.close();
-      } catch (Throwable t) {
-      }
-      tvf = null;
+    try {
+      IOUtils.closeSafely(tvx, tvd, tvf);
+    } catch (IOException ignored) {
+    }
+    try {
+      docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+    } catch (IOException ignored) {
+    }
+    try {
+      docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
+    } catch (IOException ignored) {
+    }
+    try {
+      docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
+    } catch (IOException ignored) {
     }
+    tvx = tvd = tvf = null;
     lastDocID = 0;
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java Sun Dec 19 00:24:04 2010
@@ -70,13 +70,6 @@ final class TermsHash extends InvertedDo
   }
 
   @Override
-  synchronized void closeDocStore(SegmentWriteState state) throws IOException {
-    consumer.closeDocStore(state);
-    if (nextTermsHash != null)
-      nextTermsHash.closeDocStore(state);
-  }
-
-  @Override
   synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
     Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> childThreadsAndFields = new HashMap<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>();
     Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> nextThreadsAndFields;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Sun Dec 19 00:24:04 2010
@@ -25,7 +25,6 @@ abstract class TermsHashConsumer {
   abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
   abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
   abstract void abort();
-  abstract void closeDocStore(SegmentWriteState state) throws IOException;
 
   FieldInfos fieldInfos;
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Sun Dec 19 00:24:04 2010
@@ -40,7 +40,8 @@ public class DefaultSegmentInfosReader e
     IndexInput input = null;
     try {
       input = openInput(directory, segmentsFileName);
-      int format = input.readInt();
+      final int format = input.readInt();
+      infos.setFormat(format);
   
       // check that it is a format we can understand
       if (format > DefaultSegmentInfosWriter.FORMAT_MINIMUM)

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1050738&r1=1050737&r2=1050738&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Sun Dec 19 00:24:04 2010
@@ -37,14 +37,14 @@ public class DefaultSegmentInfosWriter e
 
   /** Each segment records whether its postings are written
    *  in the new flex format */
-  public static final int FORMAT_4_0 = -10;
+  public static final int FORMAT_4_0 = -11;
 
   /** Each segment records whether it has term vectors */
-  public static final int FORMAT_HAS_VECTORS = -11;
+  public static final int FORMAT_HAS_VECTORS = -10;
 
   /** This must always point to the most recent file format.
    * whenever you add a new format, make it 1 smaller (negative version logic)! */
-  public static final int FORMAT_CURRENT = FORMAT_HAS_VECTORS;
+  public static final int FORMAT_CURRENT = FORMAT_4_0;
   
   /** This must always point to the first supported file format. */
   public static final int FORMAT_MINIMUM = FORMAT_DIAGNOSTICS;