lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From busc...@apache.org
Subject svn commit: r978805 [1/2] - in /lucene/dev/branches/realtime_search/lucene/src: java/org/apache/lucene/index/ test/org/apache/lucene/index/ test/org/apache/lucene/index/codecs/preflex/
Date Sat, 24 Jul 2010 06:04:31 GMT
Author: buschmi
Date: Sat Jul 24 06:04:30 2010
New Revision: 978805

URL: http://svn.apache.org/viewvc?rev=978805&view=rev
Log:
LUCENE-2555: Remove shared doc stores

Modified:
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocConsumer.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocInverter.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterRAMAllocator.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldsWriter.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MergePolicy.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriter.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHash.java
    lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
    lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestCodecs.java
    lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
    lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestDoc.java
    lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java
    lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java
    lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocConsumer.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocConsumer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocConsumer.java Sat Jul 24 06:04:30 2010
@@ -20,9 +20,9 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 abstract class DocConsumer {
-  abstract DocumentsWriterPerThread.DocWriter processDocument() throws IOException;
+  abstract void processDocument() throws IOException;
+  abstract void finishDocument() throws IOException;
   abstract void flush(final SegmentWriteState state) throws IOException;
-  abstract void closeDocStore(final SegmentWriteState state) throws IOException;
   abstract void abort();
   abstract boolean freeRAM();
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java Sat Jul 24 06:04:30 2010
@@ -27,10 +27,6 @@ abstract class DocFieldConsumer {
   /** Called when DocumentsWriter decides to create a new
    *  segment */
   abstract void flush(Map<FieldInfo, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
-
-  /** Called when DocumentsWriter decides to close the doc
-   *  stores */
-  abstract void closeDocStore(SegmentWriteState state) throws IOException;
   
   /** Called when an aborting exception is hit */
   abstract void abort();
@@ -44,7 +40,7 @@ abstract class DocFieldConsumer {
 
   abstract DocFieldConsumerPerField addField(FieldInfo fi);
   
-  abstract DocumentsWriterPerThread.DocWriter finishDocument() throws IOException;
+  abstract void finishDocument() throws IOException;
   
   void setFieldInfos(FieldInfos fieldInfos) {
     this.fieldInfos = fieldInfos;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java Sat Jul 24 06:04:30 2010
@@ -21,9 +21,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
-
 /** This is just a "splitter" class: it lets you wrap two
  *  DocFieldConsumer instances as a single consumer. */
 
@@ -62,15 +59,6 @@ final class DocFieldConsumers extends Do
   }
 
   @Override
-  public void closeDocStore(SegmentWriteState state) throws IOException {      
-    try {
-      one.closeDocStore(state);
-    } finally {
-      two.closeDocStore(state);
-    }
-  }
-
-  @Override
   public void abort() {
     try {
       one.abort();
@@ -86,83 +74,12 @@ final class DocFieldConsumers extends Do
     return any;
   }
 
-  PerDoc[] docFreeList = new PerDoc[1];
-  int freeCount;
-  int allocCount;
-
-  PerDoc getPerDoc() {
-    if (freeCount == 0) {
-      allocCount++;
-      if (allocCount > docFreeList.length) {
-        // Grow our free list up front to make sure we have
-        // enough space to recycle all outstanding PerDoc
-        // instances
-        assert allocCount == 1+docFreeList.length;
-        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      }
-      return new PerDoc();
-    } else
-      return docFreeList[--freeCount];
-  }
-
-  void freePerDoc(PerDoc perDoc) {
-    assert freeCount < docFreeList.length;
-    docFreeList[freeCount++] = perDoc;
-  }
-
-  class PerDoc extends DocumentsWriterPerThread.DocWriter {
-
-    DocumentsWriterPerThread.DocWriter writerOne;
-    DocumentsWriterPerThread.DocWriter writerTwo;
-
-    @Override
-    public long sizeInBytes() {
-      return writerOne.sizeInBytes() + writerTwo.sizeInBytes();
-    }
-
-    @Override
-    public void finish() throws IOException {
-      try {
-        try {
-          writerOne.finish();
-        } finally {
-          writerTwo.finish();
-        }
-      } finally {
-        freePerDoc(this);
-      }
-    }
-
-    @Override
-    public void abort() {
-      try {
-        try {
-          writerOne.abort();
-        } finally {
-          writerTwo.abort();
-        }
-      } finally {
-        freePerDoc(this);
-      }
-    }
-  }
-  
   @Override
-  public DocumentsWriterPerThread.DocWriter finishDocument() throws IOException {
-    final DocumentsWriterPerThread.DocWriter oneDoc = one.finishDocument();
-    final DocumentsWriterPerThread.DocWriter twoDoc = two.finishDocument();
-    if (oneDoc == null)
-      return twoDoc;
-    else if (twoDoc == null)
-      return oneDoc;
-    else {
-      DocFieldConsumers.PerDoc both = getPerDoc();
-      both.docID = docState.docID;
-      assert oneDoc.docID == docState.docID;
-      assert twoDoc.docID == docState.docID;
-      both.writerOne = oneDoc;
-      both.writerTwo = twoDoc;
-      return both;
+  public void finishDocument() throws IOException {
+    try {
+      one.finishDocument();
+    } finally {
+      two.finishDocument();  
     }
   }
   

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Sat Jul 24 06:04:30 2010
@@ -26,8 +26,6 @@ import java.util.Map;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
 
 
 /**
@@ -67,12 +65,6 @@ final class DocFieldProcessor extends Do
   }
 
   @Override
-  public void closeDocStore(SegmentWriteState state) throws IOException {
-    consumer.closeDocStore(state);
-    fieldsWriter.closeDocStore(state);
-  }
-
-  @Override
   public void flush(SegmentWriteState state) throws IOException {
 
     Map<FieldInfo, DocFieldConsumerPerField> childFields = new HashMap<FieldInfo, DocFieldConsumerPerField>();
@@ -105,8 +97,11 @@ final class DocFieldProcessor extends Do
       }
     }
 
-    fieldsWriter.abort();
-    consumer.abort();
+    try {
+      fieldsWriter.abort();
+    } finally {
+      consumer.abort();
+    }
   }
 
   @Override
@@ -190,7 +185,7 @@ final class DocFieldProcessor extends Do
   }
 
   @Override
-  public DocumentsWriterPerThread.DocWriter processDocument() throws IOException {
+  public void processDocument() throws IOException {
 
     consumer.startDocument();
     fieldsWriter.startDocument();
@@ -259,14 +254,9 @@ final class DocFieldProcessor extends Do
         fields[fieldCount++] = fp;
         fp.lastGen = thisFieldGen;
       }
+      
+      fp.addField(field);
 
-      if (fp.fieldCount == fp.fields.length) {
-        Fieldable[] newArray = new Fieldable[fp.fields.length*2];
-        System.arraycopy(fp.fields, 0, newArray, 0, fp.fieldCount);
-        fp.fields = newArray;
-      }
-
-      fp.fields[fp.fieldCount++] = field;
       if (field.isStored()) {
         fieldsWriter.addField(field, fp.fieldInfo);
       }
@@ -287,24 +277,18 @@ final class DocFieldProcessor extends Do
       docState.infoStream.println("WARNING: document contains at least one immense term (whose UTF8 encoding is longer than the max length " + DocumentsWriterRAMAllocator.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 + "...'"); 
       docState.maxTermPrefix = null;
     }
-
-    final DocumentsWriterPerThread.DocWriter one = fieldsWriter.finishDocument();
-    final DocumentsWriterPerThread.DocWriter two = consumer.finishDocument();
-    if (one == null) {
-      return two;
-    } else if (two == null) {
-      return one;
-    } else {
-      PerDoc both = getPerDoc();
-      both.docID = docState.docID;
-      assert one.docID == docState.docID;
-      assert two.docID == docState.docID;
-      both.one = one;
-      both.two = two;
-      return both;
+  }
+  
+  @Override
+  void finishDocument() throws IOException {
+    try {
+      fieldsWriter.finishDocument();
+    } finally {
+      consumer.finishDocument();
     }
   }
 
+
   void quickSort(DocFieldProcessorPerField[] array, int lo, int hi) {
     if (lo >= hi)
       return;
@@ -365,65 +349,4 @@ final class DocFieldProcessor extends Do
     quickSort(array, lo, left);
     quickSort(array, left + 1, hi);
   }
-
-  PerDoc[] docFreeList = new PerDoc[1];
-  int freeCount;
-  int allocCount;
-
-  PerDoc getPerDoc() {
-    if (freeCount == 0) {
-      allocCount++;
-      if (allocCount > docFreeList.length) {
-        // Grow our free list up front to make sure we have
-        // enough space to recycle all outstanding PerDoc
-        // instances
-        assert allocCount == 1+docFreeList.length;
-        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      }
-      return new PerDoc();
-    } else
-      return docFreeList[--freeCount];
-  }
-
-  void freePerDoc(PerDoc perDoc) {
-    assert freeCount < docFreeList.length;
-    docFreeList[freeCount++] = perDoc;
-  }
-
-  class PerDoc extends DocumentsWriterPerThread.DocWriter {
-
-    DocumentsWriterPerThread.DocWriter one;
-    DocumentsWriterPerThread.DocWriter two;
-
-    @Override
-    public long sizeInBytes() {
-      return one.sizeInBytes() + two.sizeInBytes();
-    }
-
-    @Override
-    public void finish() throws IOException {
-      try {
-        try {
-          one.finish();
-        } finally {
-          two.finish();
-        }
-      } finally {
-        freePerDoc(this);
-      }
-    }
-
-    @Override
-    public void abort() {
-      try {
-        try {
-          one.abort();
-        } finally {
-          two.abort();
-        }
-      } finally {
-        freePerDoc(this);
-      }
-    }
-  }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java Sat Jul 24 06:04:30 2010
@@ -18,6 +18,8 @@ package org.apache.lucene.index;
  */
 
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Holds all per thread, per field state.
@@ -39,6 +41,17 @@ final class DocFieldProcessorPerField {
     this.fieldInfo = fieldInfo;
   }
 
+  public void addField(Fieldable field) {
+    if (fieldCount == fields.length) {
+      int newSize = ArrayUtil.oversize(fieldCount + 1, RamUsageEstimator.NUM_BYTES_OBJ_REF);
+      Fieldable[] newArray = new Fieldable[newSize];
+      System.arraycopy(fields, 0, newArray, 0, fieldCount);
+      fields = newArray;
+    }
+
+    fields[fieldCount++] = field;
+  }
+  
   public void abort() {
     consumer.abort();
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocInverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocInverter.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocInverter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocInverter.java Sat Jul 24 06:04:30 2010
@@ -95,18 +95,11 @@ final class DocInverter extends DocField
   }
 
   @Override
-  public DocumentsWriterPerThread.DocWriter finishDocument() throws IOException {
+  public void finishDocument() throws IOException {
     // TODO: allow endConsumer.finishDocument to also return
     // a DocWriter
     endConsumer.finishDocument();
-    return consumer.finishDocument();
-  }
-
-
-  @Override
-  public void closeDocStore(SegmentWriteState state) throws IOException {
-    consumer.closeDocStore(state);
-    endConsumer.closeDocStore(state);
+    consumer.finishDocument();
   }
 
   @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Sat Jul 24 06:04:30 2010
@@ -130,7 +130,7 @@ final class DocumentsWriter {
   long updateDocument(final Term delTerm, final Document doc, final Analyzer analyzer)
       throws CorruptIndexException, IOException {
 
-    return threadPool.executePerThread(this, doc,
+    long seqID = threadPool.executePerThread(this, doc,
         new DocumentsWriterThreadPool.PerThreadTask<Long>() {
           @Override
           public Long process(final DocumentsWriterPerThread perThread) throws IOException {
@@ -156,11 +156,14 @@ final class DocumentsWriter {
 
             if (finishAddDocument(perThread, perThreadRAMUsedBeforeAdd)) {
               super.clearThreadBindings();
-              indexWriter.maybeMerge();
             }
             return sequenceID;
           }
         });
+    
+    indexWriter.maybeMerge();
+    
+    return seqID;
   }
 
   private final boolean finishAddDocument(DocumentsWriterPerThread perThread,
@@ -252,7 +255,7 @@ final class DocumentsWriter {
     this.flushedSequenceID = newFlushedID;
   }
 
-  final boolean flushAllThreads(final boolean flushDocStores, final boolean flushDeletes)
+  final boolean flushAllThreads(final boolean flushDeletes)
       throws IOException {
     return threadPool.executeAllThreads(new DocumentsWriterThreadPool.AllThreadsTask<Boolean>() {
       @Override
@@ -260,61 +263,29 @@ final class DocumentsWriter {
         boolean anythingFlushed = false;
         
         if (flushDeletes) {
-          synchronized (indexWriter) {
-            if (applyDeletes(indexWriter.segmentInfos)) {
-              indexWriter.checkpoint();
-            }
+          if (applyDeletes(indexWriter.segmentInfos)) {
+            indexWriter.checkpoint();
           }
         }
 
         while (threadsIterator.hasNext()) {
-          boolean perThreadFlushDocStores = flushDocStores;
           DocumentsWriterPerThread perThread = threadsIterator.next();
           final int numDocs = perThread.getNumDocsInRAM();
           
           // Always flush docs if there are any
           boolean flushDocs = numDocs > 0;
           
-          String docStoreSegment = perThread.getDocStoreSegment();
-          if (docStoreSegment == null) {
-            perThreadFlushDocStores = false;
-          }
-          int docStoreOffset = perThread.getDocStoreOffset();
-          boolean docStoreIsCompoundFile = false;
-          if (perThreadFlushDocStores
-              && (!flushDocs || !perThread.getSegment().equals(perThread.getDocStoreSegment()))) {
-            // We must separately flush the doc store
-            if (infoStream != null) {
-              message("  flush shared docStore segment " + docStoreSegment);
-            }
-            docStoreIsCompoundFile = flushDocStores(perThread);
-            flushDocStores(perThread);
-            perThreadFlushDocStores = false;
-          }
-
           String segment = perThread.getSegment();
 
           // If we are flushing docs, segment must not be null:
           assert segment != null || !flushDocs;
     
           if (flushDocs) {
-            SegmentInfo newSegment = perThread.flush(perThreadFlushDocStores);
+            SegmentInfo newSegment = perThread.flush();
             
             if (newSegment != null) {
               anythingFlushed = true;
               
-              if (0 == docStoreOffset && perThreadFlushDocStores) {
-                // This means we are flushing private doc stores
-                // with this segment, so it will not be shared
-                // with other segments
-                assert docStoreSegment != null;
-                assert docStoreSegment.equals(segment);
-                docStoreOffset = -1;
-                docStoreSegment = null;
-                docStoreIsCompoundFile = false;
-              }
-              newSegment.setDocStore(docStoreOffset, docStoreSegment, docStoreIsCompoundFile);
-              
               IndexWriter.setDiagnostics(newSegment, "flush");
               finishFlushedSegment(newSegment, perThread);
             }
@@ -361,6 +332,7 @@ final class DocumentsWriter {
     synchronized(indexWriter) {
       indexWriter.segmentInfos.add(newSegment);
       indexWriter.checkpoint();
+    
       SegmentReader reader = indexWriter.readerPool.get(newSegment, false);
       boolean any = false;
       try {
@@ -389,84 +361,15 @@ final class DocumentsWriter {
           }
         }
   
-        newSegment.setUseCompoundFile(true);
-        indexWriter.checkpoint();
-      }
-    }
-  }
-
-  
-  private boolean flushDocStores(DocumentsWriterPerThread perThread) throws IOException {
-      boolean useCompoundDocStore = false;
-  
-      String docStoreSegment;
-      
-      boolean success = false;
-      try {
-        docStoreSegment = perThread.closeDocStore();
-        success = true;
-      } finally {
-        if (!success && infoStream != null) {
-          message("hit exception closing doc store segment");
-        }
-      }
-  
-      useCompoundDocStore = indexWriter.mergePolicy.useCompoundDocStore(indexWriter.segmentInfos);
-  
-      if (useCompoundDocStore && docStoreSegment != null && perThread.closedFiles().size() != 0) {
-        // Now build compound doc store file
-  
-        if (infoStream != null) {
-          message("create compound file "
-              + IndexFileNames.segmentFileName(docStoreSegment, "",
-                  IndexFileNames.COMPOUND_FILE_STORE_EXTENSION));
-        }
-  
-        success = false;
-  
-        final int numSegments = indexWriter.segmentInfos.size();
-        final String compoundFileName = IndexFileNames.segmentFileName(docStoreSegment, "",
-            IndexFileNames.COMPOUND_FILE_STORE_EXTENSION);
-  
-        try {
-          CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
-          for (final String file : perThread.closedFiles()) {
-            cfsWriter.addFile(file);
-          }
-  
-          // Perform the merge
-          cfsWriter.close();
-          success = true;
-  
-        } finally {
-          if (!success) {
-            if (infoStream != null)
-              message("hit exception building compound file doc store for segment " + docStoreSegment);
-            synchronized(indexWriter) {
-              indexWriter.deleter.deleteFile(compoundFileName);
-            }
-            abort();
-          }
-        }
-  
         synchronized(indexWriter) {
-          for (int i = 0; i < numSegments; i++) {
-            SegmentInfo si = indexWriter.segmentInfos.info(i);
-            if (si.getDocStoreOffset() != -1 &&
-                  si.getDocStoreSegment().equals(docStoreSegment))
-              si.setDocStoreIsCompoundFile(true);
-          }
-    
+          newSegment.setUseCompoundFile(true);
           indexWriter.checkpoint();
-    
           // In case the files we just merged into a CFS were
           // not previously checkpointed:
           indexWriter.deleter.deleteNewFiles(perThread.closedFiles());
         }
       }
-  
-      return useCompoundDocStore;
-    
+    }
   }
   
   // Returns true if an abort is in progress
@@ -495,7 +398,7 @@ final class DocumentsWriter {
 
   private final boolean maybeFlushPerThread(DocumentsWriterPerThread perThread) throws IOException {
     if (perThread.getNumDocsInRAM() == maxBufferedDocs) {
-      flushSegment(perThread, false);
+      flushSegment(perThread);
       assert perThread.getNumDocsInRAM() == 0;
       return true;
     }
@@ -503,18 +406,15 @@ final class DocumentsWriter {
     return false;
   }
 
-  private boolean flushSegment(DocumentsWriterPerThread perThread, boolean flushDocStores)
+  private boolean flushSegment(DocumentsWriterPerThread perThread)
       throws IOException {
-    if (perThread.getNumDocsInRAM() == 0 && !flushDocStores) {
+    if (perThread.getNumDocsInRAM() == 0) {
       return false;
     }
 
-    int docStoreOffset = perThread.getDocStoreOffset();
-    String docStoreSegment = perThread.getDocStoreSegment();
-    SegmentInfo newSegment = perThread.flush(flushDocStores);
+    SegmentInfo newSegment = perThread.flush();
     
     if (newSegment != null) {
-      newSegment.setDocStore(docStoreOffset, docStoreSegment, false);
       finishFlushedSegment(newSegment, perThread);
       return true;
     }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Sat Jul 24 06:04:30 2010
@@ -10,7 +10,6 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.RAMFile;
 import org.apache.lucene.util.ArrayUtil;
 
 public class DocumentsWriterPerThread {
@@ -94,10 +93,6 @@ public class DocumentsWriterPerThread {
       } catch (Throwable t) {
       }
 
-      docStoreSegment = null;
-      numDocsInStore = 0;
-      docStoreOffset = 0;
-
       // Reset all postings data
       doAfterFlush();
 
@@ -121,18 +116,14 @@ public class DocumentsWriterPerThread {
   private DocFieldProcessor docFieldProcessor;
   
   String segment;                         // Current segment we are working on
-  private String docStoreSegment;         // Current doc-store segment we are writing
-  private int docStoreOffset;                     // Current starting doc-store offset of current segment
   boolean aborting;               // True if an abort is pending
   
   private final PrintStream infoStream;
   private int numDocsInRAM;
-  private int numDocsInStore;
   private int flushedDocCount;
   SegmentWriteState flushState;
 
   long[] sequenceIDs = new long[8];
-  
   final List<String> closedFiles = new ArrayList<String>();
   
   long numBytesUsed;
@@ -161,13 +152,15 @@ public class DocumentsWriterPerThread {
     docState.doc = doc;
     docState.analyzer = analyzer;
     docState.docID = numDocsInRAM;
-    initSegmentName(false);
+    if (segment == null) {
+      // this call is synchronized on IndexWriter.segmentInfos
+      segment = writer.newSegmentName();
+      assert numDocsInRAM == 0;
+    }
   
-    final DocWriter perDoc;
-    
     boolean success = false;
     try {
-      perDoc = consumer.processDocument();
+      consumer.processDocument();
       
       success = true;
     } finally {
@@ -181,9 +174,7 @@ public class DocumentsWriterPerThread {
 
     success = false;
     try {
-      if (perDoc != null) {
-        perDoc.finish();
-      }
+      consumer.finishDocument();
       
       success = true;
     } finally {
@@ -201,7 +192,6 @@ public class DocumentsWriterPerThread {
     
     sequenceIDs[numDocsInRAM] = sequenceID;
     numDocsInRAM++;
-    numDocsInStore++;
   }
   
   int getNumDocsInRAM() {
@@ -226,26 +216,6 @@ public class DocumentsWriterPerThread {
     return flushState.codec;
   }
   
-  void initSegmentName(boolean onlyDocStore) {
-    if (segment == null && (!onlyDocStore || docStoreSegment == null)) {
-      // this call is synchronized on IndexWriter.segmentInfos
-      segment = writer.newSegmentName();
-      assert numDocsInRAM == 0;
-    }
-    if (docStoreSegment == null) {
-      docStoreSegment = segment;
-      assert numDocsInStore == 0;
-    }
-  }
-
-  
-  private void initFlushState(boolean onlyDocStore) {
-    initSegmentName(onlyDocStore);
-    flushState = new SegmentWriteState(infoStream, directory, segment, docFieldProcessor.fieldInfos,
-                                       docStoreSegment, numDocsInRAM, numDocsInStore, writer.getConfig().getTermIndexInterval(),
-                                       writer.codecs);
-  }
-  
   /** Reset after a flush */
   private void doAfterFlush() throws IOException {
     segment = null;
@@ -253,12 +223,12 @@ public class DocumentsWriterPerThread {
   }
     
   /** Flush all pending docs to a new segment */
-  SegmentInfo flush(boolean closeDocStore) throws IOException {
+  SegmentInfo flush() throws IOException {
     assert numDocsInRAM > 0;
 
-    initFlushState(closeDocStore);
-
-    docStoreOffset = numDocsInStore;
+    flushState = new SegmentWriteState(infoStream, directory, segment, docFieldProcessor.fieldInfos,
+        numDocsInRAM, writer.getConfig().getTermIndexInterval(),
+        writer.codecs);
 
     if (infoStream != null) {
       message("flush postings as segment " + flushState.segmentName + " numDocs=" + numDocsInRAM);
@@ -267,22 +237,12 @@ public class DocumentsWriterPerThread {
     boolean success = false;
 
     try {
-
-      if (closeDocStore) {
-        assert flushState.docStoreSegmentName != null;
-        assert flushState.docStoreSegmentName.equals(flushState.segmentName);
-        closeDocStore();
-        flushState.numDocsInStore = 0;
-      }
-      
       consumer.flush(flushState);
 
       if (infoStream != null) {
         SegmentInfo si = new SegmentInfo(flushState.segmentName,
             flushState.numDocs,
             directory, false,
-            docStoreOffset, flushState.docStoreSegmentName,
-            false,    
             hasProx(),
             getCodec());
 
@@ -305,8 +265,6 @@ public class DocumentsWriterPerThread {
       SegmentInfo newSegment = new SegmentInfo(flushState.segmentName,
                                    flushState.numDocs,
                                    directory, false,
-                                   docStoreOffset, flushState.docStoreSegmentName,
-                                   false,    
                                    hasProx(),
                                    getCodec());
 
@@ -325,62 +283,17 @@ public class DocumentsWriterPerThread {
     }
   }
 
-  /** Closes the current open doc stores an returns the doc
-   *  store segment name.  This returns null if there are *
-   *  no buffered documents. */
-  String closeDocStore() throws IOException {
-
-    // nocommit
-//    if (infoStream != null)
-//      message("closeDocStore: " + openFiles.size() + " files to flush to segment " + docStoreSegment + " numDocs=" + numDocsInStore);
-    
-    boolean success = false;
-
-    try {
-      initFlushState(true);
-      closedFiles.clear();
-
-      consumer.closeDocStore(flushState);
-      // nocommit
-      //assert 0 == openFiles.size();
-
-      String s = docStoreSegment;
-      docStoreSegment = null;
-      docStoreOffset = 0;
-      numDocsInStore = 0;
-      success = true;
-      return s;
-    } finally {
-      if (!success) {
-        parent.abort();
-      }
-    }
-  }
-
-  
   /** Get current segment name we are writing. */
   String getSegment() {
     return segment;
   }
   
-  /** Returns the current doc store segment we are writing
-   *  to. */
-  String getDocStoreSegment() {
-    return docStoreSegment;
-  }
-
-  /** Returns the doc offset into the shared doc store for
-   *  the current buffered docs. */
-  int getDocStoreOffset() {
-    return docStoreOffset;
-  }
-
-
   @SuppressWarnings("unchecked")
   List<String> closedFiles() {
     return (List<String>) ((ArrayList<String>) closedFiles).clone();
   }
 
+  
   void addOpenFile(String name) {
     synchronized(parent.openFiles) {
       assert !parent.openFiles.contains(name);
@@ -396,58 +309,6 @@ public class DocumentsWriterPerThread {
     closedFiles.add(name);
   }
   
-  /** Consumer returns this on each doc.  This holds any
-   *  state that must be flushed synchronized "in docID
-   *  order".  We gather these and flush them in order. */
-  abstract static class DocWriter {
-    DocWriter next;
-    int docID;
-    abstract void finish() throws IOException;
-    abstract void abort();
-    abstract long sizeInBytes();
-
-    void setNext(DocWriter next) {
-      this.next = next;
-    }
-  }
-
-  /**
-   * Create and return a new DocWriterBuffer.
-   */
-  PerDocBuffer newPerDocBuffer() {
-    return new PerDocBuffer();
-  }
-
-  /**
-   * RAMFile buffer for DocWriters.
-   */
-  class PerDocBuffer extends RAMFile {
-    
-    /**
-     * Allocate bytes used from shared pool.
-     */
-    protected byte[] newBuffer(int size) {
-      assert size == DocumentsWriterRAMAllocator.PER_DOC_BLOCK_SIZE;
-      return ramAllocator.perDocAllocator.getByteBlock();
-    }
-    
-    /**
-     * Recycle the bytes used.
-     */
-    synchronized void recycle() {
-      if (buffers.size() > 0) {
-        setLength(0);
-        
-        // Recycle the blocks
-        ramAllocator.perDocAllocator.recycleByteBlocks(buffers);
-        buffers.clear();
-        sizeInBytes = 0;
-        
-        assert numBuffers() == 0;
-      }
-    }
-  }
-  
   void bytesUsed(long numBytes) {
     ramAllocator.bytesUsed(numBytes);
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterRAMAllocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterRAMAllocator.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterRAMAllocator.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterRAMAllocator.java Sat Jul 24 06:04:30 2010
@@ -144,5 +144,4 @@ class DocumentsWriterRAMAllocator {
   String toMB(long v) {
     return nf.format(v/1024./1024.);
   }
-
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java Sat Jul 24 06:04:30 2010
@@ -121,8 +121,8 @@ abstract class DocumentsWriterThreadPool
         throw new ThreadInterruptedException(ie);
       }
       
-      globalLock = true;
       pauseAllThreads();
+      globalLock = true;
     } finally {
       lock.unlock();
     }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldsWriter.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldsWriter.java Sat Jul 24 06:04:30 2010
@@ -117,10 +117,9 @@ final class FieldsWriter
     // and adds a new entry for this document into the index
     // stream.  This assumes the buffer was already written
     // in the correct fields format.
-    void flushDocument(int numStoredFields, RAMOutputStream buffer) throws IOException {
+    void startDocument(int numStoredFields) throws IOException {
       indexStream.writeLong(fieldsStream.getFilePointer());
       fieldsStream.writeVInt(numStoredFields);
-      buffer.writeTo(fieldsStream);
     }
 
     void skipDocument() throws IOException {
@@ -169,8 +168,8 @@ final class FieldsWriter
       }
     }
 
-    final void writeField(FieldInfo fi, Fieldable field) throws IOException {
-      fieldsStream.writeVInt(fi.number);
+    final void writeField(int fieldNumber, Fieldable field) throws IOException {
+      fieldsStream.writeVInt(fieldNumber);
       byte bits = 0;
       if (field.isTokenized())
         bits |= FieldsWriter.FIELD_IS_TOKENIZED;
@@ -226,7 +225,7 @@ final class FieldsWriter
 
         for (Fieldable field : fields) {
             if (field.isStored())
-              writeField(fieldInfos.fieldInfo(field.name()), field);
+              writeField(fieldInfos.fieldInfo(field.name()).number, field);
         }
     }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Sat Jul 24 06:04:30 2010
@@ -20,22 +20,15 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.lucene.index.DocumentsWriterPerThread.DocWriter;
 import org.apache.lucene.index.codecs.FieldsConsumer;
-import org.apache.lucene.index.codecs.PostingsConsumer;
-import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.util.BytesRef;
 
 final class FreqProxTermsWriter extends TermsHashConsumer {
 
   @Override
-  void closeDocStore(SegmentWriteState state) {}
-
-  @Override
   void abort() {}
 
   // TODO: would be nice to factor out more of this, eg the
@@ -112,8 +105,7 @@ final class FreqProxTermsWriter extends 
   }
 
   @Override
-  DocWriter finishDocument() throws IOException {
-    return null;
+  void finishDocument(TermsHash termsHash) throws IOException {
   }
 
   @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java Sat Jul 24 06:04:30 2010
@@ -413,7 +413,7 @@ public class IndexWriter implements Clos
     // this method is called:
     poolReaders = true;
 
-    flush(true, true, false);
+    flush(true, false);
     
     // Prevent segmentInfos from changing while opening the
     // reader; in theory we could do similar retry logic,
@@ -1440,7 +1440,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)
@@ -1961,7 +1961,7 @@ public class IndexWriter implements Clos
     if (infoStream != null)
       message("optimize: index now " + segString());
 
-    flush(true, false, true);
+    flush(true, true);
 
     synchronized(this) {
       resetMergeExceptions();
@@ -2490,7 +2490,7 @@ public class IndexWriter implements Clos
     try {
       if (infoStream != null)
         message("flush at addIndexes(Directory...)");
-      flush(true, false, true);
+      flush(true, true);
 
       int docCount = 0;
       List<SegmentInfo> infos = new ArrayList<SegmentInfo>();
@@ -2537,7 +2537,7 @@ public class IndexWriter implements Clos
           }
 
           // Update SI appropriately
-          info.setDocStore(info.getDocStoreOffset(), newDsName, info.getDocStoreIsCompoundFile());
+          info.setDocStoreSegment(newDsName);
           info.dir = directory;
           info.name = newSegName;
 
@@ -2595,8 +2595,7 @@ public class IndexWriter implements Clos
       
       SegmentInfo info = null;
       synchronized(this) {
-        info = new SegmentInfo(mergedName, docCount, directory, false, -1,
-            null, false, merger.hasProx(), merger.getCodec());
+        info = new SegmentInfo(mergedName, docCount, directory, false, merger.hasProx(), merger.getCodec());
         setDiagnostics(info, "addIndexes(IndexReader...)");
         segmentInfos.add(info);
         checkpoint();
@@ -2705,7 +2704,7 @@ public class IndexWriter implements Clos
     if (infoStream != null)
       message("prepareCommit: flush");
 
-    flush(true, true, true);
+    flush(true, true);
 
     startCommit(0, commitUserData);
   }
@@ -2826,18 +2825,18 @@ public class IndexWriter implements Clos
    * @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 {
     // 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();
   }
 
   // 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 flushDocStores, boolean flushDeletes) throws CorruptIndexException, IOException {
-    return docWriter.flushAllThreads(flushDocStores, flushDeletes);
+  private synchronized final boolean doFlush(boolean flushDeletes) throws CorruptIndexException, IOException {
+    return docWriter.flushAllThreads(flushDeletes);
     // nocommit
 //    try {
 //      try {
@@ -2998,7 +2997,6 @@ public class IndexWriter implements Clos
     // nocommit
     //docWriter.remapDeletes(segmentInfos, merger.getDocMaps(), merger.getDelCounts(), merge, mergedDocCount);
       
-    setMergeDocStoreIsCompoundFile(merge);
     merge.info.setHasProx(merger.hasProx());
 
     segmentInfos.subList(start, start + merge.segments.size()).clear();
@@ -3199,108 +3197,13 @@ public class IndexWriter implements Clos
     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;
-    // nocommit
-    //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 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
-      // nocommit
-//      if (si.getDocStoreOffset() != -1 && currentDocStoreSegment != null && si.getDocStoreSegment().equals(currentDocStoreSegment)) {
-//        doFlushDocStore = 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);
-    }
-
     merge.increfDone = true;
 
-    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,
+                                 directory, false,
                                  false,
                                  null);
 
@@ -3308,7 +3211,6 @@ public class IndexWriter implements Clos
     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));
     setDiagnostics(merge.info, "merge", details);
 
     // Also enroll the merged segment into mergingSegments;
@@ -3368,23 +3270,6 @@ public class IndexWriter implements Clos
     runningMerges.remove(merge);
   }
 
-  private synchronized void setMergeDocStoreIsCompoundFile(MergePolicy.OneMerge merge) {
-    final String mergeDocStoreSegment = merge.info.getDocStoreSegment(); 
-    if (mergeDocStoreSegment != null && !merge.info.getDocStoreIsCompoundFile()) {
-      final int size = segmentInfos.size();
-      for(int i=0;i<size;i++) {
-        final SegmentInfo info = segmentInfos.info(i);
-        final String docStoreSegment = info.getDocStoreSegment();
-        if (docStoreSegment != null &&
-            docStoreSegment.equals(mergeDocStoreSegment) && 
-            info.getDocStoreIsCompoundFile()) {
-          merge.info.setDocStoreIsCompoundFile(true);
-          break;
-        }
-      }
-    }
-  }        
-
   /** Does the actual (time-consuming) work of the merge,
    *  but without holding synchronized lock on IndexWriter
    *  instance */
@@ -3410,10 +3295,6 @@ public class IndexWriter implements Clos
     merge.readers = new SegmentReader[numSegments];
     merge.readersClone = new SegmentReader[numSegments];
 
-    boolean mergeDocStores = false;
-
-    final Set<String> dss = new HashSet<String>();
-    
     // This is try/finally to make sure merger's readers are
     // closed:
     boolean success = false;
@@ -3426,7 +3307,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());
 
@@ -3436,14 +3317,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) {
-          dss.add(info.getDocStoreSegment());
-        }
-
         totDocCount += clone.numDocs();
       }
 
@@ -3453,40 +3326,12 @@ 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:
-          // nocommit
-//          if (dss.contains(docWriter.getDocStoreSegment())) {
-//            if (infoStream != null)
-//              message("now flush at mergeMiddle");
-//            doFlush(true, false);
-//          }
-        }
-
-        for(int i=0;i<numSegments;i++) {
-          merge.readersClone[i].openDocStores();
-        }
-
-        // Clear DSS
-        synchronized(this) {
-          merge.info.setDocStore(-1, null, false);
-        }
+      for(int i=0;i<numSegments;i++) {
+        merge.readersClone[i].openDocStores();
       }
 
       // 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.setCodec(merger.getCodec());
@@ -3507,11 +3352,6 @@ public class IndexWriter implements Clos
       final boolean loadDocStores;
 
       synchronized(this) {
-        // If the doc store we are using has been closed and
-        // is in now compound format (but wasn't when we
-        // started), then we will switch to the compound
-        // format as well:
-        setMergeDocStoreIsCompoundFile(merge);
         assert merge.mergeFiles == null;
         merge.mergeFiles = merge.info.files();
         deleter.incRef(merge.mergeFiles);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java Sat Jul 24 06:04:30 2010
@@ -28,14 +28,11 @@ abstract class InvertedDocConsumer {
   /** Flush a new segment */
   abstract void flush(Map<FieldInfo, InvertedDocConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
 
-  /** Close doc stores */
-  abstract void closeDocStore(SegmentWriteState state) throws IOException;
-
   abstract InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);
   
   abstract void startDocument() throws IOException;
   
-  abstract DocumentsWriterPerThread.DocWriter finishDocument() throws IOException;
+  abstract void finishDocument() throws IOException;
   
   /** Attempt to free RAM, returning true if any RAM was
    *  freed */

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java Sat Jul 24 06:04:30 2010
@@ -22,7 +22,6 @@ import java.util.Map;
 
 abstract class InvertedDocEndConsumer {
   abstract void flush(Map<FieldInfo, InvertedDocEndConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
-  abstract void closeDocStore(SegmentWriteState state) throws IOException;
   abstract void abort();
   abstract void setFieldInfos(FieldInfos fieldInfos);
   abstract InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MergePolicy.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MergePolicy.java Sat Jul 24 06:04:30 2010
@@ -67,7 +67,6 @@ public abstract class MergePolicy implem
   public static class OneMerge {
 
     SegmentInfo info;               // used by IndexWriter
-    boolean mergeDocStores;         // used by IndexWriter
     boolean optimize;               // used by IndexWriter
     boolean increfDone;             // used by IndexWriter
     boolean registerDone;           // used by IndexWriter
@@ -157,9 +156,6 @@ public abstract class MergePolicy implem
         b.append(" into ").append(info.name);
       if (optimize)
         b.append(" [optimize]");
-      if (mergeDocStores) {
-        b.append(" [mergeDocStores]");
-      }
       return b.toString();
     }
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriter.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriter.java Sat Jul 24 06:04:30 2010
@@ -103,10 +103,6 @@ final class NormsWriter extends Inverted
   }
 
   @Override
-  void closeDocStore(SegmentWriteState state) {}
-
-  
-  @Override
   void finishDocument() throws IOException {}
 
   @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Sat Jul 24 06:04:30 2010
@@ -70,11 +70,11 @@ public final class SegmentInfo {
 
   long sizeInBytes = -1;                          // total byte size of all of our files (computed on demand)
 
-  private int docStoreOffset;                     // if this segment shares stored fields & vectors, this
+  @Deprecated private int docStoreOffset;                     // if this segment shares stored fields & vectors, this
                                                   // offset is where in that file this segment's docs begin
-  private String docStoreSegment;                 // name used to derive fields/vectors file we share with
+  @Deprecated private String docStoreSegment;                 // name used to derive fields/vectors file we share with
                                                   // other segments
-  private boolean docStoreIsCompoundFile;         // whether doc store files are stored in compound file (*.cfx)
+  @Deprecated private boolean docStoreIsCompoundFile;         // whether doc store files are stored in compound file (*.cfx)
 
   private int delCount;                           // How many deleted docs in this segment
 
@@ -87,20 +87,23 @@ public final class SegmentInfo {
 
   private Map<String,String> diagnostics;
 
-  public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, int docStoreOffset, 
-                     String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx, Codec codec) { 
+  public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, boolean hasProx, Codec codec) {
+    this(name, docCount, dir, isCompoundFile, -1, null, false, hasProx, codec);
+  }
+  
+  private SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, int docStoreOffset, 
+                            String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx, Codec codec) {
     this.name = name;
     this.docCount = docCount;
     this.dir = dir;
     delGen = NO;
     this.isCompoundFile = isCompoundFile;
-    this.docStoreOffset = docStoreOffset;
-    this.docStoreSegment = docStoreSegment;
-    this.docStoreIsCompoundFile = docStoreIsCompoundFile;
     this.hasProx = hasProx;
     this.codec = codec;
     delCount = 0;
-    assert docStoreOffset == -1 || docStoreSegment != null: "dso=" + docStoreOffset + " dss=" + docStoreSegment + " docCount=" + docCount;
+    this.docStoreOffset = docStoreOffset;
+    this.docStoreIsCompoundFile = docStoreIsCompoundFile;
+    this.docStoreSegment = docStoreSegment;
   }
 
   /**
@@ -176,11 +179,13 @@ public final class SegmentInfo {
       docStoreSegment = name;
       docStoreIsCompoundFile = false;
     }
+
     if (format > DefaultSegmentInfosWriter.FORMAT_4_0) {
       // pre-4.0 indexes write a byte if there is a single norms file
       byte b = input.readByte();
       assert 1 == b;
     }
+
     int numNormGen = input.readInt();
     if (numNormGen == NO) {
       normGen = null;
@@ -367,32 +372,24 @@ public final class SegmentInfo {
     assert delCount <= docCount;
   }
 
+  @Deprecated
   public int getDocStoreOffset() {
     return docStoreOffset;
   }
   
+  @Deprecated
   public boolean getDocStoreIsCompoundFile() {
     return docStoreIsCompoundFile;
   }
   
-  void setDocStoreIsCompoundFile(boolean v) {
-    docStoreIsCompoundFile = v;
-    clearFiles();
-  }
-  
+  @Deprecated
   public String getDocStoreSegment() {
     return docStoreSegment;
   }
-  
-  void setDocStoreOffset(int offset) {
-    docStoreOffset = offset;
-    clearFiles();
-  }
 
-  void setDocStore(int offset, String segment, boolean isCompoundFile) {        
-    docStoreOffset = offset;
-    docStoreSegment = segment;
-    docStoreIsCompoundFile = isCompoundFile;
+  @Deprecated
+  public void setDocStoreSegment(String docStoreSegment) {
+    this.docStoreSegment = docStoreSegment;
     clearFiles();
   }
   
@@ -403,12 +400,14 @@ public final class SegmentInfo {
     output.writeString(name);
     output.writeInt(docCount);
     output.writeLong(delGen);
+
     output.writeInt(docStoreOffset);
     if (docStoreOffset != -1) {
       output.writeString(docStoreSegment);
       output.writeByte((byte) (docStoreIsCompoundFile ? 1:0));
     }
 
+
     if (normGen == null) {
       output.writeInt(NO);
     } else {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Sat Jul 24 06:04:30 2010
@@ -65,12 +65,6 @@ final class SegmentMerger {
 
   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;
@@ -127,22 +121,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
@@ -154,7 +132,7 @@ final class SegmentMerger {
     mergeTerms();
     mergeNorms();
 
-    if (mergeDocStores && fieldInfos.hasVectors())
+    if (fieldInfos.hasVectors())
       mergeVectors();
 
     return mergedDocs;
@@ -179,9 +157,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));
     }
 
     codec.files(directory, info, fileSet);
@@ -197,7 +173,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));
       }
@@ -270,17 +246,20 @@ final class SegmentMerger {
    */
   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
-    }
+    //nocommit
+//    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
+//    }
 
+    fieldInfos = new FieldInfos();      // merge field names
+    
     for (IndexReader reader : readers) {
       if (reader instanceof SegmentReader) {
         SegmentReader segmentReader = (SegmentReader) reader;
@@ -310,54 +289,45 @@ 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 && fieldsReader.canReadRawDocs()) {            
-              matchingFieldsReader = fieldsReader;
-            }
+    // 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 && fieldsReader.canReadRawDocs()) {            
+            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, codecs);
+      
+    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecs);
 
     return docCount;
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java Sat Jul 24 06:04:30 2010
@@ -33,9 +33,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 final Collection<String> flushedFiles;
 
   // Actual codec used
@@ -61,16 +59,12 @@ public class SegmentWriteState {
   public final int maxSkipLevels = 10;
 
   public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
-                           String docStoreSegmentName, int numDocs,
-                           int numDocsInStore, int termIndexInterval,
-                           CodecProvider codecs) {
+                           int numDocs, int termIndexInterval, CodecProvider codecs) {
     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.codec = codecs.getWriter(this);
     flushedFiles = new HashSet<String>();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Sat Jul 24 06:04:30 2010
@@ -20,8 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -29,39 +27,38 @@ import org.apache.lucene.util.RamUsageEs
 final class StoredFieldsWriter {
 
   FieldsWriter fieldsWriter;
-  final FieldsWriter localFieldsWriter;
   final DocumentsWriterPerThread docWriter;
   final FieldInfos fieldInfos;
   int lastDocID;
-  private String docStoreSegment;
+  private String segment;
 
-  PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;
   
-  PerDoc doc;
   final DocumentsWriterPerThread.DocState docState;
 
   public StoredFieldsWriter(DocumentsWriterPerThread docWriter, FieldInfos fieldInfos) {
     this.docWriter = docWriter;
     this.fieldInfos = fieldInfos;
     this.docState = docWriter.docState;
-    localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null, fieldInfos);
+  }
+  
+  private int numStoredFields;
+  private Fieldable[] storedFields;
+  private int[] fieldNumbers;
+
+  public void reset() {
+    numStoredFields = 0;
+    storedFields = new Fieldable[1];
+    fieldNumbers = new int[1];
   }
 
   public void startDocument() {
-    if (doc != null) {
-      // Only happens if previous document hit non-aborting
-      // exception while writing stored fields into
-      // localFieldsWriter:
-      doc.reset();
-      doc.docID = docState.docID;
-    }
+    reset();
   }
-
-
+  
   public void flush(SegmentWriteState state) throws IOException {
 
-    if (state.numDocsInStore > 0) {
+    if (state.numDocs > 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:
@@ -69,86 +66,47 @@ final class StoredFieldsWriter {
 
       // 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;
-      }
-    }
-  }
-
-  public void closeDocStore(SegmentWriteState state) throws IOException {
-    final int inc = state.numDocsInStore - lastDocID;
-    if (inc > 0) {
-      initFieldsWriter();
-      fill(state.numDocsInStore - docWriter.getDocStoreOffset());
+      fill(state.numDocs);
     }
 
     if (fieldsWriter != null) {
+      fieldsWriter.flush();
       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.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));
+      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));
+      }
     }
-  }
 
-  int allocCount;
-
-  PerDoc getPerDoc() {
-    if (freeCount == 0) {
-      allocCount++;
-      if (allocCount > docFreeList.length) {
-        // Grow our free list up front to make sure we have
-        // enough space to recycle all outstanding PerDoc
-        // instances
-        assert allocCount == 1+docFreeList.length;
-        docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      }
-      return new PerDoc();
-    } else
-      return docFreeList[--freeCount];
   }
 
-  public DocumentsWriterPerThread.DocWriter finishDocument() {
-    // If there were any stored fields in this doc, doc will
-    // be non-null; else it's null.
-    try {
-      return doc;
-    } finally {
-      doc = null;
+  private void initFieldsWriter() throws IOException {
+    if (fieldsWriter == null) {
+      segment = docWriter.getSegment();
+      if (segment != null) {
+        fieldsWriter = new FieldsWriter(docWriter.directory,
+                                        segment,
+                                        fieldInfos);
+        docWriter.addOpenFile(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION));
+        docWriter.addOpenFile(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
+        lastDocID = 0;
+      }
     }
   }
 
+  int allocCount;
+
   void abort() {
-    if (doc != null) {
-      doc.abort();
-      doc = null;
-    }
+    reset();
 
     if (fieldsWriter != null) {
       try {
@@ -162,83 +120,50 @@ 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;
+    final int end = docID;
     while(lastDocID < end) {
       fieldsWriter.skipDocument();
       lastDocID++;
     }
   }
 
-  void finishDocument(PerDoc perDoc) throws IOException {
+  void finishDocument() throws IOException {
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument start");
-    initFieldsWriter();
 
-    fill(perDoc.docID);
-
-    // Append stored fields to the real FieldsWriter:
-    fieldsWriter.flushDocument(perDoc.numStoredFields, perDoc.fdt);
-    lastDocID++;
-    perDoc.reset();
-    free(perDoc);
+    initFieldsWriter();
+    fill(docState.docID);
+  
+    if (fieldsWriter != null && numStoredFields > 0) {  
+      fieldsWriter.startDocument(numStoredFields);
+      for (int i = 0; i < numStoredFields; i++) {
+        fieldsWriter.writeField(fieldNumbers[i], storedFields[i]);
+      }
+      lastDocID++;
+    }
+    
+    reset();
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
   }
 
+  
   public void addField(Fieldable field, FieldInfo fieldInfo) throws IOException {
-    if (doc == null) {
-      doc = getPerDoc();
-      doc.docID = docState.docID;
-      localFieldsWriter.setFieldsStream(doc.fdt);
-      assert doc.numStoredFields == 0: "doc.numStoredFields=" + doc.numStoredFields;
-      assert 0 == doc.fdt.length();
-      assert 0 == doc.fdt.getFilePointer();
-    }
-
-    localFieldsWriter.writeField(fieldInfo, field);
+    if (numStoredFields == storedFields.length) {
+      int newSize = ArrayUtil.oversize(numStoredFields + 1, RamUsageEstimator.NUM_BYTES_OBJ_REF);
+      Fieldable[] newArray = new Fieldable[newSize];
+      System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
+      storedFields = newArray;
+    }
+    
+    if (numStoredFields == fieldNumbers.length) {
+      fieldNumbers = ArrayUtil.grow(fieldNumbers);
+    }
+    
+    storedFields[numStoredFields] = field;
+    fieldNumbers[numStoredFields] = fieldInfo.number;
+    numStoredFields++;
+    
     assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField");
-    doc.numStoredFields++;
-  }
-  
-  public boolean freeRAM() {
-    return false;
-  }
-
-  void free(PerDoc perDoc) {
-    assert freeCount < docFreeList.length;
-    assert 0 == perDoc.numStoredFields;
-    assert 0 == perDoc.fdt.length();
-    assert 0 == perDoc.fdt.getFilePointer();
-    docFreeList[freeCount++] = perDoc;
-  }
-
-  class PerDoc extends DocumentsWriterPerThread.DocWriter {
-    final DocumentsWriterPerThread.PerDocBuffer buffer = docWriter.newPerDocBuffer();
-    RAMOutputStream fdt = new RAMOutputStream(buffer);
-    int numStoredFields;
-
-    void reset() {
-      fdt.reset();
-      buffer.recycle();
-      numStoredFields = 0;
-    }
-
-    @Override
-    public void abort() {
-      reset();
-      free(this);
-    }
-
-    @Override
-    public long sizeInBytes() {
-      return buffer.getSizeInBytes();
-    }
-
-    @Override
-    public void finish() throws IOException {
-      finishDocument(this);
-    }
   }
 }



Mime
View raw message