From e024c67561c547b5194d00ea5e5f518a1a96fe63 Mon Sep 17 00:00:00 2001 From: Ali Beyad Date: Fri, 2 Jun 2017 15:16:22 -0400 Subject: [PATCH] Checks the circuit breaker before allocating bytes for a new big array (#25010) Previously, when allocating bytes for a BigArray, the array was created (or attempted to be created) and only then would the array be checked for the amount of RAM used to see if the circuit breaker should trip. This is problematic because for very large arrays, if creating or resizing the array, it is possible to attempt to create/resize and get an OOM error before the circuit breaker trips, because the allocation happens before checking with the circuit breaker. This commit ensures that the circuit breaker is checked before all big array allocations (note, this does not effect the array allocations that are less than 16kb which use the [Type]ArrayWrapper classes found in BigArrays.java). If such an allocation or resizing would cause the circuit breaker to trip, then the breaker trips before attempting to allocate and potentially running into an OOM error from the JVM. Closes #24790 --- .../common/util/AbstractArray.java | 2 +- .../common/util/AbstractBigArray.java | 5 + .../elasticsearch/common/util/BigArrays.java | 94 +++++++++++-------- .../common/util/BigByteArray.java | 9 +- .../common/util/BigDoubleArray.java | 7 ++ .../common/util/BigFloatArray.java | 7 ++ .../common/util/BigIntArray.java | 7 ++ .../common/util/BigLongArray.java | 7 ++ .../common/util/BigObjectArray.java | 9 +- .../common/util/BigArraysTests.java | 89 +++++++++++++++--- .../common/util/MockBigArrays.java | 2 - 11 files changed, 180 insertions(+), 58 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/util/AbstractArray.java b/core/src/main/java/org/elasticsearch/common/util/AbstractArray.java index 913f1ad26a4..6a4895c7950 100644 --- a/core/src/main/java/org/elasticsearch/common/util/AbstractArray.java +++ b/core/src/main/java/org/elasticsearch/common/util/AbstractArray.java @@ -41,7 +41,7 @@ abstract class AbstractArray implements BigArray { public final void close() { if (closed.compareAndSet(false, true)) { try { - bigArrays.adjustBreaker(-ramBytesUsed()); + bigArrays.adjustBreaker(-ramBytesUsed(), true); } finally { doClose(); } diff --git a/core/src/main/java/org/elasticsearch/common/util/AbstractBigArray.java b/core/src/main/java/org/elasticsearch/common/util/AbstractBigArray.java index f26dad1fdb5..73a05f7f2cf 100644 --- a/core/src/main/java/org/elasticsearch/common/util/AbstractBigArray.java +++ b/core/src/main/java/org/elasticsearch/common/util/AbstractBigArray.java @@ -87,6 +87,11 @@ abstract class AbstractBigArray extends AbstractArray { @Override public final long ramBytesUsed() { + return ramBytesEstimated(size); + } + + /** Given the size of the array, estimate the number of bytes it will use. */ + public final long ramBytesEstimated(final long size) { // rough approximate, we only take into account the size of the values, not the overhead of the array objects return ((long) pageIndex(size - 1) + 1) * pageSize() * numBytesPerElement(); } diff --git a/core/src/main/java/org/elasticsearch/common/util/BigArrays.java b/core/src/main/java/org/elasticsearch/common/util/BigArrays.java index 728db17c2a4..8752e5b4bee 100644 --- a/core/src/main/java/org/elasticsearch/common/util/BigArrays.java +++ b/core/src/main/java/org/elasticsearch/common/util/BigArrays.java @@ -25,7 +25,6 @@ import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; -import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.recycler.Recycler; @@ -91,7 +90,7 @@ public class BigArrays implements Releasable { private abstract static class AbstractArrayWrapper extends AbstractArray implements BigArray { - protected static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(ByteArrayWrapper.class); + static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(ByteArrayWrapper.class); private final Releasable releasable; private final long size; @@ -377,6 +376,7 @@ public class BigArrays implements Releasable { // Checking the breaker is disabled if not specified this(new PageCacheRecycler(settings), breakerService, false); } + // public for tests public BigArrays(PageCacheRecycler recycler, @Nullable final CircuitBreakerService breakerService, boolean checkBreaker) { this.checkBreaker = checkBreaker; @@ -392,9 +392,12 @@ public class BigArrays implements Releasable { /** * Adjust the circuit breaker with the given delta, if the delta is * negative, or checkBreaker is false, the breaker will be adjusted - * without tripping + * without tripping. If the data was already created before calling + * this method, and the breaker trips, we add the delta without breaking + * to account for the created data. If the data has not been created yet, + * we do not add the delta to the breaker if it trips. */ - void adjustBreaker(long delta) { + void adjustBreaker(final long delta, final boolean isDataAlreadyCreated) { if (this.breakerService != null) { CircuitBreaker breaker = this.breakerService.getBreaker(CircuitBreaker.REQUEST); if (this.checkBreaker) { @@ -404,9 +407,11 @@ public class BigArrays implements Releasable { try { breaker.addEstimateBytesAndMaybeBreak(delta, ""); } catch (CircuitBreakingException e) { - // since we've already created the data, we need to - // add it so closing the stream re-adjusts properly - breaker.addWithoutBreaking(delta); + if (isDataAlreadyCreated) { + // since we've already created the data, we need to + // add it so closing the stream re-adjusts properly + breaker.addWithoutBreaking(delta); + } // re-throw the original exception throw e; } @@ -435,15 +440,21 @@ public class BigArrays implements Releasable { private T resizeInPlace(T array, long newSize) { final long oldMemSize = array.ramBytesUsed(); + assert oldMemSize == array.ramBytesEstimated(array.size) : + "ram bytes used should equal that which was previously estimated: ramBytesUsed=" + + oldMemSize + ", ramBytesEstimated=" + array.ramBytesEstimated(array.size); + final long estimatedIncreaseInBytes = array.ramBytesEstimated(newSize) - oldMemSize; + assert estimatedIncreaseInBytes >= 0 : + "estimated increase in bytes for resizing should not be negative: " + estimatedIncreaseInBytes; + adjustBreaker(estimatedIncreaseInBytes, false); array.resize(newSize); - adjustBreaker(array.ramBytesUsed() - oldMemSize); return array; } private T validate(T array) { boolean success = false; try { - adjustBreaker(array.ramBytesUsed()); + adjustBreaker(array.ramBytesUsed(), true); success = true; } finally { if (!success) { @@ -459,16 +470,17 @@ public class BigArrays implements Releasable { * @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) { - array = new BigByteArray(size, this, clearOnResize); + // when allocating big arrays, we want to first ensure we have the capacity by + // checking with the circuit breaker before attempting to allocate + adjustBreaker(BigByteArray.estimateRamBytes(size), false); + return new BigByteArray(size, this, clearOnResize); } else if (size >= BYTE_PAGE_SIZE / 2 && recycler != null) { final Recycler.V page = recycler.bytePage(clearOnResize); - array = new ByteArrayWrapper(this, page.v(), size, page, clearOnResize); + return validate(new ByteArrayWrapper(this, page.v(), size, page, clearOnResize)); } else { - array = new ByteArrayWrapper(this, new byte[(int) size], size, null, clearOnResize); + return validate(new ByteArrayWrapper(this, new byte[(int) size], size, null, clearOnResize)); } - return validate(array); } /** @@ -541,16 +553,17 @@ public class BigArrays implements Releasable { * @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) { - array = new BigIntArray(size, this, clearOnResize); + // when allocating big arrays, we want to first ensure we have the capacity by + // checking with the circuit breaker before attempting to allocate + adjustBreaker(BigIntArray.estimateRamBytes(size), false); + return new BigIntArray(size, this, clearOnResize); } else if (size >= INT_PAGE_SIZE / 2 && recycler != null) { final Recycler.V page = recycler.intPage(clearOnResize); - array = new IntArrayWrapper(this, page.v(), size, page, clearOnResize); + return validate(new IntArrayWrapper(this, page.v(), size, page, clearOnResize)); } else { - array = new IntArrayWrapper(this, new int[(int) size], size, null, clearOnResize); + return validate(new IntArrayWrapper(this, new int[(int) size], size, null, clearOnResize)); } - return validate(array); } /** @@ -591,16 +604,17 @@ public class BigArrays implements Releasable { * @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) { - array = new BigLongArray(size, this, clearOnResize); + // when allocating big arrays, we want to first ensure we have the capacity by + // checking with the circuit breaker before attempting to allocate + adjustBreaker(BigLongArray.estimateRamBytes(size), false); + return new BigLongArray(size, this, clearOnResize); } else if (size >= LONG_PAGE_SIZE / 2 && recycler != null) { final Recycler.V page = recycler.longPage(clearOnResize); - array = new LongArrayWrapper(this, page.v(), size, page, clearOnResize); + return validate(new LongArrayWrapper(this, page.v(), size, page, clearOnResize)); } else { - array = new LongArrayWrapper(this, new long[(int) size], size, null, clearOnResize); + return validate(new LongArrayWrapper(this, new long[(int) size], size, null, clearOnResize)); } - return validate(array); } /** @@ -641,16 +655,17 @@ public class BigArrays implements Releasable { * @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 > LONG_PAGE_SIZE) { - arr = new BigDoubleArray(size, this, clearOnResize); + // when allocating big arrays, we want to first ensure we have the capacity by + // checking with the circuit breaker before attempting to allocate + adjustBreaker(BigDoubleArray.estimateRamBytes(size), false); + return new BigDoubleArray(size, this, clearOnResize); } else if (size >= LONG_PAGE_SIZE / 2 && recycler != null) { final Recycler.V page = recycler.longPage(clearOnResize); - arr = new DoubleArrayWrapper(this, page.v(), size, page, clearOnResize); + return validate(new DoubleArrayWrapper(this, page.v(), size, page, clearOnResize)); } else { - arr = new DoubleArrayWrapper(this, new long[(int) size], size, null, clearOnResize); + return validate(new DoubleArrayWrapper(this, new long[(int) size], size, null, clearOnResize)); } - return validate(arr); } /** Allocate a new {@link DoubleArray} of the given capacity. */ @@ -688,16 +703,17 @@ public class BigArrays implements Releasable { * @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 > INT_PAGE_SIZE) { - array = new BigFloatArray(size, this, clearOnResize); + // when allocating big arrays, we want to first ensure we have the capacity by + // checking with the circuit breaker before attempting to allocate + adjustBreaker(BigFloatArray.estimateRamBytes(size), false); + return new BigFloatArray(size, this, clearOnResize); } else if (size >= INT_PAGE_SIZE / 2 && recycler != null) { final Recycler.V page = recycler.intPage(clearOnResize); - array = new FloatArrayWrapper(this, page.v(), size, page, clearOnResize); + return validate(new FloatArrayWrapper(this, page.v(), size, page, clearOnResize)); } else { - array = new FloatArrayWrapper(this, new int[(int) size], size, null, clearOnResize); + return validate(new FloatArrayWrapper(this, new int[(int) size], size, null, clearOnResize)); } - return validate(array); } /** Allocate a new {@link FloatArray} of the given capacity. */ @@ -736,14 +752,16 @@ public class BigArrays implements Releasable { public ObjectArray newObjectArray(long size) { final ObjectArray array; if (size > OBJECT_PAGE_SIZE) { - array = new BigObjectArray<>(size, this); + // when allocating big arrays, we want to first ensure we have the capacity by + // checking with the circuit breaker before attempting to allocate + adjustBreaker(BigObjectArray.estimateRamBytes(size), false); + return new BigObjectArray<>(size, this); } else if (size >= OBJECT_PAGE_SIZE / 2 && recycler != null) { final Recycler.V page = recycler.objectPage(); - array = new ObjectArrayWrapper<>(this, page.v(), size, page); + return validate(new ObjectArrayWrapper<>(this, page.v(), size, page)); } else { - array = new ObjectArrayWrapper<>(this, new Object[(int) size], size, null); + return validate(new ObjectArrayWrapper<>(this, new Object[(int) size], size, null)); } - return validate(array); } /** Resize the array to the exact provided size. */ diff --git a/core/src/main/java/org/elasticsearch/common/util/BigByteArray.java b/core/src/main/java/org/elasticsearch/common/util/BigByteArray.java index 02f2c627a4c..789e6dc6bba 100644 --- a/core/src/main/java/org/elasticsearch/common/util/BigByteArray.java +++ b/core/src/main/java/org/elasticsearch/common/util/BigByteArray.java @@ -33,6 +33,8 @@ import static org.elasticsearch.common.util.BigArrays.BYTE_PAGE_SIZE; */ final class BigByteArray extends AbstractBigArray implements ByteArray { + private static final BigByteArray ESTIMATOR = new BigByteArray(0, BigArrays.NON_RECYCLING_INSTANCE, false); + private byte[][] pages; /** Constructor. */ @@ -44,7 +46,7 @@ final class BigByteArray extends AbstractBigArray implements ByteArray { pages[i] = newBytePage(i); } } - + @Override public byte get(long index) { final int pageIndex = pageIndex(index); @@ -147,4 +149,9 @@ final class BigByteArray extends AbstractBigArray implements ByteArray { this.size = newSize; } + /** Estimates the number of bytes that would be consumed by an array of the given size. */ + public static long estimateRamBytes(final long size) { + return ESTIMATOR.ramBytesEstimated(size); + } + } diff --git a/core/src/main/java/org/elasticsearch/common/util/BigDoubleArray.java b/core/src/main/java/org/elasticsearch/common/util/BigDoubleArray.java index a8b4503bda6..a2c770ee995 100644 --- a/core/src/main/java/org/elasticsearch/common/util/BigDoubleArray.java +++ b/core/src/main/java/org/elasticsearch/common/util/BigDoubleArray.java @@ -32,6 +32,8 @@ import static org.elasticsearch.common.util.BigArrays.LONG_PAGE_SIZE; */ final class BigDoubleArray extends AbstractBigArray implements DoubleArray { + private static final BigDoubleArray ESTIMATOR = new BigDoubleArray(0, BigArrays.NON_RECYCLING_INSTANCE, false); + private long[][] pages; /** Constructor. */ @@ -110,4 +112,9 @@ final class BigDoubleArray extends AbstractBigArray implements DoubleArray { } } + /** Estimates the number of bytes that would be consumed by an array of the given size. */ + public static long estimateRamBytes(final long size) { + return ESTIMATOR.ramBytesEstimated(size); + } + } diff --git a/core/src/main/java/org/elasticsearch/common/util/BigFloatArray.java b/core/src/main/java/org/elasticsearch/common/util/BigFloatArray.java index cf11eba37ae..b67db2e84de 100644 --- a/core/src/main/java/org/elasticsearch/common/util/BigFloatArray.java +++ b/core/src/main/java/org/elasticsearch/common/util/BigFloatArray.java @@ -32,6 +32,8 @@ import static org.elasticsearch.common.util.BigArrays.INT_PAGE_SIZE; */ final class BigFloatArray extends AbstractBigArray implements FloatArray { + private static final BigFloatArray ESTIMATOR = new BigFloatArray(0, BigArrays.NON_RECYCLING_INSTANCE, false); + private int[][] pages; /** Constructor. */ @@ -110,4 +112,9 @@ final class BigFloatArray extends AbstractBigArray implements FloatArray { } } + /** Estimates the number of bytes that would be consumed by an array of the given size. */ + public static long estimateRamBytes(final long size) { + return ESTIMATOR.ramBytesEstimated(size); + } + } diff --git a/core/src/main/java/org/elasticsearch/common/util/BigIntArray.java b/core/src/main/java/org/elasticsearch/common/util/BigIntArray.java index 16ca3ada24d..d2a1ca3f49c 100644 --- a/core/src/main/java/org/elasticsearch/common/util/BigIntArray.java +++ b/core/src/main/java/org/elasticsearch/common/util/BigIntArray.java @@ -32,6 +32,8 @@ import static org.elasticsearch.common.util.BigArrays.INT_PAGE_SIZE; */ final class BigIntArray extends AbstractBigArray implements IntArray { + private static final BigIntArray ESTIMATOR = new BigIntArray(0, BigArrays.NON_RECYCLING_INSTANCE, false); + private int[][] pages; /** Constructor. */ @@ -108,4 +110,9 @@ final class BigIntArray extends AbstractBigArray implements IntArray { this.size = newSize; } + /** Estimates the number of bytes that would be consumed by an array of the given size. */ + public static long estimateRamBytes(final long size) { + return ESTIMATOR.ramBytesEstimated(size); + } + } diff --git a/core/src/main/java/org/elasticsearch/common/util/BigLongArray.java b/core/src/main/java/org/elasticsearch/common/util/BigLongArray.java index cb9b9e6c332..69f919382f8 100644 --- a/core/src/main/java/org/elasticsearch/common/util/BigLongArray.java +++ b/core/src/main/java/org/elasticsearch/common/util/BigLongArray.java @@ -32,6 +32,8 @@ import static org.elasticsearch.common.util.BigArrays.LONG_PAGE_SIZE; */ final class BigLongArray extends AbstractBigArray implements LongArray { + private static final BigLongArray ESTIMATOR = new BigLongArray(0, BigArrays.NON_RECYCLING_INSTANCE, false); + private long[][] pages; /** Constructor. */ @@ -111,4 +113,9 @@ final class BigLongArray extends AbstractBigArray implements LongArray { } } + /** Estimates the number of bytes that would be consumed by an array of the given size. */ + public static long estimateRamBytes(final long size) { + return ESTIMATOR.ramBytesEstimated(size); + } + } diff --git a/core/src/main/java/org/elasticsearch/common/util/BigObjectArray.java b/core/src/main/java/org/elasticsearch/common/util/BigObjectArray.java index 023c710f3aa..1ed012e2bb3 100644 --- a/core/src/main/java/org/elasticsearch/common/util/BigObjectArray.java +++ b/core/src/main/java/org/elasticsearch/common/util/BigObjectArray.java @@ -32,6 +32,8 @@ import static org.elasticsearch.common.util.BigArrays.OBJECT_PAGE_SIZE; */ final class BigObjectArray extends AbstractBigArray implements ObjectArray { + private static final BigObjectArray ESTIMATOR = new BigObjectArray(0, BigArrays.NON_RECYCLING_INSTANCE); + private Object[][] pages; /** Constructor. */ @@ -85,4 +87,9 @@ final class BigObjectArray extends AbstractBigArray implements ObjectArray this.size = newSize; } -} \ No newline at end of file + /** Estimates the number of bytes that would be consumed by an array of the given size. */ + public static long estimateRamBytes(final long size) { + return ESTIMATOR.ramBytesEstimated(size); + } + +} diff --git a/core/src/test/java/org/elasticsearch/common/util/BigArraysTests.java b/core/src/test/java/org/elasticsearch/common/util/BigArraysTests.java index 301f48f9b04..945dda446ce 100644 --- a/core/src/test/java/org/elasticsearch/common/util/BigArraysTests.java +++ b/core/src/test/java/org/elasticsearch/common/util/BigArraysTests.java @@ -33,6 +33,11 @@ import org.junit.Before; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.Arrays; +import java.util.List; +import java.util.function.Function; + +import static org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; public class BigArraysTests extends ESTestCase { @@ -330,22 +335,17 @@ public class BigArraysTests extends ESTestCase { } public void testMaxSizeExceededOnNew() throws Exception { - final int size = scaledRandomIntBetween(5, 1 << 22); - for (String type : Arrays.asList("Byte", "Int", "Long", "Float", "Double", "Object")) { - HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService( - Settings.builder() - .put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), size - 1, ByteSizeUnit.BYTES) - .build(), - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); - BigArrays bigArrays = new BigArrays(null, hcbs, false).withCircuitBreaking(); - Method create = BigArrays.class.getMethod("new" + type + "Array", long.class); + final long size = scaledRandomIntBetween(5, 1 << 22); + final long maxSize = size - 1; + for (BigArraysHelper bigArraysHelper : bigArrayCreators(maxSize, true)) { try { - create.invoke(bigArrays, size); - fail("expected an exception on " + create); - } catch (InvocationTargetException e) { - assertTrue(e.getCause() instanceof CircuitBreakingException); + bigArraysHelper.arrayAllocator.apply(size); + fail("circuit breaker should trip"); + } catch (CircuitBreakingException e) { + assertEquals(maxSize, e.getByteLimit()); + assertThat(e.getBytesWanted(), greaterThanOrEqualTo(size)); } - assertEquals(0, hcbs.getBreaker(CircuitBreaker.REQUEST).getUsed()); + assertEquals(0, bigArraysHelper.bigArrays.breakerService().getBreaker(CircuitBreaker.REQUEST).getUsed()); } } @@ -354,7 +354,7 @@ public class BigArraysTests extends ESTestCase { final long maxSize = randomIntBetween(1 << 10, 1 << 22); HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService( Settings.builder() - .put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), maxSize, ByteSizeUnit.BYTES) + .put(REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), maxSize, ByteSizeUnit.BYTES) .build(), new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); BigArrays bigArrays = new BigArrays(null, hcbs, false).withCircuitBreaking(); @@ -377,4 +377,63 @@ public class BigArraysTests extends ESTestCase { } } + public void testEstimatedBytesSameAsActualBytes() throws Exception { + final int maxSize = 1 << scaledRandomIntBetween(15, 22); + final long size = randomIntBetween((1 << 14) + 1, maxSize); + for (final BigArraysHelper bigArraysHelper : bigArrayCreators(maxSize, false)) { + final BigArray bigArray = bigArraysHelper.arrayAllocator.apply(size); + assertEquals(bigArraysHelper.ramEstimator.apply(size).longValue(), bigArray.ramBytesUsed()); + } + } + + private List bigArrayCreators(final long maxSize, final boolean withBreaking) { + final BigArrays byteBigArrays = newBigArraysInstance(maxSize, withBreaking); + BigArraysHelper byteHelper = new BigArraysHelper(byteBigArrays, + (Long size) -> byteBigArrays.newByteArray(size), + (Long size) -> BigByteArray.estimateRamBytes(size)); + final BigArrays intBigArrays = newBigArraysInstance(maxSize, withBreaking); + BigArraysHelper intHelper = new BigArraysHelper(intBigArrays, + (Long size) -> intBigArrays.newIntArray(size), + (Long size) -> BigIntArray.estimateRamBytes(size)); + final BigArrays longBigArrays = newBigArraysInstance(maxSize, withBreaking); + BigArraysHelper longHelper = new BigArraysHelper(longBigArrays, + (Long size) -> longBigArrays.newLongArray(size), + (Long size) -> BigLongArray.estimateRamBytes(size)); + final BigArrays floatBigArrays = newBigArraysInstance(maxSize, withBreaking); + BigArraysHelper floatHelper = new BigArraysHelper(floatBigArrays, + (Long size) -> floatBigArrays.newFloatArray(size), + (Long size) -> BigFloatArray.estimateRamBytes(size)); + final BigArrays doubleBigArrays = newBigArraysInstance(maxSize, withBreaking); + BigArraysHelper doubleHelper = new BigArraysHelper(doubleBigArrays, + (Long size) -> doubleBigArrays.newDoubleArray(size), + (Long size) -> BigDoubleArray.estimateRamBytes(size)); + final BigArrays objectBigArrays = newBigArraysInstance(maxSize, withBreaking); + BigArraysHelper objectHelper = new BigArraysHelper(objectBigArrays, + (Long size) -> objectBigArrays.newObjectArray(size), + (Long size) -> BigObjectArray.estimateRamBytes(size)); + return Arrays.asList(byteHelper, intHelper, longHelper, floatHelper, doubleHelper, objectHelper); + } + + private BigArrays newBigArraysInstance(final long maxSize, final boolean withBreaking) { + HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService( + Settings.builder() + .put(REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), maxSize, ByteSizeUnit.BYTES) + .build(), + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); + BigArrays bigArrays = new BigArrays(null, hcbs, false); + return (withBreaking ? bigArrays.withCircuitBreaking() : bigArrays); + } + + private static class BigArraysHelper { + final BigArrays bigArrays; + final Function arrayAllocator; + final Function ramEstimator; + + BigArraysHelper(BigArrays bigArrays, Function arrayAllocator, Function ramEstimator) { + this.bigArrays = bigArrays; + this.arrayAllocator = arrayAllocator; + this.ramEstimator = ramEstimator; + } + } + } diff --git a/test/framework/src/main/java/org/elasticsearch/common/util/MockBigArrays.java b/test/framework/src/main/java/org/elasticsearch/common/util/MockBigArrays.java index 5f76ac0030d..2f75d92d898 100644 --- a/test/framework/src/main/java/org/elasticsearch/common/util/MockBigArrays.java +++ b/test/framework/src/main/java/org/elasticsearch/common/util/MockBigArrays.java @@ -34,8 +34,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Random; -import java.util.Set; -import java.util.WeakHashMap; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference;