Index: lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java (revision 1368630) +++ lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java (working copy) @@ -90,7 +90,7 @@ w = new MockIndexWriter(r, dir, c); flushAt = _TestUtil.nextInt(r, 10, 1000); if (LuceneTestCase.VERBOSE) { - System.out.println("RIW config=" + w.getConfig()); + System.out.println("RIW dir=" + dir + " config=" + w.getConfig()); } // Make sure we sometimes test indices that don't get Index: lucene/CHANGES.txt =================================================================== --- lucene/CHANGES.txt (revision 1368657) +++ lucene/CHANGES.txt (working copy) @@ -17,6 +17,12 @@ ArrayIndexOutOfBoundsException inside ByteBlockPool (Robert Muir, Mike McCandless) +* LUCENE-4398: If you index many different field names in your + documents then due to a bug in how it measures its RAM + usage, IndexWriter would flush each segment too early eventually + reaching the point where it flushes after every doc. (Tim Smith via + Mike McCandless) + ======================= Lucene 3.6.1 ======================= More information about this release, including any errata related to the release notes, upgrade instructions, or other changes may be found online at: Index: lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (revision 1368630) +++ lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (working copy) @@ -1909,4 +1909,53 @@ r.close(); dir.close(); } + + // LUCENE-4398 + public void testRotatingFieldNames() throws Exception { + Directory dir = newFSDirectory(_TestUtil.getTempDir("TestIndexWriter.testChangingFields")); + IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)); + iwc.setRAMBufferSizeMB(0.2); + iwc.setMaxBufferedDocs(-1); + IndexWriter w = new IndexWriter(dir, iwc); + int upto = 0; + + int secondFlushDocCount = -1; + for(int iter=0;iter<10;iter++) { + final int startFlushCount = w.getFlushCount(); + int docCount = 0; + while(w.getFlushCount() == startFlushCount) { + Document doc = new Document(); + for(int i=0;i<10;i++) { + Field f = new Field("field" + (upto++), "content", Field.Store.NO, Field.Index.ANALYZED); + f.setOmitNorms(true); + doc.add(f); + } + w.addDocument(doc); + docCount++; + } + + if (VERBOSE) { + System.out.println("TEST: iter=" + iter + " flushed after docCount=" + docCount); + } + + if (iter == 1) { + // Use 2nd not 1st flush because 1st flush is + // "unusually" high because all allocations are + // fresh: + secondFlushDocCount = docCount; + } else if (iter > 1) { + assertTrue("flushed after too few docs: 2nd segment flushed at docCount=" + secondFlushDocCount + ", but current segment flushed after docCount=" + docCount + "; iter=" + iter, ((float) docCount) / secondFlushDocCount > 0.9); + } + + if (upto > 5000) { + // Start re-using field names after a while + // ... important because otherwise we can OOME due + // to too many FieldInfo instances. + upto = 0; + } + } + + w.close(); + dir.close(); + } } Index: lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java (revision 1368630) +++ lucene/core/src/java/org/apache/lucene/index/DocInverterPerField.java (working copy) @@ -61,6 +61,11 @@ } @Override + public void close() { + consumer.close(); + } + + @Override public void processFields(final Fieldable[] fields, final int count) throws IOException { Index: lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (revision 1368630) +++ lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (working copy) @@ -50,7 +50,7 @@ public void flush(Collection threads, SegmentWriteState state) throws IOException { Map> childThreadsAndFields = new HashMap>(); - for ( DocConsumerPerThread thread : threads) { + for (DocConsumerPerThread thread : threads) { DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) thread; childThreadsAndFields.put(perThread.consumer, perThread.fields()); perThread.trimFields(state); Index: lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (revision 1368630) +++ lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (working copy) @@ -79,6 +79,26 @@ nextPerField = null; } + @Override + public void close() { + if (perThread.termsHash.trackAllocations) { + try { + if (postingsHash != null) { + bytesUsed(-postingsHash.length * RamUsageEstimator.NUM_BYTES_INT); + postingsHash = null; + } + if (postingsArray != null) { + bytesUsed(-postingsArray.bytesPerPosting() * postingsArray.size); + postingsArray = null; + } + } finally { + if (nextPerField != null) { + nextPerField.close(); + } + } + } + } + private void initPostingsArray() { postingsArray = consumer.createPostingsArray(2); bytesUsed(postingsArray.size * postingsArray.bytesPerPosting()); Index: lucene/core/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java (revision 1368630) +++ lucene/core/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java (working copy) @@ -41,4 +41,6 @@ // Called on hitting an aborting exception abstract void abort(); + + abstract void close(); } Index: lucene/core/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java (revision 1368630) +++ lucene/core/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java (working copy) @@ -24,4 +24,5 @@ /** Processes all occurrences of a single field */ abstract void processFields(Fieldable[] fields, int count) throws IOException; abstract void abort(); + abstract void close(); } Index: lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (revision 1368630) +++ lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (working copy) @@ -146,6 +146,8 @@ if (state.infoStream != null) state.infoStream.println(" purge field=" + perField.fieldInfo.name); + perField.consumer.close(); + totalFieldCount--; } else {