You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2012/11/02 13:28:15 UTC

svn commit: r1404946 [1/2] - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/codecs/lucene40/values/ core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lucene/util/ core/src/test/org/apache/lucene/index/ core/src/test/org/a...

Author: simonw
Date: Fri Nov  2 12:28:14 2012
New Revision: 1404946

URL: http://svn.apache.org/viewvc?rev=1404946&view=rev
Log:
LUCENE-4515: Make MemoryIndex more memory efficient

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntBlockPool.java
      - copied, changed from r1404931, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IntBlockPool.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingIntBlockAllocator.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIntBlockPool.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingIntBlockAllocator.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IntBlockPool.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ByteSliceReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHash.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestByteSlices.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingByteBlockAllocator.java
    lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/trunk/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Fri Nov  2 12:28:14 2012
@@ -61,6 +61,9 @@ New Features
   also finds completions allowing for fuzzy edits in the input string.
   (Robert Muir, Simon Willnauer, Mike McCandless)
 
+* LUCENE-4515: MemoryIndex now supports adding the same field multiple
+  times. (Simon Willnauer)
+
 API Changes
 
 * LUCENE-4399: Deprecated AppendingCodec. Lucene's term dictionaries
@@ -148,6 +151,10 @@ Optimizations
   Instead of writing a file pointer to a VIntBlock containing the doc id, just 
   write the doc id.  (Mike McCandless, Robert Muir)
 
+* LUCENE-4515: MemoryIndex now uses Byte/IntBlockPool internally to hold terms and 
+  posting lists. All index data is represented as consecutive byte/int arrays to 
+  reduce GC cost and memory overhead. (Simon Willnauer) 
+
 Build
 
 * Upgrade randomized testing to version 2.0.4: avoid hangs on shutdown

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java Fri Nov  2 12:28:14 2012
@@ -116,7 +116,7 @@ class FixedStraightBytesImpl {
     }
     
     protected void resetPool() {
-      pool.dropBuffersAndReset();
+      pool.reset(false, false);
     }
     
     protected void writeData(IndexOutput out) throws IOException {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java Fri Nov  2 12:28:14 2012
@@ -188,7 +188,7 @@ class VarStraightBytesImpl {
         } else {
           IOUtils.closeWhileHandlingException(datOut);
         }
-        pool.dropBuffersAndReset();
+        pool.reset(false, false);
       }
 
       success = false;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ByteSliceReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ByteSliceReader.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ByteSliceReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ByteSliceReader.java Fri Nov  2 12:28:14 2012
@@ -54,7 +54,7 @@ final class ByteSliceReader extends Data
     buffer = pool.buffers[bufferUpto];
     upto = startIndex & ByteBlockPool.BYTE_BLOCK_MASK;
 
-    final int firstSize = ByteBlockPool.levelSizeArray[0];
+    final int firstSize = ByteBlockPool.LEVEL_SIZE_ARRAY[0];
 
     if (startIndex+firstSize >= endIndex) {
       // There is only this one slice to read
@@ -100,8 +100,8 @@ final class ByteSliceReader extends Data
     // Skip to our next slice
     final int nextIndex = ((buffer[limit]&0xff)<<24) + ((buffer[1+limit]&0xff)<<16) + ((buffer[2+limit]&0xff)<<8) + (buffer[3+limit]&0xff);
 
-    level = ByteBlockPool.nextLevelArray[level];
-    final int newSize = ByteBlockPool.levelSizeArray[level];
+    level = ByteBlockPool.NEXT_LEVEL_ARRAY[level];
+    final int newSize = ByteBlockPool.LEVEL_SIZE_ARRAY[level];
 
     bufferUpto = nextIndex / ByteBlockPool.BYTE_BLOCK_SIZE;
     bufferOffset = bufferUpto * ByteBlockPool.BYTE_BLOCK_SIZE;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Fri Nov  2 12:28:14 2012
@@ -59,7 +59,6 @@ final class DocFieldProcessor extends Do
   int hashMask = 1;
   int totalFieldCount;
 
-  float docBoost;
   int fieldGen;
   final DocumentsWriterPerThread.DocState docState;
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Fri Nov  2 12:28:14 2012
@@ -36,6 +36,7 @@ import org.apache.lucene.util.ByteBlockP
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.IntBlockPool;
 import org.apache.lucene.util.MutableBits;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -186,6 +187,7 @@ class DocumentsWriterPerThread {
   DeleteSlice deleteSlice;
   private final NumberFormat nf = NumberFormat.getInstance(Locale.ROOT);
   final Allocator byteBlockAllocator;
+  final IntBlockPool.Allocator intBlockAllocator;
 
   
   public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent,
@@ -201,9 +203,12 @@ class DocumentsWriterPerThread {
     this.docState.similarity = parent.indexWriter.getConfig().getSimilarity();
     bytesUsed = Counter.newCounter();
     byteBlockAllocator = new DirectTrackingAllocator(bytesUsed);
-    consumer = indexingChain.getChain(this);
     pendingDeletes = new BufferedDeletes();
+    intBlockAllocator = new IntBlockAllocator(bytesUsed);
     initialize();
+    // this should be the last call in the ctor 
+    // it really sucks that we need to pull this within the ctor and pass this ref to the chain!
+    consumer = indexingChain.getChain(this);
   }
   
   public DocumentsWriterPerThread(DocumentsWriterPerThread other, FieldInfos.Builder fieldInfos) {
@@ -619,23 +624,28 @@ class DocumentsWriterPerThread {
    * getTerms/getTermsIndex requires <= 32768 */
   final static int MAX_TERM_LENGTH_UTF8 = BYTE_BLOCK_SIZE-2;
 
-  /* Initial chunks size of the shared int[] blocks used to
-     store postings data */
-  final static int INT_BLOCK_SHIFT = 13;
-  final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT;
-  final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1;
-
-  /* Allocate another int[] from the shared pool */
-  int[] getIntBlock() {
-    int[] b = new int[INT_BLOCK_SIZE];
-    bytesUsed.addAndGet(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT);
-    return b;
-  }
-  
-  void recycleIntBlocks(int[][] blocks, int offset, int length) {
-    bytesUsed.addAndGet(-(length *(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT)));
-  }
 
+  private static class IntBlockAllocator extends IntBlockPool.Allocator {
+    private final Counter bytesUsed;
+    
+    public IntBlockAllocator(Counter bytesUsed) {
+      super(IntBlockPool.INT_BLOCK_SIZE);
+      this.bytesUsed = bytesUsed;
+    }
+    
+    /* Allocate another int[] from the shared pool */
+    public int[] getIntBlock() {
+      int[] b = new int[IntBlockPool.INT_BLOCK_SIZE];
+      bytesUsed.addAndGet(IntBlockPool.INT_BLOCK_SIZE
+          * RamUsageEstimator.NUM_BYTES_INT);
+      return b;
+    }
+    
+    public void recycleIntBlocks(int[][] blocks, int offset, int length) {
+      bytesUsed.addAndGet(-(length * (IntBlockPool.INT_BLOCK_SIZE * RamUsageEstimator.NUM_BYTES_INT)));
+    }
+    
+  }
   PerDocWriteState newPerDocWriteState(String segmentSuffix) {
     assert segmentInfo != null;
     return new PerDocWriteState(infoStream, directory, segmentInfo, bytesUsed, segmentSuffix, IOContext.DEFAULT);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHash.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHash.java Fri Nov  2 12:28:14 2012
@@ -23,6 +23,8 @@ import java.util.Map;
 
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IntBlockPool;
 
 /** This class implements {@link InvertedDocConsumer}, which
  *  is passed each token produced by the analyzer on each
@@ -36,11 +38,11 @@ final class TermsHash extends InvertedDo
 
   final TermsHashConsumer consumer;
   final TermsHash nextTermsHash;
-  final DocumentsWriterPerThread docWriter;
 
   final IntBlockPool intPool;
   final ByteBlockPool bytePool;
   ByteBlockPool termBytePool;
+  final Counter bytesUsed;
 
   final boolean primary;
   final DocumentsWriterPerThread.DocState docState;
@@ -56,11 +58,11 @@ final class TermsHash extends InvertedDo
 
   public TermsHash(final DocumentsWriterPerThread docWriter, final TermsHashConsumer consumer, boolean trackAllocations, final TermsHash nextTermsHash) {
     this.docState = docWriter.docState;
-    this.docWriter = docWriter;
     this.consumer = consumer;
     this.trackAllocations = trackAllocations; 
     this.nextTermsHash = nextTermsHash;
-    intPool = new IntBlockPool(docWriter);
+    this.bytesUsed = trackAllocations ? docWriter.bytesUsed : Counter.newCounter();
+    intPool = new IntBlockPool(docWriter.intBlockAllocator);
     bytePool = new ByteBlockPool(docWriter.byteBlockAllocator);
 
     if (nextTermsHash != null) {
@@ -87,12 +89,9 @@ final class TermsHash extends InvertedDo
 
   // Clear all state
   void reset() {
-    intPool.reset();
-    bytePool.reset();
-
-    if (primary) {
-      bytePool.reset();
-    }
+    // we don't reuse so we drop everything and don't fill with 0
+    intPool.reset(false, false);
+    bytePool.reset(false, false);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java Fri Nov  2 12:28:14 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.util.ByteBlockP
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IntBlockPool;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash.MaxBytesLengthExceededException;
 
@@ -62,8 +63,7 @@ final class TermsHashPerField extends In
     termBytePool = termsHash.termBytePool;
     docState = termsHash.docState;
     this.termsHash = termsHash;
-    bytesUsed = termsHash.trackAllocations ? termsHash.docWriter.bytesUsed
-        : Counter.newCounter();
+    bytesUsed = termsHash.bytesUsed;
     fieldState = docInverterPerField.fieldState;
     this.consumer = termsHash.consumer.addField(this, fieldInfo);
     PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
@@ -99,8 +99,8 @@ final class TermsHashPerField extends In
   public void initReader(ByteSliceReader reader, int termID, int stream) {
     assert stream < streamCount;
     int intStart = postingsArray.intStarts[termID];
-    final int[] ints = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
-    final int upto = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
+    final int[] ints = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
+    final int upto = intStart & IntBlockPool.INT_BLOCK_MASK;
     reader.init(bytePool,
                 postingsArray.byteStarts[termID]+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
                 ints[upto+stream]);
@@ -143,7 +143,7 @@ final class TermsHashPerField extends In
       // First time we are seeing this token since we last
       // flushed the hash.
       // Init stream slices
-      if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE)
+      if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE)
         intPool.nextBuffer();
 
       if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) {
@@ -167,8 +167,8 @@ final class TermsHashPerField extends In
     } else {
       termID = (-termID)-1;
       int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
-      intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
+      intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
+      intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
   }
@@ -205,7 +205,7 @@ final class TermsHashPerField extends In
     if (termID >= 0) {// New posting
       bytesHash.byteStart(termID);
       // Init stream slices
-      if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE) {
+      if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE) {
         intPool.nextBuffer();
       }
 
@@ -230,8 +230,8 @@ final class TermsHashPerField extends In
     } else {
       termID = (-termID)-1;
       final int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
-      intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
+      intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
+      intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java Fri Nov  2 12:28:14 2012
@@ -21,6 +21,8 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.IntBlockPool.SliceReader;
+import org.apache.lucene.util.IntBlockPool.SliceWriter;
 
 import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
 
@@ -113,10 +115,14 @@ public final class ByteBlockPool {
     }
   };
 
-
+  /**
+   * array of buffers currently used in the pool. Buffers are allocated if
+   * needed don't modify this outside of this class.
+   */
   public byte[][] buffers = new byte[10][];
-
-  int bufferUpto = -1;                        // Which buffer we are upto
+  
+  /** index into the buffers array pointing to the current buffer used as the head */
+  private int bufferUpto = -1;                        // Which buffer we are upto
   /** Where we are in head buffer */
   public int byteUpto = BYTE_BLOCK_SIZE;
 
@@ -131,43 +137,64 @@ public final class ByteBlockPool {
     this.allocator = allocator;
   }
   
-  public void dropBuffersAndReset() {
-    if (bufferUpto != -1) {
-      // Recycle all but the first buffer
-      allocator.recycleByteBlocks(buffers, 0, 1+bufferUpto);
-
-      // Re-use the first buffer
-      bufferUpto = -1;
-      byteUpto = BYTE_BLOCK_SIZE;
-      byteOffset = -BYTE_BLOCK_SIZE;
-      buffers = new byte[10][];
-      buffer = null;
-    }
-  }
-
+  /**
+   * Resets the pool to its initial state reusing the first buffer and fills all
+   * buffers with <tt>0</tt> bytes before they reused or passed to
+   * {@link Allocator#recycleByteBlocks(byte[][], int, int)}. Calling
+   * {@link ByteBlockPool#nextBuffer()} is not needed after reset.
+   */
   public void reset() {
+    reset(true, true);
+  }
+  
+  /**
+   * Expert: Resets the pool to its initial state reusing the first buffer. Calling
+   * {@link ByteBlockPool#nextBuffer()} is not needed after reset. 
+   * @param zeroFillBuffers if <code>true</code> the buffers are filled with <tt>0</tt>. 
+   *        This should be set to <code>true</code> if this pool is used with slices.
+   * @param reuseFirst if <code>true</code> the first buffer will be reused and calling
+   *        {@link ByteBlockPool#nextBuffer()} is not needed after reset iff the 
+   *        block pool was used before ie. {@link ByteBlockPool#nextBuffer()} was called before.
+   */
+  public void reset(boolean zeroFillBuffers, boolean reuseFirst) {
     if (bufferUpto != -1) {
       // We allocated at least one buffer
 
-      for(int i=0;i<bufferUpto;i++)
-        // Fully zero fill buffers that we fully used
-        Arrays.fill(buffers[i], (byte) 0);
-
-      // Partial zero fill the final buffer
-      Arrays.fill(buffers[bufferUpto], 0, byteUpto, (byte) 0);
-          
-      if (bufferUpto > 0)
-        // Recycle all but the first buffer
-        allocator.recycleByteBlocks(buffers, 1, 1+bufferUpto);
-
-      // Re-use the first buffer
-      bufferUpto = 0;
-      byteUpto = 0;
-      byteOffset = 0;
-      buffer = buffers[0];
+      if (zeroFillBuffers) {
+        for(int i=0;i<bufferUpto;i++) {
+          // Fully zero fill buffers that we fully used
+          Arrays.fill(buffers[i], (byte) 0);
+        }
+        // Partial zero fill the final buffer
+        Arrays.fill(buffers[bufferUpto], 0, byteUpto, (byte) 0);
+      }
+     
+     if (bufferUpto > 0 || !reuseFirst) {
+       final int offset = reuseFirst ? 1 : 0;  
+       // Recycle all but the first buffer
+       allocator.recycleByteBlocks(buffers, offset, 1+bufferUpto);
+       Arrays.fill(buffers, offset, 1+bufferUpto, null);
+     }
+     if (reuseFirst) {
+       // Re-use the first buffer
+       bufferUpto = 0;
+       byteUpto = 0;
+       byteOffset = 0;
+       buffer = buffers[0];
+     } else {
+       bufferUpto = -1;
+       byteUpto = BYTE_BLOCK_SIZE;
+       byteOffset = -BYTE_BLOCK_SIZE;
+       buffer = null;
+     }
     }
   }
-  
+  /**
+   * Advances the pool to its next buffer. This method should be called once
+   * after the constructor to initialize the pool. In contrast to the
+   * constructor a {@link ByteBlockPool#reset()} call will advance the pool to
+   * its first buffer immediately.
+   */
   public void nextBuffer() {
     if (1+bufferUpto == buffers.length) {
       byte[][] newBuffers = new byte[ArrayUtil.oversize(buffers.length+1,
@@ -181,7 +208,11 @@ public final class ByteBlockPool {
     byteUpto = 0;
     byteOffset += BYTE_BLOCK_SIZE;
   }
-
+  
+  /**
+   * Allocates a new slice with the given size. 
+   * @see ByteBlockPool#FIRST_LEVEL_SIZE
+   */
   public int newSlice(final int size) {
     if (byteUpto > BYTE_BLOCK_SIZE-size)
       nextBuffer();
@@ -197,15 +228,32 @@ public final class ByteBlockPool {
   // array is the length of each slice, ie first slice is 5
   // bytes, next slice is 14 bytes, etc.
   
-  public final static int[] nextLevelArray = {1, 2, 3, 4, 5, 6, 7, 8, 9, 9};
-  public final static int[] levelSizeArray = {5, 14, 20, 30, 40, 40, 80, 80, 120, 200};
-  public final static int FIRST_LEVEL_SIZE = levelSizeArray[0];
+  /**
+   * An array holding the offset into the {@link ByteBlockPool#LEVEL_SIZE_ARRAY}
+   * to quickly navigate to the next slice level.
+   */
+  public final static int[] NEXT_LEVEL_ARRAY = {1, 2, 3, 4, 5, 6, 7, 8, 9, 9};
+  
+  /**
+   * An array holding the level sizes for byte slices.
+   */
+  public final static int[] LEVEL_SIZE_ARRAY = {5, 14, 20, 30, 40, 40, 80, 80, 120, 200};
+  
+  /**
+   * The first level size for new slices
+   * @see ByteBlockPool#newSlice(int)
+   */
+  public final static int FIRST_LEVEL_SIZE = LEVEL_SIZE_ARRAY[0];
 
+  /**
+   * Creates a new byte slice with the given starting size and 
+   * returns the slices offset in the pool.
+   */
   public int allocSlice(final byte[] slice, final int upto) {
 
     final int level = slice[upto] & 15;
-    final int newLevel = nextLevelArray[level];
-    final int newSize = levelSizeArray[newLevel];
+    final int newLevel = NEXT_LEVEL_ARRAY[level];
+    final int newSize = LEVEL_SIZE_ARRAY[newLevel];
 
     // Maybe allocate another block
     if (byteUpto > BYTE_BLOCK_SIZE-newSize)
@@ -288,13 +336,14 @@ public final class ByteBlockPool {
   }
   
   /**
-   *
+   * Copies bytes from the pool starting at the given offset with the given  
+   * length into the given {@link BytesRef} at offset <tt>0</tt> and returns it.
+   * <p>Note: this method allows to copy across block boundaries.</p>
    */
-  public final BytesRef copyFrom(final BytesRef bytes) {
-    final int length = bytes.length;
-    final int offset = bytes.offset;
+  public final BytesRef copyFrom(final BytesRef bytes, final int offset, final int length) {
     bytes.offset = 0;
     bytes.grow(length);
+    bytes.length = length;
     int bufferIndex = offset >> BYTE_BLOCK_SHIFT;
     byte[] buffer = buffers[bufferIndex];
     int pos = offset & BYTE_BLOCK_MASK;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/BytesRefHash.java Fri Nov  2 12:28:14 2012
@@ -228,7 +228,7 @@ public final class BytesRefHash {
     lastCount = count;
     count = 0;
     if (resetPool) {
-      pool.dropBuffersAndReset();
+      pool.reset(false, false); // we don't need to 0-fill the buffers
     }
     bytesStart = bytesStartArray.clear();
     if (lastCount != -1 && shrink(lastCount)) {

Copied: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntBlockPool.java (from r1404931, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IntBlockPool.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntBlockPool.java?p2=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntBlockPool.java&p1=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IntBlockPool.java&r1=1404931&r2=1404946&rev=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IntBlockPool.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IntBlockPool.java Fri Nov  2 12:28:14 2012
@@ -1,6 +1,4 @@
-package org.apache.lucene.index;
-
-import java.util.Arrays;
+package org.apache.lucene.util;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,47 +17,364 @@ import java.util.Arrays;
  * limitations under the License.
  */
 
-final class IntBlockPool {
+import java.util.Arrays;
 
-  public int[][] buffers = new int[10][];
+/**
+ * A pool for int blocks similar to {@link ByteBlockPool}
+ * @lucene.internal
+ */
+public final class IntBlockPool {
+  public final static int INT_BLOCK_SHIFT = 13;
+  public final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT;
+  public final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1;
+  
+  /** Abstract class for allocating and freeing int
+   *  blocks. */
+  public abstract static class Allocator {
+    protected final int blockSize;
 
-  int bufferUpto = -1;                        // Which buffer we are upto
-  public int intUpto = DocumentsWriterPerThread.INT_BLOCK_SIZE;             // Where we are in head buffer
+    public Allocator(int blockSize) {
+      this.blockSize = blockSize;
+    }
 
-  public int[] buffer;                              // Current head buffer
-  public int intOffset = -DocumentsWriterPerThread.INT_BLOCK_SIZE;          // Current head offset
+    public abstract void recycleIntBlocks(int[][] blocks, int start, int end);
 
-  final private DocumentsWriterPerThread docWriter;
+    public int[] getIntBlock() {
+      return new int[blockSize];
+    }
+  }
+  
+  /** A simple {@link Allocator} that never recycles. */
+  public static final class DirectAllocator extends Allocator {
+
+    /**
+     * Creates a new {@link DirectAllocator} with a default block size
+     */
+    public DirectAllocator() {
+      super(INT_BLOCK_SIZE);
+    }
 
-  public IntBlockPool(DocumentsWriterPerThread docWriter) {
-    this.docWriter = docWriter;
+    @Override
+    public void recycleIntBlocks(int[][] blocks, int start, int end) {
+    }
   }
+  
+  /** array of buffers currently used in the pool. Buffers are allocated if needed don't modify this outside of this class */
+  public int[][] buffers = new int[10][];
 
+  /** index into the buffers array pointing to the current buffer used as the head */
+  private int bufferUpto = -1;   
+  /** Pointer to the current position in head buffer */
+  public int intUpto = INT_BLOCK_SIZE;
+  /** Current head buffer */
+  public int[] buffer;
+  /** Current head offset */
+  public int intOffset = -INT_BLOCK_SIZE;
+
+  private final Allocator allocator;
+
+  /**
+   * Creates a new {@link IntBlockPool} with a default {@link Allocator}.
+   * @see IntBlockPool#nextBuffer()
+   */
+  public IntBlockPool() {
+    this(new DirectAllocator());
+  }
+  
+  /**
+   * Creates a new {@link IntBlockPool} with the given {@link Allocator}.
+   * @see IntBlockPool#nextBuffer()
+   */
+  public IntBlockPool(Allocator allocator) {
+    this.allocator = allocator;
+  }
+  
+  /**
+   * Resets the pool to its initial state reusing the first buffer. Calling
+   * {@link IntBlockPool#nextBuffer()} is not needed after reset.
+   */
   public void reset() {
+    this.reset(true, true);
+  }
+  
+  /**
+   * Expert: Resets the pool to its initial state reusing the first buffer. 
+   * @param zeroFillBuffers if <code>true</code> the buffers are filled with <tt>0</tt>. 
+   *        This should be set to <code>true</code> if this pool is used with 
+   *        {@link SliceWriter}.
+   * @param reuseFirst if <code>true</code> the first buffer will be reused and calling
+   *        {@link IntBlockPool#nextBuffer()} is not needed after reset iff the 
+   *        block pool was used before ie. {@link IntBlockPool#nextBuffer()} was called before.
+   */
+  public void reset(boolean zeroFillBuffers, boolean reuseFirst) {
     if (bufferUpto != -1) {
-      // Reuse first buffer
-      if (bufferUpto > 0) {
-        docWriter.recycleIntBlocks(buffers, 1, bufferUpto-1);
-        Arrays.fill(buffers, 1, bufferUpto, null);
+      // We allocated at least one buffer
+
+      if (zeroFillBuffers) {
+        for(int i=0;i<bufferUpto;i++) {
+          // Fully zero fill buffers that we fully used
+          Arrays.fill(buffers[i], 0);
+        }
+        // Partial zero fill the final buffer
+        Arrays.fill(buffers[bufferUpto], 0, intUpto, 0);
       }
-      bufferUpto = 0;
-      intUpto = 0;
-      intOffset = 0;
-      buffer = buffers[0];
+     
+      if (bufferUpto > 0 || !reuseFirst) {
+        final int offset = reuseFirst ? 1 : 0;  
+       // Recycle all but the first buffer
+       allocator.recycleIntBlocks(buffers, offset, 1+bufferUpto);
+       Arrays.fill(buffers, offset, bufferUpto+1, null);
+     }
+     if (reuseFirst) {
+       // Re-use the first buffer
+       bufferUpto = 0;
+       intUpto = 0;
+       intOffset = 0;
+       buffer = buffers[0];
+     } else {
+       bufferUpto = -1;
+       intUpto = INT_BLOCK_SIZE;
+       intOffset = -INT_BLOCK_SIZE;
+       buffer = null;
+     }
     }
   }
-
+  
+  /**
+   * Advances the pool to its next buffer. This method should be called once
+   * after the constructor to initialize the pool. In contrast to the
+   * constructor a {@link IntBlockPool#reset()} call will advance the pool to
+   * its first buffer immediately.
+   */
   public void nextBuffer() {
     if (1+bufferUpto == buffers.length) {
       int[][] newBuffers = new int[(int) (buffers.length*1.5)][];
       System.arraycopy(buffers, 0, newBuffers, 0, buffers.length);
       buffers = newBuffers;
     }
-    buffer = buffers[1+bufferUpto] = docWriter.getIntBlock();
+    buffer = buffers[1+bufferUpto] = allocator.getIntBlock();
     bufferUpto++;
 
     intUpto = 0;
-    intOffset += DocumentsWriterPerThread.INT_BLOCK_SIZE;
+    intOffset += INT_BLOCK_SIZE;
+  }
+  
+  /**
+   * Creates a new int slice with the given starting size and returns the slices offset in the pool.
+   * @see SliceReader
+   */
+  private int newSlice(final int size) {
+    if (intUpto > INT_BLOCK_SIZE-size) {
+      nextBuffer();
+      assert assertSliceBuffer(buffer);
+    }
+      
+    final int upto = intUpto;
+    intUpto += size;
+    buffer[intUpto-1] = 1;
+    return upto;
+  }
+  
+  private static final boolean assertSliceBuffer(int[] buffer) {
+    int count = 0;
+    for (int i = 0; i < buffer.length; i++) {
+      count += buffer[i]; // for slices the buffer must only have 0 values
+    }
+    return count == 0;
+  }
+  
+  
+  // no need to make this public unless we support different sizes
+  // TODO make the levels and the sizes configurable
+  /**
+   * An array holding the offset into the {@link IntBlockPool#LEVEL_SIZE_ARRAY}
+   * to quickly navigate to the next slice level.
+   */
+  private final static int[] NEXT_LEVEL_ARRAY = {1, 2, 3, 4, 5, 6, 7, 8, 9, 9};
+  
+  /**
+   * An array holding the level sizes for int slices.
+   */
+  private final static int[] LEVEL_SIZE_ARRAY = {2, 4, 8, 16, 32, 64, 128, 256, 512, 1024};
+  
+  /**
+   * The first level size for new slices
+   */
+  private final static int FIRST_LEVEL_SIZE = LEVEL_SIZE_ARRAY[0];
+
+  /**
+   * Allocates a new slice from the given offset
+   */
+  private int allocSlice(final int[] slice, final int sliceOffset) {
+    final int level = slice[sliceOffset];
+    final int newLevel = NEXT_LEVEL_ARRAY[level-1];
+    final int newSize = LEVEL_SIZE_ARRAY[newLevel];
+    // Maybe allocate another block
+    if (intUpto > INT_BLOCK_SIZE-newSize) {
+      nextBuffer();
+      assert assertSliceBuffer(buffer);
+    }
+
+    final int newUpto = intUpto;
+    final int offset = newUpto + intOffset;
+    intUpto += newSize;
+    // Write forwarding address at end of last slice:
+    slice[sliceOffset] = offset;
+        
+    // Write new level:
+    buffer[intUpto-1] = newLevel;
+
+    return newUpto;
+  }
+  
+  /**
+   * A {@link SliceWriter} that allows to write multiple integer slices into a given {@link IntBlockPool}.
+   * 
+   *  @see SliceReader
+   *  @lucene.internal
+   */
+  public static class SliceWriter {
+    
+    private int offset;
+    private final IntBlockPool pool;
+    
+    
+    public SliceWriter(IntBlockPool pool) {
+      this.pool = pool;
+    }
+    /**
+     * 
+     */
+    public void reset(int sliceOffset) {
+      this.offset = sliceOffset;
+    }
+    
+    /**
+     * Writes the given value into the slice and resizes the slice if needed
+     */
+    public void writeInt(int value) {
+      int[] ints = pool.buffers[offset >> INT_BLOCK_SHIFT];
+      assert ints != null;
+      int relativeOffset = offset & INT_BLOCK_MASK;
+      if (ints[relativeOffset] != 0) {
+        // End of slice; allocate a new one
+          relativeOffset = pool.allocSlice(ints, relativeOffset);
+        ints = pool.buffer;
+        offset = relativeOffset + pool.intOffset;
+      }
+      ints[relativeOffset] = value;
+      offset++; 
+    }
+    
+    /**
+     * starts a new slice and returns the start offset. The returned value
+     * should be used as the start offset to initialize a {@link SliceReader}.
+     */
+    public int startNewSlice() {
+      return offset = pool.newSlice(FIRST_LEVEL_SIZE) + pool.intOffset;
+      
+    }
+    
+    /**
+     * Returns the offset of the currently written slice. The returned value
+     * should be used as the end offset to initialize a {@link SliceReader} once
+     * this slice is fully written or to reset the this writer if another slice
+     * needs to be written.
+     */
+    public int getCurrentOffset() {
+      return offset;
+    }
+  }
+  
+  /**
+   * A {@link SliceReader} that can read int slices written by a {@link SliceWriter}
+   * @lucene.internal
+   */
+  public static final class SliceReader {
+    
+    private final IntBlockPool pool;
+    private int upto;
+    private int bufferUpto;
+    private int bufferOffset;
+    private int[] buffer;
+    private int limit;
+    private int level;
+    private int end;
+    
+    /**
+     * Creates a new {@link SliceReader} on the given pool
+     */
+    public SliceReader(IntBlockPool pool) {
+      this.pool = pool;
+    }
+
+    /**
+     * Resets the reader to a slice give the slices absolute start and end offset in the pool
+     */
+    public void reset(int startOffset, int endOffset) {
+      bufferUpto = startOffset / INT_BLOCK_SIZE;
+      bufferOffset = bufferUpto * INT_BLOCK_SIZE;
+      this.end = endOffset;
+      upto = startOffset;
+      level = 1;
+      
+      buffer = pool.buffers[bufferUpto];
+      upto = startOffset & INT_BLOCK_MASK;
+
+      final int firstSize = IntBlockPool.LEVEL_SIZE_ARRAY[0];
+      if (startOffset+firstSize >= endOffset) {
+        // There is only this one slice to read
+        limit = endOffset & INT_BLOCK_MASK;
+      } else {
+        limit = upto+firstSize-1;
+      }
+
+    }
+    
+    /**
+     * Returns <code>true</code> iff the current slice is fully read. If this
+     * method returns <code>true</code> {@link SliceReader#readInt()} should not
+     * be called again on this slice.
+     */
+    public boolean endOfSlice() {
+      assert upto + bufferOffset <= end;
+      return upto + bufferOffset == end;
+    }
+    
+    /**
+     * Reads the next int from the current slice and returns it.
+     * @see SliceReader#endOfSlice()
+     */
+    public int readInt() {
+      assert !endOfSlice();
+      assert upto <= limit;
+      if (upto == limit)
+        nextSlice();
+      return buffer[upto++];
+    }
+    
+    private void nextSlice() {
+      // Skip to our next slice
+      final int nextIndex = buffer[limit];
+      level = NEXT_LEVEL_ARRAY[level-1];
+      final int newSize = LEVEL_SIZE_ARRAY[level];
+
+      bufferUpto = nextIndex / INT_BLOCK_SIZE;
+      bufferOffset = bufferUpto * INT_BLOCK_SIZE;
+
+      buffer = pool.buffers[bufferUpto];
+      upto = nextIndex & INT_BLOCK_MASK;
+
+      if (nextIndex + newSize >= end) {
+        // We are advancing to the final slice
+        assert end - nextIndex > 0;
+        limit = end - bufferOffset;
+      } else {
+        // This is not the final slice (subtract 4 for the
+        // forwarding address at the end of this new slice)
+        limit = upto+newSize-1;
+      }
+    }
   }
 }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java Fri Nov  2 12:28:14 2012
@@ -1,7 +1,5 @@
 package org.apache.lucene.util;
 
-import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.lucene.util.ByteBlockPool.Allocator;
 
 /*
@@ -22,17 +20,19 @@ import org.apache.lucene.util.ByteBlockP
  */
 
 /**
- * A threadsafe {@link ByteBlockPool.Allocator} implementation that recycles unused byte
+ * A {@link ByteBlockPool.Allocator} implementation that recycles unused byte
  * blocks in a buffer and reuses them in subsequent calls to
  * {@link #getByteBlock()}.
- * 
+ * <p>
+ * Note: This class is not thread-safe
+ * </p>
  * @lucene.internal
  */
 public final class RecyclingByteBlockAllocator extends ByteBlockPool.Allocator {
   private byte[][] freeByteBlocks;
   private final int maxBufferedBlocks;
   private int freeBlocks = 0;
-  private final AtomicLong bytesUsed;
+  private final Counter bytesUsed;
   public static final int DEFAULT_BUFFERED_BLOCKS = 64;
 
   /**
@@ -43,12 +43,12 @@ public final class RecyclingByteBlockAll
    * @param maxBufferedBlocks
    *          maximum number of buffered byte block
    * @param bytesUsed
-   *          {@link AtomicLong} reference counting internally allocated bytes
+   *          {@link Counter} reference counting internally allocated bytes
    */
   public RecyclingByteBlockAllocator(int blockSize, int maxBufferedBlocks,
-      AtomicLong bytesUsed) {
+      Counter bytesUsed) {
     super(blockSize);
-    freeByteBlocks = new byte[Math.min(10, maxBufferedBlocks)][];
+    freeByteBlocks = new byte[maxBufferedBlocks][];
     this.maxBufferedBlocks = maxBufferedBlocks;
     this.bytesUsed = bytesUsed;
   }
@@ -62,7 +62,7 @@ public final class RecyclingByteBlockAll
    *          maximum number of buffered byte block
    */
   public RecyclingByteBlockAllocator(int blockSize, int maxBufferedBlocks) {
-    this(blockSize, maxBufferedBlocks, new AtomicLong());
+    this(blockSize, maxBufferedBlocks, Counter.newCounter(false));
   }
 
   /**
@@ -72,11 +72,11 @@ public final class RecyclingByteBlockAll
    * 
    */
   public RecyclingByteBlockAllocator() {
-    this(ByteBlockPool.BYTE_BLOCK_SIZE, 64, new AtomicLong());
+    this(ByteBlockPool.BYTE_BLOCK_SIZE, 64, Counter.newCounter(false));
   }
 
   @Override
-  public synchronized byte[] getByteBlock() {
+  public byte[] getByteBlock() {
     if (freeBlocks == 0) {
       bytesUsed.addAndGet(blockSize);
       return new byte[blockSize];
@@ -87,7 +87,7 @@ public final class RecyclingByteBlockAll
   }
 
   @Override
-  public synchronized void recycleByteBlocks(byte[][] blocks, int start, int end) {
+  public void recycleByteBlocks(byte[][] blocks, int start, int end) {
     final int numBlocks = Math.min(maxBufferedBlocks - freeBlocks, end - start);
     final int size = freeBlocks + numBlocks;
     if (size >= freeByteBlocks.length) {
@@ -111,14 +111,14 @@ public final class RecyclingByteBlockAll
   /**
    * @return the number of currently buffered blocks
    */
-  public synchronized int numBufferedBlocks() {
+  public int numBufferedBlocks() {
     return freeBlocks;
   }
 
   /**
    * @return the number of bytes currently allocated by this {@link Allocator}
    */
-  public synchronized long bytesUsed() {
+  public long bytesUsed() {
     return bytesUsed.get();
   }
 
@@ -136,8 +136,8 @@ public final class RecyclingByteBlockAll
    *          the number of byte blocks to remove
    * @return the number of actually removed buffers
    */
-  public synchronized int freeBlocks(int num) {
-    assert num >= 0;
+  public int freeBlocks(int num) {
+    assert num >= 0 : "free blocks must be >= 0 but was: "+ num;
     final int stop;
     final int count;
     if (num > freeBlocks) {

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingIntBlockAllocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingIntBlockAllocator.java?rev=1404946&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingIntBlockAllocator.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/RecyclingIntBlockAllocator.java Fri Nov  2 12:28:14 2012
@@ -0,0 +1,157 @@
+package org.apache.lucene.util;
+
+import org.apache.lucene.util.IntBlockPool.Allocator;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * A {@link Allocator} implementation that recycles unused int
+ * blocks in a buffer and reuses them in subsequent calls to
+ * {@link #getIntBlock()}.
+ * <p>
+ * Note: This class is not thread-safe
+ * </p>
+ * @lucene.internal
+ */
+public final class RecyclingIntBlockAllocator extends Allocator {
+  private int[][] freeByteBlocks;
+  private final int maxBufferedBlocks;
+  private int freeBlocks = 0;
+  private final Counter bytesUsed;
+  public static final int DEFAULT_BUFFERED_BLOCKS = 64;
+
+  /**
+   * Creates a new {@link RecyclingIntBlockAllocator}
+   * 
+   * @param blockSize
+   *          the block size in bytes
+   * @param maxBufferedBlocks
+   *          maximum number of buffered int block
+   * @param bytesUsed
+   *          {@link Counter} reference counting internally allocated bytes
+   */
+  public RecyclingIntBlockAllocator(int blockSize, int maxBufferedBlocks,
+      Counter bytesUsed) {
+    super(blockSize);
+    freeByteBlocks = new int[maxBufferedBlocks][];
+    this.maxBufferedBlocks = maxBufferedBlocks;
+    this.bytesUsed = bytesUsed;
+  }
+
+  /**
+   * Creates a new {@link RecyclingIntBlockAllocator}.
+   * 
+   * @param blockSize
+   *          the size of each block returned by this allocator
+   * @param maxBufferedBlocks
+   *          maximum number of buffered int blocks
+   */
+  public RecyclingIntBlockAllocator(int blockSize, int maxBufferedBlocks) {
+    this(blockSize, maxBufferedBlocks, Counter.newCounter(false));
+  }
+
+  /**
+   * Creates a new {@link RecyclingIntBlockAllocator} with a block size of
+   * {@link IntBlockPool#INT_BLOCK_SIZE}, upper buffered docs limit of
+   * {@link #DEFAULT_BUFFERED_BLOCKS} ({@value #DEFAULT_BUFFERED_BLOCKS}).
+   * 
+   */
+  public RecyclingIntBlockAllocator() {
+    this(IntBlockPool.INT_BLOCK_SIZE, 64, Counter.newCounter(false));
+  }
+
+  @Override
+  public int[] getIntBlock() {
+    if (freeBlocks == 0) {
+      bytesUsed.addAndGet(blockSize*RamUsageEstimator.NUM_BYTES_INT);
+      return new int[blockSize];
+    }
+    final int[] b = freeByteBlocks[--freeBlocks];
+    freeByteBlocks[freeBlocks] = null;
+    return b;
+  }
+
+  @Override
+  public void recycleIntBlocks(int[][] blocks, int start, int end) {
+    final int numBlocks = Math.min(maxBufferedBlocks - freeBlocks, end - start);
+    final int size = freeBlocks + numBlocks;
+    if (size >= freeByteBlocks.length) {
+      final int[][] newBlocks = new int[ArrayUtil.oversize(size,
+          RamUsageEstimator.NUM_BYTES_OBJECT_REF)][];
+      System.arraycopy(freeByteBlocks, 0, newBlocks, 0, freeBlocks);
+      freeByteBlocks = newBlocks;
+    }
+    final int stop = start + numBlocks;
+    for (int i = start; i < stop; i++) {
+      freeByteBlocks[freeBlocks++] = blocks[i];
+      blocks[i] = null;
+    }
+    for (int i = stop; i < end; i++) {
+      blocks[i] = null;
+    }
+    bytesUsed.addAndGet(-(end - stop) * (blockSize * RamUsageEstimator.NUM_BYTES_INT));
+    assert bytesUsed.get() >= 0;
+  }
+
+  /**
+   * @return the number of currently buffered blocks
+   */
+  public int numBufferedBlocks() {
+    return freeBlocks;
+  }
+
+  /**
+   * @return the number of bytes currently allocated by this {@link Allocator}
+   */
+  public long bytesUsed() {
+    return bytesUsed.get();
+  }
+
+  /**
+   * @return the maximum number of buffered byte blocks
+   */
+  public int maxBufferedBlocks() {
+    return maxBufferedBlocks;
+  }
+
+  /**
+   * Removes the given number of int blocks from the buffer if possible.
+   * 
+   * @param num
+   *          the number of int blocks to remove
+   * @return the number of actually removed buffers
+   */
+  public int freeBlocks(int num) {
+    assert num >= 0 : "free blocks must be >= 0 but was: "+ num;
+    final int stop;
+    final int count;
+    if (num > freeBlocks) {
+      stop = 0;
+      count = freeBlocks;
+    } else {
+      stop = freeBlocks - num;
+      count = num;
+    }
+    while (freeBlocks > stop) {
+      freeByteBlocks[--freeBlocks] = null;
+    }
+    bytesUsed.addAndGet(-count*blockSize* RamUsageEstimator.NUM_BYTES_INT);
+    assert bytesUsed.get() >= 0;
+    return count;
+  }
+}
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestByteSlices.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestByteSlices.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestByteSlices.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestByteSlices.java Fri Nov  2 12:28:14 2012
@@ -21,7 +21,7 @@ import org.apache.lucene.util.RecyclingB
 public class TestByteSlices extends LuceneTestCase {
 
   public void testBasic() throws Throwable {
-    ByteBlockPool pool = new ByteBlockPool(new RecyclingByteBlockAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, Integer.MAX_VALUE));
+    ByteBlockPool pool = new ByteBlockPool(new RecyclingByteBlockAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, random().nextInt(100)));
 
     final int NUM_STREAM = atLeast(100);
 

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIntBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIntBlockPool.java?rev=1404946&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIntBlockPool.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIntBlockPool.java Fri Nov  2 12:28:14 2012
@@ -0,0 +1,157 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IntBlockPool;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * tests basic {@link IntBlockPool} functionality
+ */
+public class TestIntBlockPool extends LuceneTestCase {
+  
+  public void testSingleWriterReader() {
+    Counter bytesUsed = Counter.newCounter();
+    IntBlockPool pool = new IntBlockPool(new ByteTrackingAllocator(bytesUsed));
+    
+    for (int j = 0; j < 2; j++) {
+      IntBlockPool.SliceWriter writer = new IntBlockPool.SliceWriter(pool);
+      int start = writer.startNewSlice();
+      int num = atLeast(100);
+      for (int i = 0; i < num; i++) {
+        writer.writeInt(i);
+      }
+      
+      int upto = writer.getCurrentOffset();
+      IntBlockPool.SliceReader reader = new IntBlockPool.SliceReader(pool);
+      reader.reset(start, upto);
+      for (int i = 0; i < num; i++) {
+        assertEquals(i, reader.readInt());
+      }
+      assertTrue(reader.endOfSlice());
+      if (random().nextBoolean()) {
+        pool.reset(true, false);
+        assertEquals(0, bytesUsed.get());
+      } else {
+        pool.reset(true, true);
+        assertEquals(IntBlockPool.INT_BLOCK_SIZE
+            * RamUsageEstimator.NUM_BYTES_INT, bytesUsed.get());
+      }
+    }
+  }
+  
+  public void testMultipleWriterReader() {
+    Counter bytesUsed = Counter.newCounter();
+    IntBlockPool pool = new IntBlockPool(new ByteTrackingAllocator(bytesUsed));
+    for (int j = 0; j < 2; j++) {
+      List<StartEndAndValues> holders = new ArrayList<TestIntBlockPool.StartEndAndValues>();
+      int num = atLeast(4);
+      for (int i = 0; i < num; i++) {
+        holders.add(new StartEndAndValues(random().nextInt(1000)));
+      }
+      IntBlockPool.SliceWriter writer = new IntBlockPool.SliceWriter(pool);
+      IntBlockPool.SliceReader reader = new IntBlockPool.SliceReader(pool);
+      
+      int numValuesToWrite = atLeast(10000);
+      for (int i = 0; i < numValuesToWrite; i++) {
+        StartEndAndValues values = holders
+            .get(random().nextInt(holders.size()));
+        if (values.valueCount == 0) {
+          values.start = writer.startNewSlice();
+        } else {
+          writer.reset(values.end);
+        }
+        writer.writeInt(values.nextValue());
+        values.end = writer.getCurrentOffset();
+        if (random().nextInt(5) == 0) {
+          // pick one and reader the ints
+          assertReader(reader, holders.get(random().nextInt(holders.size())));
+        }
+      }
+      
+      while (!holders.isEmpty()) {
+        StartEndAndValues values = holders.remove(random().nextInt(
+            holders.size()));
+        assertReader(reader, values);
+      }
+      if (random().nextBoolean()) {
+        pool.reset(true, false);
+        assertEquals(0, bytesUsed.get());
+      } else {
+        pool.reset(true, true);
+        assertEquals(IntBlockPool.INT_BLOCK_SIZE
+            * RamUsageEstimator.NUM_BYTES_INT, bytesUsed.get());
+      }
+    }
+  }
+  
+  private static class ByteTrackingAllocator extends IntBlockPool.Allocator {
+    private final Counter bytesUsed;
+    
+    public ByteTrackingAllocator(Counter bytesUsed) {
+      this(IntBlockPool.INT_BLOCK_SIZE, bytesUsed);
+    }
+    
+    public ByteTrackingAllocator(int blockSize, Counter bytesUsed) {
+      super(blockSize);
+      this.bytesUsed = bytesUsed;
+    }
+    
+    public int[] getIntBlock() {
+      bytesUsed.addAndGet(blockSize * RamUsageEstimator.NUM_BYTES_INT);
+      return new int[blockSize];
+    }
+    
+    @Override
+    public void recycleIntBlocks(int[][] blocks, int start, int end) {
+      bytesUsed
+          .addAndGet(-((end - start) * blockSize * RamUsageEstimator.NUM_BYTES_INT));
+    }
+    
+  }
+  
+  private void assertReader(IntBlockPool.SliceReader reader,
+      StartEndAndValues values) {
+    reader.reset(values.start, values.end);
+    for (int i = 0; i < values.valueCount; i++) {
+      assertEquals(values.valueOffset + i, reader.readInt());
+    }
+    assertTrue(reader.endOfSlice());
+  }
+  
+  private static class StartEndAndValues {
+    int valueOffset;
+    int valueCount;
+    int start;
+    int end;
+    
+    public StartEndAndValues(int valueOffset) {
+      this.valueOffset = valueOffset;
+    }
+    
+    public int nextValue() {
+      return valueOffset + valueCount++;
+    }
+    
+  }
+  
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestByteBlockPool.java Fri Nov  2 12:28:14 2012
@@ -28,41 +28,53 @@ import org.apache.lucene.store.RAMDirect
 public class TestByteBlockPool extends LuceneTestCase {
 
   public void testCopyRefAndWrite() throws IOException {
-    List<String> list = new ArrayList<String>();
-    int maxLength = atLeast(500);
-    ByteBlockPool pool = new ByteBlockPool(new ByteBlockPool.DirectAllocator());
+    Counter bytesUsed = Counter.newCounter();
+    ByteBlockPool pool = new ByteBlockPool(new ByteBlockPool.DirectTrackingAllocator(bytesUsed));
     pool.nextBuffer();
-    final int numValues = atLeast(100);
-    BytesRef ref = new BytesRef();
-    for (int i = 0; i < numValues; i++) {
-      final String value = _TestUtil.randomRealisticUnicodeString(random(),
-          maxLength);
-      list.add(value);
-      ref.copyChars(value);
-      pool.copy(ref);
+    boolean reuseFirst = random().nextBoolean();
+    for (int j = 0; j < 2; j++) {
+        
+      List<String> list = new ArrayList<String>();
+      int maxLength = atLeast(500);
+      final int numValues = atLeast(100);
+      BytesRef ref = new BytesRef();
+      for (int i = 0; i < numValues; i++) {
+        final String value = _TestUtil.randomRealisticUnicodeString(random(),
+            maxLength);
+        list.add(value);
+        ref.copyChars(value);
+        pool.copy(ref);
+      }
+      RAMDirectory dir = new RAMDirectory();
+      IndexOutput stream = dir.createOutput("foo.txt", newIOContext(random()));
+      pool.writePool(stream);
+      stream.flush();
+      stream.close();
+      IndexInput input = dir.openInput("foo.txt", newIOContext(random()));
+      assertEquals(pool.byteOffset + pool.byteUpto, stream.length());
+      BytesRef expected = new BytesRef();
+      BytesRef actual = new BytesRef();
+      for (String string : list) {
+        expected.copyChars(string);
+        actual.grow(expected.length);
+        actual.length = expected.length;
+        input.readBytes(actual.bytes, 0, actual.length);
+        assertEquals(expected, actual);
+      }
+      try {
+        input.readByte();
+        fail("must be EOF");
+      } catch (EOFException e) {
+        // expected - read past EOF
+      }
+      pool.reset(random().nextBoolean(), reuseFirst);
+      if (reuseFirst) {
+        assertEquals(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed.get());
+      } else {
+        assertEquals(0, bytesUsed.get());
+        pool.nextBuffer(); // prepare for next iter
+      }
+      dir.close();
     }
-    RAMDirectory dir = new RAMDirectory();
-    IndexOutput stream = dir.createOutput("foo.txt", newIOContext(random()));
-    pool.writePool(stream);
-    stream.flush();
-    stream.close();
-    IndexInput input = dir.openInput("foo.txt", newIOContext(random()));
-    assertEquals(pool.byteOffset + pool.byteUpto, stream.length());
-    BytesRef expected = new BytesRef();
-    BytesRef actual = new BytesRef();
-    for (String string : list) {
-      expected.copyChars(string);
-      actual.grow(expected.length);
-      actual.length = expected.length;
-      input.readBytes(actual.bytes, 0, actual.length);
-      assertEquals(expected, actual);
-    }
-    try {
-      input.readByte();
-      fail("must be EOF");
-    } catch (EOFException e) {
-      // expected - read past EOF
-    }
-    dir.close();
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingByteBlockAllocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingByteBlockAllocator.java?rev=1404946&r1=1404945&r2=1404946&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingByteBlockAllocator.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingByteBlockAllocator.java Fri Nov  2 12:28:14 2012
@@ -39,7 +39,7 @@ public class TestRecyclingByteBlockAlloc
 
   private RecyclingByteBlockAllocator newAllocator() {
     return new RecyclingByteBlockAllocator(1 << (2 + random().nextInt(15)),
-        random().nextInt(97), new AtomicLong());
+        random().nextInt(97), Counter.newCounter());
   }
 
   @Test

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingIntBlockAllocator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingIntBlockAllocator.java?rev=1404946&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingIntBlockAllocator.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestRecyclingIntBlockAllocator.java Fri Nov  2 12:28:14 2012
@@ -0,0 +1,143 @@
+package org.apache.lucene.util;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+import org.junit.Before;
+import org.junit.Test;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Testcase for {@link RecyclingIntBlockAllocator}
+ */
+public class TestRecyclingIntBlockAllocator extends LuceneTestCase {
+
+  /**
+   */
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  private RecyclingIntBlockAllocator newAllocator() {
+    return new RecyclingIntBlockAllocator(1 << (2 + random().nextInt(15)),
+        random().nextInt(97), Counter.newCounter());
+  }
+
+  @Test
+  public void testAllocate() {
+    RecyclingIntBlockAllocator allocator = newAllocator();
+    HashSet<int[]> set = new HashSet<int[]>();
+    int[] block = allocator.getIntBlock();
+    set.add(block);
+    assertNotNull(block);
+    final int size = block.length;
+
+    int num = atLeast(97);
+    for (int i = 0; i < num; i++) {
+      block = allocator.getIntBlock();
+      assertNotNull(block);
+      assertEquals(size, block.length);
+      assertTrue("block is returned twice", set.add(block));
+      assertEquals(4 * size * (i + 2), allocator.bytesUsed()); // zero based + 1
+      assertEquals(0, allocator.numBufferedBlocks());
+    }
+  }
+
+  @Test
+  public void testAllocateAndRecycle() {
+    RecyclingIntBlockAllocator allocator = newAllocator();
+    HashSet<int[]> allocated = new HashSet<int[]>();
+
+    int[] block = allocator.getIntBlock();
+    allocated.add(block);
+    assertNotNull(block);
+    final int size = block.length;
+
+    int numIters = atLeast(97);
+    for (int i = 0; i < numIters; i++) {
+      int num = 1 + random().nextInt(39);
+      for (int j = 0; j < num; j++) {
+        block = allocator.getIntBlock();
+        assertNotNull(block);
+        assertEquals(size, block.length);
+        assertTrue("block is returned twice", allocated.add(block));
+        assertEquals(4 * size * (allocated.size() +  allocator.numBufferedBlocks()), allocator
+            .bytesUsed());
+      }
+      int[][] array = allocated.toArray(new int[0][]);
+      int begin = random().nextInt(array.length);
+      int end = begin + random().nextInt(array.length - begin);
+      List<int[]> selected = new ArrayList<int[]>();
+      for (int j = begin; j < end; j++) {
+        selected.add(array[j]);
+      }
+      allocator.recycleIntBlocks(array, begin, end);
+      for (int j = begin; j < end; j++) {
+        assertNull(array[j]);
+        int[] b = selected.remove(0);
+        assertTrue(allocated.remove(b));
+      }
+    }
+  }
+
+  @Test
+  public void testAllocateAndFree() {
+    RecyclingIntBlockAllocator allocator = newAllocator();
+    HashSet<int[]> allocated = new HashSet<int[]>();
+    int freeButAllocated = 0;
+    int[] block = allocator.getIntBlock();
+    allocated.add(block);
+    assertNotNull(block);
+    final int size = block.length;
+
+    int numIters = atLeast(97);
+    for (int i = 0; i < numIters; i++) {
+      int num = 1 + random().nextInt(39);
+      for (int j = 0; j < num; j++) {
+        block = allocator.getIntBlock();
+        freeButAllocated = Math.max(0, freeButAllocated - 1);
+        assertNotNull(block);
+        assertEquals(size, block.length);
+        assertTrue("block is returned twice", allocated.add(block));
+        assertEquals("" + (4 * size * (allocated.size() + allocator.numBufferedBlocks()) - allocator.bytesUsed()), 4 * size * (allocated.size() + allocator.numBufferedBlocks()),
+            allocator.bytesUsed());
+      }
+
+      int[][] array = allocated.toArray(new int[0][]);
+      int begin = random().nextInt(array.length);
+      int end = begin + random().nextInt(array.length - begin);
+      for (int j = begin; j < end; j++) {
+        int[] b = array[j];
+        assertTrue(allocated.remove(b));
+      }
+      allocator.recycleIntBlocks(array, begin, end);
+      for (int j = begin; j < end; j++) {
+        assertNull(array[j]);
+      }
+      // randomly free blocks
+      int numFreeBlocks = allocator.numBufferedBlocks();
+      int freeBlocks = allocator.freeBlocks(random().nextInt(7 + allocator
+          .maxBufferedBlocks()));
+      assertEquals(allocator.numBufferedBlocks(), numFreeBlocks - freeBlocks);
+    }
+  }
+}
\ No newline at end of file