You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2010/07/24 08:04:31 UTC

svn commit: r978805 [2/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/

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Sat Jul 24 06:04:30 2010
@@ -20,9 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Map;
 
-import org.apache.lucene.index.DocumentsWriterPerThread.DocWriter;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -31,7 +29,6 @@ final class TermVectorsTermsWriter exten
 
   final DocumentsWriterPerThread docWriter;
   TermVectorsWriter termVectorsWriter;
-  PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;
   IndexOutput tvx;
   IndexOutput tvd;
@@ -40,7 +37,6 @@ final class TermVectorsTermsWriter exten
   
   final DocumentsWriterPerThread.DocState docState;
   final BytesRef flushTerm = new BytesRef();
-  TermVectorsTermsWriter.PerDoc doc;
   
   // Used by perField when serializing the term vectors
   final ByteSliceReader vectorSliceReader = new ByteSliceReader();
@@ -55,40 +51,26 @@ final class TermVectorsTermsWriter exten
 
     if (tvx != null) {
 
-      if (state.numDocsInStore > 0)
+      if (state.numDocs > 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());
+        fill(state.numDocs);
+      }
 
       tvx.flush();
       tvd.flush();
       tvf.flush();
-    }
-
-    for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) {
-      TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field;
-      perField.termsHashPerField.reset();
-      perField.shrinkHash();
-    }
-  }
-
-  @Override
-  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 idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
+      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));
 
-      String fldName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
-      String docName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+      String fldName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
+      String docName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
       state.flushedFiles.add(idxName);
       state.flushedFiles.add(fldName);
       state.flushedFiles.add(docName);
@@ -98,31 +80,20 @@ final class TermVectorsTermsWriter exten
       docWriter.removeOpenFile(docName);
 
       lastDocID = 0;
-    }    
-  }
 
-  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];
+    for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) {
+      TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field;
+      perField.termsHashPerField.reset();
+      perField.shrinkHash();
+    }
   }
 
   /** 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;
+    final int end = docID;
     if (lastDocID < end) {
       final long tvfPosition = tvf.getFilePointer();
       while(lastDocID < end) {
@@ -137,18 +108,18 @@ final class TermVectorsTermsWriter exten
   void initTermVectorsWriter() throws IOException {        
     if (tvx == null) {
       
-      final String docStoreSegment = docWriter.getDocStoreSegment();
+      final String segment = docWriter.getSegment();
 
-      if (docStoreSegment == null)
+      if (segment == 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);
+      String idxName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
+      String docName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+      String fldName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
       tvx = docWriter.directory.createOutput(idxName);
       tvd = docWriter.directory.createOutput(docName);
       tvf = docWriter.directory.createOutput(fldName);
@@ -165,53 +136,44 @@ final class TermVectorsTermsWriter exten
     }
   }
 
-  void finishDocument(PerDoc perDoc) throws IOException {
+  @Override
+  void finishDocument(TermsHash termsHash) throws IOException {
 
     assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument start");
 
     initTermVectorsWriter();
 
-    fill(perDoc.docID);
-
+    fill(docState.docID);
+    
     // Append term vectors to the real outputs:
     tvx.writeLong(tvd.getFilePointer());
     tvx.writeLong(tvf.getFilePointer());
-    tvd.writeVInt(perDoc.numVectorFields);
-    if (perDoc.numVectorFields > 0) {
-      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++) {
-        long pos = perDoc.fieldPointers[i];
+    tvd.writeVInt(numVectorFields);
+    if (numVectorFields > 0) {
+      for(int i=0;i<numVectorFields;i++) {
+        tvd.writeVInt(perFields[i].fieldInfo.number);
+      }
+      long lastPos = tvf.getFilePointer();
+      perFields[0].finishDocument();
+      for(int i=1;i<numVectorFields;i++) {
+        long pos = tvf.getFilePointer();
         tvd.writeVLong(pos-lastPos);
         lastPos = pos;
+        perFields[i].finishDocument();
       }
-      perDoc.perDocTvf.writeTo(tvf);
-      perDoc.numVectorFields = 0;
     }
 
-    assert lastDocID == perDoc.docID + docWriter.getDocStoreOffset();
+    assert lastDocID == docState.docID;
 
     lastDocID++;
 
-    perDoc.reset();
-    free(perDoc);
+    termsHash.reset();
+    reset();
     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() {
-    if (doc != null) {
-      doc.abort();
-      doc = null;
-    }
 
     if (tvx != null) {
       try {
@@ -239,54 +201,13 @@ final class TermVectorsTermsWriter exten
 
   }
 
-  void free(PerDoc doc) {
-    assert freeCount < docFreeList.length;
-    docFreeList[freeCount++] = doc;
-  }
-
-  class PerDoc extends DocumentsWriterPerThread.DocWriter {
-
-    final DocumentsWriterPerThread.PerDocBuffer buffer = docWriter.newPerDocBuffer();
-    RAMOutputStream perDocTvf = new RAMOutputStream(buffer);
-
-    int numVectorFields;
-
-    int[] fieldNumbers = new int[1];
-    long[] fieldPointers = new long[1];
-
-    void reset() {
-      perDocTvf.reset();
-      buffer.recycle();
-      numVectorFields = 0;
-    }
-
-    @Override
-    public void abort() {
-      reset();
-      free(this);
-    }
-
-    void addField(final int fieldNumber) {
-      if (numVectorFields == fieldNumbers.length) {
-        fieldNumbers = ArrayUtil.grow(fieldNumbers);
-      }
-      if (numVectorFields == fieldPointers.length) {
-        fieldPointers = ArrayUtil.grow(fieldPointers);
-      }
-      fieldNumbers[numVectorFields] = fieldNumber;
-      fieldPointers[numVectorFields] = perDocTvf.getFilePointer();
-      numVectorFields++;
-    }
+  int numVectorFields;
 
-    @Override
-    public long sizeInBytes() {
-      return buffer.getSizeInBytes();
-    }
+  TermVectorsTermsWriterPerField[] perFields;
 
-    @Override
-    public void finish() throws IOException {
-      finishDocument(this);
-    }
+  void reset() {
+    numVectorFields = 0;
+    perFields = new TermVectorsTermsWriterPerField[1];
   }
 
   @Override
@@ -294,22 +215,22 @@ final class TermVectorsTermsWriter exten
     return new TermVectorsTermsWriterPerField(termsHashPerField, this, fieldInfo);
   }
 
-  @Override
-  DocWriter finishDocument() throws IOException {
-    try {
-      return doc;
-    } finally {
-      doc = null;
+  void addFieldToFlush(TermVectorsTermsWriterPerField fieldToFlush) {
+    if (numVectorFields == perFields.length) {
+      int newSize = ArrayUtil.oversize(numVectorFields + 1, RamUsageEstimator.NUM_BYTES_OBJ_REF);
+      TermVectorsTermsWriterPerField[] newArray = new TermVectorsTermsWriterPerField[newSize];
+      System.arraycopy(perFields, 0, newArray, 0, numVectorFields);
+      perFields = newArray;
     }
-  }
 
+    perFields[numVectorFields++] = fieldToFlush;
+  }
+  
   @Override
   void startDocument() throws IOException {
     assert clearLastVectorFieldName();
-    if (doc != null) {
-      doc.reset();
-      doc.docID = docState.docID;
-    }
+    perFields = new TermVectorsTermsWriterPerField[1];
+    reset();
   }
   
   // Called only by assert

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java Sat Jul 24 06:04:30 2010
@@ -68,15 +68,7 @@ final class TermVectorsTermsWriterPerFie
     }
 
     if (doVectors) {
-      if (termsWriter.doc == null) {
-        termsWriter.doc = termsWriter.getPerDoc();
-        termsWriter.doc.docID = docState.docID;
-        assert termsWriter.doc.numVectorFields == 0;
-        assert 0 == termsWriter.doc.perDocTvf.length();
-        assert 0 == termsWriter.doc.perDocTvf.getFilePointer();
-      } else {
-        assert termsWriter.doc.docID == docState.docID;
-
+      if (termsWriter.tvx != null) {
         if (termsHashPerField.numPostings != 0)
           // Only necessary if previous doc hit a
           // non-aborting exception while writing vectors in
@@ -93,13 +85,15 @@ final class TermVectorsTermsWriterPerFie
 
   public void abort() {}
 
-  /** Called once per field per document if term vectors
-   *  are enabled, to write the vectors to
-   *  RAMOutputStream, which is then quickly flushed to
-   *  * the real term vectors files in the Directory. */
   @Override
   void finish() throws IOException {
+    if (!doVectors || termsHashPerField.numPostings == 0)
+      return;
 
+    termsWriter.addFieldToFlush(this);
+  }
+  
+  void finishDocument() throws IOException {
     assert docState.testPoint("TermVectorsTermsWriterPerField.finish start");
 
     final int numPostings = termsHashPerField.numPostings;
@@ -108,14 +102,9 @@ final class TermVectorsTermsWriterPerFie
 
     assert numPostings >= 0;
 
-    if (!doVectors || numPostings == 0)
-      return;
-
     if (numPostings > maxNumPostings)
       maxNumPostings = numPostings;
 
-    final IndexOutput tvf = termsWriter.doc.perDocTvf;
-
     // This is called once, after inverting all occurrences
     // of a given field in the doc.  At this point we flush
     // our hash into the DocWriter.
@@ -123,8 +112,8 @@ final class TermVectorsTermsWriterPerFie
     assert fieldInfo.storeTermVector;
     assert termsWriter.vectorFieldsInOrder(fieldInfo);
 
-    termsWriter.doc.addField(termsHashPerField.fieldInfo.number);
     TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
+    final IndexOutput tvf = termsWriter.tvf;
 
     // TODO: we may want to make this sort in same order
     // as Codec's terms dict?
@@ -186,7 +175,6 @@ final class TermVectorsTermsWriterPerFie
     }
 
     termsHashPerField.reset();
-    termsHashPerField.termsHash.reset();
   }
 
   void shrinkHash() {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHash.java Sat Jul 24 06:04:30 2010
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.index.DocumentsWriterPerThread.DocWriter;
 import org.apache.lucene.util.BytesRef;
 
 /** This class implements {@link InvertedDocConsumer}, which
@@ -99,14 +98,6 @@ final class TermsHash extends InvertedDo
     }
   }
 
-
-  @Override
-  void closeDocStore(SegmentWriteState state) throws IOException {
-    consumer.closeDocStore(state);
-    if (nextTermsHash != null)
-      nextTermsHash.closeDocStore(state);
-  }
-
   @Override
   void flush(Map<FieldInfo,InvertedDocConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
     Map<FieldInfo,TermsHashConsumerPerField> childFields = new HashMap<FieldInfo,TermsHashConsumerPerField>();
@@ -144,20 +135,13 @@ final class TermsHash extends InvertedDo
   }
 
   @Override
-  DocWriter finishDocument() throws IOException {
-    final DocumentsWriterPerThread.DocWriter doc = consumer.finishDocument();
-
-    final DocumentsWriterPerThread.DocWriter doc2;
-    if (nextTermsHash != null) {
-      doc2 = nextTermsHash.consumer.finishDocument();
-    } else {
-      doc2 = null;
-    }
-    if (doc == null) {
-      return doc2;
-    } else {
-      doc.setNext(doc2);
-      return doc;
+  void finishDocument() throws IOException {
+    try {
+      consumer.finishDocument(this);
+    } finally {
+      if (nextTermsHash != null) {
+        nextTermsHash.consumer.finishDocument(nextTermsHash);
+      }
     }
   }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Sat Jul 24 06:04:30 2010
@@ -23,10 +23,8 @@ import java.util.Map;
 abstract class TermsHashConsumer {
   abstract void flush(Map<FieldInfo, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException;
   abstract void abort();
-  abstract void closeDocStore(SegmentWriteState state) throws IOException;
-
   abstract void startDocument() throws IOException;
-  abstract DocumentsWriterPerThread.DocWriter finishDocument() throws IOException;
+  abstract void finishDocument(TermsHash termsHash) throws IOException;
   abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo);
   FieldInfos fieldInfos;
 

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestCodecs.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestCodecs.java Sat Jul 24 06:04:30 2010
@@ -281,7 +281,7 @@ public class TestCodecs extends MultiCod
 
     final Directory dir = new MockRAMDirectory();
     this.write(fieldInfos, dir, fields);
-    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, CodecProvider.getDefault().getWriter(null));
+    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, CodecProvider.getDefault().getWriter(null));
     si.setHasProx(false);
 
     final FieldsProducer reader = si.getCodec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 64, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
@@ -319,7 +319,7 @@ public class TestCodecs extends MultiCod
     final Directory dir = new MockRAMDirectory();
 
     this.write(fieldInfos, dir, fields);
-    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, -1, SEGMENT, false, true, CodecProvider.getDefault().getWriter(null));
+    final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, CodecProvider.getDefault().getWriter(null));
 
     final FieldsProducer terms = si.getCodec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 1024, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
 
@@ -602,7 +602,7 @@ public class TestCodecs extends MultiCod
 
     final int termIndexInterval = this.nextInt(13, 27);
 
-    final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, null, 10000, 10000, termIndexInterval,
+    final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, 10000, termIndexInterval,
                                                     CodecProvider.getDefault());
 
     final FieldsConsumer consumer = state.codec.fieldsConsumer(state);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java Sat Jul 24 06:04:30 2010
@@ -84,7 +84,7 @@ public class TestConcurrentMergeSchedule
         writer.addDocument(doc);
         failure.setDoFail();
         try {
-          writer.flush(true, false, true);
+          writer.flush(true, true);
           if (failure.hitExc) {
             fail("failed to hit IOException");
           }

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestDoc.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestDoc.java Sat Jul 24 06:04:30 2010
@@ -194,7 +194,7 @@ public class TestDoc extends LuceneTestC
       merger.closeReaders();
       
       final SegmentInfo info = new SegmentInfo(merged, si1.docCount + si2.docCount, si1.dir,
-                                               useCompoundFile, -1, null, false, merger.hasProx(), merger.getCodec());
+                                               useCompoundFile, merger.hasProx(), merger.getCodec());
       
       if (useCompoundFile) {
         List<String> filesToDelete = merger.createCompoundFile(merged + ".cfs", info);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Sat Jul 24 06:04:30 2010
@@ -1437,7 +1437,7 @@ public class TestIndexWriter extends Luc
       doc.add(new Field("field", "aaa", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
       for(int i=0;i<19;i++)
         writer.addDocument(doc);
-      writer.flush(false, true, true);
+      writer.flush(false, true);
       writer.close();
       SegmentInfos sis = new SegmentInfos();
       sis.read(dir);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java Sat Jul 24 06:04:30 2010
@@ -146,7 +146,7 @@ public class TestIndexWriterReader exten
     writer.setInfoStream(infoStream);
     // create the index
     createIndexNoClose(!optimize, "index1", writer);
-    writer.flush(false, true, true);
+    writer.flush(false, true);
 
     // create a 2nd index
     Directory dir2 = new MockRAMDirectory();
@@ -224,7 +224,7 @@ public class TestIndexWriterReader exten
     writer.setInfoStream(infoStream);
     // create the index
     createIndexNoClose(!optimize, "index1", writer);
-    writer.flush(false, true, true);
+    writer.flush(false, true);
     // get a reader
     IndexReader r1 = writer.getReader();
 
@@ -426,7 +426,7 @@ public class TestIndexWriterReader exten
     IndexReader r1 = writer.getReader();
     assertEquals(0, r1.maxDoc());
     createIndexNoClose(false, "index1", writer);
-    writer.flush(!optimize, true, true);
+    writer.flush(!optimize, true);
 
     IndexReader iwr1 = writer.getReader();
     assertEquals(100, iwr1.maxDoc());
@@ -438,7 +438,7 @@ public class TestIndexWriterReader exten
       Document d = createDocument(x, "index1", 5);
       writer.addDocument(d);
     }
-    writer.flush(false, true, true);
+    writer.flush(false, true);
     // verify the reader was reopened internally
     IndexReader iwr2 = writer.getReader();
     assertTrue(iwr2 != r1);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java Sat Jul 24 06:04:30 2010
@@ -73,8 +73,8 @@ public class TestSegmentMerger extends L
     merger.closeReaders();
     assertTrue(docsMerged == 2);
     //Should be able to open a new SegmentReader against the new directory
-    SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment, docsMerged, mergedDir, false, -1,
-        null, false, merger.hasProx(), merger.getCodec()), BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, null);
+    SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment, docsMerged, mergedDir, false, 
+        merger.hasProx(), merger.getCodec()), BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, null);
 
     assertTrue(mergedReader != null);
     assertTrue(mergedReader.numDocs() == 2);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java?rev=978805&r1=978804&r2=978805&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java Sat Jul 24 06:04:30 2010
@@ -134,7 +134,7 @@ public class TestSurrogates extends Luce
     dir.createOutput(segName + ".frq").close();
 
     // !!hack alert!! stuffing uniqueTermCount in as docCount
-    return new SegmentInfo(segName, uniqueTermCount, dir, false, -1, null, false, true, codec);
+    return new SegmentInfo(segName, uniqueTermCount, dir, false, true, codec);
   }
 
   private String toHexString(Term t) {