You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2010/05/05 02:44:16 UTC

svn commit: r941125 - in /lucene/java/branches/lucene_2_9: ./ contrib/ contrib/highlighter/src/test/ contrib/instantiated/src/test/org/apache/lucene/store/instantiated/ src/java/org/apache/lucene/analysis/ src/java/org/apache/lucene/index/ src/java/org...

Author: mikemccand
Date: Wed May  5 00:44:15 2010
New Revision: 941125

URL: http://svn.apache.org/viewvc?rev=941125&view=rev
Log:
LUCENE-2283: merge to 29x

Modified:
    lucene/java/branches/lucene_2_9/   (props changed)
    lucene/java/branches/lucene_2_9/CHANGES.txt   (contents, props changed)
    lucene/java/branches/lucene_2_9/build.xml   (props changed)
    lucene/java/branches/lucene_2_9/contrib/   (props changed)
    lucene/java/branches/lucene_2_9/contrib/CHANGES.txt   (props changed)
    lucene/java/branches/lucene_2_9/contrib/highlighter/src/test/   (props changed)
    lucene/java/branches/lucene_2_9/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java   (props changed)
    lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/analysis/Tokenizer.java   (props changed)
    lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/StoredFieldsWriter.java
    lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
    lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
    lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java   (props changed)
    lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMFile.java
    lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMOutputStream.java
    lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/util/AttributeSource.java   (props changed)
    lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/analysis/BaseTokenStreamTestCase.java   (props changed)
    lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java   (props changed)
    lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestDateTools.java   (props changed)
    lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestNumberTools.java   (props changed)
    lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java   (props changed)
    lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/store/TestHugeRamFile.java
    lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/util/TestAttributeSource.java   (props changed)

Propchange: lucene/java/branches/lucene_2_9/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,4 +1,4 @@
 /lucene/dev/trunk/lucene:932398
 /lucene/java/branches/lucene_2_4:748824
 /lucene/java/branches/lucene_3_0:886275,889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk:821888,824125,826029,826385,830871,833095,833297,833886,881819,882672,883554,884870,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk:821888,824125,826029,826385,830871,833095,833297,833886,881819,882672,883554,884870,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060

Modified: lucene/java/branches/lucene_2_9/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_9/CHANGES.txt?rev=941125&r1=941124&r2=941125&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_9/CHANGES.txt (original)
+++ lucene/java/branches/lucene_2_9/CHANGES.txt Wed May  5 00:44:15 2010
@@ -9,6 +9,17 @@ Bug fixes
    in IndexWriter, nor the Reader in Tokenizer after close is
    called.  (Ruben Laguna, Uwe Schindler, Mike McCandless)
 
+ * LUCENE-2422: Don't reuse byte[] in IndexInput/Output -- it gains
+   little performance, and ties up possibly large amounts of memory
+   for apps that index large docs.  (Ross Woolf via Mike McCandless)
+
+ * LUCENE-2283: Use shared memory pool for term vector and stored
+   fields buffers. This memory will be reclaimed if needed according
+   to the configured RAM Buffer Size for the IndexWriter.  This also
+   fixes potentially excessive memory usage when many threads are
+   indexing a mix of small and large documents.  (Tim Smith via Mike
+   McCandless)
+  
 ======================= Release 2.9.2 2010-02-26 =======================
 
 Bug fixes
@@ -56,10 +67,6 @@ Bug fixes
    (for example, text:foo^0) sorted incorrectly and produced
    invalid docids. (yonik)
 
- * LUCENE-2422: Don't reuse byte[] in IndexInput/Output -- it gains
-   little performance, and ties up possibly large amounts of memory
-   for apps that index large docs.  (Ross Woolf via Mike McCandless)
-
 API Changes
 
  * LUCENE-2190: Added a new class CustomScoreProvider to function package

Propchange: lucene/java/branches/lucene_2_9/CHANGES.txt
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/CHANGES.txt:932398
 /lucene/java/branches/lucene_3_0/CHANGES.txt:889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/CHANGES.txt:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/CHANGES.txt:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/build.xml
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/build.xml:932398
 /lucene/java/branches/lucene_3_0/build.xml:889688,891211,900212,908479,909401,912383,940993
-/lucene/java/trunk/build.xml:821888,889431-889432,891209,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/build.xml:821888,889431-889432,891209,899627,900196,908477,908975,909398,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/contrib/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/contrib:932398
 /lucene/java/branches/lucene_3_0/contrib:889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/contrib:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/contrib:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/contrib/CHANGES.txt
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/contrib/CHANGES.txt:932398
 /lucene/java/branches/lucene_3_0/contrib/CHANGES.txt:889688,891211,900212,908479,909401,912383,940993
-/lucene/java/trunk/contrib/CHANGES.txt:821888,889431-889432,891209,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/contrib/CHANGES.txt:821888,889431-889432,891209,899627,900196,908477,908975,909398,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/contrib/highlighter/src/test/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/contrib/highlighter/src/test:932398
 /lucene/java/branches/lucene_3_0/contrib/highlighter/src/test:889688,891211,900212,908479,909401,912383,940993
-/lucene/java/trunk/contrib/highlighter/src/test:821888,889431-889432,891209,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/contrib/highlighter/src/test:821888,889431-889432,891209,899627,900196,908477,908975,909398,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -3,4 +3,4 @@
 /lucene/java/branches/lucene_2_9/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java:817269-818599
 /lucene/java/branches/lucene_2_9_back_compat_tests/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java:818601-821336
 /lucene/java/branches/lucene_3_0/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java:889463,889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java:821888,881213,881315,881466,881819,882374,882672,882807,882888,882977,883074-883075,883554,884870,886257,886911,887347,887532,887602,888247,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java:821888,881213,881315,881466,881819,882374,882672,882807,882888,882977,883074-883075,883554,884870,886257,886911,887347,887532,887602,888247,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/analysis/Tokenizer.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/src/java/org/apache/lucene/analysis/Tokenizer.java:932398
 /lucene/java/branches/lucene_3_0/src/java/org/apache/lucene/analysis/Tokenizer.java:889688,891211,900212,908479,909401,912383,940993
-/lucene/java/trunk/src/java/org/apache/lucene/analysis/Tokenizer.java:821888,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/src/java/org/apache/lucene/analysis/Tokenizer.java:821888,899627,900196,908477,908975,909398,910034,910078,912407,919060

Modified: lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=941125&r1=941124&r2=941125&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/DocumentsWriter.java Wed May  5 00:44:15 2010
@@ -38,6 +38,7 @@ import org.apache.lucene.search.Similari
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMFile;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Constants;
 
@@ -175,6 +176,46 @@ final class DocumentsWriter {
       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 == PER_DOC_BLOCK_SIZE;
+      return perDocAllocator.getByteBlock(false);
+    }
+    
+    /**
+     * Recycle the bytes used.
+     */
+    synchronized void recycle() {
+      if (buffers.size() > 0) {
+        setLength(0);
+        
+        // Recycle the blocks
+        final int blockCount = buffers.size();
+        
+        final byte[][] blocks = (byte[][]) buffers.toArray( new byte[blockCount][] );
+        perDocAllocator.recycleByteBlocks(blocks, 0, blockCount);
+        buffers.clear();
+        sizeInBytes = 0;
+        
+        assert numBuffers() == 0;
+      }
+    }
+  }
   
   /**
    * The IndexingChain must define the {@link #getChain(DocumentsWriter)} method
@@ -1201,9 +1242,14 @@ final class DocumentsWriter {
   final static int BYTE_BLOCK_NOT_MASK = ~BYTE_BLOCK_MASK;
 
   private class ByteBlockAllocator extends ByteBlockPool.Allocator {
+    final int blockSize;
 
     ArrayList freeByteBlocks = new ArrayList();
-    
+
+    ByteBlockAllocator(int blockSize) {
+      this.blockSize = blockSize;
+    }
+
     /* Allocate another byte[] from the shared pool */
     byte[] getByteBlock(boolean trackAllocations) {
       synchronized(DocumentsWriter.this) {
@@ -1216,12 +1262,12 @@ final class DocumentsWriter {
           // things that don't track allocations (term
           // vectors) and things that do (freq/prox
           // postings).
-          numBytesAlloc += BYTE_BLOCK_SIZE;
-          b = new byte[BYTE_BLOCK_SIZE];
+          numBytesAlloc += blockSize;
+          b = new byte[blockSize];
         } else
           b = (byte[]) freeByteBlocks.remove(size-1);
         if (trackAllocations)
-          numBytesUsed += BYTE_BLOCK_SIZE;
+          numBytesUsed += blockSize;
         assert numBytesUsed <= numBytesAlloc;
         return b;
       }
@@ -1281,7 +1327,12 @@ final class DocumentsWriter {
       freeIntBlocks.add(blocks[i]);
   }
 
-  ByteBlockAllocator byteBlockAllocator = new ByteBlockAllocator();
+  ByteBlockAllocator byteBlockAllocator = new ByteBlockAllocator(BYTE_BLOCK_SIZE);
+
+  final static int PER_DOC_BLOCK_SIZE = 1024;
+
+  final ByteBlockAllocator perDocAllocator = new ByteBlockAllocator(PER_DOC_BLOCK_SIZE);
+
 
   /* Initial chunk size of the shared char[] blocks used to
      store term text */
@@ -1321,10 +1372,12 @@ final class DocumentsWriter {
     return nf.format(v/1024./1024.);
   }
 
-  /* We have three pools of RAM: Postings, byte blocks
-   * (holds freq/prox posting data) and char blocks (holds
-   * characters in the term).  Different docs require
-   * varying amount of storage from these three classes.
+  /* We have four pools of RAM: Postings, byte blocks
+   * (holds freq/prox posting data), char blocks (holds
+   * characters in the term) and per-doc buffers (stored fields/term vectors).  
+   * Different docs require varying amount of storage from 
+   * these four classes.
+   * 
    * For example, docs with many unique single-occurrence
    * short terms will use up the Postings RAM and hardly any
    * of the other two.  Whereas docs with very large terms
@@ -1348,6 +1401,7 @@ final class DocumentsWriter {
                 " deletesMB=" + toMB(deletesRAMUsed) +
                 " vs trigger=" + toMB(freeTrigger) +
                 " byteBlockFree=" + toMB(byteBlockAllocator.freeByteBlocks.size()*BYTE_BLOCK_SIZE) +
+                " perDocFree=" + toMB(perDocAllocator.freeByteBlocks.size()*PER_DOC_BLOCK_SIZE) +
                 " charBlockFree=" + toMB(freeCharBlocks.size()*CHAR_BLOCK_SIZE*CHAR_NUM_BYTE));
 
       final long startBytesAlloc = numBytesAlloc + deletesRAMUsed;
@@ -1363,7 +1417,11 @@ final class DocumentsWriter {
       while(numBytesAlloc+deletesRAMUsed > freeLevel) {
       
         synchronized(this) {
-          if (0 == byteBlockAllocator.freeByteBlocks.size() && 0 == freeCharBlocks.size() && 0 == freeIntBlocks.size() && !any) {
+          if (0 == perDocAllocator.freeByteBlocks.size() 
+              && 0 == byteBlockAllocator.freeByteBlocks.size() 
+              && 0 == freeCharBlocks.size() 
+              && 0 == freeIntBlocks.size() 
+              && !any) {
             // Nothing else to free -- must flush now.
             bufferIsFull = numBytesUsed+deletesRAMUsed > flushTrigger;
             if (infoStream != null) {
@@ -1376,23 +1434,34 @@ final class DocumentsWriter {
             break;
           }
 
-          if ((0 == iter % 4) && byteBlockAllocator.freeByteBlocks.size() > 0) {
+          if ((0 == iter % 5) && byteBlockAllocator.freeByteBlocks.size() > 0) {
             byteBlockAllocator.freeByteBlocks.remove(byteBlockAllocator.freeByteBlocks.size()-1);
             numBytesAlloc -= BYTE_BLOCK_SIZE;
           }
 
-          if ((1 == iter % 4) && freeCharBlocks.size() > 0) {
+          if ((1 == iter % 5) && freeCharBlocks.size() > 0) {
             freeCharBlocks.remove(freeCharBlocks.size()-1);
             numBytesAlloc -= CHAR_BLOCK_SIZE * CHAR_NUM_BYTE;
           }
 
-          if ((2 == iter % 4) && freeIntBlocks.size() > 0) {
+          if ((2 == iter % 5) && freeIntBlocks.size() > 0) {
             freeIntBlocks.remove(freeIntBlocks.size()-1);
             numBytesAlloc -= INT_BLOCK_SIZE * INT_NUM_BYTE;
           }
+
+          if ((3 == iter % 5) && perDocAllocator.freeByteBlocks.size() > 0) {
+            // Remove upwards of 32 blocks (each block is 1K)
+            for (int i = 0; i < 32; ++i) {
+              perDocAllocator.freeByteBlocks.remove(perDocAllocator.freeByteBlocks.size() - 1);
+              numBytesAlloc -= PER_DOC_BLOCK_SIZE;
+              if (perDocAllocator.freeByteBlocks.size() == 0) {
+                break;
+              }
+            }
+          }
         }
 
-        if ((3 == iter % 4) && any)
+        if ((4 == iter % 5) && any)
           // Ask consumer to free any recycled state
           any = consumer.freeRAM();
 

Modified: lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=941125&r1=941124&r2=941125&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/StoredFieldsWriter.java Wed May  5 00:44:15 2010
@@ -166,14 +166,13 @@ final class StoredFieldsWriter {
   }
 
   class PerDoc extends DocumentsWriter.DocWriter {
-
-    // TODO: use something more memory efficient; for small
-    // docs the 1024 buffer size of RAMOutputStream wastes alot
-    RAMOutputStream fdt = new RAMOutputStream();
+    final DocumentsWriter.PerDocBuffer buffer = docWriter.newPerDocBuffer();
+    RAMOutputStream fdt = new RAMOutputStream(buffer);
     int numStoredFields;
 
     void reset() {
       fdt.reset();
+      buffer.recycle();
       numStoredFields = 0;
     }
 
@@ -183,7 +182,7 @@ final class StoredFieldsWriter {
     }
 
     public long sizeInBytes() {
-      return fdt.sizeInBytes();
+      return buffer.getSizeInBytes();
     }
 
     public void finish() throws IOException {

Modified: lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=941125&r1=941124&r2=941125&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Wed May  5 00:44:15 2010
@@ -191,8 +191,8 @@ final class TermVectorsTermsWriter exten
         tvd.writeVLong(pos-lastPos);
         lastPos = pos;
       }
-      perDoc.tvf.writeTo(tvf);
-      perDoc.tvf.reset();
+      perDoc.perDocTvf.writeTo(tvf);
+      perDoc.perDocTvf.reset();
       perDoc.numVectorFields = 0;
     }
 
@@ -242,16 +242,17 @@ final class TermVectorsTermsWriter exten
 
   class PerDoc extends DocumentsWriter.DocWriter {
 
-    // TODO: use something more memory efficient; for small
-    // docs the 1024 buffer size of RAMOutputStream wastes alot
-    RAMOutputStream tvf = new RAMOutputStream();
+    final DocumentsWriter.PerDocBuffer buffer = docWriter.newPerDocBuffer();
+    RAMOutputStream perDocTvf = new RAMOutputStream(buffer);
+
     int numVectorFields;
 
     int[] fieldNumbers = new int[1];
     long[] fieldPointers = new long[1];
 
     void reset() {
-      tvf.reset();
+      perDocTvf.reset();
+      buffer.recycle();
       numVectorFields = 0;
     }
 
@@ -266,12 +267,12 @@ final class TermVectorsTermsWriter exten
         fieldPointers = ArrayUtil.grow(fieldPointers);
       }
       fieldNumbers[numVectorFields] = fieldNumber;
-      fieldPointers[numVectorFields] = tvf.getFilePointer();
+      fieldPointers[numVectorFields] = perDocTvf.getFilePointer();
       numVectorFields++;
     }
 
     public long sizeInBytes() {
-      return tvf.sizeInBytes();
+      return buffer.getSizeInBytes();
     }
 
     public void finish() throws IOException {

Modified: lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java?rev=941125&r1=941124&r2=941125&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (original)
+++ lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java Wed May  5 00:44:15 2010
@@ -72,8 +72,8 @@ final class TermVectorsTermsWriterPerFie
         perThread.doc = termsWriter.getPerDoc();
         perThread.doc.docID = docState.docID;
         assert perThread.doc.numVectorFields == 0;
-        assert 0 == perThread.doc.tvf.length();
-        assert 0 == perThread.doc.tvf.getFilePointer();
+        assert 0 == perThread.doc.perDocTvf.length();
+        assert 0 == perThread.doc.perDocTvf.getFilePointer();
       } else {
         assert perThread.doc.docID == docState.docID;
 
@@ -111,7 +111,7 @@ final class TermVectorsTermsWriterPerFie
     if (numPostings > maxNumPostings)
       maxNumPostings = numPostings;
 
-    final IndexOutput tvf = perThread.doc.tvf;
+    final IndexOutput tvf = perThread.doc.perDocTvf;
 
     // This is called once, after inverting all occurrences
     // of a given field in the doc.  At this point we flush

Propchange: lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:932398
 /lucene/java/branches/lucene_3_0/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060

Modified: lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMFile.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMFile.java?rev=941125&r1=941124&r2=941125&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMFile.java (original)
+++ lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMFile.java Wed May  5 00:44:15 2010
@@ -20,44 +20,46 @@ package org.apache.lucene.store;
 import java.util.ArrayList;
 import java.io.Serializable;
 
-class RAMFile implements Serializable {
+/** For Lucene internal use */
+public class RAMFile implements Serializable {
 
   private static final long serialVersionUID = 1l;
 
-  private ArrayList buffers = new ArrayList();
+  protected ArrayList buffers = new ArrayList();
+
   long length;
   RAMDirectory directory;
-  long sizeInBytes;
+  protected long sizeInBytes;
 
   // This is publicly modifiable via Directory.touchFile(), so direct access not supported
   private long lastModified = System.currentTimeMillis();
 
   // File used as buffer, in no RAMDirectory
-  RAMFile() {}
+  protected RAMFile() {}
   
   RAMFile(RAMDirectory directory) {
     this.directory = directory;
   }
 
   // For non-stream access from thread that might be concurrent with writing
-  synchronized long getLength() {
+  public synchronized long getLength() {
     return length;
   }
 
-  synchronized void setLength(long length) {
+  protected synchronized void setLength(long length) {
     this.length = length;
   }
 
   // For non-stream access from thread that might be concurrent with writing
-  synchronized long getLastModified() {
+  public synchronized long getLastModified() {
     return lastModified;
   }
 
-  synchronized void setLastModified(long lastModified) {
+  protected synchronized void setLastModified(long lastModified) {
     this.lastModified = lastModified;
   }
 
-  final byte[] addBuffer(int size) {
+  protected final byte[] addBuffer(int size) {
     byte[] buffer = newBuffer(size);
     synchronized(this) {
       buffers.add(buffer);
@@ -72,11 +74,11 @@ class RAMFile implements Serializable {
     return buffer;
   }
 
-  final synchronized byte[] getBuffer(int index) {
+  protected final synchronized byte[] getBuffer(int index) {
     return (byte[]) buffers.get(index);
   }
 
-  final synchronized int numBuffers() {
+  protected final synchronized int numBuffers() {
     return buffers.size();
   }
 
@@ -86,14 +88,12 @@ class RAMFile implements Serializable {
    * @param size size of allocated buffer.
    * @return allocated buffer.
    */
-  byte[] newBuffer(int size) {
+  protected byte[] newBuffer(int size) {
     return new byte[size];
   }
 
-  synchronized long getSizeInBytes() {
-    synchronized(directory) {
-      return sizeInBytes;
-    }
+  public synchronized long getSizeInBytes() {
+    return sizeInBytes;
   }
   
 }

Modified: lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=941125&r1=941124&r2=941125&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/store/RAMOutputStream.java Wed May  5 00:44:15 2010
@@ -22,6 +22,7 @@ import java.io.IOException;
 /**
  * A memory-resident {@link IndexOutput} implementation.
  * 
+ * <p>For Lucene internal use</p>
  * @version $Id$
  */
 
@@ -42,7 +43,7 @@ public class RAMOutputStream extends Ind
     this(new RAMFile());
   }
 
-  RAMOutputStream(RAMFile f) {
+  public RAMOutputStream(RAMFile f) {
     file = f;
 
     // make sure that we switch to the
@@ -68,14 +69,13 @@ public class RAMOutputStream extends Ind
     }
   }
 
-  /** Resets this to an empty buffer. */
+  /** Resets this to an empty file. */
   public void reset() {
-    try {
-      seek(0);
-    } catch (IOException e) {                     // should never happen
-      throw new RuntimeException(e.toString());
-    }
-
+    currentBuffer = null;
+    currentBufferIndex = -1;
+    bufferPosition = 0;
+    bufferStart = 0;
+    bufferLength = 0;
     file.setLength(0);
   }
 

Propchange: lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/util/AttributeSource.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,4 +1,4 @@
 /lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/AttributeSource.java:932398
 /lucene/java/branches/lucene_2_4/src/java/org/apache/lucene/util/AttributeSource.java:748824
 /lucene/java/branches/lucene_3_0/src/java/org/apache/lucene/util/AttributeSource.java:886275,889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/src/java/org/apache/lucene/util/AttributeSource.java:821888,824125,826029,826385,830871,833095,833297,833886,881819,882672,883554,884870,886257,887347,887532,891189,891363,894348,897672,908975,910034,910078,912407
+/lucene/java/trunk/src/java/org/apache/lucene/util/AttributeSource.java:821888,824125,826029,826385,830871,833095,833297,833886,881819,882672,883554,884870,886257,887347,887532,891189,891363,894348,897672,908975,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,4 +1,4 @@
 /lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/BaseTokenStreamTestCase.java:932398
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/analysis/BaseTokenStreamTestCase.java:748824
 /lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/analysis/BaseTokenStreamTestCase.java:886275,889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/src/test/org/apache/lucene/analysis/BaseTokenStreamTestCase.java:818920,821888,824125,826029,826385,830871,833095,833297,833886,881819,882672,883554,884870,887347,887532,891189,891363,897672,908975,910034,910078,912407
+/lucene/java/trunk/src/test/org/apache/lucene/analysis/BaseTokenStreamTestCase.java:818920,821888,824125,826029,826385,830871,833095,833297,833886,881819,882672,883554,884870,887347,887532,891189,891363,897672,908975,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:932398
 /lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/src/test/org/apache/lucene/analysis/TestISOLatin1AccentFilter.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestDateTools.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/src/test/org/apache/lucene/document/TestDateTools.java:932398
 /lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/document/TestDateTools.java:889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/src/test/org/apache/lucene/document/TestDateTools.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/src/test/org/apache/lucene/document/TestDateTools.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestNumberTools.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/src/test/org/apache/lucene/document/TestNumberTools.java:932398
 /lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/document/TestNumberTools.java:889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/src/test/org/apache/lucene/document/TestNumberTools.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/src/test/org/apache/lucene/document/TestNumberTools.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060

Propchange: lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:932398
 /lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060

Modified: lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/store/TestHugeRamFile.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/store/TestHugeRamFile.java?rev=941125&r1=941124&r2=941125&view=diff
==============================================================================
--- lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/store/TestHugeRamFile.java (original)
+++ lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/store/TestHugeRamFile.java Wed May  5 00:44:15 2010
@@ -32,7 +32,7 @@ public class TestHugeRamFile extends Luc
   private static class DenseRAMFile extends RAMFile {
     private long capacity = 0;
     private HashMap singleBuffers = new HashMap();
-    byte[] newBuffer(int size) {
+    protected byte[] newBuffer(int size) {
       capacity += size;
       if (capacity <= MAX_VALUE) {
         // below maxint we reuse buffers

Propchange: lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/util/TestAttributeSource.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed May  5 00:44:15 2010
@@ -1,3 +1,3 @@
 /lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestAttributeSource.java:932398
 /lucene/java/branches/lucene_3_0/src/test/org/apache/lucene/util/TestAttributeSource.java:889688,891211,899639,900212,908479,909401,912383,940993
-/lucene/java/trunk/src/test/org/apache/lucene/util/TestAttributeSource.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407
+/lucene/java/trunk/src/test/org/apache/lucene/util/TestAttributeSource.java:821888,881819,886257,887347,887532,889431-889432,891189,891209,891363,897672,899627,900196,908477,908975,909398,910034,910078,912407,919060