Limit the number of bytes that can be allocated to process requests.

This should prevent costly requests from killing the whole cluster.

Close #6050
This commit is contained in:
Adrien Grand 2014-05-05 13:48:20 +02:00
parent 9ed34b5a9e
commit c4f127fb6f
20 changed files with 339 additions and 321 deletions

View File

@ -19,21 +19,26 @@
package org.elasticsearch.common.util;
import org.elasticsearch.common.lease.Releasable;
abstract class AbstractArray implements Releasable {
abstract class AbstractArray implements BigArray {
private final BigArrays bigArrays;
public final boolean clearOnResize;
private boolean released = false;
AbstractArray(boolean clearOnResize) {
AbstractArray(BigArrays bigArrays, boolean clearOnResize) {
this.bigArrays = bigArrays;
this.clearOnResize = clearOnResize;
}
@Override
public void close() {
public final void close() {
bigArrays.ramBytesUsed.addAndGet(-sizeInBytes());
assert !released : "double release";
released = true;
doClose();
}
protected abstract void doClose();
}

View File

@ -32,6 +32,8 @@ import java.util.Arrays;
/** Common implementation for array lists that slice data into fixed-size blocks. */
abstract class AbstractBigArray extends AbstractArray {
private static final long EMPTY_SIZE = RamUsageEstimator.shallowSizeOfInstance(AbstractBigArray.class) + RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
private final PageCacheRecycler recycler;
private Recycler.V<?>[] cache;
@ -39,9 +41,9 @@ abstract class AbstractBigArray extends AbstractArray {
private final int pageMask;
protected long size;
protected AbstractBigArray(int pageSize, PageCacheRecycler recycler, boolean clearOnResize) {
super(clearOnResize);
this.recycler = recycler;
protected AbstractBigArray(int pageSize, BigArrays bigArrays, boolean clearOnResize) {
super(bigArrays, clearOnResize);
this.recycler = bigArrays.recycler;
Preconditions.checkArgument(pageSize >= 128, "pageSize must be >= 128");
Preconditions.checkArgument((pageSize & (pageSize - 1)) == 0, "pageSize must be a power of two");
this.pageShift = Integer.numberOfTrailingZeros(pageSize);
@ -76,6 +78,8 @@ abstract class AbstractBigArray extends AbstractArray {
return size;
}
public abstract void resize(long newSize);
protected abstract int numBytesPerElement();
public final long sizeInBytes() {
@ -161,8 +165,7 @@ abstract class AbstractBigArray extends AbstractArray {
}
@Override
public final void close() {
super.close();
protected final void doClose() {
if (recycler != null) {
Releasables.close(cache);
cache = null;

View File

@ -27,4 +27,9 @@ public interface BigArray extends Releasable {
/** Return the length of this array. */
public long size();
/**
* Return an estimated memory usage of this instance.
*/
public long sizeInBytes();
}

View File

@ -23,6 +23,7 @@ import com.google.common.base.Preconditions;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
@ -33,11 +34,14 @@ import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import java.util.Arrays;
import java.util.concurrent.atomic.AtomicLong;
/** Utility class to work with arrays. */
public class BigArrays extends AbstractComponent {
public static final BigArrays NON_RECYCLING_INSTANCE = new BigArrays(ImmutableSettings.EMPTY, null);
// TODO: switch to a circuit breaker that is shared not only on big arrays level, and applies to other request level data structures
public static final String MAX_SIZE_IN_BYTES_SETTING = "requests.memory.breaker.limit";
public static final BigArrays NON_RECYCLING_INSTANCE = new BigArrays(ImmutableSettings.EMPTY, null, Long.MAX_VALUE);
/** Page size in bytes: 16KB */
public static final int PAGE_SIZE_IN_BYTES = 1 << 14;
@ -80,13 +84,15 @@ public class BigArrays extends AbstractComponent {
return index == (int) index;
}
private static abstract class AbstractArrayWrapper extends AbstractArray {
private static abstract class AbstractArrayWrapper extends AbstractArray implements BigArray {
protected static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(ByteArrayWrapper.class);
private final Releasable releasable;
private final long size;
AbstractArrayWrapper(long size, Releasable releasable, boolean clearOnResize) {
super(clearOnResize);
AbstractArrayWrapper(BigArrays bigArrays, long size, Releasable releasable, boolean clearOnResize) {
super(bigArrays, clearOnResize);
this.releasable = releasable;
this.size = size;
}
@ -96,7 +102,7 @@ public class BigArrays extends AbstractComponent {
}
@Override
public final void close() {
protected final void doClose() {
Releasables.close(releasable);
}
@ -106,11 +112,16 @@ public class BigArrays extends AbstractComponent {
private final byte[] array;
ByteArrayWrapper(byte[] array, long size, Recycler.V<byte[]> releasable, boolean clearOnResize) {
super(size, releasable, clearOnResize);
ByteArrayWrapper(BigArrays bigArrays, byte[] array, long size, Recycler.V<byte[]> releasable, boolean clearOnResize) {
super(bigArrays, size, releasable, clearOnResize);
this.array = array;
}
@Override
public long sizeInBytes() {
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(array);
}
@Override
public byte get(long index) {
assert indexIsInt(index);
@ -152,11 +163,16 @@ public class BigArrays extends AbstractComponent {
private final int[] array;
IntArrayWrapper(int[] array, long size, Recycler.V<int[]> releasable, boolean clearOnResize) {
super(size, releasable, clearOnResize);
IntArrayWrapper(BigArrays bigArrays, int[] array, long size, Recycler.V<int[]> releasable, boolean clearOnResize) {
super(bigArrays, size, releasable, clearOnResize);
this.array = array;
}
@Override
public long sizeInBytes() {
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(array);
}
@Override
public int get(long index) {
assert indexIsInt(index);
@ -190,11 +206,16 @@ public class BigArrays extends AbstractComponent {
private final long[] array;
LongArrayWrapper(long[] array, long size, Recycler.V<long[]> releasable, boolean clearOnResize) {
super(size, releasable, clearOnResize);
LongArrayWrapper(BigArrays bigArrays, long[] array, long size, Recycler.V<long[]> releasable, boolean clearOnResize) {
super(bigArrays, size, releasable, clearOnResize);
this.array = array;
}
@Override
public long sizeInBytes() {
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(array);
}
@Override
public long get(long index) {
assert indexIsInt(index);
@ -227,11 +248,16 @@ public class BigArrays extends AbstractComponent {
private final double[] array;
DoubleArrayWrapper(double[] array, long size, Recycler.V<double[]> releasable, boolean clearOnResize) {
super(size, releasable, clearOnResize);
DoubleArrayWrapper(BigArrays bigArrays, double[] array, long size, Recycler.V<double[]> releasable, boolean clearOnResize) {
super(bigArrays, size, releasable, clearOnResize);
this.array = array;
}
@Override
public long sizeInBytes() {
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(array);
}
@Override
public double get(long index) {
assert indexIsInt(index);
@ -265,11 +291,16 @@ public class BigArrays extends AbstractComponent {
private final float[] array;
FloatArrayWrapper(float[] array, long size, Recycler.V<float[]> releasable, boolean clearOnResize) {
super(size, releasable, clearOnResize);
FloatArrayWrapper(BigArrays bigArrays, float[] array, long size, Recycler.V<float[]> releasable, boolean clearOnResize) {
super(bigArrays, size, releasable, clearOnResize);
this.array = array;
}
@Override
public long sizeInBytes() {
return SHALLOW_SIZE + RamUsageEstimator.sizeOf(array);
}
@Override
public float get(long index) {
assert indexIsInt(index);
@ -303,11 +334,16 @@ public class BigArrays extends AbstractComponent {
private final Object[] array;
ObjectArrayWrapper(Object[] array, long size, Recycler.V<Object[]> releasable) {
super(size, releasable, true);
ObjectArrayWrapper(BigArrays bigArrays, Object[] array, long size, Recycler.V<Object[]> releasable) {
super(bigArrays, size, releasable, true);
this.array = array;
}
@Override
public long sizeInBytes() {
return SHALLOW_SIZE + RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + RamUsageEstimator.NUM_BYTES_OBJECT_REF * size());
}
@SuppressWarnings("unchecked")
@Override
public T get(long index) {
@ -327,11 +363,46 @@ public class BigArrays extends AbstractComponent {
}
final PageCacheRecycler recycler;
final AtomicLong ramBytesUsed;
final long maxSizeInBytes;
@Inject
public BigArrays(Settings settings, PageCacheRecycler recycler) {
this(settings, recycler, settings.getAsMemory(MAX_SIZE_IN_BYTES_SETTING, "20%").bytes());
}
private BigArrays(Settings settings, PageCacheRecycler recycler, final long maxSizeInBytes) {
super(settings);
this.maxSizeInBytes = maxSizeInBytes;
this.recycler = recycler;
ramBytesUsed = new AtomicLong();
}
private void validate(long delta) {
final long totalSizeInBytes = ramBytesUsed.addAndGet(delta);
if (totalSizeInBytes > maxSizeInBytes) {
throw new ElasticsearchIllegalStateException("Maximum number of bytes allocated exceeded: [" + totalSizeInBytes + "] (> " + maxSizeInBytes + ")");
}
}
private <T extends AbstractBigArray> T resizeInPlace(T array, long newSize) {
final long oldMemSize = array.sizeInBytes();
array.resize(newSize);
validate(array.sizeInBytes() - oldMemSize);
return array;
}
private <T extends BigArray> T validate(T array) {
boolean success = false;
try {
validate(array.sizeInBytes());
success = true;
} finally {
if (!success) {
Releasables.closeWhileHandlingException(array);
}
}
return array;
}
/**
@ -340,14 +411,16 @@ 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) {
final ByteArray array;
if (size > BYTE_PAGE_SIZE) {
return new BigByteArray(size, recycler, clearOnResize);
array = new BigByteArray(size, this, 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);
array = new ByteArrayWrapper(this, page.v(), size, page, clearOnResize);
} else {
return new ByteArrayWrapper(new byte[(int) size], size, null, clearOnResize);
array = new ByteArrayWrapper(this, new byte[(int) size], size, null, clearOnResize);
}
return validate(array);
}
/**
@ -361,14 +434,13 @@ public class BigArrays extends AbstractComponent {
/** Resize the array to the exact provided size. */
public ByteArray resize(ByteArray array, long size) {
if (array instanceof BigByteArray) {
((BigByteArray) array).resize(size);
return array;
return resizeInPlace((BigByteArray) array, size);
} else {
AbstractArray arr = (AbstractArray) array;
final ByteArray newArray = newByteArray(size, arr.clearOnResize);
final byte[] rawArray = ((ByteArrayWrapper) array).array;
newArray.set(0, rawArray, 0, (int) Math.min(rawArray.length, newArray.size()));
array.close();
arr.close();
return newArray;
}
}
@ -421,14 +493,16 @@ 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) {
final IntArray array;
if (size > INT_PAGE_SIZE) {
return new BigIntArray(size, recycler, clearOnResize);
array = new BigIntArray(size, this, 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);
array = new IntArrayWrapper(this, page.v(), size, page, clearOnResize);
} else {
return new IntArrayWrapper(new int[(int) size], size, null, clearOnResize);
array = new IntArrayWrapper(this, new int[(int) size], size, null, clearOnResize);
}
return validate(array);
}
/**
@ -442,8 +516,7 @@ public class BigArrays extends AbstractComponent {
/** Resize the array to the exact provided size. */
public IntArray resize(IntArray array, long size) {
if (array instanceof BigIntArray) {
((BigIntArray) array).resize(size);
return array;
return resizeInPlace((BigIntArray) array, size);
} else {
AbstractArray arr = (AbstractArray) array;
final IntArray newArray = newIntArray(size, arr.clearOnResize);
@ -470,14 +543,16 @@ 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) {
final LongArray array;
if (size > LONG_PAGE_SIZE) {
return new BigLongArray(size, recycler, clearOnResize);
array = new BigLongArray(size, this, 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);
array = new LongArrayWrapper(this, page.v(), size, page, clearOnResize);
} else {
return new LongArrayWrapper(new long[(int) size], size, null, clearOnResize);
array = new LongArrayWrapper(this, new long[(int) size], size, null, clearOnResize);
}
return validate(array);
}
/**
@ -491,8 +566,7 @@ public class BigArrays extends AbstractComponent {
/** Resize the array to the exact provided size. */
public LongArray resize(LongArray array, long size) {
if (array instanceof BigLongArray) {
((BigLongArray) array).resize(size);
return array;
return resizeInPlace((BigLongArray) array, size);
} else {
AbstractArray arr = (AbstractArray) array;
final LongArray newArray = newLongArray(size, arr.clearOnResize);
@ -519,14 +593,16 @@ 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) {
final DoubleArray arr;
if (size > DOUBLE_PAGE_SIZE) {
return new BigDoubleArray(size, recycler, clearOnResize);
arr = new BigDoubleArray(size, this, 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);
arr = new DoubleArrayWrapper(this, page.v(), size, page, clearOnResize);
} else {
return new DoubleArrayWrapper(new double[(int) size], size, null, clearOnResize);
arr = new DoubleArrayWrapper(this, new double[(int) size], size, null, clearOnResize);
}
return validate(arr);
}
/** Allocate a new {@link DoubleArray} of the given capacity. */
@ -537,8 +613,7 @@ public class BigArrays extends AbstractComponent {
/** Resize the array to the exact provided size. */
public DoubleArray resize(DoubleArray array, long size) {
if (array instanceof BigDoubleArray) {
((BigDoubleArray) array).resize(size);
return array;
return resizeInPlace((BigDoubleArray) array, size);
} else {
AbstractArray arr = (AbstractArray) array;
final DoubleArray newArray = newDoubleArray(size, arr.clearOnResize);
@ -565,14 +640,16 @@ 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) {
final FloatArray array;
if (size > FLOAT_PAGE_SIZE) {
return new BigFloatArray(size, recycler, clearOnResize);
array = new BigFloatArray(size, this, 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);
array = new FloatArrayWrapper(this, page.v(), size, page, clearOnResize);
} else {
return new FloatArrayWrapper(new float[(int) size], size, null, clearOnResize);
array = new FloatArrayWrapper(this, new float[(int) size], size, null, clearOnResize);
}
return validate(array);
}
/** Allocate a new {@link FloatArray} of the given capacity. */
@ -583,14 +660,14 @@ public class BigArrays extends AbstractComponent {
/** Resize the array to the exact provided size. */
public FloatArray resize(FloatArray array, long size) {
if (array instanceof BigFloatArray) {
((BigFloatArray) array).resize(size);
return array;
return resizeInPlace((BigFloatArray) array, size);
} else {
AbstractArray arr = (AbstractArray) array;
final FloatArray newArray = newFloatArray(size, arr.clearOnResize);
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
newArray.set(i, array.get(i));
}
arr.close();
return newArray;
}
}
@ -609,21 +686,22 @@ public class BigArrays extends AbstractComponent {
* @param size the initial length of the array
*/
public <T> ObjectArray<T> newObjectArray(long size) {
final ObjectArray<T> array;
if (size > OBJECT_PAGE_SIZE) {
return new BigObjectArray<>(size, recycler);
array = new BigObjectArray<>(size, this);
} else if (size >= OBJECT_PAGE_SIZE / 2 && recycler != null) {
final Recycler.V<Object[]> page = recycler.objectPage();
return new ObjectArrayWrapper<>(page.v(), size, page);
array = new ObjectArrayWrapper<>(this, page.v(), size, page);
} else {
return new ObjectArrayWrapper<>(new Object[(int) size], size, null);
array = new ObjectArrayWrapper<>(this, new Object[(int) size], size, null);
}
return validate(array);
}
/** Resize the array to the exact provided size. */
public <T> ObjectArray<T> resize(ObjectArray<T> array, long size) {
if (array instanceof BigObjectArray) {
((BigObjectArray<?>) array).resize(size);
return array;
return resizeInPlace((BigObjectArray<T>) array, size);
} else {
final ObjectArray<T> newArray = newObjectArray(size);
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
@ -643,4 +721,10 @@ public class BigArrays extends AbstractComponent {
return resize(array, newSize);
}
/**
* Return an approximate number of bytes that have been allocated but not released yet.
*/
public long sizeInBytes() {
return ramBytesUsed.get();
}
}

View File

@ -23,7 +23,6 @@ import com.google.common.base.Preconditions;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import java.util.Arrays;
@ -38,15 +37,15 @@ final class BigByteArray extends AbstractBigArray implements ByteArray {
private byte[][] pages;
/** Constructor. */
public BigByteArray(long size, PageCacheRecycler recycler, boolean clearOnResize) {
super(BYTE_PAGE_SIZE, recycler, clearOnResize);
public BigByteArray(long size, BigArrays bigArrays, boolean clearOnResize) {
super(BYTE_PAGE_SIZE, bigArrays, clearOnResize);
this.size = size;
pages = new byte[numPages(size)][];
for (int i = 0; i < pages.length; ++i) {
pages[i] = newBytePage(i);
}
}
@Override
public byte get(long index) {
final int pageIndex = pageIndex(index);

View File

@ -22,7 +22,6 @@ package org.elasticsearch.common.util;
import com.google.common.base.Preconditions;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import java.util.Arrays;
@ -37,8 +36,8 @@ final class BigDoubleArray extends AbstractBigArray implements DoubleArray {
private double[][] pages;
/** Constructor. */
public BigDoubleArray(long size, PageCacheRecycler recycler, boolean clearOnResize) {
super(DOUBLE_PAGE_SIZE, recycler, clearOnResize);
public BigDoubleArray(long size, BigArrays bigArrays, boolean clearOnResize) {
super(DOUBLE_PAGE_SIZE, bigArrays, clearOnResize);
this.size = size;
pages = new double[numPages(size)][];
for (int i = 0; i < pages.length; ++i) {

View File

@ -1,79 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.common.util;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import java.util.Arrays;
/**
* Float array abstraction able to support more than 2B values. This implementation slices data into fixed-sized blocks of
* configurable length.
*/
public final class BigDoubleArrayList extends AbstractBigArray {
/**
* Default page size, 16KB of memory per page.
*/
private static final int DEFAULT_PAGE_SIZE = 1 << 11;
private double[][] pages;
public BigDoubleArrayList(int pageSize, long initialCapacity) {
super(pageSize, null, true);
pages = new double[numPages(initialCapacity)][];
}
public BigDoubleArrayList(long initialCapacity) {
this(DEFAULT_PAGE_SIZE, initialCapacity);
}
public BigDoubleArrayList() {
this(1024);
}
public double get(long index) {
assert index >= 0 && index < size;
final int pageIndex = pageIndex(index);
final int indexInPage = indexInPage(index);
return pages[pageIndex][indexInPage];
}
public void add(double d) {
final int pageIndex = pageIndex(size);
if (pageIndex >= pages.length) {
final int newLength = ArrayUtil.oversize(pageIndex + 1, numBytesPerElement());
pages = Arrays.copyOf(pages, newLength);
}
if (pages[pageIndex] == null) {
pages[pageIndex] = new double[pageSize()];
}
final int indexInPage = indexInPage(size);
pages[pageIndex][indexInPage] = d;
++size;
}
@Override
protected int numBytesPerElement() {
return RamUsageEstimator.NUM_BYTES_DOUBLE;
}
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.common.util;
import com.google.common.base.Preconditions;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import java.util.Arrays;
@ -37,8 +36,8 @@ final class BigFloatArray extends AbstractBigArray implements FloatArray {
private float[][] pages;
/** Constructor. */
public BigFloatArray(long size, PageCacheRecycler recycler, boolean clearOnResize) {
super(FLOAT_PAGE_SIZE, recycler, clearOnResize);
public BigFloatArray(long size, BigArrays bigArrays, boolean clearOnResize) {
super(FLOAT_PAGE_SIZE, bigArrays, clearOnResize);
this.size = size;
pages = new float[numPages(size)][];
for (int i = 0; i < pages.length; ++i) {

View File

@ -1,74 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.common.util;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Float array abstraction able to support more than 2B values. This implementation slices data into fixed-sized blocks of
* configurable length.
*/
public final class BigFloatArrayList extends AbstractBigArray {
/**
* Default page size, 16KB of memory per page.
*/
private static final int DEFAULT_PAGE_SIZE = 1 << 12;
private float[][] pages;
public BigFloatArrayList(int pageSize, long initialCapacity) {
super(pageSize, null, true);
pages = new float[numPages(initialCapacity)][];
}
public BigFloatArrayList(long initialCapacity) {
this(DEFAULT_PAGE_SIZE, initialCapacity);
}
public BigFloatArrayList() {
this(1024);
}
public float get(long index) {
assert index >= 0 && index < size;
final int pageIndex = pageIndex(index);
final int indexInPage = indexInPage(index);
return pages[pageIndex][indexInPage];
}
public void add(float f) {
final int pageIndex = pageIndex(size);
pages = ArrayUtil.grow(pages, pageIndex + 1);
if (pages[pageIndex] == null) {
pages[pageIndex] = new float[pageSize()];
}
final int indexInPage = indexInPage(size);
pages[pageIndex][indexInPage] = f;
++size;
}
@Override
protected int numBytesPerElement() {
return RamUsageEstimator.NUM_BYTES_FLOAT;
}
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.common.util;
import com.google.common.base.Preconditions;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import java.util.Arrays;
@ -37,8 +36,8 @@ final class BigIntArray extends AbstractBigArray implements IntArray {
private int[][] pages;
/** Constructor. */
public BigIntArray(long size, PageCacheRecycler recycler, boolean clearOnResize) {
super(INT_PAGE_SIZE, recycler, clearOnResize);
public BigIntArray(long size, BigArrays bigArrays, boolean clearOnResize) {
super(INT_PAGE_SIZE, bigArrays, clearOnResize);
this.size = size;
pages = new int[numPages(size)][];
for (int i = 0; i < pages.length; ++i) {

View File

@ -22,7 +22,6 @@ package org.elasticsearch.common.util;
import com.google.common.base.Preconditions;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import java.util.Arrays;
@ -37,8 +36,8 @@ final class BigLongArray extends AbstractBigArray implements LongArray {
private long[][] pages;
/** Constructor. */
public BigLongArray(long size, PageCacheRecycler recycler, boolean clearOnResize) {
super(LONG_PAGE_SIZE, recycler, clearOnResize);
public BigLongArray(long size, BigArrays bigArrays, boolean clearOnResize) {
super(LONG_PAGE_SIZE, bigArrays, clearOnResize);
this.size = size;
pages = new long[numPages(size)][];
for (int i = 0; i < pages.length; ++i) {

View File

@ -21,7 +21,6 @@ package org.elasticsearch.common.util;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import java.util.Arrays;
@ -36,8 +35,8 @@ final class BigObjectArray<T> extends AbstractBigArray implements ObjectArray<T>
private Object[][] pages;
/** Constructor. */
public BigObjectArray(long size, PageCacheRecycler recycler) {
super(OBJECT_PAGE_SIZE, recycler, true);
public BigObjectArray(long size, BigArrays bigArrays) {
super(OBJECT_PAGE_SIZE, bigArrays, true);
this.size = size;
pages = new Object[numPages(size)][];
for (int i = 0; i < pages.length; ++i) {

View File

@ -21,7 +21,7 @@ package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.common.util.BigDoubleArrayList;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
@ -87,10 +87,10 @@ public abstract class DoubleArrayAtomicFieldData extends AbstractAtomicNumericFi
public static class WithOrdinals extends DoubleArrayAtomicFieldData {
private final BigDoubleArrayList values;
private final DoubleArray values;
private final Ordinals ordinals;
public WithOrdinals(BigDoubleArrayList values, Ordinals ordinals) {
public WithOrdinals(DoubleArray values, Ordinals ordinals) {
super();
this.values = values;
this.ordinals = ordinals;
@ -128,9 +128,9 @@ public abstract class DoubleArrayAtomicFieldData extends AbstractAtomicNumericFi
static class LongValues extends org.elasticsearch.index.fielddata.LongValues.WithOrdinals {
private final BigDoubleArrayList values;
private final DoubleArray values;
LongValues(BigDoubleArrayList values, Ordinals.Docs ordinals) {
LongValues(DoubleArray values, Ordinals.Docs ordinals) {
super(ordinals);
this.values = values;
}
@ -144,9 +144,9 @@ public abstract class DoubleArrayAtomicFieldData extends AbstractAtomicNumericFi
static class DoubleValues extends org.elasticsearch.index.fielddata.DoubleValues.WithOrdinals {
private final BigDoubleArrayList values;
private final DoubleArray values;
DoubleValues(BigDoubleArrayList values, Ordinals.Docs ordinals) {
DoubleValues(DoubleArray values, Ordinals.Docs ordinals) {
super(ordinals);
this.values = values;
}
@ -165,11 +165,11 @@ public abstract class DoubleArrayAtomicFieldData extends AbstractAtomicNumericFi
*/
public static class SingleFixedSet extends DoubleArrayAtomicFieldData {
private final BigDoubleArrayList values;
private final DoubleArray values;
private final FixedBitSet set;
private final long numOrds;
public SingleFixedSet(BigDoubleArrayList values, FixedBitSet set, long numOrds) {
public SingleFixedSet(DoubleArray values, FixedBitSet set, long numOrds) {
super();
this.values = values;
this.set = set;
@ -206,10 +206,10 @@ public abstract class DoubleArrayAtomicFieldData extends AbstractAtomicNumericFi
static class LongValues extends org.elasticsearch.index.fielddata.LongValues {
private final BigDoubleArrayList values;
private final DoubleArray values;
private final FixedBitSet set;
LongValues(BigDoubleArrayList values, FixedBitSet set) {
LongValues(DoubleArray values, FixedBitSet set) {
super(false);
this.values = values;
this.set = set;
@ -229,10 +229,10 @@ public abstract class DoubleArrayAtomicFieldData extends AbstractAtomicNumericFi
static class DoubleValues extends org.elasticsearch.index.fielddata.DoubleValues {
private final BigDoubleArrayList values;
private final DoubleArray values;
private final FixedBitSet set;
DoubleValues(BigDoubleArrayList values, FixedBitSet set) {
DoubleValues(DoubleArray values, FixedBitSet set) {
super(false);
this.values = values;
this.set = set;
@ -256,14 +256,14 @@ public abstract class DoubleArrayAtomicFieldData extends AbstractAtomicNumericFi
*/
public static class Single extends DoubleArrayAtomicFieldData {
private final BigDoubleArrayList values;
private final DoubleArray values;
private final long numOrds;
/**
* Note, here, we assume that there is no offset by 1 from docId, so position 0
* is the value for docId 0.
*/
public Single(BigDoubleArrayList values, long numOrds) {
public Single(DoubleArray values, long numOrds) {
super();
this.values = values;
this.numOrds = numOrds;
@ -299,9 +299,9 @@ public abstract class DoubleArrayAtomicFieldData extends AbstractAtomicNumericFi
static final class LongValues extends DenseLongValues {
private final BigDoubleArrayList values;
private final DoubleArray values;
LongValues(BigDoubleArrayList values) {
LongValues(DoubleArray values) {
super(false);
this.values = values;
}
@ -314,9 +314,9 @@ public abstract class DoubleArrayAtomicFieldData extends AbstractAtomicNumericFi
static final class DoubleValues extends DenseDoubleValues {
private final BigDoubleArrayList values;
private final DoubleArray values;
DoubleValues(BigDoubleArrayList values) {
DoubleValues(DoubleArray values) {
super(false);
this.values = values;
}

View File

@ -25,11 +25,11 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.util.*;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigDoubleArrayList;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.fieldcomparator.DoubleValuesComparatorSource;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsBuilder;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.Ordinals.Docs;
@ -38,6 +38,7 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.indices.fielddata.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
/**
*/
@ -86,16 +87,19 @@ public class DoubleArrayIndexFieldData extends AbstractIndexFieldData<DoubleArra
return data;
}
// TODO: how can we guess the number of terms? numerics end up creating more terms per value...
final BigDoubleArrayList values = new BigDoubleArrayList();
DoubleArray values = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(128);
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat("acceptable_transient_overhead_ratio", OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO);
boolean success = false;
try (OrdinalsBuilder builder = new OrdinalsBuilder(reader.maxDoc(), acceptableTransientOverheadRatio)) {
final BytesRefIterator iter = builder.buildFromTerms(getNumericType().wrapTermsEnum(terms.iterator(null)));
BytesRef term;
long numTerms = 0;
while ((term = iter.next()) != null) {
values.add(NumericUtils.sortableLongToDouble(NumericUtils.prefixCodedToLong(term)));
values = BigArrays.NON_RECYCLING_INSTANCE.grow(values, numTerms + 1);
values.set(numTerms++, NumericUtils.sortableLongToDouble(NumericUtils.prefixCodedToLong(term)));
}
values = BigArrays.NON_RECYCLING_INSTANCE.resize(values, numTerms);
Ordinals build = builder.build(fieldDataType.getSettings());
if (build.isMultiValued() || CommonSettings.getMemoryStorageHint(fieldDataType) == CommonSettings.MemoryStorageFormat.ORDINALS) {
data = new DoubleArrayAtomicFieldData.WithOrdinals(values, build);
@ -114,13 +118,11 @@ public class DoubleArrayIndexFieldData extends AbstractIndexFieldData<DoubleArra
}
int maxDoc = reader.maxDoc();
BigDoubleArrayList sValues = new BigDoubleArrayList(maxDoc);
DoubleArray sValues = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(maxDoc);
for (int i = 0; i < maxDoc; i++) {
final long ordinal = ordinals.getOrd(i);
if (ordinal == Ordinals.MISSING_ORDINAL) {
sValues.add(0);
} else {
sValues.add(values.get(ordinal));
if (ordinal != Ordinals.MISSING_ORDINAL) {
sValues.set(i, values.get(ordinal));
}
}
assert sValues.size() == maxDoc;

View File

@ -20,7 +20,7 @@ package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.common.util.BigFloatArrayList;
import org.elasticsearch.common.util.FloatArray;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
@ -87,9 +87,9 @@ public abstract class FloatArrayAtomicFieldData extends AbstractAtomicNumericFie
public static class WithOrdinals extends FloatArrayAtomicFieldData {
private final Ordinals ordinals;
private final BigFloatArrayList values;
private final FloatArray values;
public WithOrdinals(BigFloatArrayList values, Ordinals ordinals) {
public WithOrdinals(FloatArray values, Ordinals ordinals) {
super();
this.values = values;
this.ordinals = ordinals;
@ -125,9 +125,9 @@ public abstract class FloatArrayAtomicFieldData extends AbstractAtomicNumericFie
static class LongValues extends org.elasticsearch.index.fielddata.LongValues.WithOrdinals {
private final BigFloatArrayList values;
private final FloatArray values;
LongValues(BigFloatArrayList values, Ordinals.Docs ordinals) {
LongValues(FloatArray values, Ordinals.Docs ordinals) {
super(ordinals);
this.values = values;
}
@ -141,9 +141,9 @@ public abstract class FloatArrayAtomicFieldData extends AbstractAtomicNumericFie
static class DoubleValues extends org.elasticsearch.index.fielddata.DoubleValues.WithOrdinals {
private final BigFloatArrayList values;
private final FloatArray values;
DoubleValues(BigFloatArrayList values, Ordinals.Docs ordinals) {
DoubleValues(FloatArray values, Ordinals.Docs ordinals) {
super(ordinals);
this.values = values;
}
@ -161,11 +161,11 @@ public abstract class FloatArrayAtomicFieldData extends AbstractAtomicNumericFie
*/
public static class SingleFixedSet extends FloatArrayAtomicFieldData {
private final BigFloatArrayList values;
private final FloatArray values;
private final FixedBitSet set;
private final long numOrd;
public SingleFixedSet(BigFloatArrayList values, FixedBitSet set, long numOrd) {
public SingleFixedSet(FloatArray values, FixedBitSet set, long numOrd) {
super();
this.values = values;
this.set = set;
@ -203,10 +203,10 @@ public abstract class FloatArrayAtomicFieldData extends AbstractAtomicNumericFie
static class LongValues extends org.elasticsearch.index.fielddata.LongValues {
private final BigFloatArrayList values;
private final FloatArray values;
private final FixedBitSet set;
LongValues(BigFloatArrayList values, FixedBitSet set) {
LongValues(FloatArray values, FixedBitSet set) {
super(false);
this.values = values;
this.set = set;
@ -226,10 +226,10 @@ public abstract class FloatArrayAtomicFieldData extends AbstractAtomicNumericFie
static class DoubleValues extends org.elasticsearch.index.fielddata.DoubleValues {
private final BigFloatArrayList values;
private final FloatArray values;
private final FixedBitSet set;
DoubleValues(BigFloatArrayList values, FixedBitSet set) {
DoubleValues(FloatArray values, FixedBitSet set) {
super(false);
this.values = values;
this.set = set;
@ -254,14 +254,14 @@ public abstract class FloatArrayAtomicFieldData extends AbstractAtomicNumericFie
*/
public static class Single extends FloatArrayAtomicFieldData {
private final BigFloatArrayList values;
private final FloatArray values;
private final long numOrd;
/**
* Note, here, we assume that there is no offset by 1 from docId, so position 0
* is the value for docId 0.
*/
public Single(BigFloatArrayList values, long numOrd) {
public Single(FloatArray values, long numOrd) {
super();
this.values = values;
this.numOrd = numOrd;
@ -298,9 +298,9 @@ public abstract class FloatArrayAtomicFieldData extends AbstractAtomicNumericFie
static class LongValues extends DenseLongValues {
private final BigFloatArrayList values;
private final FloatArray values;
LongValues(BigFloatArrayList values) {
LongValues(FloatArray values) {
super(false);
this.values = values;
}
@ -313,9 +313,9 @@ public abstract class FloatArrayAtomicFieldData extends AbstractAtomicNumericFie
static class DoubleValues extends DenseDoubleValues {
private final BigFloatArrayList values;
private final FloatArray values;
DoubleValues(BigFloatArrayList values) {
DoubleValues(FloatArray values) {
super(false);
this.values = values;
}

View File

@ -24,11 +24,11 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.util.*;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigFloatArrayList;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.FloatArray;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.fieldcomparator.FloatValuesComparatorSource;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsBuilder;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.Ordinals.Docs;
@ -37,6 +37,7 @@ import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.indices.fielddata.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
/**
*/
@ -84,16 +85,19 @@ public class FloatArrayIndexFieldData extends AbstractIndexFieldData<FloatArrayA
return data;
}
// TODO: how can we guess the number of terms? numerics end up creating more terms per value...
final BigFloatArrayList values = new BigFloatArrayList();
FloatArray values = BigArrays.NON_RECYCLING_INSTANCE.newFloatArray(128);
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat("acceptable_transient_overhead_ratio", OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO);
boolean success = false;
try (OrdinalsBuilder builder = new OrdinalsBuilder(reader.maxDoc(), acceptableTransientOverheadRatio)) {
BytesRefIterator iter = builder.buildFromTerms(getNumericType().wrapTermsEnum(terms.iterator(null)));
BytesRef term;
long numTerms = 0;
while ((term = iter.next()) != null) {
values.add(NumericUtils.sortableIntToFloat(NumericUtils.prefixCodedToInt(term)));
values = BigArrays.NON_RECYCLING_INSTANCE.grow(values, numTerms + 1);
values.set(numTerms++, NumericUtils.sortableIntToFloat(NumericUtils.prefixCodedToInt(term)));
}
values = BigArrays.NON_RECYCLING_INSTANCE.resize(values, numTerms);
Ordinals build = builder.build(fieldDataType.getSettings());
if (build.isMultiValued() || CommonSettings.getMemoryStorageHint(fieldDataType) == CommonSettings.MemoryStorageFormat.ORDINALS) {
data = new FloatArrayAtomicFieldData.WithOrdinals(values, build);
@ -112,13 +116,11 @@ public class FloatArrayIndexFieldData extends AbstractIndexFieldData<FloatArrayA
}
int maxDoc = reader.maxDoc();
BigFloatArrayList sValues = new BigFloatArrayList(maxDoc);
FloatArray sValues = BigArrays.NON_RECYCLING_INSTANCE.newFloatArray(maxDoc);
for (int i = 0; i < maxDoc; i++) {
final long ordinal = ordinals.getOrd(i);
if (ordinal == Ordinals.MISSING_ORDINAL) {
sValues.add(0);
} else {
sValues.add(values.get(ordinal));
if (ordinal != Ordinals.MISSING_ORDINAL) {
sValues.set(i, values.get(ordinal));
}
}
assert sValues.size() == maxDoc;

View File

@ -21,7 +21,7 @@ package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.util.BigDoubleArrayList;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.fielddata.AtomicGeoPointFieldData;
import org.elasticsearch.index.fielddata.GeoPointValues;
import org.elasticsearch.index.fielddata.ScriptDocValues;
@ -44,10 +44,10 @@ public abstract class GeoPointDoubleArrayAtomicFieldData extends AtomicGeoPointF
static class WithOrdinals extends GeoPointDoubleArrayAtomicFieldData {
private final BigDoubleArrayList lon, lat;
private final DoubleArray lon, lat;
private final Ordinals ordinals;
public WithOrdinals(BigDoubleArrayList lon, BigDoubleArrayList lat, Ordinals ordinals) {
public WithOrdinals(DoubleArray lon, DoubleArray lat, Ordinals ordinals) {
super();
this.lon = lon;
this.lat = lat;
@ -79,12 +79,12 @@ public abstract class GeoPointDoubleArrayAtomicFieldData extends AtomicGeoPointF
public static class GeoPointValuesWithOrdinals extends GeoPointValues {
private final BigDoubleArrayList lon, lat;
private final DoubleArray lon, lat;
private final Ordinals.Docs ordinals;
private final GeoPoint scratch = new GeoPoint();
GeoPointValuesWithOrdinals(BigDoubleArrayList lon, BigDoubleArrayList lat, Ordinals.Docs ordinals) {
GeoPointValuesWithOrdinals(DoubleArray lon, DoubleArray lat, Ordinals.Docs ordinals) {
super(ordinals.isMultiValued());
this.lon = lon;
this.lat = lat;
@ -110,11 +110,11 @@ public abstract class GeoPointDoubleArrayAtomicFieldData extends AtomicGeoPointF
*/
public static class SingleFixedSet extends GeoPointDoubleArrayAtomicFieldData {
private final BigDoubleArrayList lon, lat;
private final DoubleArray lon, lat;
private final FixedBitSet set;
private final long numOrds;
public SingleFixedSet(BigDoubleArrayList lon, BigDoubleArrayList lat, FixedBitSet set, long numOrds) {
public SingleFixedSet(DoubleArray lon, DoubleArray lat, FixedBitSet set, long numOrds) {
super();
this.lon = lon;
this.lat = lat;
@ -148,13 +148,13 @@ public abstract class GeoPointDoubleArrayAtomicFieldData extends AtomicGeoPointF
static class GeoPointValuesSingleFixedSet extends GeoPointValues {
private final BigDoubleArrayList lon;
private final BigDoubleArrayList lat;
private final DoubleArray lon;
private final DoubleArray lat;
private final FixedBitSet set;
private final GeoPoint scratch = new GeoPoint();
GeoPointValuesSingleFixedSet(BigDoubleArrayList lon, BigDoubleArrayList lat, FixedBitSet set) {
GeoPointValuesSingleFixedSet(DoubleArray lon, DoubleArray lat, FixedBitSet set) {
super(false);
this.lon = lon;
this.lat = lat;
@ -179,10 +179,10 @@ public abstract class GeoPointDoubleArrayAtomicFieldData extends AtomicGeoPointF
*/
public static class Single extends GeoPointDoubleArrayAtomicFieldData {
private final BigDoubleArrayList lon, lat;
private final DoubleArray lon, lat;
private final long numOrds;
public Single(BigDoubleArrayList lon, BigDoubleArrayList lat, long numOrds) {
public Single(DoubleArray lon, DoubleArray lat, long numOrds) {
super();
this.lon = lon;
this.lat = lat;
@ -215,13 +215,13 @@ public abstract class GeoPointDoubleArrayAtomicFieldData extends AtomicGeoPointF
static class GeoPointValuesSingle extends GeoPointValues {
private final BigDoubleArrayList lon;
private final BigDoubleArrayList lat;
private final DoubleArray lon;
private final DoubleArray lat;
private final GeoPoint scratch = new GeoPoint();
GeoPointValuesSingle(BigDoubleArrayList lon, BigDoubleArrayList lat) {
GeoPointValuesSingle(DoubleArray lon, DoubleArray lat) {
super(false);
this.lon = lon;
this.lat = lat;

View File

@ -24,7 +24,8 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.util.FixedBitSet;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigDoubleArrayList;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.*;
import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsBuilder;
@ -70,32 +71,35 @@ public class GeoPointDoubleArrayIndexFieldData extends AbstractGeoPointIndexFiel
estimator.afterLoad(null, data.getMemorySizeInBytes());
return data;
}
final BigDoubleArrayList lat = new BigDoubleArrayList();
final BigDoubleArrayList lon = new BigDoubleArrayList();
DoubleArray lat = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(128);
DoubleArray lon = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(128);
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat("acceptable_transient_overhead_ratio", OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO);
boolean success = false;
try (OrdinalsBuilder builder = new OrdinalsBuilder(terms.size(), reader.maxDoc(), acceptableTransientOverheadRatio)) {
final GeoPointEnum iter = new GeoPointEnum(builder.buildFromTerms(terms.iterator(null)));
GeoPoint point;
long numTerms = 0;
while ((point = iter.next()) != null) {
lat.add(point.getLat());
lon.add(point.getLon());
lat = BigArrays.NON_RECYCLING_INSTANCE.resize(lat, numTerms + 1);
lon = BigArrays.NON_RECYCLING_INSTANCE.resize(lon, numTerms + 1);
lat.set(numTerms, point.getLat());
lon.set(numTerms, point.getLon());
++numTerms;
}
lat = BigArrays.NON_RECYCLING_INSTANCE.resize(lat, numTerms);
lon = BigArrays.NON_RECYCLING_INSTANCE.resize(lon, numTerms);
Ordinals build = builder.build(fieldDataType.getSettings());
if (!(build.isMultiValued() || CommonSettings.getMemoryStorageHint(fieldDataType) == CommonSettings.MemoryStorageFormat.ORDINALS)) {
Docs ordinals = build.ordinals();
int maxDoc = reader.maxDoc();
BigDoubleArrayList sLat = new BigDoubleArrayList(reader.maxDoc());
BigDoubleArrayList sLon = new BigDoubleArrayList(reader.maxDoc());
DoubleArray sLat = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(reader.maxDoc());
DoubleArray sLon = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(reader.maxDoc());
for (int i = 0; i < maxDoc; i++) {
long nativeOrdinal = ordinals.getOrd(i);
if (nativeOrdinal == Ordinals.MISSING_ORDINAL) {
sLat.add(0);
sLon.add(0);
} else {
sLat.add(lat.get(nativeOrdinal));
sLon.add(lon.get(nativeOrdinal));
if (nativeOrdinal != Ordinals.MISSING_ORDINAL) {
sLat.set(i, lat.get(nativeOrdinal));
sLon.set(i, lon.get(nativeOrdinal));
}
}
FixedBitSet set = builder.buildDocsWithValuesSet();

View File

@ -20,14 +20,17 @@
package org.elasticsearch.common.util;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.Before;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.Arrays;
public class BigArraysTests extends ElasticsearchTestCase {
@ -326,4 +329,59 @@ public class BigArraysTests extends ElasticsearchTestCase {
return bytearray;
}
public void testByteAccounting() throws Exception {
for (String type : Arrays.asList("Byte", "Int", "Long", "Float", "Double", "Object")) {
BigArrays bigArrays = new BigArrays(ImmutableSettings.builder().put(BigArrays.MAX_SIZE_IN_BYTES_SETTING, Long.MAX_VALUE).build(), null);
Method create = BigArrays.class.getMethod("new" + type + "Array", long.class);
final int size = scaledRandomIntBetween(5, 1 << 16);
BigArray array = (BigArray) create.invoke(bigArrays, size);
assertEquals(array.sizeInBytes(), bigArrays.sizeInBytes());
Method resize = BigArrays.class.getMethod("resize", array.getClass().getInterfaces()[0], long.class);
int newSize = scaledRandomIntBetween(5, 1 << 16);
array = (BigArray) resize.invoke(bigArrays, array, newSize);
assertEquals(array.sizeInBytes(), bigArrays.sizeInBytes());
array.close();
assertEquals(0, bigArrays.sizeInBytes());
}
}
public void testMaxSizeExceededOnNew() throws Exception {
final int size = scaledRandomIntBetween(5, 1 << 22);
for (String type : Arrays.asList("Byte", "Int", "Long", "Float", "Double", "Object")) {
BigArrays bigArrays = new BigArrays(ImmutableSettings.builder().put(BigArrays.MAX_SIZE_IN_BYTES_SETTING, randomIntBetween(1, size)).build(), null);
Method create = BigArrays.class.getMethod("new" + type + "Array", long.class);
try {
create.invoke(bigArrays, size);
fail("expected an exception on " + create);
} catch (InvocationTargetException e) {
assertTrue(e.getCause() instanceof ElasticsearchIllegalStateException);
}
assertEquals(0, bigArrays.sizeInBytes());
}
}
public void testMaxSizeExceededOnResize() throws Exception {
for (String type : Arrays.asList("Byte", "Int", "Long", "Float", "Double", "Object")) {
final long maxSize = randomIntBetween(1 << 10, 1 << 22);
BigArrays bigArrays = new BigArrays(ImmutableSettings.builder().put(BigArrays.MAX_SIZE_IN_BYTES_SETTING, maxSize).build(), null);
Method create = BigArrays.class.getMethod("new" + type + "Array", long.class);
final int size = scaledRandomIntBetween(1, 20);
BigArray array = (BigArray) create.invoke(bigArrays, size);
Method resize = BigArrays.class.getMethod("resize", array.getClass().getInterfaces()[0], long.class);
while (true) {
long newSize = array.size() * 2;
assertEquals(array.sizeInBytes(), bigArrays.sizeInBytes());
try {
array = (BigArray) resize.invoke(bigArrays, array, newSize);
} catch (InvocationTargetException e) {
assertTrue(e.getCause() instanceof ElasticsearchIllegalStateException);
break;
}
}
assertEquals(array.sizeInBytes(), bigArrays.sizeInBytes());
array.close();
assertEquals(0, bigArrays.sizeInBytes());
}
}
}

View File

@ -45,6 +45,7 @@ public class MockBigArrays extends BigArrays {
*/
private static final boolean TRACK_ALLOCATIONS = false;
private static volatile BigArrays INSTANCE = null;
private static ConcurrentMap<Object, Object> ACQUIRED_ARRAYS = new ConcurrentHashMap<>();
public static void ensureAllArraysAreReleased() throws Exception {
@ -68,6 +69,12 @@ public class MockBigArrays extends BigArrays {
}
}
}
if (INSTANCE != null) {
final long sizeInBytes = INSTANCE.sizeInBytes();
if (sizeInBytes != 0) {
throw new AssertionError("Expected 0 bytes, got " + sizeInBytes);
}
}
}
private final Random random;
@ -82,6 +89,7 @@ public class MockBigArrays extends BigArrays {
seed = 0;
}
random = new Random(seed);
INSTANCE = this;
}
@Override
@ -234,11 +242,13 @@ public class MockBigArrays extends BigArrays {
private static abstract class AbstractArrayWrapper {
final BigArray in;
boolean clearOnResize;
AtomicBoolean released;
AbstractArrayWrapper(boolean clearOnResize) {
AbstractArrayWrapper(BigArray in, boolean clearOnResize) {
ACQUIRED_ARRAYS.put(this, TRACK_ALLOCATIONS ? new RuntimeException() : Boolean.TRUE);
this.in = in;
this.clearOnResize = clearOnResize;
released = new AtomicBoolean(false);
}
@ -251,6 +261,10 @@ public class MockBigArrays extends BigArrays {
return getDelegate().size();
}
public long sizeInBytes() {
return in.sizeInBytes();
}
public void close() {
if (!released.compareAndSet(false, true)) {
throw new IllegalStateException("Double release");
@ -267,7 +281,7 @@ public class MockBigArrays extends BigArrays {
private final ByteArray in;
ByteArrayWrapper(ByteArray in, boolean clearOnResize) {
super(clearOnResize);
super(in, clearOnResize);
this.in = in;
}
@ -313,7 +327,7 @@ public class MockBigArrays extends BigArrays {
private final IntArray in;
IntArrayWrapper(IntArray in, boolean clearOnResize) {
super(clearOnResize);
super(in, clearOnResize);
this.in = in;
}
@ -354,7 +368,7 @@ public class MockBigArrays extends BigArrays {
private final LongArray in;
LongArrayWrapper(LongArray in, boolean clearOnResize) {
super(clearOnResize);
super(in, clearOnResize);
this.in = in;
}
@ -395,7 +409,7 @@ public class MockBigArrays extends BigArrays {
private final FloatArray in;
FloatArrayWrapper(FloatArray in, boolean clearOnResize) {
super(clearOnResize);
super(in, clearOnResize);
this.in = in;
}
@ -436,7 +450,7 @@ public class MockBigArrays extends BigArrays {
private final DoubleArray in;
DoubleArrayWrapper(DoubleArray in, boolean clearOnResize) {
super(clearOnResize);
super(in, clearOnResize);
this.in = in;
}
@ -477,7 +491,7 @@ public class MockBigArrays extends BigArrays {
private final ObjectArray<T> in;
ObjectArrayWrapper(ObjectArray<T> in) {
super(false);
super(in, false);
this.in = in;
}