Index: src/test/org/apache/lucene/index/TestDoc.java =================================================================== --- src/test/org/apache/lucene/index/TestDoc.java (revision 966819) +++ src/test/org/apache/lucene/index/TestDoc.java (working copy) @@ -194,7 +194,7 @@ 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 filesToDelete = merger.createCompoundFile(merged + ".cfs", info); Index: src/test/org/apache/lucene/index/TestIndexWriter.java =================================================================== --- src/test/org/apache/lucene/index/TestIndexWriter.java (revision 966819) +++ src/test/org/apache/lucene/index/TestIndexWriter.java (working copy) @@ -1437,7 +1437,7 @@ 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); Index: src/test/org/apache/lucene/index/TestCodecs.java =================================================================== --- src/test/org/apache/lucene/index/TestCodecs.java (revision 966818) +++ src/test/org/apache/lucene/index/TestCodecs.java (working copy) @@ -281,7 +281,7 @@ 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 @@ 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 @@ 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); Index: src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java =================================================================== --- src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (revision 966818) +++ src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (working copy) @@ -84,7 +84,7 @@ writer.addDocument(doc); failure.setDoFail(); try { - writer.flush(true, false, true); + writer.flush(true, true); if (failure.hitExc) { fail("failed to hit IOException"); } Index: src/test/org/apache/lucene/index/TestIndexWriterReader.java =================================================================== --- src/test/org/apache/lucene/index/TestIndexWriterReader.java (revision 966819) +++ src/test/org/apache/lucene/index/TestIndexWriterReader.java (working copy) @@ -146,7 +146,7 @@ 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 @@ 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 @@ 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 @@ 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); Index: src/test/org/apache/lucene/index/TestSegmentMerger.java =================================================================== --- src/test/org/apache/lucene/index/TestSegmentMerger.java (revision 966819) +++ src/test/org/apache/lucene/index/TestSegmentMerger.java (working copy) @@ -73,8 +73,8 @@ 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); Index: src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java =================================================================== --- src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java (revision 966819) +++ src/test/org/apache/lucene/index/codecs/preflex/TestSurrogates.java (working copy) @@ -134,7 +134,7 @@ 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) { Index: src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java =================================================================== --- src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java (revision 966818) +++ src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java (working copy) @@ -121,8 +121,8 @@ throw new ThreadInterruptedException(ie); } + pauseAllThreads(); globalLock = true; - pauseAllThreads(); } finally { lock.unlock(); } Index: src/java/org/apache/lucene/index/DocumentsWriterPerThread.java =================================================================== --- src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (revision 966818) +++ src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (working copy) @@ -10,7 +10,6 @@ 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 @@ } catch (Throwable t) { } - docStoreSegment = null; - numDocsInStore = 0; - docStoreOffset = 0; - // Reset all postings data doAfterFlush(); @@ -121,18 +116,14 @@ 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 closedFiles = new ArrayList(); long numBytesUsed; @@ -161,13 +152,15 @@ 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 @@ success = false; try { - if (perDoc != null) { - perDoc.finish(); - } + consumer.finishDocument(); success = true; } finally { @@ -201,7 +192,6 @@ sequenceIDs[numDocsInRAM] = sequenceID; numDocsInRAM++; - numDocsInStore++; } int getNumDocsInRAM() { @@ -226,26 +216,6 @@ 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,13 +223,13 @@ } /** Flush all pending docs to a new segment */ - SegmentInfo flush(boolean closeDocStore) throws IOException { + SegmentInfo flush() throws IOException { assert numDocsInRAM > 0; - initFlushState(closeDocStore); + flushState = new SegmentWriteState(infoStream, directory, segment, docFieldProcessor.fieldInfos, + numDocsInRAM, writer.getConfig().getTermIndexInterval(), + writer.codecs); - docStoreOffset = numDocsInStore; - if (infoStream != null) { message("flush postings as segment " + flushState.segmentName + " numDocs=" + numDocsInRAM); } @@ -267,22 +237,12 @@ 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 @@ SegmentInfo newSegment = new SegmentInfo(flushState.segmentName, flushState.numDocs, directory, false, - docStoreOffset, flushState.docStoreSegmentName, - false, hasProx(), getCodec()); @@ -325,62 +283,17 @@ } } - /** 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 closedFiles() { return (List) ((ArrayList) closedFiles).clone(); } + void addOpenFile(String name) { synchronized(parent.openFiles) { assert !parent.openFiles.contains(name); @@ -396,58 +309,6 @@ 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); } Index: src/java/org/apache/lucene/index/TermsHash.java =================================================================== --- src/java/org/apache/lucene/index/TermsHash.java (revision 966818) +++ src/java/org/apache/lucene/index/TermsHash.java (working copy) @@ -21,7 +21,6 @@ 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,15 +98,7 @@ } } - @Override - void closeDocStore(SegmentWriteState state) throws IOException { - consumer.closeDocStore(state); - if (nextTermsHash != null) - nextTermsHash.closeDocStore(state); - } - - @Override void flush(Map fieldsToFlush, final SegmentWriteState state) throws IOException { Map childFields = new HashMap(); Map nextChildFields; @@ -144,21 +135,14 @@ } @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; + void finishDocument() throws IOException { + try { + consumer.finishDocument(this); + } finally { + if (nextTermsHash != null) { + nextTermsHash.consumer.finishDocument(nextTermsHash); + } } - if (doc == null) { - return doc2; - } else { - doc.setNext(doc2); - return doc; - } } @Override Index: src/java/org/apache/lucene/index/FieldsWriter.java =================================================================== --- src/java/org/apache/lucene/index/FieldsWriter.java (revision 966819) +++ src/java/org/apache/lucene/index/FieldsWriter.java (working copy) @@ -117,10 +117,9 @@ // 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 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 @@ for (Fieldable field : fields) { if (field.isStored()) - writeField(fieldInfos.fieldInfo(field.name()), field); + writeField(fieldInfos.fieldInfo(field.name()).number, field); } } } Index: src/java/org/apache/lucene/index/SegmentMerger.java =================================================================== --- src/java/org/apache/lucene/index/SegmentMerger.java (revision 966818) +++ src/java/org/apache/lucene/index/SegmentMerger.java (working copy) @@ -65,12 +65,6 @@ 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 @@ * @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 @@ mergeTerms(); mergeNorms(); - if (mergeDocStores && fieldInfos.hasVectors()) + if (fieldInfos.hasVectors()) mergeVectors(); return mergedDocs; @@ -179,9 +157,7 @@ // 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 @@ } // 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 @@ */ 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,55 +289,46 @@ setMatchingSegmentReaders(); - if (mergeDocStores) { - // merge field values - final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos); + // 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; - } + 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); + 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"); - - } 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, docCount, termIndexInterval, codecs); - segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval, codecs); - return docCount; } Index: src/java/org/apache/lucene/index/DocFieldProcessor.java =================================================================== --- src/java/org/apache/lucene/index/DocFieldProcessor.java (revision 966818) +++ src/java/org/apache/lucene/index/DocFieldProcessor.java (working copy) @@ -26,8 +26,6 @@ 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 @@ } @Override - public void closeDocStore(SegmentWriteState state) throws IOException { - consumer.closeDocStore(state); - fieldsWriter.closeDocStore(state); - } - - @Override public void flush(SegmentWriteState state) throws IOException { Map childFields = new HashMap(); @@ -105,8 +97,11 @@ } } - fieldsWriter.abort(); - consumer.abort(); + try { + fieldsWriter.abort(); + } finally { + consumer.abort(); + } } @Override @@ -190,7 +185,7 @@ } @Override - public DocumentsWriterPerThread.DocWriter processDocument() throws IOException { + public void processDocument() throws IOException { consumer.startDocument(); fieldsWriter.startDocument(); @@ -259,14 +254,9 @@ 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 @@ 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 @@ 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); - } - } - } } Index: src/java/org/apache/lucene/index/SegmentInfo.java =================================================================== --- src/java/org/apache/lucene/index/SegmentInfo.java (revision 966819) +++ src/java/org/apache/lucene/index/SegmentInfo.java (working copy) @@ -70,11 +70,11 @@ 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 @@ private Map 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 @@ 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 @@ 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 @@ 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 { Index: src/java/org/apache/lucene/index/FreqProxTermsWriter.java =================================================================== --- src/java/org/apache/lucene/index/FreqProxTermsWriter.java (revision 966818) +++ src/java/org/apache/lucene/index/FreqProxTermsWriter.java (working copy) @@ -20,22 +20,15 @@ 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 @@ } @Override - DocWriter finishDocument() throws IOException { - return null; + void finishDocument(TermsHash termsHash) throws IOException { } @Override Index: src/java/org/apache/lucene/index/InvertedDocConsumer.java =================================================================== --- src/java/org/apache/lucene/index/InvertedDocConsumer.java (revision 966818) +++ src/java/org/apache/lucene/index/InvertedDocConsumer.java (working copy) @@ -28,14 +28,11 @@ /** Flush a new segment */ abstract void flush(Map 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 */ Index: src/java/org/apache/lucene/index/DocFieldProcessorPerField.java =================================================================== --- src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (revision 966818) +++ src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (working copy) @@ -18,6 +18,8 @@ */ 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 @@ 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(); } Index: src/java/org/apache/lucene/index/DocFieldConsumer.java =================================================================== --- src/java/org/apache/lucene/index/DocFieldConsumer.java (revision 966818) +++ src/java/org/apache/lucene/index/DocFieldConsumer.java (working copy) @@ -27,10 +27,6 @@ /** Called when DocumentsWriter decides to create a new * segment */ abstract void flush(Map 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 DocFieldConsumerPerField addField(FieldInfo fi); - abstract DocumentsWriterPerThread.DocWriter finishDocument() throws IOException; + abstract void finishDocument() throws IOException; void setFieldInfos(FieldInfos fieldInfos) { this.fieldInfos = fieldInfos; Index: src/java/org/apache/lucene/index/DocumentsWriterRAMAllocator.java =================================================================== --- src/java/org/apache/lucene/index/DocumentsWriterRAMAllocator.java (revision 966818) +++ src/java/org/apache/lucene/index/DocumentsWriterRAMAllocator.java (working copy) @@ -144,5 +144,4 @@ String toMB(long v) { return nf.format(v/1024./1024.); } - } Index: src/java/org/apache/lucene/index/DocFieldConsumers.java =================================================================== --- src/java/org/apache/lucene/index/DocFieldConsumers.java (revision 966818) +++ src/java/org/apache/lucene/index/DocFieldConsumers.java (working copy) @@ -21,9 +21,6 @@ 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 @@ } @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 @@ 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(); } } Index: src/java/org/apache/lucene/index/InvertedDocEndConsumer.java =================================================================== --- src/java/org/apache/lucene/index/InvertedDocEndConsumer.java (revision 966818) +++ src/java/org/apache/lucene/index/InvertedDocEndConsumer.java (working copy) @@ -22,7 +22,6 @@ abstract class InvertedDocEndConsumer { abstract void flush(Map 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); Index: src/java/org/apache/lucene/index/TermsHashConsumer.java =================================================================== --- src/java/org/apache/lucene/index/TermsHashConsumer.java (revision 966818) +++ src/java/org/apache/lucene/index/TermsHashConsumer.java (working copy) @@ -23,10 +23,8 @@ abstract class TermsHashConsumer { abstract void flush(Map 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; Index: src/java/org/apache/lucene/index/MergePolicy.java =================================================================== --- src/java/org/apache/lucene/index/MergePolicy.java (revision 966818) +++ src/java/org/apache/lucene/index/MergePolicy.java (working copy) @@ -67,7 +67,6 @@ 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 @@ b.append(" into ").append(info.name); if (optimize) b.append(" [optimize]"); - if (mergeDocStores) { - b.append(" [mergeDocStores]"); - } return b.toString(); } } Index: src/java/org/apache/lucene/index/StoredFieldsWriter.java =================================================================== --- src/java/org/apache/lucene/index/StoredFieldsWriter.java (revision 966818) +++ src/java/org/apache/lucene/index/StoredFieldsWriter.java (working copy) @@ -20,8 +20,6 @@ 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 @@ 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 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; - } + public void reset() { + numStoredFields = 0; + storedFields = new Fieldable[1]; + fieldNumbers = new int[1]; } - + public void startDocument() { + 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 @@ // Fill fdx file to include any final docs that we // skipped because they hit non-aborting exceptions - fill(state.numDocsInStore - docWriter.getDocStoreOffset()); + fill(state.numDocs); } - 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()); - } - 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 @@ /** 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(); + 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; } - - localFieldsWriter.writeField(fieldInfo, field); + + 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); - } - } } Index: src/java/org/apache/lucene/index/TermVectorsTermsWriter.java =================================================================== --- src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (revision 966818) +++ src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (working copy) @@ -20,9 +20,7 @@ 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 DocumentsWriterPerThread docWriter; TermVectorsWriter termVectorsWriter; - PerDoc[] docFreeList = new PerDoc[1]; int freeCount; IndexOutput tvx; IndexOutput tvd; @@ -40,7 +37,6 @@ 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 @@ 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 @@ 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 @@ 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 @@ } } - 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 0) { + for(int i=0;i() { @Override public Long process(final DocumentsWriterPerThread perThread) throws IOException { @@ -156,11 +156,14 @@ if (finishAddDocument(perThread, perThreadRAMUsedBeforeAdd)) { super.clearThreadBindings(); - indexWriter.maybeMerge(); } return sequenceID; } }); + + indexWriter.maybeMerge(); + + return seqID; } private final boolean finishAddDocument(DocumentsWriterPerThread perThread, @@ -252,7 +255,7 @@ 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() { @Override @@ -260,61 +263,29 @@ 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 @@ synchronized(indexWriter) { indexWriter.segmentInfos.add(newSegment); indexWriter.checkpoint(); + SegmentReader reader = indexWriter.readerPool.get(newSegment, false); boolean any = false; try { @@ -389,84 +361,15 @@ } } - 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 @@ 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 @@ 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; } Index: src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java =================================================================== --- src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (revision 966819) +++ src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (working copy) @@ -68,15 +68,7 @@ } 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 @@ 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 @@ 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 @@ 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 @@ } termsHashPerField.reset(); - termsHashPerField.termsHash.reset(); } void shrinkHash() { Index: src/java/org/apache/lucene/index/IndexWriter.java =================================================================== --- src/java/org/apache/lucene/index/IndexWriter.java (revision 966818) +++ src/java/org/apache/lucene/index/IndexWriter.java (working copy) @@ -413,7 +413,7 @@ // 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 @@ // 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 @@ if (infoStream != null) message("optimize: index now " + segString()); - flush(true, false, true); + flush(true, true); synchronized(this) { resetMergeExceptions(); @@ -2490,7 +2490,7 @@ try { if (infoStream != null) message("flush at addIndexes(Directory...)"); - flush(true, false, true); + flush(true, true); int docCount = 0; List infos = new ArrayList(); @@ -2537,7 +2537,7 @@ } // Update SI appropriately - info.setDocStore(info.getDocStoreOffset(), newDsName, info.getDocStoreIsCompoundFile()); + info.setDocStoreSegment(newDsName); info.dir = directory; info.name = newSegName; @@ -2595,8 +2595,7 @@ 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 @@ if (infoStream != null) message("prepareCommit: flush"); - flush(true, true, true); + flush(true, true); startCommit(0, commitUserData); } @@ -2826,18 +2825,18 @@ * @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 @@ // 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 @@ 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 @@ Map details = new HashMap(); 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 @@ 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 dss = new HashSet(); - // This is try/finally to make sure merger's readers are // closed: boolean success = false; @@ -3426,7 +3307,7 @@ // 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 @@ 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 @@ 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 flushedFiles; // Actual codec used @@ -61,16 +59,12 @@ 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(); Index: src/java/org/apache/lucene/index/DocConsumer.java =================================================================== --- src/java/org/apache/lucene/index/DocConsumer.java (revision 966818) +++ src/java/org/apache/lucene/index/DocConsumer.java (working copy) @@ -20,9 +20,9 @@ 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(); } Index: src/java/org/apache/lucene/index/DocInverter.java =================================================================== --- src/java/org/apache/lucene/index/DocInverter.java (revision 966818) +++ src/java/org/apache/lucene/index/DocInverter.java (working copy) @@ -95,21 +95,14 @@ } @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(); + consumer.finishDocument(); } - @Override - public void closeDocStore(SegmentWriteState state) throws IOException { - consumer.closeDocStore(state); - endConsumer.closeDocStore(state); - } - - @Override void abort() { try { consumer.abort(); Index: src/java/org/apache/lucene/index/NormsWriter.java =================================================================== --- src/java/org/apache/lucene/index/NormsWriter.java (revision 966818) +++ src/java/org/apache/lucene/index/NormsWriter.java (working copy) @@ -103,10 +103,6 @@ } @Override - void closeDocStore(SegmentWriteState state) {} - - - @Override void finishDocument() throws IOException {} @Override