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