Reuse pages more agressively in BigArrays.
Pages are now going to be reused when the size of the array is greater than ${page_size}/2 (as opposed to ${page_size} currently). Close #5299
This commit is contained in:
parent
5723603987
commit
2c5d77cde1
|
@ -26,6 +26,9 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.recycler.Recycler;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
|
@ -77,18 +80,36 @@ public class BigArrays extends AbstractComponent {
|
|||
return index == (int) index;
|
||||
}
|
||||
|
||||
private static class ByteArrayWrapper extends AbstractArray implements ByteArray {
|
||||
private static abstract class AbstractArrayWrapper extends AbstractArray {
|
||||
|
||||
private final byte[] array;
|
||||
private final Releasable releasable;
|
||||
private final long size;
|
||||
|
||||
ByteArrayWrapper(byte[] array, PageCacheRecycler recycler, boolean clearOnResize) {
|
||||
AbstractArrayWrapper(long size, Releasable releasable, boolean clearOnResize) {
|
||||
super(clearOnResize);
|
||||
this.array = array;
|
||||
this.releasable = releasable;
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
public final long size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return array.length;
|
||||
public final boolean release() {
|
||||
Releasables.release(releasable);
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class ByteArrayWrapper extends AbstractArrayWrapper implements ByteArray {
|
||||
|
||||
private final byte[] array;
|
||||
|
||||
ByteArrayWrapper(byte[] array, long size, Recycler.V<byte[]> releasable, boolean clearOnResize) {
|
||||
super(size, releasable, clearOnResize);
|
||||
this.array = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -121,20 +142,15 @@ public class BigArrays extends AbstractComponent {
|
|||
|
||||
}
|
||||
|
||||
private static class IntArrayWrapper extends AbstractArray implements IntArray {
|
||||
private static class IntArrayWrapper extends AbstractArrayWrapper implements IntArray {
|
||||
|
||||
private final int[] array;
|
||||
|
||||
IntArrayWrapper(int[] array, PageCacheRecycler recycler, boolean clearOnResize) {
|
||||
super(clearOnResize);
|
||||
IntArrayWrapper(int[] array, long size, Recycler.V<int[]> releasable, boolean clearOnResize) {
|
||||
super(size, releasable, clearOnResize);
|
||||
this.array = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return array.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int get(long index) {
|
||||
assert indexIsInt(index);
|
||||
|
@ -157,20 +173,15 @@ public class BigArrays extends AbstractComponent {
|
|||
|
||||
}
|
||||
|
||||
private static class LongArrayWrapper extends AbstractArray implements LongArray {
|
||||
private static class LongArrayWrapper extends AbstractArrayWrapper implements LongArray {
|
||||
|
||||
private final long[] array;
|
||||
|
||||
LongArrayWrapper(long[] array, PageCacheRecycler recycler, boolean clearOnResize) {
|
||||
super(clearOnResize);
|
||||
LongArrayWrapper(long[] array, long size, Recycler.V<long[]> releasable, boolean clearOnResize) {
|
||||
super(size, releasable, clearOnResize);
|
||||
this.array = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return array.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long get(long index) {
|
||||
assert indexIsInt(index);
|
||||
|
@ -199,20 +210,15 @@ public class BigArrays extends AbstractComponent {
|
|||
}
|
||||
}
|
||||
|
||||
private static class DoubleArrayWrapper extends AbstractArray implements DoubleArray {
|
||||
private static class DoubleArrayWrapper extends AbstractArrayWrapper implements DoubleArray {
|
||||
|
||||
private final double[] array;
|
||||
|
||||
DoubleArrayWrapper(double[] array, PageCacheRecycler recycler, boolean clearOnResize) {
|
||||
super(clearOnResize);
|
||||
DoubleArrayWrapper(double[] array, long size, Recycler.V<double[]> releasable, boolean clearOnResize) {
|
||||
super(size, releasable, clearOnResize);
|
||||
this.array = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return array.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double get(long index) {
|
||||
assert indexIsInt(index);
|
||||
|
@ -242,20 +248,15 @@ public class BigArrays extends AbstractComponent {
|
|||
|
||||
}
|
||||
|
||||
private static class FloatArrayWrapper extends AbstractArray implements FloatArray {
|
||||
private static class FloatArrayWrapper extends AbstractArrayWrapper implements FloatArray {
|
||||
|
||||
private final float[] array;
|
||||
|
||||
FloatArrayWrapper(float[] array, PageCacheRecycler recycler, boolean clearOnResize) {
|
||||
super(clearOnResize);
|
||||
FloatArrayWrapper(float[] array, long size, Recycler.V<float[]> releasable, boolean clearOnResize) {
|
||||
super(size, releasable, clearOnResize);
|
||||
this.array = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return array.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float get(long index) {
|
||||
assert indexIsInt(index);
|
||||
|
@ -285,20 +286,15 @@ public class BigArrays extends AbstractComponent {
|
|||
|
||||
}
|
||||
|
||||
private static class ObjectArrayWrapper<T> extends AbstractArray implements ObjectArray<T> {
|
||||
private static class ObjectArrayWrapper<T> extends AbstractArrayWrapper implements ObjectArray<T> {
|
||||
|
||||
private final Object[] array;
|
||||
|
||||
ObjectArrayWrapper(Object[] array, PageCacheRecycler recycler) {
|
||||
super(true);
|
||||
ObjectArrayWrapper(Object[] array, long size, Recycler.V<Object[]> releasable) {
|
||||
super(size, releasable, true);
|
||||
this.array = array;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return array.length;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public T get(long index) {
|
||||
|
@ -317,7 +313,7 @@ public class BigArrays extends AbstractComponent {
|
|||
|
||||
}
|
||||
|
||||
private final PageCacheRecycler recycler;
|
||||
final PageCacheRecycler recycler;
|
||||
|
||||
@Inject
|
||||
public BigArrays(Settings settings, PageCacheRecycler recycler) {
|
||||
|
@ -331,10 +327,13 @@ public class BigArrays extends AbstractComponent {
|
|||
* @param clearOnResize whether values should be set to 0 on initialization and resize
|
||||
*/
|
||||
public ByteArray newByteArray(long size, boolean clearOnResize) {
|
||||
if (size <= BYTE_PAGE_SIZE) {
|
||||
return new ByteArrayWrapper(new byte[(int) size], recycler, clearOnResize);
|
||||
} else {
|
||||
if (size > BYTE_PAGE_SIZE) {
|
||||
return new BigByteArray(size, recycler, clearOnResize);
|
||||
} else if (size >= BYTE_PAGE_SIZE / 2 && recycler != null) {
|
||||
final Recycler.V<byte[]> page = recycler.bytePage(clearOnResize);
|
||||
return new ByteArrayWrapper(page.v(), size, page, clearOnResize);
|
||||
} else {
|
||||
return new ByteArrayWrapper(new byte[(int) size], size, null, clearOnResize);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -376,10 +375,13 @@ public class BigArrays extends AbstractComponent {
|
|||
* @param clearOnResize whether values should be set to 0 on initialization and resize
|
||||
*/
|
||||
public IntArray newIntArray(long size, boolean clearOnResize) {
|
||||
if (size <= INT_PAGE_SIZE) {
|
||||
return new IntArrayWrapper(new int[(int) size], recycler, clearOnResize);
|
||||
} else {
|
||||
if (size > INT_PAGE_SIZE) {
|
||||
return new BigIntArray(size, recycler, clearOnResize);
|
||||
} else if (size >= INT_PAGE_SIZE / 2 && recycler != null) {
|
||||
final Recycler.V<int[]> page = recycler.intPage(clearOnResize);
|
||||
return new IntArrayWrapper(page.v(), size, page, clearOnResize);
|
||||
} else {
|
||||
return new IntArrayWrapper(new int[(int) size], size, null, clearOnResize);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -422,10 +424,13 @@ public class BigArrays extends AbstractComponent {
|
|||
* @param clearOnResize whether values should be set to 0 on initialization and resize
|
||||
*/
|
||||
public LongArray newLongArray(long size, boolean clearOnResize) {
|
||||
if (size <= LONG_PAGE_SIZE) {
|
||||
return new LongArrayWrapper(new long[(int) size], recycler, clearOnResize);
|
||||
} else {
|
||||
if (size > LONG_PAGE_SIZE) {
|
||||
return new BigLongArray(size, recycler, clearOnResize);
|
||||
} else if (size >= LONG_PAGE_SIZE / 2 && recycler != null) {
|
||||
final Recycler.V<long[]> page = recycler.longPage(clearOnResize);
|
||||
return new LongArrayWrapper(page.v(), size, page, clearOnResize);
|
||||
} else {
|
||||
return new LongArrayWrapper(new long[(int) size], size, null, clearOnResize);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -468,10 +473,13 @@ public class BigArrays extends AbstractComponent {
|
|||
* @param clearOnResize whether values should be set to 0 on initialization and resize
|
||||
*/
|
||||
public DoubleArray newDoubleArray(long size, boolean clearOnResize) {
|
||||
if (size <= LONG_PAGE_SIZE) {
|
||||
return new DoubleArrayWrapper(new double[(int) size], recycler, clearOnResize);
|
||||
} else {
|
||||
if (size > DOUBLE_PAGE_SIZE) {
|
||||
return new BigDoubleArray(size, recycler, clearOnResize);
|
||||
} else if (size >= DOUBLE_PAGE_SIZE / 2 && recycler != null) {
|
||||
final Recycler.V<double[]> page = recycler.doublePage(clearOnResize);
|
||||
return new DoubleArrayWrapper(page.v(), size, page, clearOnResize);
|
||||
} else {
|
||||
return new DoubleArrayWrapper(new double[(int) size], size, null, clearOnResize);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -511,10 +519,13 @@ public class BigArrays extends AbstractComponent {
|
|||
* @param clearOnResize whether values should be set to 0 on initialization and resize
|
||||
*/
|
||||
public FloatArray newFloatArray(long size, boolean clearOnResize) {
|
||||
if (size <= FLOAT_PAGE_SIZE) {
|
||||
return new FloatArrayWrapper(new float[(int) size], recycler, clearOnResize);
|
||||
} else {
|
||||
if (size > FLOAT_PAGE_SIZE) {
|
||||
return new BigFloatArray(size, recycler, clearOnResize);
|
||||
} else if (size >= FLOAT_PAGE_SIZE / 2 && recycler != null) {
|
||||
final Recycler.V<float[]> page = recycler.floatPage(clearOnResize);
|
||||
return new FloatArrayWrapper(page.v(), size, page, clearOnResize);
|
||||
} else {
|
||||
return new FloatArrayWrapper(new float[(int) size], size, null, clearOnResize);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -552,10 +563,13 @@ public class BigArrays extends AbstractComponent {
|
|||
* @param size the initial length of the array
|
||||
*/
|
||||
public <T> ObjectArray<T> newObjectArray(long size) {
|
||||
if (size <= OBJECT_PAGE_SIZE) {
|
||||
return new ObjectArrayWrapper<T>(new Object[(int) size], recycler);
|
||||
} else {
|
||||
if (size > OBJECT_PAGE_SIZE) {
|
||||
return new BigObjectArray<T>(size, recycler);
|
||||
} else if (size >= OBJECT_PAGE_SIZE / 2 && recycler != null) {
|
||||
final Recycler.V<Object[]> page = recycler.objectPage();
|
||||
return new ObjectArrayWrapper<T>(page.v(), size, page);
|
||||
} else {
|
||||
return new ObjectArrayWrapper<T>(new Object[(int) size], size, null);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue