Refactor ByteBlockPool so it is just a "shift/mask big array" (#12625)

Moved all the hairy allocSlice stuff as static method in TermsHashPerField and I introduce a BytesRefBlockPool to
 encapsulate of the BytesRefHash write/read logic.
This commit is contained in:
Ignacio Vera 2023-10-18 08:10:48 +02:00 committed by GitHub
parent ea272d0eda
commit 17ea6d5a0d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 335 additions and 279 deletions

View File

@ -363,6 +363,8 @@ Other
* GITHUB#12512: Remove unused variable in BKDWriter. (Chao Zhang) * GITHUB#12512: Remove unused variable in BKDWriter. (Chao Zhang)
* GITHUB#12625: Refactor ByteBlockPool so it is just a "shift/mask big array". (Ignacio Vera)
======================== Lucene 9.7.0 ======================= ======================== Lucene 9.7.0 =======================
API Changes API Changes

View File

@ -53,7 +53,7 @@ final class ByteSliceReader extends DataInput {
buffer = pool.buffers[bufferUpto]; buffer = pool.buffers[bufferUpto];
upto = startIndex & ByteBlockPool.BYTE_BLOCK_MASK; upto = startIndex & ByteBlockPool.BYTE_BLOCK_MASK;
final int firstSize = ByteBlockPool.LEVEL_SIZE_ARRAY[0]; final int firstSize = TermsHashPerField.LEVEL_SIZE_ARRAY[0];
if (startIndex + firstSize >= endIndex) { if (startIndex + firstSize >= endIndex) {
// There is only this one slice to read // There is only this one slice to read
@ -97,8 +97,8 @@ final class ByteSliceReader extends DataInput {
// Skip to our next slice // Skip to our next slice
final int nextIndex = (int) BitUtil.VH_LE_INT.get(buffer, limit); final int nextIndex = (int) BitUtil.VH_LE_INT.get(buffer, limit);
level = ByteBlockPool.NEXT_LEVEL_ARRAY[level]; level = TermsHashPerField.NEXT_LEVEL_ARRAY[level];
final int newSize = ByteBlockPool.LEVEL_SIZE_ARRAY[level]; final int newSize = TermsHashPerField.LEVEL_SIZE_ARRAY[level];
bufferUpto = nextIndex / ByteBlockPool.BYTE_BLOCK_SIZE; bufferUpto = nextIndex / ByteBlockPool.BYTE_BLOCK_SIZE;
bufferOffset = bufferUpto * ByteBlockPool.BYTE_BLOCK_SIZE; bufferOffset = bufferUpto * ByteBlockPool.BYTE_BLOCK_SIZE;

View File

@ -24,6 +24,7 @@ import java.util.Map;
import org.apache.lucene.index.FreqProxTermsWriterPerField.FreqProxPostingsArray; import org.apache.lucene.index.FreqProxTermsWriterPerField.FreqProxPostingsArray;
import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBlockPool;
import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.BytesRefBuilder;
/** /**
@ -120,6 +121,7 @@ class FreqProxFields extends Fields {
private static class FreqProxTermsEnum extends BaseTermsEnum { private static class FreqProxTermsEnum extends BaseTermsEnum {
final FreqProxTermsWriterPerField terms; final FreqProxTermsWriterPerField terms;
final BytesRefBlockPool termsPool;
final int[] sortedTermIDs; final int[] sortedTermIDs;
final FreqProxPostingsArray postingsArray; final FreqProxPostingsArray postingsArray;
final BytesRef scratch = new BytesRef(); final BytesRef scratch = new BytesRef();
@ -128,6 +130,7 @@ class FreqProxFields extends Fields {
FreqProxTermsEnum(FreqProxTermsWriterPerField terms) { FreqProxTermsEnum(FreqProxTermsWriterPerField terms) {
this.terms = terms; this.terms = terms;
this.termsPool = new BytesRefBlockPool(terms.bytePool);
this.numTerms = terms.getNumTerms(); this.numTerms = terms.getNumTerms();
sortedTermIDs = terms.getSortedTermIDs(); sortedTermIDs = terms.getSortedTermIDs();
assert sortedTermIDs != null; assert sortedTermIDs != null;
@ -149,7 +152,7 @@ class FreqProxFields extends Fields {
while (hi >= lo) { while (hi >= lo) {
int mid = (lo + hi) >>> 1; int mid = (lo + hi) >>> 1;
int textStart = postingsArray.textStarts[sortedTermIDs[mid]]; int textStart = postingsArray.textStarts[sortedTermIDs[mid]];
terms.bytePool.setBytesRef(scratch, textStart); termsPool.fillBytesRef(scratch, textStart);
int cmp = scratch.compareTo(text); int cmp = scratch.compareTo(text);
if (cmp < 0) { if (cmp < 0) {
lo = mid + 1; lo = mid + 1;
@ -169,7 +172,7 @@ class FreqProxFields extends Fields {
return SeekStatus.END; return SeekStatus.END;
} else { } else {
int textStart = postingsArray.textStarts[sortedTermIDs[ord]]; int textStart = postingsArray.textStarts[sortedTermIDs[ord]];
terms.bytePool.setBytesRef(scratch, textStart); termsPool.fillBytesRef(scratch, textStart);
assert term().compareTo(text) > 0; assert term().compareTo(text) > 0;
return SeekStatus.NOT_FOUND; return SeekStatus.NOT_FOUND;
} }
@ -179,7 +182,7 @@ class FreqProxFields extends Fields {
public void seekExact(long ord) { public void seekExact(long ord) {
this.ord = (int) ord; this.ord = (int) ord;
int textStart = postingsArray.textStarts[sortedTermIDs[this.ord]]; int textStart = postingsArray.textStarts[sortedTermIDs[this.ord]];
terms.bytePool.setBytesRef(scratch, textStart); termsPool.fillBytesRef(scratch, textStart);
} }
@Override @Override
@ -189,7 +192,7 @@ class FreqProxFields extends Fields {
return null; return null;
} else { } else {
int textStart = postingsArray.textStarts[sortedTermIDs[ord]]; int textStart = postingsArray.textStarts[sortedTermIDs[ord]];
terms.bytePool.setBytesRef(scratch, textStart); termsPool.fillBytesRef(scratch, textStart);
return scratch; return scratch;
} }
} }

View File

@ -21,8 +21,8 @@ import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute; import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute; import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
import org.apache.lucene.codecs.TermVectorsWriter; import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBlockPool;
final class TermVectorsConsumerPerField extends TermsHashPerField { final class TermVectorsConsumerPerField extends TermsHashPerField {
@ -40,7 +40,7 @@ final class TermVectorsConsumerPerField extends TermsHashPerField {
private OffsetAttribute offsetAttribute; private OffsetAttribute offsetAttribute;
private PayloadAttribute payloadAttribute; private PayloadAttribute payloadAttribute;
private TermFrequencyAttribute termFreqAtt; private TermFrequencyAttribute termFreqAtt;
private final ByteBlockPool termBytePool; private final BytesRefBlockPool termBytePool;
private boolean hasPayloads; // if enabled, and we actually saw any for this field private boolean hasPayloads; // if enabled, and we actually saw any for this field
@ -58,7 +58,7 @@ final class TermVectorsConsumerPerField extends TermsHashPerField {
this.termsWriter = termsHash; this.termsWriter = termsHash;
this.fieldInfo = fieldInfo; this.fieldInfo = fieldInfo;
this.fieldState = invertState; this.fieldState = invertState;
termBytePool = termsHash.termBytePool; termBytePool = new BytesRefBlockPool(termsHash.termBytePool);
} }
/** /**
@ -106,7 +106,7 @@ final class TermVectorsConsumerPerField extends TermsHashPerField {
final int freq = postings.freqs[termID]; final int freq = postings.freqs[termID];
// Get BytesRef // Get BytesRef
termBytePool.setBytesRef(flushTerm, postings.textStarts[termID]); termBytePool.fillBytesRef(flushTerm, postings.textStarts[termID]);
tv.startTerm(flushTerm, freq); tv.startTerm(flushTerm, freq);
if (doVectorPositions || doVectorOffsets) { if (doVectorPositions || doVectorOffsets) {

View File

@ -17,6 +17,7 @@
package org.apache.lucene.index; package org.apache.lucene.index;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.ByteBlockPool; import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash; import org.apache.lucene.util.BytesRefHash;
@ -97,7 +98,7 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
final int offsetInAddressBuffer = streamStartOffset & IntBlockPool.INT_BLOCK_MASK; final int offsetInAddressBuffer = streamStartOffset & IntBlockPool.INT_BLOCK_MASK;
reader.init( reader.init(
bytePool, bytePool,
postingsArray.byteStarts[termID] + stream * ByteBlockPool.FIRST_LEVEL_SIZE, postingsArray.byteStarts[termID] + stream * FIRST_LEVEL_SIZE,
streamAddressBuffer[offsetInAddressBuffer + stream]); streamAddressBuffer[offsetInAddressBuffer + stream]);
} }
@ -153,8 +154,7 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
intPool.nextBuffer(); intPool.nextBuffer();
} }
if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < (2 * streamCount) * FIRST_LEVEL_SIZE) {
< (2 * streamCount) * ByteBlockPool.FIRST_LEVEL_SIZE) {
// can we fit at least one byte per stream in the current buffer, if not allocate a new one // can we fit at least one byte per stream in the current buffer, if not allocate a new one
bytePool.nextBuffer(); bytePool.nextBuffer();
} }
@ -168,7 +168,7 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
for (int i = 0; i < streamCount; i++) { for (int i = 0; i < streamCount; i++) {
// initialize each stream with a slice we start with ByteBlockPool.FIRST_LEVEL_SIZE) // initialize each stream with a slice we start with ByteBlockPool.FIRST_LEVEL_SIZE)
// and grow as we need more space. see ByteBlockPool.LEVEL_SIZE_ARRAY // and grow as we need more space. see ByteBlockPool.LEVEL_SIZE_ARRAY
final int upto = bytePool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE); final int upto = newSlice(bytePool, FIRST_LEVEL_SIZE, 0);
termStreamAddressBuffer[streamAddressOffset + i] = upto + bytePool.byteOffset; termStreamAddressBuffer[streamAddressOffset + i] = upto + bytePool.byteOffset;
} }
postingsArray.byteStarts[termID] = termStreamAddressBuffer[streamAddressOffset]; postingsArray.byteStarts[termID] = termStreamAddressBuffer[streamAddressOffset];
@ -221,7 +221,7 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
int offset = upto & ByteBlockPool.BYTE_BLOCK_MASK; int offset = upto & ByteBlockPool.BYTE_BLOCK_MASK;
if (bytes[offset] != 0) { if (bytes[offset] != 0) {
// End of slice; allocate a new one // End of slice; allocate a new one
offset = bytePool.allocSlice(bytes, offset); offset = allocSlice(bytePool, bytes, offset);
bytes = bytePool.buffer; bytes = bytePool.buffer;
termStreamAddressBuffer[streamAddress] = offset + bytePool.byteOffset; termStreamAddressBuffer[streamAddress] = offset + bytePool.byteOffset;
} }
@ -243,7 +243,7 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
} }
while (offset < end) { while (offset < end) {
int offsetAndLength = bytePool.allocKnownSizeSlice(slice, sliceOffset); int offsetAndLength = allocKnownSizeSlice(bytePool, slice, sliceOffset);
sliceOffset = offsetAndLength >> 8; sliceOffset = offsetAndLength >> 8;
int sliceLength = offsetAndLength & 0xff; int sliceLength = offsetAndLength & 0xff;
slice = bytePool.buffer; slice = bytePool.buffer;
@ -255,6 +255,81 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
} }
} }
// Size of each slice. These arrays should be at most 16
// elements (index is encoded with 4 bits). First array
// is just a compact way to encode X+1 with a max. Second
// array is the length of each slice, ie first slice is 5
// bytes, next slice is 14 bytes, etc.
/**
* An array holding the offset into the {@link #LEVEL_SIZE_ARRAY} to quickly navigate to the next
* slice level.
*/
static final int[] NEXT_LEVEL_ARRAY = {1, 2, 3, 4, 5, 6, 7, 8, 9, 9};
/** An array holding the level sizes for byte slices. */
static final int[] LEVEL_SIZE_ARRAY = {5, 14, 20, 30, 40, 40, 80, 80, 120, 200};
/** The first level size for new slices */
static final int FIRST_LEVEL_SIZE = LEVEL_SIZE_ARRAY[0];
/**
* Allocates a new slice with the given size. As each slice is filled with 0's initially, we mark
* the end with a non-zero byte. This way we don't need to record its length and instead allocate
* new slice once they hit a non-zero byte.
*/
// pkg private for access by tests
static int newSlice(ByteBlockPool bytePool, final int size, final int level) {
assert LEVEL_SIZE_ARRAY[level] == size;
// Maybe allocate another block
if (bytePool.byteUpto > ByteBlockPool.BYTE_BLOCK_SIZE - size) {
bytePool.nextBuffer();
}
final int upto = bytePool.byteUpto;
bytePool.byteUpto += size;
bytePool.buffer[bytePool.byteUpto - 1] = (byte) (16 | level);
return upto;
}
/**
* Creates a new byte slice with the given starting size and returns the slices offset in the
* pool.
*/
// pkg private for access by tests
static int allocSlice(ByteBlockPool bytePool, final byte[] slice, final int upto) {
return allocKnownSizeSlice(bytePool, slice, upto) >> 8;
}
/**
* Create a new byte slice with the given starting size return the slice offset in the pool and
* length. The lower 8 bits of the returned int represent the length of the slice, and the upper
* 24 bits represent the offset.
*/
// pkg private for access by tests
static int allocKnownSizeSlice(ByteBlockPool bytePool, final byte[] slice, final int upto) {
// The idea is to allocate the next slice and then write the address of the new slice
// into the last 4 bytes of the previous slice (the "forwarding address").
final int level = slice[upto] & 15;
final int newLevel = NEXT_LEVEL_ARRAY[level];
final int newSize = LEVEL_SIZE_ARRAY[newLevel];
final int newUpto = newSlice(bytePool, newSize, newLevel);
final int offset = newUpto + bytePool.byteOffset;
// Copy forward the past 3 bytes (which we are about to overwrite with the forwarding address).
// We actually copy 4 bytes at once since VarHandles make it cheap.
final int past3Bytes = ((int) BitUtil.VH_LE_INT.get(slice, upto - 3)) & 0xFFFFFF;
// Ensure we're not changing the content of `buffer` by setting 4 bytes instead of 3. This
// should never happen since the next `newSize` bytes must be equal to 0.
assert bytePool.buffer[newUpto + 3] == 0;
BitUtil.VH_LE_INT.set(bytePool.buffer, newUpto, past3Bytes);
// Write forwarding address at end of last slice:
BitUtil.VH_LE_INT.set(slice, upto - 3, offset);
return ((newUpto + 3) << 8) | (newSize - 3);
}
final void writeVInt(int stream, int i) { final void writeVInt(int stream, int i) {
assert stream < streamCount; assert stream < streamCount;
while ((i & ~0x7F) != 0) { while ((i & ~0x7F) != 0) {

View File

@ -22,15 +22,9 @@ import java.util.Arrays;
import java.util.List; import java.util.List;
/** /**
* Class that Posting and PostingVector use to write byte streams into shared fixed-size byte[] * Represents a logical byte[] as a series of blocks. You can write into it by using append and read
* arrays. The idea is to allocate slices of increasing lengths For example, the first slice is 5 * using the offset position (random access). The buffers can be reset to reuse the allocated
* bytes, the next slice is 14, etc. We start by writing our bytes into the first 5 bytes. When we * buffers.
* hit the end of the slice, we allocate the next slice and then write the address of the new slice
* into the last 4 bytes of the previous slice (the "forwarding address").
*
* <p>Each slice is filled with 0's initially, and we mark the end with a non-zero byte. This way
* the methods that are writing into the slice don't need to record its length and instead allocate
* a new slice once they hit a non-zero byte.
* *
* @lucene.internal * @lucene.internal
*/ */
@ -66,11 +60,7 @@ public final class ByteBlockPool implements Accountable {
public static final class DirectAllocator extends Allocator { public static final class DirectAllocator extends Allocator {
public DirectAllocator() { public DirectAllocator() {
this(BYTE_BLOCK_SIZE); super(BYTE_BLOCK_SIZE);
}
public DirectAllocator(int blockSize) {
super(blockSize);
} }
@Override @Override
@ -82,11 +72,7 @@ public final class ByteBlockPool implements Accountable {
private final Counter bytesUsed; private final Counter bytesUsed;
public DirectTrackingAllocator(Counter bytesUsed) { public DirectTrackingAllocator(Counter bytesUsed) {
this(BYTE_BLOCK_SIZE, bytesUsed); super(BYTE_BLOCK_SIZE);
}
public DirectTrackingAllocator(int blockSize, Counter bytesUsed) {
super(blockSize);
this.bytesUsed = bytesUsed; this.bytesUsed = bytesUsed;
} }
@ -202,85 +188,6 @@ public final class ByteBlockPool implements Accountable {
byteOffset = Math.addExact(byteOffset, BYTE_BLOCK_SIZE); byteOffset = Math.addExact(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();
final int upto = byteUpto;
byteUpto += size;
buffer[byteUpto - 1] = 16;
return upto;
}
// Size of each slice. These arrays should be at most 16
// elements (index is encoded with 4 bits). First array
// is just a compact way to encode X+1 with a max. Second
// array is the length of each slice, ie first slice is 5
// bytes, next slice is 14 bytes, etc.
/**
* An array holding the offset into the {@link ByteBlockPool#LEVEL_SIZE_ARRAY} to quickly navigate
* to the next slice level.
*/
public static final int[] NEXT_LEVEL_ARRAY = {1, 2, 3, 4, 5, 6, 7, 8, 9, 9};
/** An array holding the level sizes for byte slices. */
public static final 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 static final 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) {
return allocKnownSizeSlice(slice, upto) >> 8;
}
/**
* Create a new byte slice with the given starting size return the slice offset in the pool and
* length. The lower 8 bits of the returned int represent the length of the slice, and the upper
* 24 bits represent the offset.
*/
public int allocKnownSizeSlice(final byte[] slice, final int upto) {
final int level = slice[upto] & 15;
final int newLevel = NEXT_LEVEL_ARRAY[level];
final int newSize = LEVEL_SIZE_ARRAY[newLevel];
// Maybe allocate another block
if (byteUpto > BYTE_BLOCK_SIZE - newSize) {
nextBuffer();
}
final int newUpto = byteUpto;
final int offset = newUpto + byteOffset;
byteUpto += newSize;
// Copy forward the past 3 bytes (which we are about to overwrite with the forwarding address).
// We actually copy 4 bytes at once since VarHandles make it cheap.
int past3Bytes = ((int) BitUtil.VH_LE_INT.get(slice, upto - 3)) & 0xFFFFFF;
// Ensure we're not changing the content of `buffer` by setting 4 bytes instead of 3. This
// should never happen since the next `newSize` bytes must be equal to 0.
assert buffer[newUpto + 3] == 0;
BitUtil.VH_LE_INT.set(buffer, newUpto, past3Bytes);
// Write forwarding address at end of last slice:
BitUtil.VH_LE_INT.set(slice, upto - 3, offset);
// Write new level:
buffer[byteUpto - 1] = (byte) (16 | newLevel);
return ((newUpto + 3) << 8) | (newSize - 3);
}
/** /**
* Fill the provided {@link BytesRef} with the bytes at the specified offset/length slice. This * Fill the provided {@link BytesRef} with the bytes at the specified offset/length slice. This
* will avoid copying the bytes, if the slice fits into a single block; otherwise, it uses the * will avoid copying the bytes, if the slice fits into a single block; otherwise, it uses the
@ -306,23 +213,6 @@ public final class ByteBlockPool implements Accountable {
} }
} }
// Fill in a BytesRef from term's length & bytes encoded in
// byte block
public void setBytesRef(BytesRef term, int textStart) {
final byte[] bytes = term.bytes = buffers[textStart >> BYTE_BLOCK_SHIFT];
int pos = textStart & BYTE_BLOCK_MASK;
if ((bytes[pos] & 0x80) == 0) {
// length is 1 byte
term.length = bytes[pos];
term.offset = pos + 1;
} else {
// length is 2 bytes
term.length = ((short) BitUtil.VH_BE_SHORT.get(bytes, pos)) & 0x7FFF;
term.offset = pos + 2;
}
assert term.length >= 0;
}
/** Appends the bytes in the provided {@link BytesRef} at the current position. */ /** Appends the bytes in the provided {@link BytesRef} at the current position. */
public void append(final BytesRef bytes) { public void append(final BytesRef bytes) {
int bytesLeft = bytes.length; int bytesLeft = bytes.length;
@ -366,26 +256,6 @@ public final class ByteBlockPool implements Accountable {
} }
} }
/**
* Set the given {@link BytesRef} so that its content is equal to the {@code ref.length} bytes
* starting at {@code offset}. Most of the time this method will set pointers to internal
* data-structures. However, in case a value crosses a boundary, a fresh copy will be returned. On
* the contrary to {@link #setBytesRef(BytesRef, int)}, this does not expect the length to be
* encoded with the data.
*/
public void setRawBytesRef(BytesRef ref, final long offset) {
int bufferIndex = (int) (offset >> BYTE_BLOCK_SHIFT);
int pos = (int) (offset & BYTE_BLOCK_MASK);
if (pos + ref.length <= BYTE_BLOCK_SIZE) {
ref.bytes = buffers[bufferIndex];
ref.offset = pos;
} else {
ref.bytes = new byte[ref.length];
ref.offset = 0;
readBytes(offset, ref.bytes, 0, ref.length);
}
}
@Override @Override
public long ramBytesUsed() { public long ramBytesUsed() {
long size = BASE_RAM_BYTES; long size = BASE_RAM_BYTES;

View File

@ -0,0 +1,176 @@
/*
* 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.
*/
package org.apache.lucene.util;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SHIFT;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
import java.util.Arrays;
/**
* Represents a logical list of ByteRef backed by a {@link ByteBlockPool}. It uses up to two bytes
* to record the length of the BytesRef followed by the actual bytes. They can be read using the
* start position returned when they are appended.
*
* <p>The {@link BytesRef} is written so it never crosses the {@link ByteBlockPool#BYTE_BLOCK_SIZE}
* boundary. The limit of the largest {@link BytesRef} is therefore {@link
* ByteBlockPool#BYTE_BLOCK_SIZE}-2 bytes.
*
* @lucene.internal
*/
public class BytesRefBlockPool implements Accountable {
private static final long BASE_RAM_BYTES =
RamUsageEstimator.shallowSizeOfInstance(BytesRefBlockPool.class);
private final ByteBlockPool byteBlockPool;
public BytesRefBlockPool() {
this.byteBlockPool = new ByteBlockPool(new ByteBlockPool.DirectAllocator());
}
public BytesRefBlockPool(ByteBlockPool byteBlockPool) {
this.byteBlockPool = byteBlockPool;
}
/** Reset this buffer to the empty state. */
void reset() {
byteBlockPool.reset(false, false); // we don't need to 0-fill the buffers
}
/**
* Populates the given BytesRef with the term starting at <i>start</i>.
*
* @see #fillBytesRef(BytesRef, int)
*/
public void fillBytesRef(BytesRef term, int start) {
final byte[] bytes = term.bytes = byteBlockPool.buffers[start >> BYTE_BLOCK_SHIFT];
int pos = start & BYTE_BLOCK_MASK;
if ((bytes[pos] & 0x80) == 0) {
// length is 1 byte
term.length = bytes[pos];
term.offset = pos + 1;
} else {
// length is 2 bytes
term.length = ((short) BitUtil.VH_BE_SHORT.get(bytes, pos)) & 0x7FFF;
term.offset = pos + 2;
}
assert term.length >= 0;
}
/**
* Add a term returning the start position on the underlying {@link ByteBlockPool}. THis can be
* used to read back the value using {@link #fillBytesRef(BytesRef, int)}.
*
* @see #fillBytesRef(BytesRef, int)
*/
public int addBytesRef(BytesRef bytes) {
final int length = bytes.length;
final int len2 = 2 + bytes.length;
if (len2 + byteBlockPool.byteUpto > BYTE_BLOCK_SIZE) {
if (len2 > BYTE_BLOCK_SIZE) {
throw new BytesRefHash.MaxBytesLengthExceededException(
"bytes can be at most " + (BYTE_BLOCK_SIZE - 2) + " in length; got " + bytes.length);
}
byteBlockPool.nextBuffer();
}
final byte[] buffer = byteBlockPool.buffer;
final int bufferUpto = byteBlockPool.byteUpto;
final int textStart = bufferUpto + byteBlockPool.byteOffset;
// We first encode the length, followed by the
// bytes. Length is encoded as vInt, but will consume
// 1 or 2 bytes at most (we reject too-long terms,
// above).
if (length < 128) {
// 1 byte to store length
buffer[bufferUpto] = (byte) length;
byteBlockPool.byteUpto += length + 1;
assert length >= 0 : "Length must be positive: " + length;
System.arraycopy(bytes.bytes, bytes.offset, buffer, bufferUpto + 1, length);
} else {
// 2 byte to store length
BitUtil.VH_BE_SHORT.set(buffer, bufferUpto, (short) (length | 0x8000));
byteBlockPool.byteUpto += length + 2;
System.arraycopy(bytes.bytes, bytes.offset, buffer, bufferUpto + 2, length);
}
return textStart;
}
/**
* Computes the hash of the BytesRef at the given start. This is equivalent of doing:
*
* <pre>
* BytesRef bytes = new BytesRef();
* fillTerm(bytes, start);
* BytesRefHash.doHash(bytes.bytes, bytes.pos, bytes.len);
* </pre>
*
* It just saves the work of filling the BytesRef.
*/
int hash(int start) {
final int offset = start & BYTE_BLOCK_MASK;
final byte[] bytes = byteBlockPool.buffers[start >> BYTE_BLOCK_SHIFT];
final int len;
int pos;
if ((bytes[offset] & 0x80) == 0) {
// length is 1 byte
len = bytes[offset];
pos = offset + 1;
} else {
len = ((short) BitUtil.VH_BE_SHORT.get(bytes, offset)) & 0x7FFF;
pos = offset + 2;
}
return BytesRefHash.doHash(bytes, pos, len);
}
/**
* Computes the equality between the BytesRef at the start position with the provided BytesRef.
* This is equivalent of doing:
*
* <pre>
* BytesRef bytes = new BytesRef();
* fillTerm(bytes, start);
* Arrays.equals(bytes.bytes, bytes.offset, bytes.offset + length, b.bytes, b.offset, b.offset + b.length);
* </pre>
*
* It just saves the work of filling the BytesRef.
*/
boolean equals(int start, BytesRef b) {
final byte[] bytes = byteBlockPool.buffers[start >> BYTE_BLOCK_SHIFT];
int pos = start & BYTE_BLOCK_MASK;
final int length;
final int offset;
if ((bytes[pos] & 0x80) == 0) {
// length is 1 byte
length = bytes[pos];
offset = pos + 1;
} else {
// length is 2 bytes
length = ((short) BitUtil.VH_BE_SHORT.get(bytes, pos)) & 0x7FFF;
offset = pos + 2;
}
return Arrays.equals(bytes, offset, offset + length, b.bytes, b.offset, b.offset + b.length);
}
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES + byteBlockPool.ramBytesUsed();
}
}

View File

@ -16,10 +16,6 @@
*/ */
package org.apache.lucene.util; package org.apache.lucene.util;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SHIFT;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
import java.util.Arrays; import java.util.Arrays;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.util.ByteBlockPool.DirectAllocator; import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
@ -48,7 +44,7 @@ public final class BytesRefHash implements Accountable {
// the following fields are needed by comparator, // the following fields are needed by comparator,
// so package private to prevent access$-methods: // so package private to prevent access$-methods:
final ByteBlockPool pool; final BytesRefBlockPool pool;
int[] bytesStart; int[] bytesStart;
private int hashSize; private int hashSize;
@ -58,7 +54,7 @@ public final class BytesRefHash implements Accountable {
private int lastCount = -1; private int lastCount = -1;
private int[] ids; private int[] ids;
private final BytesStartArray bytesStartArray; private final BytesStartArray bytesStartArray;
private Counter bytesUsed; private final Counter bytesUsed;
/** /**
* Creates a new {@link BytesRefHash} with a {@link ByteBlockPool} using a {@link * Creates a new {@link BytesRefHash} with a {@link ByteBlockPool} using a {@link
@ -78,7 +74,7 @@ public final class BytesRefHash implements Accountable {
hashSize = capacity; hashSize = capacity;
hashHalfSize = hashSize >> 1; hashHalfSize = hashSize >> 1;
hashMask = hashSize - 1; hashMask = hashSize - 1;
this.pool = pool; this.pool = new BytesRefBlockPool(pool);
ids = new int[hashSize]; ids = new int[hashSize];
Arrays.fill(ids, -1); Arrays.fill(ids, -1);
this.bytesStartArray = bytesStartArray; this.bytesStartArray = bytesStartArray;
@ -110,7 +106,7 @@ public final class BytesRefHash implements Accountable {
public BytesRef get(int bytesID, BytesRef ref) { public BytesRef get(int bytesID, BytesRef ref) {
assert bytesStart != null : "bytesStart is null - not initialized"; assert bytesStart != null : "bytesStart is null - not initialized";
assert bytesID < bytesStart.length : "bytesID exceeds byteStart len: " + bytesStart.length; assert bytesID < bytesStart.length : "bytesID exceeds byteStart len: " + bytesStart.length;
pool.setBytesRef(ref, bytesStart[bytesID]); pool.fillBytesRef(ref, bytesStart[bytesID]);
return ref; return ref;
} }
@ -160,30 +156,12 @@ public final class BytesRefHash implements Accountable {
@Override @Override
protected void get(BytesRefBuilder builder, BytesRef result, int i) { protected void get(BytesRefBuilder builder, BytesRef result, int i) {
pool.setBytesRef(result, bytesStart[compact[i]]); pool.fillBytesRef(result, bytesStart[compact[i]]);
} }
}.sort(0, count); }.sort(0, count);
return compact; return compact;
} }
private boolean equals(int id, BytesRef b) {
final int textStart = bytesStart[id];
final byte[] bytes = pool.buffers[textStart >> BYTE_BLOCK_SHIFT];
int pos = textStart & BYTE_BLOCK_MASK;
final int length;
final int offset;
if ((bytes[pos] & 0x80) == 0) {
// length is 1 byte
length = bytes[pos];
offset = pos + 1;
} else {
// length is 2 bytes
length = ((short) BitUtil.VH_BE_SHORT.get(bytes, pos)) & 0x7FFF;
offset = pos + 2;
}
return Arrays.equals(bytes, offset, offset + length, b.bytes, b.offset, b.offset + b.length);
}
private boolean shrink(int targetSize) { private boolean shrink(int targetSize) {
// Cannot use ArrayUtil.shrink because we require power // Cannot use ArrayUtil.shrink because we require power
// of 2: // of 2:
@ -209,7 +187,7 @@ public final class BytesRefHash implements Accountable {
lastCount = count; lastCount = count;
count = 0; count = 0;
if (resetPool) { if (resetPool) {
pool.reset(false, false); // we don't need to 0-fill the buffers pool.reset();
} }
bytesStart = bytesStartArray.clear(); bytesStart = bytesStartArray.clear();
if (lastCount != -1 && shrink(lastCount)) { if (lastCount != -1 && shrink(lastCount)) {
@ -242,47 +220,18 @@ public final class BytesRefHash implements Accountable {
*/ */
public int add(BytesRef bytes) { public int add(BytesRef bytes) {
assert bytesStart != null : "Bytesstart is null - not initialized"; assert bytesStart != null : "Bytesstart is null - not initialized";
final int length = bytes.length;
// final position // final position
final int hashPos = findHash(bytes); final int hashPos = findHash(bytes);
int e = ids[hashPos]; int e = ids[hashPos];
if (e == -1) { if (e == -1) {
// new entry // new entry
final int len2 = 2 + bytes.length;
if (len2 + pool.byteUpto > BYTE_BLOCK_SIZE) {
if (len2 > BYTE_BLOCK_SIZE) {
throw new MaxBytesLengthExceededException(
"bytes can be at most " + (BYTE_BLOCK_SIZE - 2) + " in length; got " + bytes.length);
}
pool.nextBuffer();
}
final byte[] buffer = pool.buffer;
final int bufferUpto = pool.byteUpto;
if (count >= bytesStart.length) { if (count >= bytesStart.length) {
bytesStart = bytesStartArray.grow(); bytesStart = bytesStartArray.grow();
assert count < bytesStart.length + 1 : "count: " + count + " len: " + bytesStart.length; assert count < bytesStart.length + 1 : "count: " + count + " len: " + bytesStart.length;
} }
bytesStart[count] = pool.addBytesRef(bytes);
e = count++; e = count++;
bytesStart[e] = bufferUpto + pool.byteOffset;
// We first encode the length, followed by the
// bytes. Length is encoded as vInt, but will consume
// 1 or 2 bytes at most (we reject too-long terms,
// above).
if (length < 128) {
// 1 byte to store length
buffer[bufferUpto] = (byte) length;
pool.byteUpto += length + 1;
assert length >= 0 : "Length must be positive: " + length;
System.arraycopy(bytes.bytes, bytes.offset, buffer, bufferUpto + 1, length);
} else {
// 2 byte to store length
BitUtil.VH_BE_SHORT.set(buffer, bufferUpto, (short) (length | 0x8000));
pool.byteUpto += length + 2;
System.arraycopy(bytes.bytes, bytes.offset, buffer, bufferUpto + 2, length);
}
assert ids[hashPos] == -1; assert ids[hashPos] == -1;
ids[hashPos] = e; ids[hashPos] = e;
@ -312,14 +261,14 @@ public final class BytesRefHash implements Accountable {
// final position // final position
int hashPos = code & hashMask; int hashPos = code & hashMask;
int e = ids[hashPos]; int e = ids[hashPos];
if (e != -1 && !equals(e, bytes)) { if (e != -1 && pool.equals(bytesStart[e], bytes) == false) {
// Conflict; use linear probe to find an open slot // Conflict; use linear probe to find an open slot
// (see LUCENE-5604): // (see LUCENE-5604):
do { do {
code++; code++;
hashPos = code & hashMask; hashPos = code & hashMask;
e = ids[hashPos]; e = ids[hashPos];
} while (e != -1 && !equals(e, bytes)); } while (e != -1 && pool.equals(bytesStart[e], bytes) == false);
} }
return hashPos; return hashPos;
@ -378,20 +327,7 @@ public final class BytesRefHash implements Accountable {
if (e0 != -1) { if (e0 != -1) {
int code; int code;
if (hashOnData) { if (hashOnData) {
final int off = bytesStart[e0]; code = pool.hash(bytesStart[e0]);
final int start = off & BYTE_BLOCK_MASK;
final byte[] bytes = pool.buffers[off >> BYTE_BLOCK_SHIFT];
final int len;
int pos;
if ((bytes[start] & 0x80) == 0) {
// length is 1 byte
len = bytes[start];
pos = start + 1;
} else {
len = ((short) BitUtil.VH_BE_SHORT.get(bytes, start)) & 0x7FFF;
pos = start + 2;
}
code = doHash(bytes, pos, len);
} else { } else {
code = bytesStart[e0]; code = bytesStart[e0];
} }
@ -418,7 +354,7 @@ public final class BytesRefHash implements Accountable {
} }
// TODO: maybe use long? But our keys are typically short... // TODO: maybe use long? But our keys are typically short...
private int doHash(byte[] bytes, int offset, int length) { static int doHash(byte[] bytes, int offset, int length) {
return StringHelper.murmurhash3_x86_32(bytes, offset, length, StringHelper.GOOD_FAST_HASH_SEED); return StringHelper.murmurhash3_x86_32(bytes, offset, length, StringHelper.GOOD_FAST_HASH_SEED);
} }

View File

@ -36,12 +36,11 @@ public final class RecyclingByteBlockAllocator extends ByteBlockPool.Allocator {
/** /**
* Creates a new {@link RecyclingByteBlockAllocator} * Creates a new {@link RecyclingByteBlockAllocator}
* *
* @param blockSize the block size in bytes
* @param maxBufferedBlocks maximum number of buffered byte block * @param maxBufferedBlocks maximum number of buffered byte block
* @param bytesUsed {@link Counter} reference counting internally allocated bytes * @param bytesUsed {@link Counter} reference counting internally allocated bytes
*/ */
public RecyclingByteBlockAllocator(int blockSize, int maxBufferedBlocks, Counter bytesUsed) { public RecyclingByteBlockAllocator(int maxBufferedBlocks, Counter bytesUsed) {
super(blockSize); super(ByteBlockPool.BYTE_BLOCK_SIZE);
freeByteBlocks = new byte[maxBufferedBlocks][]; freeByteBlocks = new byte[maxBufferedBlocks][];
this.maxBufferedBlocks = maxBufferedBlocks; this.maxBufferedBlocks = maxBufferedBlocks;
this.bytesUsed = bytesUsed; this.bytesUsed = bytesUsed;
@ -50,11 +49,10 @@ public final class RecyclingByteBlockAllocator extends ByteBlockPool.Allocator {
/** /**
* Creates a new {@link RecyclingByteBlockAllocator}. * Creates a new {@link RecyclingByteBlockAllocator}.
* *
* @param blockSize the block size in bytes
* @param maxBufferedBlocks maximum number of buffered byte block * @param maxBufferedBlocks maximum number of buffered byte block
*/ */
public RecyclingByteBlockAllocator(int blockSize, int maxBufferedBlocks) { public RecyclingByteBlockAllocator(int maxBufferedBlocks) {
this(blockSize, maxBufferedBlocks, Counter.newCounter(false)); this(maxBufferedBlocks, Counter.newCounter(false));
} }
/** /**
@ -63,7 +61,7 @@ public final class RecyclingByteBlockAllocator extends ByteBlockPool.Allocator {
* ({@value #DEFAULT_BUFFERED_BLOCKS}). * ({@value #DEFAULT_BUFFERED_BLOCKS}).
*/ */
public RecyclingByteBlockAllocator() { public RecyclingByteBlockAllocator() {
this(ByteBlockPool.BYTE_BLOCK_SIZE, 64, Counter.newCounter(false)); this(DEFAULT_BUFFERED_BLOCKS, Counter.newCounter(false));
} }
@Override @Override

View File

@ -37,10 +37,10 @@ public class TestByteSliceReader extends LuceneTestCase {
BLOCK_POOL = new ByteBlockPool(new ByteBlockPool.DirectAllocator()); BLOCK_POOL = new ByteBlockPool(new ByteBlockPool.DirectAllocator());
BLOCK_POOL.nextBuffer(); BLOCK_POOL.nextBuffer();
byte[] buffer = BLOCK_POOL.buffer; byte[] buffer = BLOCK_POOL.buffer;
int upto = BLOCK_POOL.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE); int upto = TermsHashPerField.newSlice(BLOCK_POOL, TermsHashPerField.FIRST_LEVEL_SIZE, 0);
for (byte randomByte : RANDOM_DATA) { for (byte randomByte : RANDOM_DATA) {
if ((buffer[upto] & 16) != 0) { if ((buffer[upto] & 16) != 0) {
upto = BLOCK_POOL.allocSlice(buffer, upto); upto = TermsHashPerField.allocSlice(BLOCK_POOL, buffer, upto);
buffer = BLOCK_POOL.buffer; buffer = BLOCK_POOL.buffer;
} }
buffer[upto++] = randomByte; buffer[upto++] = randomByte;

View File

@ -323,4 +323,38 @@ public class TestTermsHashPerField extends LuceneTestCase {
} }
} }
} }
public void testAllocKnowSizeSlice() {
Counter bytesUsed = Counter.newCounter();
ByteBlockPool pool = new ByteBlockPool(new ByteBlockPool.DirectTrackingAllocator(bytesUsed));
pool.nextBuffer();
for (int i = 0; i < 100; i++) {
int size;
if (random().nextBoolean()) {
size = TestUtil.nextInt(random(), 100, 1000);
} else {
size = TestUtil.nextInt(random(), 50000, 100000);
}
byte[] randomData = new byte[size];
random().nextBytes(randomData);
int upto = TermsHashPerField.newSlice(pool, TermsHashPerField.FIRST_LEVEL_SIZE, 0);
for (int offset = 0; offset < size; ) {
if ((pool.buffer[upto] & 16) == 0) {
pool.buffer[upto++] = randomData[offset++];
} else {
int offsetAndLength = TermsHashPerField.allocKnownSizeSlice(pool, pool.buffer, upto);
int sliceLength = offsetAndLength & 0xff;
upto = offsetAndLength >> 8;
assertNotEquals(0, pool.buffer[upto + sliceLength - 1]);
assertEquals(0, pool.buffer[upto]);
int writeLength = Math.min(sliceLength - 1, size - offset);
System.arraycopy(randomData, offset, pool.buffer, upto, writeLength);
offset += writeLength;
upto += writeLength;
}
}
}
}
} }

View File

@ -44,6 +44,7 @@ public class TestByteBlockPool extends LuceneTestCase {
} }
// verify // verify
long position = 0; long position = 0;
BytesRefBuilder builder = new BytesRefBuilder();
for (BytesRef expected : list) { for (BytesRef expected : list) {
ref.grow(expected.length); ref.grow(expected.length);
ref.setLength(expected.length); ref.setLength(expected.length);
@ -54,8 +55,7 @@ public class TestByteBlockPool extends LuceneTestCase {
break; break;
case 1: case 1:
BytesRef scratch = new BytesRef(); BytesRef scratch = new BytesRef();
scratch.length = ref.length(); pool.setBytesRef(builder, scratch, position, ref.length());
pool.setRawBytesRef(scratch, position);
System.arraycopy(scratch.bytes, scratch.offset, ref.bytes(), 0, ref.length()); System.arraycopy(scratch.bytes, scratch.offset, ref.bytes(), 0, ref.length());
break; break;
default: default:
@ -102,40 +102,6 @@ public class TestByteBlockPool extends LuceneTestCase {
} }
} }
public void testAllocKnowSizeSlice() throws IOException {
Counter bytesUsed = Counter.newCounter();
ByteBlockPool pool = new ByteBlockPool(new ByteBlockPool.DirectTrackingAllocator(bytesUsed));
pool.nextBuffer();
for (int i = 0; i < 100; i++) {
int size;
if (random().nextBoolean()) {
size = TestUtil.nextInt(random(), 100, 1000);
} else {
size = TestUtil.nextInt(random(), 50000, 100000);
}
byte[] randomData = new byte[size];
random().nextBytes(randomData);
int upto = pool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE);
for (int offset = 0; offset < size; ) {
if ((pool.buffer[upto] & 16) == 0) {
pool.buffer[upto++] = randomData[offset++];
} else {
int offsetAndLength = pool.allocKnownSizeSlice(pool.buffer, upto);
int sliceLength = offsetAndLength & 0xff;
upto = offsetAndLength >> 8;
assertNotEquals(0, pool.buffer[upto + sliceLength - 1]);
assertEquals(0, pool.buffer[upto]);
int writeLength = Math.min(sliceLength - 1, size - offset);
System.arraycopy(randomData, offset, pool.buffer, upto, writeLength);
offset += writeLength;
upto += writeLength;
}
}
}
}
public void testTooManyAllocs() { public void testTooManyAllocs() {
// Use a mock allocator that doesn't waste memory // Use a mock allocator that doesn't waste memory
ByteBlockPool pool = ByteBlockPool pool =

View File

@ -50,8 +50,7 @@ public class TestBytesRefHash extends LuceneTestCase {
private ByteBlockPool newPool() { private ByteBlockPool newPool() {
return random().nextBoolean() && pool != null return random().nextBoolean() && pool != null
? pool ? pool
: new ByteBlockPool( : new ByteBlockPool(new RecyclingByteBlockAllocator(random().nextInt(25)));
new RecyclingByteBlockAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, random().nextInt(25)));
} }
private BytesRefHash newHash(ByteBlockPool blockPool) { private BytesRefHash newHash(ByteBlockPool blockPool) {

View File

@ -34,8 +34,7 @@ public class TestRecyclingByteBlockAllocator extends LuceneTestCase {
} }
private RecyclingByteBlockAllocator newAllocator() { private RecyclingByteBlockAllocator newAllocator() {
return new RecyclingByteBlockAllocator( return new RecyclingByteBlockAllocator(random().nextInt(97), Counter.newCounter());
1 << (2 + random().nextInt(15)), random().nextInt(97), Counter.newCounter());
} }
@Test @Test

View File

@ -446,9 +446,7 @@ public class MemoryIndex {
+ (maxBufferedIntBlocks * SlicedIntBlockPool.INT_BLOCK_SIZE * Integer.BYTES) + (maxBufferedIntBlocks * SlicedIntBlockPool.INT_BLOCK_SIZE * Integer.BYTES)
<= maxReusedBytes; <= maxReusedBytes;
byteBlockPool = byteBlockPool =
new ByteBlockPool( new ByteBlockPool(new RecyclingByteBlockAllocator(maxBufferedByteBlocks, bytesUsed));
new RecyclingByteBlockAllocator(
ByteBlockPool.BYTE_BLOCK_SIZE, maxBufferedByteBlocks, bytesUsed));
slicedIntBlockPool = slicedIntBlockPool =
new SlicedIntBlockPool( new SlicedIntBlockPool(
new RecyclingIntBlockAllocator( new RecyclingIntBlockAllocator(