diff --git a/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java b/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java index 57fc187e3cb..4607e01266f 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java @@ -36,6 +36,7 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.query.IndexQueryParserService; import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.query.QueryParsingException; @@ -69,13 +70,16 @@ public class TransportValidateQueryAction extends TransportBroadcastOperationAct private final PageCacheRecycler pageCacheRecycler; + private final BigArrays bigArrays; + @Inject - public TransportValidateQueryAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, IndicesService indicesService, ScriptService scriptService, CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler) { + public TransportValidateQueryAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, IndicesService indicesService, ScriptService scriptService, CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler, BigArrays bigArrays) { super(settings, threadPool, clusterService, transportService); this.indicesService = indicesService; this.scriptService = scriptService; this.cacheRecycler = cacheRecycler; this.pageCacheRecycler = pageCacheRecycler; + this.bigArrays = bigArrays; } @Override @@ -184,7 +188,7 @@ public class TransportValidateQueryAction extends TransportBroadcastOperationAct SearchContext.setCurrent(new DefaultSearchContext(0, new ShardSearchRequest().types(request.types()).nowInMillis(request.nowInMillis()), null, indexShard.acquireSearcher("validate_query"), indexService, indexShard, - scriptService, cacheRecycler, pageCacheRecycler)); + scriptService, cacheRecycler, pageCacheRecycler, bigArrays)); try { ParsedQuery parsedQuery = queryParserService.parseQuery(request.source()); valid = true; diff --git a/src/main/java/org/elasticsearch/action/count/TransportCountAction.java b/src/main/java/org/elasticsearch/action/count/TransportCountAction.java index 3d06a38c8ad..bddd6fd7e0d 100644 --- a/src/main/java/org/elasticsearch/action/count/TransportCountAction.java +++ b/src/main/java/org/elasticsearch/action/count/TransportCountAction.java @@ -37,6 +37,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.shard.service.IndexShard; @@ -70,14 +71,18 @@ public class TransportCountAction extends TransportBroadcastOperationAction extends in.seek(metaDataPosition); this.totalUncompressedLength = in.readVLong(); int size = in.readVInt(); - offsets = BigArrays.newLongArray(size); + offsets = BigArrays.NON_RECYCLING_INSTANCE.newLongArray(size); for (int i = 0; i < size; i++) { offsets.set(i, in.readVLong()); } diff --git a/src/main/java/org/elasticsearch/common/lease/Releasables.java b/src/main/java/org/elasticsearch/common/lease/Releasables.java index b0b35c798ff..d60bb6b8772 100644 --- a/src/main/java/org/elasticsearch/common/lease/Releasables.java +++ b/src/main/java/org/elasticsearch/common/lease/Releasables.java @@ -35,7 +35,7 @@ public enum Releasables { throw new RuntimeException(t); } - private static void release(Iterable releasables, boolean ignoreException) { + private static void release(Iterable releasables, boolean ignoreException) { Throwable th = null; for (Releasable releasable : releasables) { if (releasable != null) { @@ -54,7 +54,7 @@ public enum Releasables { } /** Release the provided {@link Releasable}s. */ - public static void release(Iterable releasables) { + public static void release(Iterable releasables) { release(releasables, false); } diff --git a/src/main/java/org/elasticsearch/common/util/AbstractArray.java b/src/main/java/org/elasticsearch/common/util/AbstractArray.java index f075ca5ae2e..6c48fa7402f 100644 --- a/src/main/java/org/elasticsearch/common/util/AbstractArray.java +++ b/src/main/java/org/elasticsearch/common/util/AbstractArray.java @@ -19,17 +19,14 @@ package org.elasticsearch.common.util; -import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.lease.Releasable; abstract class AbstractArray implements Releasable { - public final PageCacheRecycler recycler; public final boolean clearOnResize; private boolean released = false; - AbstractArray(PageCacheRecycler recycler, boolean clearOnResize) { - this.recycler = recycler; + AbstractArray(boolean clearOnResize) { this.clearOnResize = clearOnResize; } diff --git a/src/main/java/org/elasticsearch/common/util/AbstractBigArray.java b/src/main/java/org/elasticsearch/common/util/AbstractBigArray.java index ccffd22ed69..5c9b7e78df1 100644 --- a/src/main/java/org/elasticsearch/common/util/AbstractBigArray.java +++ b/src/main/java/org/elasticsearch/common/util/AbstractBigArray.java @@ -32,6 +32,7 @@ import java.util.Arrays; /** Common implementation for array lists that slice data into fixed-size blocks. */ abstract class AbstractBigArray extends AbstractArray { + private final PageCacheRecycler recycler; private Recycler.V[] cache; private final int pageShift; @@ -39,7 +40,8 @@ abstract class AbstractBigArray extends AbstractArray { protected long size; protected AbstractBigArray(int pageSize, PageCacheRecycler recycler, boolean clearOnResize) { - super(recycler, clearOnResize); + super(clearOnResize); + this.recycler = 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); diff --git a/src/main/java/org/elasticsearch/common/util/AbstractHash.java b/src/main/java/org/elasticsearch/common/util/AbstractHash.java index da2ba8e4ac5..8b08b6c14f3 100644 --- a/src/main/java/org/elasticsearch/common/util/AbstractHash.java +++ b/src/main/java/org/elasticsearch/common/util/AbstractHash.java @@ -20,7 +20,6 @@ package org.elasticsearch.common.util; import com.google.common.base.Preconditions; -import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; @@ -34,14 +33,16 @@ abstract class AbstractHash implements Releasable { // collisions may result into worse lookup performance. static final float DEFAULT_MAX_LOAD_FACTOR = 0.6f; + final BigArrays bigArrays; final float maxLoadFactor; long size, maxSize; LongArray ids; long mask; - AbstractHash(long capacity, float maxLoadFactor, PageCacheRecycler recycler) { + AbstractHash(long capacity, float maxLoadFactor, BigArrays bigArrays) { Preconditions.checkArgument(capacity >= 0, "capacity must be >= 0"); Preconditions.checkArgument(maxLoadFactor > 0 && maxLoadFactor < 1, "maxLoadFactor must be > 0 and < 1"); + this.bigArrays = bigArrays; this.maxLoadFactor = maxLoadFactor; long buckets = 1L + (long) (capacity / maxLoadFactor); buckets = Math.max(1, Long.highestOneBit(buckets - 1) << 1); // next power of two @@ -49,7 +50,7 @@ abstract class AbstractHash implements Releasable { maxSize = (long) (buckets * maxLoadFactor); assert maxSize >= capacity; size = 0; - ids = BigArrays.newLongArray(buckets, recycler, true); + ids = bigArrays.newLongArray(buckets, true); mask = buckets - 1; } @@ -102,7 +103,7 @@ abstract class AbstractHash implements Releasable { final long newBuckets = buckets << 1; assert newBuckets == Long.highestOneBit(newBuckets) : newBuckets; // power of 2 resizeKeys(newBuckets); - ids = BigArrays.resize(ids, newBuckets); + ids = bigArrays.resize(ids, newBuckets); mask = newBuckets - 1; // First let's remap in-place: most data will be put in its final position directly for (long i = 0; i < buckets; ++i) { diff --git a/src/main/java/org/elasticsearch/common/util/BigArray.java b/src/main/java/org/elasticsearch/common/util/BigArray.java index 418e76b04e9..70b525e05aa 100644 --- a/src/main/java/org/elasticsearch/common/util/BigArray.java +++ b/src/main/java/org/elasticsearch/common/util/BigArray.java @@ -22,7 +22,7 @@ package org.elasticsearch.common.util; import org.elasticsearch.common.lease.Releasable; /** Base abstraction of an array. */ -interface BigArray extends Releasable { +public interface BigArray extends Releasable { /** Return the length of this array. */ public long size(); diff --git a/src/main/java/org/elasticsearch/common/util/BigArrays.java b/src/main/java/org/elasticsearch/common/util/BigArrays.java index b3b76272027..d797e4353ba 100644 --- a/src/main/java/org/elasticsearch/common/util/BigArrays.java +++ b/src/main/java/org/elasticsearch/common/util/BigArrays.java @@ -24,12 +24,17 @@ 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 org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; import java.util.Arrays; /** Utility class to work with arrays. */ -public enum BigArrays { - ; +public class BigArrays extends AbstractComponent { + + public static final BigArrays NON_RECYCLING_INSTANCE = new BigArrays(ImmutableSettings.EMPTY, null); /** Page size in bytes: 16KB */ public static final int PAGE_SIZE_IN_BYTES = 1 << 14; @@ -77,7 +82,7 @@ public enum BigArrays { private final byte[] array; ByteArrayWrapper(byte[] array, PageCacheRecycler recycler, boolean clearOnResize) { - super(recycler, clearOnResize); + super(clearOnResize); this.array = array; } @@ -121,7 +126,7 @@ public enum BigArrays { private final int[] array; IntArrayWrapper(int[] array, PageCacheRecycler recycler, boolean clearOnResize) { - super(recycler, clearOnResize); + super(clearOnResize); this.array = array; } @@ -157,7 +162,7 @@ public enum BigArrays { private final long[] array; LongArrayWrapper(long[] array, PageCacheRecycler recycler, boolean clearOnResize) { - super(recycler, clearOnResize); + super(clearOnResize); this.array = array; } @@ -199,7 +204,7 @@ public enum BigArrays { private final double[] array; DoubleArrayWrapper(double[] array, PageCacheRecycler recycler, boolean clearOnResize) { - super(recycler, clearOnResize); + super(clearOnResize); this.array = array; } @@ -242,7 +247,7 @@ public enum BigArrays { private final float[] array; FloatArrayWrapper(float[] array, PageCacheRecycler recycler, boolean clearOnResize) { - super(recycler, clearOnResize); + super(clearOnResize); this.array = array; } @@ -285,7 +290,7 @@ public enum BigArrays { private final Object[] array; ObjectArrayWrapper(Object[] array, PageCacheRecycler recycler) { - super(recycler, true); + super(true); this.array = array; } @@ -312,8 +317,20 @@ public enum BigArrays { } - /** Allocate a new {@link ByteArray} of the given capacity. */ - public static ByteArray newByteArray(long size, PageCacheRecycler recycler, boolean clearOnResize) { + private final PageCacheRecycler recycler; + + @Inject + public BigArrays(Settings settings, PageCacheRecycler recycler) { + super(settings); + this.recycler = recycler; + } + + /** + * Allocate a new {@link ByteArray}. + * @param size the initial length of the array + * @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 { @@ -321,27 +338,31 @@ public enum BigArrays { } } - /** Allocate a new {@link ByteArray} of the given capacity. */ - public static ByteArray newByteArray(long size) { - return newByteArray(size, null, true); + /** + * Allocate a new {@link ByteArray} initialized with zeros. + * @param size the initial length of the array + */ + public ByteArray newByteArray(long size) { + return newByteArray(size, true); } /** Resize the array to the exact provided size. */ - public static ByteArray resize(ByteArray array, long size) { + public ByteArray resize(ByteArray array, long size) { if (array instanceof BigByteArray) { ((BigByteArray) array).resize(size); return array; } else { AbstractArray arr = (AbstractArray) array; - final ByteArray newArray = newByteArray(size, arr.recycler, arr.clearOnResize); + 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.release(); return newArray; } } /** Grow an array to a size that is larger than minSize, preserving content, and potentially reusing part of the provided array. */ - public static ByteArray grow(ByteArray array, long minSize) { + public ByteArray grow(ByteArray array, long minSize) { if (minSize <= array.size()) { return array; } @@ -349,8 +370,12 @@ public enum BigArrays { return resize(array, newSize); } - /** Allocate a new {@link IntArray} of the given capacity. */ - public static IntArray newIntArray(long size, PageCacheRecycler recycler, boolean clearOnResize) { + /** + * Allocate a new {@link IntArray}. + * @param size the initial length of the array + * @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 { @@ -358,28 +383,32 @@ public enum BigArrays { } } - /** Allocate a new {@link IntArray} of the given capacity. */ - public static IntArray newIntArray(long size) { - return newIntArray(size, null, true); + /** + * Allocate a new {@link IntArray}. + * @param size the initial length of the array + */ + public IntArray newIntArray(long size) { + return newIntArray(size, true); } /** Resize the array to the exact provided size. */ - public static IntArray resize(IntArray array, long size) { + public IntArray resize(IntArray array, long size) { if (array instanceof BigIntArray) { ((BigIntArray) array).resize(size); return array; } else { AbstractArray arr = (AbstractArray) array; - final IntArray newArray = newIntArray(size, arr.recycler, arr.clearOnResize); + final IntArray newArray = newIntArray(size, arr.clearOnResize); for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) { newArray.set(i, array.get(i)); } + array.release(); return newArray; } } /** Grow an array to a size that is larger than minSize, preserving content, and potentially reusing part of the provided array. */ - public static IntArray grow(IntArray array, long minSize) { + public IntArray grow(IntArray array, long minSize) { if (minSize <= array.size()) { return array; } @@ -387,8 +416,12 @@ public enum BigArrays { return resize(array, newSize); } - /** Allocate a new {@link LongArray} of the given capacity. */ - public static LongArray newLongArray(long size, PageCacheRecycler recycler, boolean clearOnResize) { + /** + * Allocate a new {@link LongArray}. + * @param size the initial length of the array + * @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 { @@ -396,28 +429,32 @@ public enum BigArrays { } } - /** Allocate a new {@link LongArray} of the given capacity. */ - public static LongArray newLongArray(long size) { - return newLongArray(size, null, true); + /** + * Allocate a new {@link LongArray}. + * @param size the initial length of the array + */ + public LongArray newLongArray(long size) { + return newLongArray(size, true); } /** Resize the array to the exact provided size. */ - public static LongArray resize(LongArray array, long size) { + public LongArray resize(LongArray array, long size) { if (array instanceof BigLongArray) { ((BigLongArray) array).resize(size); return array; } else { AbstractArray arr = (AbstractArray) array; - final LongArray newArray = newLongArray(size, arr.recycler, arr.clearOnResize); + final LongArray newArray = newLongArray(size, arr.clearOnResize); for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) { newArray.set(i, array.get(i)); } + array.release(); return newArray; } } /** Grow an array to a size that is larger than minSize, preserving content, and potentially reusing part of the provided array. */ - public static LongArray grow(LongArray array, long minSize) { + public LongArray grow(LongArray array, long minSize) { if (minSize <= array.size()) { return array; } @@ -425,8 +462,12 @@ public enum BigArrays { return resize(array, newSize); } - /** Allocate a new {@link DoubleArray} of the given capacity. */ - public static DoubleArray newDoubleArray(long size, PageCacheRecycler recycler, boolean clearOnResize) { + /** + * Allocate a new {@link DoubleArray}. + * @param size the initial length of the array + * @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 { @@ -435,27 +476,28 @@ public enum BigArrays { } /** Allocate a new {@link DoubleArray} of the given capacity. */ - public static DoubleArray newDoubleArray(long size) { - return newDoubleArray(size, null, true); + public DoubleArray newDoubleArray(long size) { + return newDoubleArray(size, true); } /** Resize the array to the exact provided size. */ - public static DoubleArray resize(DoubleArray array, long size) { + public DoubleArray resize(DoubleArray array, long size) { if (array instanceof BigDoubleArray) { ((BigDoubleArray) array).resize(size); return array; } else { AbstractArray arr = (AbstractArray) array; - final DoubleArray newArray = newDoubleArray(size, arr.recycler, arr.clearOnResize); + final DoubleArray newArray = newDoubleArray(size, arr.clearOnResize); for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) { newArray.set(i, array.get(i)); } + array.release(); return newArray; } } /** Grow an array to a size that is larger than minSize, preserving content, and potentially reusing part of the provided array. */ - public static DoubleArray grow(DoubleArray array, long minSize) { + public DoubleArray grow(DoubleArray array, long minSize) { if (minSize <= array.size()) { return array; } @@ -463,8 +505,12 @@ public enum BigArrays { return resize(array, newSize); } - /** Allocate a new {@link FloatArray} of the given capacity. */ - public static FloatArray newFloatArray(long size, PageCacheRecycler recycler, boolean clearOnResize) { + /** + * Allocate a new {@link FloatArray}. + * @param size the initial length of the array + * @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 { @@ -473,18 +519,18 @@ public enum BigArrays { } /** Allocate a new {@link FloatArray} of the given capacity. */ - public static FloatArray newFloatArray(long size) { - return newFloatArray(size, null, true); + public FloatArray newFloatArray(long size) { + return newFloatArray(size, true); } /** Resize the array to the exact provided size. */ - public static FloatArray resize(FloatArray array, long size) { + public FloatArray resize(FloatArray array, long size) { if (array instanceof BigFloatArray) { ((BigFloatArray) array).resize(size); return array; } else { AbstractArray arr = (AbstractArray) array; - final FloatArray newArray = newFloatArray(size, arr.recycler, arr.clearOnResize); + 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)); } @@ -493,7 +539,7 @@ public enum BigArrays { } /** Grow an array to a size that is larger than minSize, preserving content, and potentially reusing part of the provided array. */ - public static FloatArray grow(FloatArray array, long minSize) { + public FloatArray grow(FloatArray array, long minSize) { if (minSize <= array.size()) { return array; } @@ -501,8 +547,11 @@ public enum BigArrays { return resize(array, newSize); } - /** Allocate a new {@link ObjectArray} of the given capacity. */ - public static ObjectArray newObjectArray(long size, PageCacheRecycler recycler) { + /** + * Allocate a new {@link ObjectArray}. + * @param size the initial length of the array + */ + public ObjectArray newObjectArray(long size) { if (size <= OBJECT_PAGE_SIZE) { return new ObjectArrayWrapper(new Object[(int) size], recycler); } else { @@ -510,27 +559,23 @@ public enum BigArrays { } } - /** Allocate a new {@link ObjectArray} of the given capacity. */ - public static ObjectArray newObjectArray(long size) { - return newObjectArray(size, null); - } - /** Resize the array to the exact provided size. */ - public static ObjectArray resize(ObjectArray array, long size) { + public ObjectArray resize(ObjectArray array, long size) { if (array instanceof BigObjectArray) { ((BigObjectArray) array).resize(size); return array; } else { - final ObjectArray newArray = newObjectArray(size, ((AbstractArray) array).recycler); + final ObjectArray newArray = newObjectArray(size); for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) { newArray.set(i, array.get(i)); } + array.release(); return newArray; } } /** Grow an array to a size that is larger than minSize, preserving content, and potentially reusing part of the provided array. */ - public static ObjectArray grow(ObjectArray array, long minSize) { + public ObjectArray grow(ObjectArray array, long minSize) { if (minSize <= array.size()) { return array; } diff --git a/src/main/java/org/elasticsearch/common/util/BigArraysModule.java b/src/main/java/org/elasticsearch/common/util/BigArraysModule.java new file mode 100644 index 00000000000..b2fc2b06b7b --- /dev/null +++ b/src/main/java/org/elasticsearch/common/util/BigArraysModule.java @@ -0,0 +1,50 @@ +/* + * 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 com.google.common.collect.ImmutableList; +import org.elasticsearch.common.inject.AbstractModule; +import org.elasticsearch.common.inject.Module; +import org.elasticsearch.common.inject.SpawnModules; +import org.elasticsearch.common.settings.Settings; + +import static org.elasticsearch.common.inject.Modules.createModule; + +/** + */ +public class BigArraysModule extends AbstractModule implements SpawnModules { + + public static final String IMPL = "common.util.big_arrays_impl"; + + private final Settings settings; + + public BigArraysModule(Settings settings) { + this.settings = settings; + } + + @Override + protected void configure() { + } + + @Override + public Iterable spawnModules() { + return ImmutableList.of(createModule(settings.getAsClass(IMPL, DefaultBigArraysModule.class), settings)); + } +} diff --git a/src/main/java/org/elasticsearch/common/util/BytesRefHash.java b/src/main/java/org/elasticsearch/common/util/BytesRefHash.java index 5ed15cc432e..119b0be9b0d 100644 --- a/src/main/java/org/elasticsearch/common/util/BytesRefHash.java +++ b/src/main/java/org/elasticsearch/common/util/BytesRefHash.java @@ -21,7 +21,6 @@ package org.elasticsearch.common.util; import com.carrotsearch.hppc.hash.MurmurHash3; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.lease.Releasables; /** @@ -38,16 +37,17 @@ public final class BytesRefHash extends AbstractHash { private final BytesRef spare; // Constructor with configurable capacity and default maximum load factor. - public BytesRefHash(long capacity, PageCacheRecycler recycler) { - this(capacity, DEFAULT_MAX_LOAD_FACTOR, recycler); + public BytesRefHash(long capacity, BigArrays bigArrays) { + this(capacity, DEFAULT_MAX_LOAD_FACTOR, bigArrays); } //Constructor with configurable capacity and load factor. - public BytesRefHash(long capacity, float maxLoadFactor, PageCacheRecycler recycler) { - super(capacity, maxLoadFactor, recycler); - startOffsets = BigArrays.newLongArray(capacity + 1, recycler, false); - bytes = BigArrays.newByteArray(capacity * 3, recycler, false); - hashes = BigArrays.newIntArray(capacity, recycler, false); + public BytesRefHash(long capacity, float maxLoadFactor, BigArrays bigArrays) { + super(capacity, maxLoadFactor, bigArrays); + startOffsets = bigArrays.newLongArray(capacity + 1, false); + startOffsets.set(0, 0); + bytes = bigArrays.newByteArray(capacity * 3, false); + hashes = bigArrays.newIntArray(capacity, false); spare = new BytesRef(); } @@ -106,11 +106,11 @@ public final class BytesRefHash extends AbstractHash { private void append(long id, BytesRef key, int code) { assert size == id; final long startOffset = startOffsets.get(size); - bytes = BigArrays.grow(bytes, startOffset + key.length); + bytes = bigArrays.grow(bytes, startOffset + key.length); bytes.set(startOffset, key.bytes, key.offset, key.length); - startOffsets = BigArrays.grow(startOffsets, size + 2); + startOffsets = bigArrays.grow(startOffsets, size + 2); startOffsets.set(size + 1, startOffset + key.length); - hashes = BigArrays.grow(hashes, id + 1); + hashes = bigArrays.grow(hashes, id + 1); hashes.set(id, code); } diff --git a/src/main/java/org/elasticsearch/common/util/DefaultBigArraysModule.java b/src/main/java/org/elasticsearch/common/util/DefaultBigArraysModule.java new file mode 100644 index 00000000000..ce73a51225a --- /dev/null +++ b/src/main/java/org/elasticsearch/common/util/DefaultBigArraysModule.java @@ -0,0 +1,39 @@ +/* + * 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.elasticsearch.common.inject.AbstractModule; +import org.elasticsearch.common.settings.Settings; + +/** + */ +public class DefaultBigArraysModule extends AbstractModule { + + private final Settings settings; + + public DefaultBigArraysModule(Settings settings) { + this.settings = settings; + } + + @Override + protected void configure() { + bind(BigArrays.class).asEagerSingleton(); + } +} diff --git a/src/main/java/org/elasticsearch/common/util/LongHash.java b/src/main/java/org/elasticsearch/common/util/LongHash.java index f235cfd262a..f77053d96ed 100644 --- a/src/main/java/org/elasticsearch/common/util/LongHash.java +++ b/src/main/java/org/elasticsearch/common/util/LongHash.java @@ -20,7 +20,6 @@ package org.elasticsearch.common.util; import com.carrotsearch.hppc.hash.MurmurHash3; -import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.lease.Releasables; /** @@ -35,14 +34,14 @@ public final class LongHash extends AbstractHash { private LongArray keys; // Constructor with configurable capacity and default maximum load factor. - public LongHash(long capacity, PageCacheRecycler recycler) { - this(capacity, DEFAULT_MAX_LOAD_FACTOR, recycler); + public LongHash(long capacity, BigArrays bigArrays) { + this(capacity, DEFAULT_MAX_LOAD_FACTOR, bigArrays); } //Constructor with configurable capacity and load factor. - public LongHash(long capacity, float maxLoadFactor, PageCacheRecycler recycler) { - super(capacity, maxLoadFactor, recycler); - keys = BigArrays.newLongArray(capacity(), recycler, false); + public LongHash(long capacity, float maxLoadFactor, BigArrays bigArrays) { + super(capacity, maxLoadFactor, bigArrays); + keys = bigArrays.newLongArray(capacity(), false); } private static long hash(long value) { @@ -116,7 +115,7 @@ public final class LongHash extends AbstractHash { @Override protected void resizeKeys(long capacity) { - keys = BigArrays.resize(keys, capacity); + keys = bigArrays.resize(keys, capacity); } @Override diff --git a/src/main/java/org/elasticsearch/index/fielddata/plain/FSTBytesAtomicFieldData.java b/src/main/java/org/elasticsearch/index/fielddata/plain/FSTBytesAtomicFieldData.java index bec4e9335dd..bb4be792309 100644 --- a/src/main/java/org/elasticsearch/index/fielddata/plain/FSTBytesAtomicFieldData.java +++ b/src/main/java/org/elasticsearch/index/fielddata/plain/FSTBytesAtomicFieldData.java @@ -95,7 +95,7 @@ public class FSTBytesAtomicFieldData implements AtomicFieldData.WithOrdinals fstEnum = new BytesRefFSTEnum(fst); - IntArray hashes = BigArrays.newIntArray(ordinals.getMaxOrd()); + IntArray hashes = BigArrays.NON_RECYCLING_INSTANCE.newIntArray(ordinals.getMaxOrd()); // we don't store an ord 0 in the FST since we could have an empty string in there and FST don't support // empty strings twice. ie. them merge fails for long output. hashes.set(0, new BytesRef().hashCode()); diff --git a/src/main/java/org/elasticsearch/index/fielddata/plain/PagedBytesAtomicFieldData.java b/src/main/java/org/elasticsearch/index/fielddata/plain/PagedBytesAtomicFieldData.java index 46e9bc561a7..3664f950871 100644 --- a/src/main/java/org/elasticsearch/index/fielddata/plain/PagedBytesAtomicFieldData.java +++ b/src/main/java/org/elasticsearch/index/fielddata/plain/PagedBytesAtomicFieldData.java @@ -94,7 +94,7 @@ public class PagedBytesAtomicFieldData implements AtomicFieldData.WithOrdinals= 0) { - scores = BigArrays.grow(scores, index + 1); + scores = bigArrays.grow(scores, index + 1); scores.set(index, scorer.score()); } } @@ -242,10 +244,11 @@ public class ParentQuery extends Query { Ordinals.Docs ordinals = bytesValues.ordinals(); final int maxOrd = (int) ordinals.getMaxOrd(); + final BigArrays bigArrays = searchContext.bigArrays(); if (parentIdsIndexCache == null) { - parentIdsIndexCache = BigArrays.newLongArray(BigArrays.overSize(maxOrd), searchContext.pageCacheRecycler(), false); + parentIdsIndexCache = bigArrays.newLongArray(BigArrays.overSize(maxOrd), false); } else if (parentIdsIndexCache.size() < maxOrd) { - parentIdsIndexCache = BigArrays.grow(parentIdsIndexCache, maxOrd); + parentIdsIndexCache = bigArrays.grow(parentIdsIndexCache, maxOrd); } parentIdsIndexCache.fill(0, maxOrd, -1L); if (seenOrdinalsCache == null || seenOrdinalsCache.length() < maxOrd) { @@ -258,7 +261,7 @@ public class ParentQuery extends Query { @Override public boolean release() throws ElasticsearchException { - Releasables.release(parentIds, scores); + Releasables.release(parentIds, scores, parentIdsIndexCache); return true; } } diff --git a/src/main/java/org/elasticsearch/node/internal/InternalNode.java b/src/main/java/org/elasticsearch/node/internal/InternalNode.java index 49fc205c34c..ccb7a91e4c4 100644 --- a/src/main/java/org/elasticsearch/node/internal/InternalNode.java +++ b/src/main/java/org/elasticsearch/node/internal/InternalNode.java @@ -51,6 +51,7 @@ import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsModule; +import org.elasticsearch.common.util.BigArraysModule; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoveryService; @@ -151,6 +152,7 @@ public final class InternalNode implements Node { modules.add(new Version.Module(version)); modules.add(new CacheRecyclerModule(settings)); modules.add(new PageCacheRecyclerModule(settings)); + modules.add(new BigArraysModule(settings)); modules.add(new PluginsModule(settings, pluginsService)); modules.add(new SettingsModule(settings)); modules.add(new NodeModule(this)); diff --git a/src/main/java/org/elasticsearch/percolator/PercolateContext.java b/src/main/java/org/elasticsearch/percolator/PercolateContext.java index 846b8bd26d1..eefde3c41d5 100644 --- a/src/main/java/org/elasticsearch/percolator/PercolateContext.java +++ b/src/main/java/org/elasticsearch/percolator/PercolateContext.java @@ -34,6 +34,7 @@ import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lucene.HashedBytesRef; import org.elasticsearch.common.text.StringText; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.docset.DocSetCache; import org.elasticsearch.index.cache.filter.FilterCache; @@ -93,6 +94,7 @@ public class PercolateContext extends SearchContext { private final IndexShard indexShard; private final CacheRecycler cacheRecycler; private final PageCacheRecycler pageCacheRecycler; + private final BigArrays bigArrays; private final ScriptService scriptService; private final ConcurrentMap percolateQueries; private String[] types; @@ -115,7 +117,7 @@ public class PercolateContext extends SearchContext { public PercolateContext(PercolateShardRequest request, SearchShardTarget searchShardTarget, IndexShard indexShard, IndexService indexService, CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler, - ScriptService scriptService) { + BigArrays bigArrays, ScriptService scriptService) { this.request = request; this.indexShard = indexShard; this.indexService = indexService; @@ -125,6 +127,7 @@ public class PercolateContext extends SearchContext { this.types = new String[]{request.documentType()}; this.cacheRecycler = cacheRecycler; this.pageCacheRecycler = pageCacheRecycler; + this.bigArrays = bigArrays; this.querySearchResult = new QuerySearchResult(0, searchShardTarget); this.engineSearcher = indexShard.acquireSearcher("percolate"); this.searcher = new ContextIndexSearcher(this, engineSearcher); @@ -477,6 +480,11 @@ public class PercolateContext extends SearchContext { return pageCacheRecycler; } + @Override + public BigArrays bigArrays() { + return bigArrays; + } + @Override public FilterCache filterCache() { return indexService.cache().filter(); diff --git a/src/main/java/org/elasticsearch/percolator/PercolatorService.java b/src/main/java/org/elasticsearch/percolator/PercolatorService.java index 8d44eade8ab..b2438df19cb 100644 --- a/src/main/java/org/elasticsearch/percolator/PercolatorService.java +++ b/src/main/java/org/elasticsearch/percolator/PercolatorService.java @@ -47,6 +47,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.text.BytesText; import org.elasticsearch.common.text.StringText; import org.elasticsearch.common.text.Text; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; @@ -64,7 +65,10 @@ import org.elasticsearch.index.query.ParsedQuery; import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.percolator.QueryCollector.*; +import org.elasticsearch.percolator.QueryCollector.Count; +import org.elasticsearch.percolator.QueryCollector.Match; +import org.elasticsearch.percolator.QueryCollector.MatchAndScore; +import org.elasticsearch.percolator.QueryCollector.MatchAndSort; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.SearchParseElement; import org.elasticsearch.search.SearchShardTarget; @@ -98,6 +102,7 @@ public class PercolatorService extends AbstractComponent { private final ByteObjectOpenHashMap percolatorTypes; private final CacheRecycler cacheRecycler; private final PageCacheRecycler pageCacheRecycler; + private final BigArrays bigArrays; private final ClusterService clusterService; private final PercolatorIndex single; @@ -110,13 +115,15 @@ public class PercolatorService extends AbstractComponent { private final ScriptService scriptService; @Inject - public PercolatorService(Settings settings, IndicesService indicesService, CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler, + public PercolatorService(Settings settings, IndicesService indicesService, CacheRecycler cacheRecycler, + PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, HighlightPhase highlightPhase, ClusterService clusterService, FacetPhase facetPhase, AggregationPhase aggregationPhase, ScriptService scriptService) { super(settings); this.indicesService = indicesService; this.cacheRecycler = cacheRecycler; this.pageCacheRecycler = pageCacheRecycler; + this.bigArrays = bigArrays; this.clusterService = clusterService; this.highlightPhase = highlightPhase; this.facetPhase = facetPhase; @@ -152,7 +159,7 @@ public class PercolatorService extends AbstractComponent { SearchShardTarget searchShardTarget = new SearchShardTarget(clusterService.localNode().id(), request.index(), request.shardId()); final PercolateContext context = new PercolateContext( - request, searchShardTarget, indexShard, percolateIndexService, cacheRecycler, pageCacheRecycler, scriptService + request, searchShardTarget, indexShard, percolateIndexService, cacheRecycler, pageCacheRecycler, bigArrays, scriptService ); try { ParsedDocument parsedDocument = parseRequest(percolateIndexService, request, context); diff --git a/src/main/java/org/elasticsearch/search/SearchService.java b/src/main/java/org/elasticsearch/search/SearchService.java index 9a6fbe81574..579b09e9788 100644 --- a/src/main/java/org/elasticsearch/search/SearchService.java +++ b/src/main/java/org/elasticsearch/search/SearchService.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.ConcurrentMapLong; import org.elasticsearch.common.xcontent.XContentFactory; @@ -108,6 +109,8 @@ public class SearchService extends AbstractLifecycleComponent { private final PageCacheRecycler pageCacheRecycler; + private final BigArrays bigArrays; + private final DfsPhase dfsPhase; private final QueryPhase queryPhase; @@ -126,7 +129,7 @@ public class SearchService extends AbstractLifecycleComponent { @Inject public SearchService(Settings settings, ClusterService clusterService, IndicesService indicesService, IndicesLifecycle indicesLifecycle, IndicesWarmer indicesWarmer, ThreadPool threadPool, - ScriptService scriptService, CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler, DfsPhase dfsPhase, QueryPhase queryPhase, FetchPhase fetchPhase) { + ScriptService scriptService, CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, DfsPhase dfsPhase, QueryPhase queryPhase, FetchPhase fetchPhase) { super(settings); this.threadPool = threadPool; this.clusterService = clusterService; @@ -135,6 +138,7 @@ public class SearchService extends AbstractLifecycleComponent { this.scriptService = scriptService; this.cacheRecycler = cacheRecycler; this.pageCacheRecycler = pageCacheRecycler; + this.bigArrays = bigArrays; this.dfsPhase = dfsPhase; this.queryPhase = queryPhase; this.fetchPhase = fetchPhase; @@ -490,7 +494,7 @@ public class SearchService extends AbstractLifecycleComponent { SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().id(), request.index(), request.shardId()); Engine.Searcher engineSearcher = searcher == null ? indexShard.acquireSearcher("search") : searcher; - SearchContext context = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher, indexService, indexShard, scriptService, cacheRecycler, pageCacheRecycler); + SearchContext context = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher, indexService, indexShard, scriptService, cacheRecycler, pageCacheRecycler, bigArrays); SearchContext.setCurrent(context); try { context.scroll(request.scroll()); diff --git a/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java b/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java index b713da1ad4e..d5cf64d88d8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java @@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.ReaderContextAware; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext; @@ -50,6 +51,7 @@ public abstract class Aggregator implements Releasable, ReaderContextAware { protected final String name; protected final Aggregator parent; protected final AggregationContext context; + protected final BigArrays bigArrays; protected final int depth; protected final long estimatedBucketCount; @@ -72,6 +74,7 @@ public abstract class Aggregator implements Releasable, ReaderContextAware { this.parent = parent; this.estimatedBucketCount = estimatedBucketsCount; this.context = context; + this.bigArrays = context.bigArrays(); this.depth = parent == null ? 0 : 1 + parent.depth(); this.bucketAggregationMode = bucketAggregationMode; assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead"; diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java index 047de1f1a13..9135343516f 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java @@ -18,15 +18,18 @@ */ package org.elasticsearch.search.aggregations; +import com.google.common.collect.Iterables; +import com.google.common.collect.UnmodifiableIterator; import org.apache.lucene.index.AtomicReaderContext; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.search.aggregations.Aggregator.BucketAggregationMode; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; import java.util.List; /** @@ -77,7 +80,7 @@ public class AggregatorFactories { // The estimated count is just an estimation and we can't rely on how it's estimated (that is, an // estimation of 0 should not imply that we'll end up without any buckets) long arraySize = estimatedBucketsCount > 0 ? estimatedBucketsCount : 1; - aggregators = BigArrays.newObjectArray(arraySize , context.pageCacheRecycler()); + aggregators = bigArrays.newObjectArray(arraySize); aggregators.set(0, first); for (long i = 1; i < arraySize; ++i) { aggregators.set(i, createAndRegisterContextAware(parent.context(), factory, parent, estimatedBucketsCount)); @@ -101,7 +104,7 @@ public class AggregatorFactories { @Override public void collect(int doc, long owningBucketOrdinal) throws IOException { - aggregators = BigArrays.grow(aggregators, owningBucketOrdinal + 1); + aggregators = bigArrays.grow(aggregators, owningBucketOrdinal + 1); Aggregator aggregator = aggregators.get(owningBucketOrdinal); if (aggregator == null) { aggregator = createAndRegisterContextAware(parent.context(), factory, parent, estimatedBucketsCount); @@ -126,7 +129,29 @@ public class AggregatorFactories { @Override public void doRelease() { - Releasables.release(aggregators); + final Iterable aggregatorsIter = new Iterable() { + + @Override + public Iterator iterator() { + return new UnmodifiableIterator() { + + long i = 0; + + @Override + public boolean hasNext() { + return i < aggregators.size(); + } + + @Override + public Aggregator next() { + return aggregators.get(i++); + } + + }; + } + + }; + Releasables.release(Iterables.concat(aggregatorsIter, Collections.singleton(aggregators))); } }; } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java index 9d8a1544352..8c1558077aa 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.LongArray; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; @@ -44,7 +43,7 @@ public abstract class BucketsAggregator extends Aggregator { public BucketsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent) { super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent); - docCounts = BigArrays.newLongArray(estimatedBucketsCount, context.pageCacheRecycler(), true); + docCounts = bigArrays.newLongArray(estimatedBucketsCount, true); List collectables = new ArrayList(subAggregators.length); for (int i = 0; i < subAggregators.length; i++) { if (subAggregators[i].shouldCollect()) { @@ -58,7 +57,7 @@ public abstract class BucketsAggregator extends Aggregator { * Utility method to collect the given doc in the given bucket (identified by the bucket ordinal) */ protected final void collectBucket(int doc, long bucketOrd) throws IOException { - docCounts = BigArrays.grow(docCounts, bucketOrd + 1); + docCounts = bigArrays.grow(docCounts, bucketOrd + 1); docCounts.increment(bucketOrd, 1); for (int i = 0; i < collectableSugAggregators.length; i++) { collectableSugAggregators[i].collect(doc, bucketOrd); @@ -78,7 +77,7 @@ public abstract class BucketsAggregator extends Aggregator { * Utility method to increment the doc counts of the given bucket (identified by the bucket ordinal) */ protected final void incrementBucketDocCount(int inc, long bucketOrd) throws IOException { - docCounts = BigArrays.grow(docCounts, bucketOrd + 1); + docCounts = bigArrays.grow(docCounts, bucketOrd + 1); docCounts.increment(bucketOrd, inc); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java index e5f70b3f681..f428adbf6fa 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java @@ -54,7 +54,7 @@ public class GeoHashGridAggregator extends BucketsAggregator { this.valuesSource = valuesSource; this.requiredSize = requiredSize; this.shardSize = shardSize; - bucketOrds = new LongHash(INITIAL_CAPACITY,aggregationContext.pageCacheRecycler()); + bucketOrds = new LongHash(INITIAL_CAPACITY, aggregationContext.bigArrays()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java index 55ca02073c2..5544c21c673 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java @@ -72,7 +72,7 @@ public class HistogramAggregator extends BucketsAggregator { this.minDocCount = minDocCount; this.histogramFactory = histogramFactory; - bucketOrds = new LongHash(initialCapacity, aggregationContext.pageCacheRecycler()); + bucketOrds = new LongHash(initialCapacity, aggregationContext.bigArrays()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java index 16330da1d83..1f0787cc69a 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java @@ -54,7 +54,7 @@ public class DoubleTermsAggregator extends BucketsAggregator { this.requiredSize = requiredSize; this.shardSize = shardSize; this.minDocCount = minDocCount; - bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.pageCacheRecycler()); + bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java index faacc467d77..798f4e3c8fc 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java @@ -54,7 +54,7 @@ public class LongTermsAggregator extends BucketsAggregator { this.requiredSize = requiredSize; this.shardSize = shardSize; this.minDocCount = minDocCount; - bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.pageCacheRecycler()); + bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java index 5dd01a3e2d9..74f1f5a65fe 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java @@ -68,7 +68,7 @@ public class StringTermsAggregator extends BucketsAggregator { this.shardSize = shardSize; this.minDocCount = minDocCount; this.includeExclude = includeExclude; - bucketOrds = new BytesRefHash(estimatedBucketCount, aggregationContext.pageCacheRecycler()); + bucketOrds = new BytesRefHash(estimatedBucketCount, aggregationContext.bigArrays()); } @Override @@ -278,7 +278,7 @@ public class StringTermsAggregator extends BucketsAggregator { if (ordinalToBucket != null) { ordinalToBucket.release(); } - ordinalToBucket = BigArrays.newLongArray(BigArrays.overSize(maxOrd), context().pageCacheRecycler(), false); + ordinalToBucket = context().bigArrays().newLongArray(BigArrays.overSize(maxOrd), false); } ordinalToBucket.fill(0, maxOrd, -1L); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregator.java index e9d235dc5e8..1c52dfb2875 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregator.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.metrics.avg; import org.apache.lucene.index.AtomicReaderContext; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.common.util.LongArray; import org.elasticsearch.index.fielddata.DoubleValues; @@ -50,8 +49,8 @@ public class AvgAggregator extends MetricsAggregator.SingleValue { this.valuesSource = valuesSource; if (valuesSource != null) { final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - counts = BigArrays.newLongArray(initialSize, context.pageCacheRecycler(), true); - sums = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), true); + counts = bigArrays.newLongArray(initialSize, true); + sums = bigArrays.newDoubleArray(initialSize, true); } } @@ -67,8 +66,8 @@ public class AvgAggregator extends MetricsAggregator.SingleValue { @Override public void collect(int doc, long owningBucketOrdinal) throws IOException { - counts = BigArrays.grow(counts, owningBucketOrdinal + 1); - sums = BigArrays.grow(sums, owningBucketOrdinal + 1); + counts = bigArrays.grow(counts, owningBucketOrdinal + 1); + sums = bigArrays.grow(sums, owningBucketOrdinal + 1); final int valueCount = values.setDocument(doc); counts.increment(owningBucketOrdinal, valueCount); diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java index afa6356f19b..3a1e8e43852 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.metrics.max; import org.apache.lucene.index.AtomicReaderContext; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.index.fielddata.DoubleValues; import org.elasticsearch.search.aggregations.Aggregator; @@ -48,7 +47,7 @@ public class MaxAggregator extends MetricsAggregator.SingleValue { this.valuesSource = valuesSource; if (valuesSource != null) { final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - maxes = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), false); + maxes = bigArrays.newDoubleArray(initialSize, false); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); } } @@ -67,7 +66,7 @@ public class MaxAggregator extends MetricsAggregator.SingleValue { public void collect(int doc, long owningBucketOrdinal) throws IOException { if (owningBucketOrdinal >= maxes.size()) { long from = maxes.size(); - maxes = BigArrays.grow(maxes, owningBucketOrdinal + 1); + maxes = bigArrays.grow(maxes, owningBucketOrdinal + 1); maxes.fill(from, maxes.size(), Double.NEGATIVE_INFINITY); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java index 4901cafe501..3ef01ccc833 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.metrics.min; import org.apache.lucene.index.AtomicReaderContext; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.index.fielddata.DoubleValues; import org.elasticsearch.search.aggregations.Aggregator; @@ -48,7 +47,7 @@ public class MinAggregator extends MetricsAggregator.SingleValue { this.valuesSource = valuesSource; if (valuesSource != null) { final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - mins = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), false); + mins = bigArrays.newDoubleArray(initialSize, false); mins.fill(0, mins.size(), Double.POSITIVE_INFINITY); } } @@ -71,7 +70,7 @@ public class MinAggregator extends MetricsAggregator.SingleValue { if (owningBucketOrdinal >= mins.size()) { long from = mins.size(); - mins = BigArrays.grow(mins, owningBucketOrdinal + 1); + mins = bigArrays.grow(mins, owningBucketOrdinal + 1); mins.fill(from, mins.size(), Double.POSITIVE_INFINITY); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java index d0b1fb21e6d..239d08d1e85 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java @@ -53,11 +53,11 @@ public class StatsAggegator extends MetricsAggregator.MultiValue { this.valuesSource = valuesSource; if (valuesSource != null) { final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - counts = BigArrays.newLongArray(initialSize, context.pageCacheRecycler(), true); - sums = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), true); - mins = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), false); + counts = bigArrays.newLongArray(initialSize, true); + sums = bigArrays.newDoubleArray(initialSize, true); + mins = bigArrays.newDoubleArray(initialSize, false); mins.fill(0, mins.size(), Double.POSITIVE_INFINITY); - maxes = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), false); + maxes = bigArrays.newDoubleArray(initialSize, false); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); } } @@ -77,10 +77,10 @@ public class StatsAggegator extends MetricsAggregator.MultiValue { if (owningBucketOrdinal >= counts.size()) { final long from = counts.size(); final long overSize = BigArrays.overSize(owningBucketOrdinal + 1); - counts = BigArrays.resize(counts, overSize); - sums = BigArrays.resize(sums, overSize); - mins = BigArrays.resize(mins, overSize); - maxes = BigArrays.resize(maxes, overSize); + counts = bigArrays.resize(counts, overSize); + sums = bigArrays.resize(sums, overSize); + mins = bigArrays.resize(mins, overSize); + maxes = bigArrays.resize(maxes, overSize); mins.fill(from, overSize, Double.POSITIVE_INFINITY); maxes.fill(from, overSize, Double.NEGATIVE_INFINITY); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java index 7d405ab287a..e593b055234 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java @@ -54,13 +54,13 @@ public class ExtendedStatsAggregator extends MetricsAggregator.MultiValue { this.valuesSource = valuesSource; if (valuesSource != null) { final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - counts = BigArrays.newLongArray(initialSize, context.pageCacheRecycler(), true); - sums = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), true); - mins = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), false); + counts = bigArrays.newLongArray(initialSize, true); + sums = bigArrays.newDoubleArray(initialSize, true); + mins = bigArrays.newDoubleArray(initialSize, false); mins.fill(0, mins.size(), Double.POSITIVE_INFINITY); - maxes = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), false); + maxes = bigArrays.newDoubleArray(initialSize, false); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); - sumOfSqrs = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), true); + sumOfSqrs = bigArrays.newDoubleArray(initialSize, true); } } @@ -79,11 +79,11 @@ public class ExtendedStatsAggregator extends MetricsAggregator.MultiValue { if (owningBucketOrdinal >= counts.size()) { final long from = counts.size(); final long overSize = BigArrays.overSize(owningBucketOrdinal + 1); - counts = BigArrays.resize(counts, overSize); - sums = BigArrays.resize(sums, overSize); - mins = BigArrays.resize(mins, overSize); - maxes = BigArrays.resize(maxes, overSize); - sumOfSqrs = BigArrays.resize(sumOfSqrs, overSize); + counts = bigArrays.resize(counts, overSize); + sums = bigArrays.resize(sums, overSize); + mins = bigArrays.resize(mins, overSize); + maxes = bigArrays.resize(maxes, overSize); + sumOfSqrs = bigArrays.resize(sumOfSqrs, overSize); mins.fill(from, overSize, Double.POSITIVE_INFINITY); maxes.fill(from, overSize, Double.NEGATIVE_INFINITY); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java index 1f037b75e52..21c67a8c0fb 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.metrics.sum; import org.apache.lucene.index.AtomicReaderContext; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.DoubleArray; import org.elasticsearch.index.fielddata.DoubleValues; import org.elasticsearch.search.aggregations.Aggregator; @@ -48,7 +47,7 @@ public class SumAggregator extends MetricsAggregator.SingleValue { this.valuesSource = valuesSource; if (valuesSource != null) { final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - sums = BigArrays.newDoubleArray(initialSize, context.pageCacheRecycler(), true); + sums = bigArrays.newDoubleArray(initialSize, true); } } @@ -64,7 +63,7 @@ public class SumAggregator extends MetricsAggregator.SingleValue { @Override public void collect(int doc, long owningBucketOrdinal) throws IOException { - sums = BigArrays.grow(sums, owningBucketOrdinal + 1); + sums = bigArrays.grow(sums, owningBucketOrdinal + 1); final int valuesCount = values.setDocument(doc); double sum = 0; diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java index f0d31f11f78..60d11fdd7a8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.metrics.valuecount; import org.apache.lucene.index.AtomicReaderContext; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.LongArray; import org.elasticsearch.index.fielddata.BytesValues; import org.elasticsearch.search.aggregations.Aggregator; @@ -53,7 +52,7 @@ public class ValueCountAggregator extends MetricsAggregator.SingleValue { if (valuesSource != null) { // expectedBucketsCount == 0 means it's a top level bucket final long initialSize = expectedBucketsCount < 2 ? 1 : expectedBucketsCount; - counts = BigArrays.newLongArray(initialSize, context.pageCacheRecycler(), true); + counts = bigArrays.newLongArray(initialSize, true); } } @@ -69,7 +68,7 @@ public class ValueCountAggregator extends MetricsAggregator.SingleValue { @Override public void collect(int doc, long owningBucketOrdinal) throws IOException { - counts = BigArrays.grow(counts, owningBucketOrdinal + 1); + counts = bigArrays.grow(counts, owningBucketOrdinal + 1); counts.increment(owningBucketOrdinal, values.setDocument(doc)); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java b/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java index d5da18a0e2a..78829722e3c 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java @@ -27,6 +27,7 @@ import org.elasticsearch.cache.recycler.CacheRecycler; import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.lucene.ReaderContextAware; import org.elasticsearch.common.lucene.ScorerAware; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexGeoPointFieldData; import org.elasticsearch.index.fielddata.IndexNumericFieldData; @@ -71,6 +72,10 @@ public class AggregationContext implements ReaderContextAware, ScorerAware { return searchContext.pageCacheRecycler(); } + public BigArrays bigArrays() { + return searchContext.bigArrays(); + } + public AtomicReaderContext currentReader() { return reader; } diff --git a/src/main/java/org/elasticsearch/search/facet/terms/strings/TermsStringOrdinalsFacetExecutor.java b/src/main/java/org/elasticsearch/search/facet/terms/strings/TermsStringOrdinalsFacetExecutor.java index c89dfca9088..919fd434628 100644 --- a/src/main/java/org/elasticsearch/search/facet/terms/strings/TermsStringOrdinalsFacetExecutor.java +++ b/src/main/java/org/elasticsearch/search/facet/terms/strings/TermsStringOrdinalsFacetExecutor.java @@ -27,6 +27,8 @@ import org.apache.lucene.util.PriorityQueue; import org.apache.lucene.util.UnicodeUtil; import org.elasticsearch.cache.recycler.CacheRecycler; import org.elasticsearch.common.collect.BoundedTreeSet; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.IntArray; import org.elasticsearch.index.fielddata.BytesValues; @@ -53,6 +55,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor { private final IndexFieldData.WithOrdinals indexFieldData; final CacheRecycler cacheRecycler; + final BigArrays bigArrays; private final TermsFacet.ComparatorType comparatorType; private final int size; private final int shardSize; @@ -88,6 +91,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor { } this.cacheRecycler = context.cacheRecycler(); + this.bigArrays = context.bigArrays(); this.aggregators = new ArrayList(context.searcher().getIndexReader().leaves().size()); } @@ -147,6 +151,8 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor { list[i] = (InternalStringTermsFacet.TermEntry) ordered.pop(); } + Releasables.release(aggregators); + return new InternalStringTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total); } @@ -183,6 +189,8 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor { } } + Releasables.release(aggregators); + return new InternalStringTermsFacet(facetName, comparatorType, size, ordered, missing, total); } @@ -227,6 +235,8 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor { // if we have values for this one, add it if (current.values.ordinals().getNumOrds() > 0) { aggregators.add(current); + } else { + Releasables.release(current); } current = null; } @@ -235,7 +245,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor { } } - public static final class ReaderAggregator { + public final class ReaderAggregator implements Releasable { private final long maxOrd; @@ -249,7 +259,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor { public ReaderAggregator(BytesValues.WithOrdinals values, int ordinalsCacheLimit, CacheRecycler cacheRecycler) { this.values = values; this.maxOrd = values.ordinals().getMaxOrd(); - this.counts = BigArrays.newIntArray(maxOrd); + this.counts = bigArrays.newIntArray(maxOrd); } final void onOrdinal(int docId, long ordinal) { @@ -273,6 +283,13 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor { public BytesRef copyCurrent() { return values.copyShared(); } + + @Override + public boolean release() { + Releasables.release(counts); + return true; + } + } public static class AggregatorPriorityQueue extends PriorityQueue { diff --git a/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java b/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java index 6b630a90039..d1a56cc12c2 100644 --- a/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java +++ b/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java @@ -30,12 +30,14 @@ import org.elasticsearch.cache.recycler.CacheRecycler; import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.search.AndFilter; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.XConstantScoreQuery; import org.elasticsearch.common.lucene.search.XFilteredQuery; import org.elasticsearch.common.lucene.search.function.BoostScoreFunction; import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.docset.DocSetCache; import org.elasticsearch.index.cache.filter.FilterCache; @@ -93,6 +95,8 @@ public class DefaultSearchContext extends SearchContext { private final PageCacheRecycler pageCacheRecycler; + private final BigArrays bigArrays; + private final IndexShard indexShard; private final IndexService indexService; @@ -175,7 +179,8 @@ public class DefaultSearchContext extends SearchContext { public DefaultSearchContext(long id, ShardSearchRequest request, SearchShardTarget shardTarget, Engine.Searcher engineSearcher, IndexService indexService, IndexShard indexShard, - ScriptService scriptService, CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler) { + ScriptService scriptService, CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler, + BigArrays bigArrays) { this.id = id; this.request = request; this.searchType = request.searchType(); @@ -184,6 +189,7 @@ public class DefaultSearchContext extends SearchContext { this.scriptService = scriptService; this.cacheRecycler = cacheRecycler; this.pageCacheRecycler = pageCacheRecycler; + this.bigArrays = bigArrays; this.dfsResult = new DfsSearchResult(id, shardTarget); this.queryResult = new QuerySearchResult(id, shardTarget); this.fetchResult = new FetchSearchResult(id, shardTarget); @@ -447,6 +453,10 @@ public class DefaultSearchContext extends SearchContext { return pageCacheRecycler; } + public BigArrays bigArrays() { + return bigArrays; + } + public FilterCache filterCache() { return indexService.cache().filter(); } @@ -665,19 +675,10 @@ public class DefaultSearchContext extends SearchContext { @Override public void clearReleasables() { if (clearables != null) { - Throwable th = null; - for (Releasable releasable : clearables) { - try { - releasable.release(); - } catch (Throwable t) { - if (th == null) { - th = t; - } - } - } - clearables.clear(); - if (th != null) { - throw new RuntimeException(th); + try { + Releasables.release(clearables); + } finally { + clearables.clear(); } } } diff --git a/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/src/main/java/org/elasticsearch/search/internal/SearchContext.java index c431b4e239d..21ddcad8148 100644 --- a/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -26,6 +26,7 @@ import org.elasticsearch.cache.recycler.CacheRecycler; import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.docset.DocSetCache; import org.elasticsearch.index.cache.filter.FilterCache; @@ -182,6 +183,8 @@ public abstract class SearchContext implements Releasable { public abstract PageCacheRecycler pageCacheRecycler(); + public abstract BigArrays bigArrays(); + public abstract FilterCache filterCache(); public abstract DocSetCache docSetCache(); diff --git a/src/test/java/org/elasticsearch/common/util/BigArraysTests.java b/src/test/java/org/elasticsearch/common/util/BigArraysTests.java index 514caadb539..07f5e08dbc3 100644 --- a/src/test/java/org/elasticsearch/common/util/BigArraysTests.java +++ b/src/test/java/org/elasticsearch/common/util/BigArraysTests.java @@ -24,24 +24,34 @@ import org.elasticsearch.cache.recycler.MockPageCacheRecycler; 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.threadpool.ThreadPool; +import org.junit.Before; import java.util.Arrays; public class BigArraysTests extends ElasticsearchTestCase { - public static PageCacheRecycler randomCacheRecycler() { - return randomBoolean() ? null : new MockPageCacheRecycler(ImmutableSettings.EMPTY, new ThreadPool()); + public static BigArrays randombigArrays() { + final PageCacheRecycler recycler = randomBoolean() ? null : new MockPageCacheRecycler(ImmutableSettings.EMPTY, new ThreadPool()); + return new MockBigArrays(ImmutableSettings.EMPTY, recycler); + } + + private BigArrays bigArrays; + + @Before + public void init() { + bigArrays = randombigArrays(); } public void testByteArrayGrowth() { final int totalLen = randomIntBetween(1, 4000000); final int startLen = randomIntBetween(1, randomBoolean() ? 1000 : totalLen); - ByteArray array = BigArrays.newByteArray(startLen, randomCacheRecycler(), randomBoolean()); + ByteArray array = bigArrays.newByteArray(startLen, randomBoolean()); byte[] ref = new byte[totalLen]; for (int i = 0; i < totalLen; ++i) { ref[i] = randomByte(); - array = BigArrays.grow(array, i + 1); + array = bigArrays.grow(array, i + 1); array.set(i, ref[i]); } for (int i = 0; i < totalLen; ++i) { @@ -53,11 +63,11 @@ public class BigArraysTests extends ElasticsearchTestCase { public void testIntArrayGrowth() { final int totalLen = randomIntBetween(1, 1000000); final int startLen = randomIntBetween(1, randomBoolean() ? 1000 : totalLen); - IntArray array = BigArrays.newIntArray(startLen, randomCacheRecycler(), randomBoolean()); + IntArray array = bigArrays.newIntArray(startLen, randomBoolean()); int[] ref = new int[totalLen]; for (int i = 0; i < totalLen; ++i) { ref[i] = randomInt(); - array = BigArrays.grow(array, i + 1); + array = bigArrays.grow(array, i + 1); array.set(i, ref[i]); } for (int i = 0; i < totalLen; ++i) { @@ -69,11 +79,11 @@ public class BigArraysTests extends ElasticsearchTestCase { public void testLongArrayGrowth() { final int totalLen = randomIntBetween(1, 1000000); final int startLen = randomIntBetween(1, randomBoolean() ? 1000 : totalLen); - LongArray array = BigArrays.newLongArray(startLen, randomCacheRecycler(), randomBoolean()); + LongArray array = bigArrays.newLongArray(startLen, randomBoolean()); long[] ref = new long[totalLen]; for (int i = 0; i < totalLen; ++i) { ref[i] = randomLong(); - array = BigArrays.grow(array, i + 1); + array = bigArrays.grow(array, i + 1); array.set(i, ref[i]); } for (int i = 0; i < totalLen; ++i) { @@ -82,14 +92,30 @@ public class BigArraysTests extends ElasticsearchTestCase { array.release(); } + public void testFloatArrayGrowth() { + final int totalLen = randomIntBetween(1, 1000000); + final int startLen = randomIntBetween(1, randomBoolean() ? 1000 : totalLen); + FloatArray array = bigArrays.newFloatArray(startLen, randomBoolean()); + float[] ref = new float[totalLen]; + for (int i = 0; i < totalLen; ++i) { + ref[i] = randomFloat(); + array = bigArrays.grow(array, i + 1); + array.set(i, ref[i]); + } + for (int i = 0; i < totalLen; ++i) { + assertEquals(ref[i], array.get(i), 0.001d); + } + array.release(); + } + public void testDoubleArrayGrowth() { final int totalLen = randomIntBetween(1, 1000000); final int startLen = randomIntBetween(1, randomBoolean() ? 1000 : totalLen); - DoubleArray array = BigArrays.newDoubleArray(startLen, randomCacheRecycler(), randomBoolean()); + DoubleArray array = bigArrays.newDoubleArray(startLen, randomBoolean()); double[] ref = new double[totalLen]; for (int i = 0; i < totalLen; ++i) { ref[i] = randomDouble(); - array = BigArrays.grow(array, i + 1); + array = bigArrays.grow(array, i + 1); array.set(i, ref[i]); } for (int i = 0; i < totalLen; ++i) { @@ -101,7 +127,7 @@ public class BigArraysTests extends ElasticsearchTestCase { public void testObjectArrayGrowth() { final int totalLen = randomIntBetween(1, 1000000); final int startLen = randomIntBetween(1, randomBoolean() ? 1000 : totalLen); - ObjectArray array = BigArrays.newObjectArray(startLen, randomCacheRecycler()); + ObjectArray array = bigArrays.newObjectArray(startLen); final Object[] pool = new Object[100]; for (int i = 0; i < pool.length; ++i) { pool[i] = new Object(); @@ -109,7 +135,7 @@ public class BigArraysTests extends ElasticsearchTestCase { Object[] ref = new Object[totalLen]; for (int i = 0; i < totalLen; ++i) { ref[i] = randomFrom(pool); - array = BigArrays.grow(array, i + 1); + array = bigArrays.grow(array, i + 1); array.set(i, ref[i]); } for (int i = 0; i < totalLen; ++i) { @@ -118,13 +144,34 @@ public class BigArraysTests extends ElasticsearchTestCase { array.release(); } + public void testFloatArrayFill() { + final int len = randomIntBetween(1, 100000); + final int fromIndex = randomIntBetween(0, len - 1); + final int toIndex = randomBoolean() + ? Math.min(fromIndex + randomInt(100), len) // single page + : randomIntBetween(fromIndex, len); // likely multiple pages + final FloatArray array2 = bigArrays.newFloatArray(len, randomBoolean()); + final float[] array1 = new float[len]; + for (int i = 0; i < len; ++i) { + array1[i] = randomFloat(); + array2.set(i, array1[i]); + } + final float rand = randomFloat(); + Arrays.fill(array1, fromIndex, toIndex, rand); + array2.fill(fromIndex, toIndex, rand); + for (int i = 0; i < len; ++i) { + assertEquals(array1[i], array2.get(i), 0.001d); + } + array2.release(); + } + public void testDoubleArrayFill() { final int len = randomIntBetween(1, 100000); final int fromIndex = randomIntBetween(0, len - 1); final int toIndex = randomBoolean() ? Math.min(fromIndex + randomInt(100), len) // single page : randomIntBetween(fromIndex, len); // likely multiple pages - final DoubleArray array2 = BigArrays.newDoubleArray(len, randomCacheRecycler(), randomBoolean()); + final DoubleArray array2 = bigArrays.newDoubleArray(len, randomBoolean()); final double[] array1 = new double[len]; for (int i = 0; i < len; ++i) { array1[i] = randomDouble(); @@ -145,7 +192,7 @@ public class BigArraysTests extends ElasticsearchTestCase { final int toIndex = randomBoolean() ? Math.min(fromIndex + randomInt(100), len) // single page : randomIntBetween(fromIndex, len); // likely multiple pages - final LongArray array2 = BigArrays.newLongArray(len, randomCacheRecycler(), randomBoolean()); + final LongArray array2 = bigArrays.newLongArray(len, randomBoolean()); final long[] array1 = new long[len]; for (int i = 0; i < len; ++i) { array1[i] = randomLong(); @@ -163,7 +210,7 @@ public class BigArraysTests extends ElasticsearchTestCase { public void testByteArrayBulkGet() { final byte[] array1 = new byte[randomIntBetween(1, 4000000)]; getRandom().nextBytes(array1); - final ByteArray array2 = BigArrays.newByteArray(array1.length, randomCacheRecycler(), randomBoolean()); + final ByteArray array2 = bigArrays.newByteArray(array1.length, randomBoolean()); for (int i = 0; i < array1.length; ++i) { array2.set(i, array1[i]); } @@ -180,7 +227,7 @@ public class BigArraysTests extends ElasticsearchTestCase { public void testByteArrayBulkSet() { final byte[] array1 = new byte[randomIntBetween(1, 4000000)]; getRandom().nextBytes(array1); - final ByteArray array2 = BigArrays.newByteArray(array1.length, randomCacheRecycler(), randomBoolean()); + final ByteArray array2 = bigArrays.newByteArray(array1.length, randomBoolean()); for (int i = 0; i < array1.length; ) { final int len = Math.min(array1.length - i, randomBoolean() ? randomInt(10) : randomInt(3 * BigArrays.BYTE_PAGE_SIZE)); array2.set(i, array1, i, len); diff --git a/src/test/java/org/elasticsearch/common/util/BytesRefHashTests.java b/src/test/java/org/elasticsearch/common/util/BytesRefHashTests.java index 43db0c52bac..91e933fe2cd 100644 --- a/src/test/java/org/elasticsearch/common/util/BytesRefHashTests.java +++ b/src/test/java/org/elasticsearch/common/util/BytesRefHashTests.java @@ -40,7 +40,7 @@ public class BytesRefHashTests extends ElasticsearchTestCase { } // Test high load factors to make sure that collision resolution works fine final float maxLoadFactor = 0.6f + randomFloat() * 0.39f; - hash = new BytesRefHash(randomIntBetween(0, 100), maxLoadFactor, BigArraysTests.randomCacheRecycler()); + hash = new BytesRefHash(randomIntBetween(0, 100), maxLoadFactor, BigArraysTests.randombigArrays()); } @Override diff --git a/src/test/java/org/elasticsearch/common/util/LongHashTests.java b/src/test/java/org/elasticsearch/common/util/LongHashTests.java index f62f05016a1..042b1cbb613 100644 --- a/src/test/java/org/elasticsearch/common/util/LongHashTests.java +++ b/src/test/java/org/elasticsearch/common/util/LongHashTests.java @@ -39,7 +39,7 @@ public class LongHashTests extends ElasticsearchTestCase { final long[] idToValue = new long[values.length]; // Test high load factors to make sure that collision resolution works fine final float maxLoadFactor = 0.6f + randomFloat() * 0.39f; - final LongHash longHash = new LongHash(randomIntBetween(0, 100), maxLoadFactor, BigArraysTests.randomCacheRecycler()); + final LongHash longHash = new LongHash(randomIntBetween(0, 100), maxLoadFactor, BigArraysTests.randombigArrays()); final int iters = randomInt(1000000); for (int i = 0; i < iters; ++i) { final Long value = randomFrom(values); diff --git a/src/test/java/org/elasticsearch/index/search/child/ChildrenConstantScoreQueryTests.java b/src/test/java/org/elasticsearch/index/search/child/ChildrenConstantScoreQueryTests.java index 755c5cbc10b..f27ae67f950 100644 --- a/src/test/java/org/elasticsearch/index/search/child/ChildrenConstantScoreQueryTests.java +++ b/src/test/java/org/elasticsearch/index/search/child/ChildrenConstantScoreQueryTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.common.lucene.search.XConstantScoreQuery; import org.elasticsearch.common.lucene.search.XFilteredQuery; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.Index; import org.elasticsearch.index.cache.filter.weighted.WeightedFilterCache; import org.elasticsearch.index.engine.Engine; @@ -56,6 +57,7 @@ import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.search.internal.ContextIndexSearcher; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.ElasticsearchLuceneTestCase; +import org.elasticsearch.test.cache.recycler.MockBigArrays; import org.elasticsearch.test.index.service.StubIndexService; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.Description; @@ -332,6 +334,7 @@ public class ChildrenConstantScoreQueryTests extends ElasticsearchLuceneTestCase final Index index = new Index(indexName); final CacheRecycler cacheRecycler = new CacheRecycler(ImmutableSettings.EMPTY); final PageCacheRecycler pageCacheRecycler = new PageCacheRecycler(ImmutableSettings.EMPTY, new ThreadPool()); + final BigArrays bigArrays = new MockBigArrays(ImmutableSettings.EMPTY, pageCacheRecycler); Settings settings = ImmutableSettings.EMPTY; MapperService mapperService = MapperTestUtils.newMapperService(index, settings); IndexFieldDataService indexFieldDataService = new IndexFieldDataService(index, new DummyCircuitBreakerService()); @@ -346,7 +349,7 @@ public class ChildrenConstantScoreQueryTests extends ElasticsearchLuceneTestCase NodeSettingsService nodeSettingsService = new NodeSettingsService(settings); IndicesFilterCache indicesFilterCache = new IndicesFilterCache(settings, threadPool, cacheRecycler, nodeSettingsService); WeightedFilterCache filterCache = new WeightedFilterCache(index, settings, indicesFilterCache); - return new TestSearchContext(cacheRecycler, pageCacheRecycler, indexService, filterCache, indexFieldDataService); + return new TestSearchContext(cacheRecycler, pageCacheRecycler, bigArrays, indexService, filterCache, indexFieldDataService); } } diff --git a/src/test/java/org/elasticsearch/index/search/child/TestSearchContext.java b/src/test/java/org/elasticsearch/index/search/child/TestSearchContext.java index d2c382bbf9d..367181dfd31 100644 --- a/src/test/java/org/elasticsearch/index/search/child/TestSearchContext.java +++ b/src/test/java/org/elasticsearch/index/search/child/TestSearchContext.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cache.recycler.CacheRecycler; import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.docset.DocSetCache; import org.elasticsearch.index.cache.filter.FilterCache; @@ -66,6 +67,7 @@ public class TestSearchContext extends SearchContext { final CacheRecycler cacheRecycler; final PageCacheRecycler pageCacheRecycler; + final BigArrays bigArrays; final IndexService indexService; final FilterCache filterCache; final IndexFieldDataService indexFieldDataService; @@ -73,9 +75,10 @@ public class TestSearchContext extends SearchContext { ContextIndexSearcher searcher; int size; - public TestSearchContext(CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler, IndexService indexService, FilterCache filterCache, IndexFieldDataService indexFieldDataService) { + public TestSearchContext(CacheRecycler cacheRecycler, PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, IndexService indexService, FilterCache filterCache, IndexFieldDataService indexFieldDataService) { this.cacheRecycler = cacheRecycler; this.pageCacheRecycler = pageCacheRecycler; + this.bigArrays = bigArrays; this.indexService = indexService; this.filterCache = filterCache; this.indexFieldDataService = indexFieldDataService; @@ -84,6 +87,7 @@ public class TestSearchContext extends SearchContext { public TestSearchContext() { this.cacheRecycler = null; this.pageCacheRecycler = null; + this.bigArrays = null; this.indexService = null; this.filterCache = null; this.indexFieldDataService = null; @@ -319,6 +323,11 @@ public class TestSearchContext extends SearchContext { return pageCacheRecycler; } + @Override + public BigArrays bigArrays() { + return bigArrays; + } + @Override public FilterCache filterCache() { return filterCache; diff --git a/src/test/java/org/elasticsearch/search/aggregations/bucket/DoubleTermsTests.java b/src/test/java/org/elasticsearch/search/aggregations/bucket/DoubleTermsTests.java index c4ea08dbd4e..8c682442562 100644 --- a/src/test/java/org/elasticsearch/search/aggregations/bucket/DoubleTermsTests.java +++ b/src/test/java/org/elasticsearch/search/aggregations/bucket/DoubleTermsTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.search.aggregations.metrics.stats.Stats; import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats; import org.elasticsearch.search.aggregations.metrics.sum.Sum; import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.cache.recycler.MockBigArrays; import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; @@ -675,6 +676,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByMissingSubAggregation() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") @@ -693,6 +695,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByNonMetricsSubAggregation() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") @@ -712,6 +715,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByMultiValuedSubAggregation_WithUknownMetric() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") @@ -732,6 +736,7 @@ public class DoubleTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByMultiValuedSubAggregation_WithoutMetric() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") diff --git a/src/test/java/org/elasticsearch/search/aggregations/bucket/LongTermsTests.java b/src/test/java/org/elasticsearch/search/aggregations/bucket/LongTermsTests.java index dc04fe999e3..7b66f64db46 100644 --- a/src/test/java/org/elasticsearch/search/aggregations/bucket/LongTermsTests.java +++ b/src/test/java/org/elasticsearch/search/aggregations/bucket/LongTermsTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.metrics.stats.Stats; import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats; import org.elasticsearch.search.aggregations.metrics.sum.Sum; import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.cache.recycler.MockBigArrays; import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; @@ -667,6 +668,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByMissingSubAggregation() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") @@ -685,6 +687,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByNonMetricsSubAggregation() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") @@ -704,6 +707,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByMultiValuedSubAggregation_WithUknownMetric() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") @@ -724,6 +728,7 @@ public class LongTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByMultiValuedSubAggregation_WithoutMetric() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") diff --git a/src/test/java/org/elasticsearch/search/aggregations/bucket/NestedTests.java b/src/test/java/org/elasticsearch/search/aggregations/bucket/NestedTests.java index 0fafc93b11d..e73a1e22339 100644 --- a/src/test/java/org/elasticsearch/search/aggregations/bucket/NestedTests.java +++ b/src/test/java/org/elasticsearch/search/aggregations/bucket/NestedTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket; import org.elasticsearch.search.aggregations.metrics.max.Max; import org.elasticsearch.search.aggregations.metrics.stats.Stats; import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.cache.recycler.MockBigArrays; import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; @@ -141,6 +142,7 @@ public class NestedTests extends ElasticsearchIntegrationTest { @Test public void onNonNestedField() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx") .addAggregation(nested("nested").path("value") diff --git a/src/test/java/org/elasticsearch/search/aggregations/bucket/StringTermsTests.java b/src/test/java/org/elasticsearch/search/aggregations/bucket/StringTermsTests.java index d0c5c53730c..e7b87c97906 100644 --- a/src/test/java/org/elasticsearch/search/aggregations/bucket/StringTermsTests.java +++ b/src/test/java/org/elasticsearch/search/aggregations/bucket/StringTermsTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.search.aggregations.metrics.stats.Stats; import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStats; import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCount; import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.cache.recycler.MockBigArrays; import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; @@ -835,6 +836,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByMissingSubAggregation() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") @@ -854,6 +856,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByNonMetricsSubAggregation() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") @@ -874,6 +877,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByMultiValuedSubAggregation_WithUknownMetric() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") @@ -895,6 +899,7 @@ public class StringTermsTests extends ElasticsearchIntegrationTest { @Test public void singleValuedField_OrderedByMultiValuedSubAggregation_WithoutMetric() throws Exception { + MockBigArrays.discardNextCheck(); try { client().prepareSearch("idx").setTypes("type") diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java b/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java index 2898290c56d..895c17b621e 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java @@ -33,11 +33,13 @@ import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.util.concurrent.EsAbortPolicy; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.test.cache.recycler.MockBigArrays; import org.elasticsearch.test.engine.MockInternalEngine; import org.elasticsearch.test.junit.listeners.LoggingListener; import org.elasticsearch.test.store.MockDirectoryHelper; import org.junit.After; import org.junit.AfterClass; +import org.junit.Before; import org.junit.BeforeClass; import java.io.Closeable; @@ -125,6 +127,17 @@ public abstract class ElasticsearchTestCase extends AbstractRandomizedTest { MockPageCacheRecycler.ensureAllPagesAreReleased(); } + @Before + public void resetArrayTracking() { + // useful if there are tests that use MockBigArrays but don't inherit from ElasticsearchTestCase + MockBigArrays.reset(); + } + + @After + public void ensureAllArraysReleased() { + MockBigArrays.ensureAllArraysAreReleased(); + } + public static void ensureAllFilesClosed() throws IOException { try { for (MockDirectoryHelper.ElasticsearchMockDirectoryWrapper w : MockDirectoryHelper.wrappers) { diff --git a/src/test/java/org/elasticsearch/test/TestCluster.java b/src/test/java/org/elasticsearch/test/TestCluster.java index fe729d43823..4ec664f1077 100644 --- a/src/test/java/org/elasticsearch/test/TestCluster.java +++ b/src/test/java/org/elasticsearch/test/TestCluster.java @@ -46,11 +46,13 @@ import org.elasticsearch.common.settings.ImmutableSettings.Builder; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.BigArraysModule; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.engine.IndexEngineModule; import org.elasticsearch.node.Node; import org.elasticsearch.node.internal.InternalNode; import org.elasticsearch.search.SearchService; +import org.elasticsearch.test.cache.recycler.MockBigArraysModule; import org.elasticsearch.test.cache.recycler.MockPageCacheRecyclerModule; import org.elasticsearch.test.engine.MockEngineModule; import org.elasticsearch.test.store.MockFSIndexStoreModule; @@ -233,6 +235,7 @@ public final class TestCluster implements Iterable { builder.put("index.store.type", MockFSIndexStoreModule.class.getName()); // no RAM dir for now! builder.put(IndexEngineModule.EngineSettings.ENGINE_TYPE, MockEngineModule.class.getName()); builder.put(PageCacheRecyclerModule.CACHE_IMPL, MockPageCacheRecyclerModule.class.getName()); + builder.put(BigArraysModule.IMPL, MockBigArraysModule.class.getName()); } if (isLocalTransportConfigured()) { builder.put(TransportModule.TRANSPORT_TYPE_KEY, AssertingLocalTransportModule.class.getName()); diff --git a/src/test/java/org/elasticsearch/test/cache/recycler/MockBigArrays.java b/src/test/java/org/elasticsearch/test/cache/recycler/MockBigArrays.java new file mode 100644 index 00000000000..0cefc8dc15c --- /dev/null +++ b/src/test/java/org/elasticsearch/test/cache/recycler/MockBigArrays.java @@ -0,0 +1,501 @@ +/* + * 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.test.cache.recycler; + +import com.carrotsearch.randomizedtesting.RandomizedContext; +import com.carrotsearch.randomizedtesting.SeedUtils; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.cache.recycler.PageCacheRecycler; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.*; + +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicBoolean; + +public class MockBigArrays extends BigArrays { + + /** + * Tracking allocations is useful when debugging a leak but shouldn't be enabled by default as this would also be very costly + * since it creates a new Exception every time a new array is created. + */ + private static final boolean TRACK_ALLOCATIONS = false; + + private static boolean DISCARD = false; + + private static ConcurrentMap ACQUIRED_ARRAYS = new ConcurrentHashMap(); + + /** + * Discard the next check that all arrays should be released. This can be useful if for a specific test, the cost to make + * sure the array is released is higher than the cost the user would experience if the array would not be released. + */ + public static void discardNextCheck() { + DISCARD = true; + } + + public static void reset() { + ACQUIRED_ARRAYS.clear(); + } + + public static void ensureAllArraysAreReleased() { + if (DISCARD) { + DISCARD = false; + } else if (ACQUIRED_ARRAYS.size() > 0) { + final Object cause = ACQUIRED_ARRAYS.entrySet().iterator().next().getValue(); + throw new RuntimeException(ACQUIRED_ARRAYS.size() + " arrays have not been released", cause instanceof Throwable ? (Throwable) cause : null); + } + } + + private final Random random; + + @Inject + public MockBigArrays(Settings settings, PageCacheRecycler recycler) { + super(settings, recycler); + long seed; + try { + seed = SeedUtils.parseSeed(RandomizedContext.current().getRunnerSeedAsString()); + } catch (IllegalStateException e) { // rest tests don't run randomized and have no context + seed = 0; + } + random = new Random(seed); + } + + @Override + public ByteArray newByteArray(long size, boolean clearOnResize) { + final ByteArrayWrapper array = new ByteArrayWrapper(super.newByteArray(size, clearOnResize), clearOnResize); + if (!clearOnResize) { + array.randomizeContent(0, size); + } + return array; + } + + @Override + public ByteArray resize(ByteArray array, long size) { + ByteArrayWrapper arr = (ByteArrayWrapper) array; + final long originalSize = arr.size(); + array = super.resize(arr.in, size); + ACQUIRED_ARRAYS.remove(arr); + if (array instanceof ByteArrayWrapper) { + arr = (ByteArrayWrapper) array; + } else { + arr = new ByteArrayWrapper(array, arr.clearOnResize); + } + if (!arr.clearOnResize) { + arr.randomizeContent(originalSize, size); + } + return arr; + } + + @Override + public IntArray newIntArray(long size, boolean clearOnResize) { + final IntArrayWrapper array = new IntArrayWrapper(super.newIntArray(size, clearOnResize), clearOnResize); + if (!clearOnResize) { + array.randomizeContent(0, size); + } + return array; + } + + @Override + public IntArray resize(IntArray array, long size) { + IntArrayWrapper arr = (IntArrayWrapper) array; + final long originalSize = arr.size(); + array = super.resize(arr.in, size); + ACQUIRED_ARRAYS.remove(arr); + if (array instanceof IntArrayWrapper) { + arr = (IntArrayWrapper) array; + } else { + arr = new IntArrayWrapper(array, arr.clearOnResize); + } + if (!arr.clearOnResize) { + arr.randomizeContent(originalSize, size); + } + return arr; + } + + @Override + public LongArray newLongArray(long size, boolean clearOnResize) { + final LongArrayWrapper array = new LongArrayWrapper(super.newLongArray(size, clearOnResize), clearOnResize); + if (!clearOnResize) { + array.randomizeContent(0, size); + } + return array; + } + + @Override + public LongArray resize(LongArray array, long size) { + LongArrayWrapper arr = (LongArrayWrapper) array; + final long originalSize = arr.size(); + array = super.resize(arr.in, size); + ACQUIRED_ARRAYS.remove(arr); + if (array instanceof LongArrayWrapper) { + arr = (LongArrayWrapper) array; + } else { + arr = new LongArrayWrapper(array, arr.clearOnResize); + } + if (!arr.clearOnResize) { + arr.randomizeContent(originalSize, size); + } + return arr; + } + + @Override + public FloatArray newFloatArray(long size, boolean clearOnResize) { + final FloatArrayWrapper array = new FloatArrayWrapper(super.newFloatArray(size, clearOnResize), clearOnResize); + if (!clearOnResize) { + array.randomizeContent(0, size); + } + return array; + } + + @Override + public FloatArray resize(FloatArray array, long size) { + FloatArrayWrapper arr = (FloatArrayWrapper) array; + final long originalSize = arr.size(); + array = super.resize(arr.in, size); + ACQUIRED_ARRAYS.remove(arr); + if (array instanceof FloatArrayWrapper) { + arr = (FloatArrayWrapper) array; + } else { + arr = new FloatArrayWrapper(array, arr.clearOnResize); + } + if (!arr.clearOnResize) { + arr.randomizeContent(originalSize, size); + } + return arr; + } + + @Override + public DoubleArray newDoubleArray(long size, boolean clearOnResize) { + final DoubleArrayWrapper array = new DoubleArrayWrapper(super.newDoubleArray(size, clearOnResize), clearOnResize); + if (!clearOnResize) { + array.randomizeContent(0, size); + } + return array; + } + + @Override + public DoubleArray resize(DoubleArray array, long size) { + DoubleArrayWrapper arr = (DoubleArrayWrapper) array; + final long originalSize = arr.size(); + array = super.resize(arr.in, size); + ACQUIRED_ARRAYS.remove(arr); + if (array instanceof DoubleArrayWrapper) { + arr = (DoubleArrayWrapper) array; + } else { + arr = new DoubleArrayWrapper(array, arr.clearOnResize); + } + if (!arr.clearOnResize) { + arr.randomizeContent(originalSize, size); + } + return arr; + } + + @Override + public ObjectArray newObjectArray(long size) { + return new ObjectArrayWrapper(super.newObjectArray(size)); + } + + @Override + public ObjectArray resize(ObjectArray array, long size) { + ObjectArrayWrapper arr = (ObjectArrayWrapper) array; + array = super.resize(arr.in, size); + ACQUIRED_ARRAYS.remove(arr); + if (array instanceof ObjectArrayWrapper) { + arr = (ObjectArrayWrapper) array; + } else { + arr = new ObjectArrayWrapper(array); + } + return arr; + } + + private static abstract class AbstractArrayWrapper { + + boolean clearOnResize; + AtomicBoolean released; + + AbstractArrayWrapper(boolean clearOnResize) { + ACQUIRED_ARRAYS.put(this, TRACK_ALLOCATIONS ? new RuntimeException() : Boolean.TRUE); + this.clearOnResize = clearOnResize; + released = new AtomicBoolean(false); + } + + protected abstract BigArray getDelegate(); + + protected abstract void randomizeContent(long from, long to); + + public long size() { + return getDelegate().size(); + } + + public boolean release() { + if (!released.compareAndSet(false, true)) { + throw new IllegalStateException("Double release"); + } + ACQUIRED_ARRAYS.remove(this); + randomizeContent(0, size()); + return getDelegate().release(); + } + + } + + private class ByteArrayWrapper extends AbstractArrayWrapper implements ByteArray { + + private final ByteArray in; + + ByteArrayWrapper(ByteArray in, boolean clearOnResize) { + super(clearOnResize); + this.in = in; + } + + @Override + protected BigArray getDelegate() { + return in; + } + + @Override + protected void randomizeContent(long from, long to) { + for (long i = from; i < to; ++i) { + set(i, (byte) random.nextInt(1 << 8)); + } + } + + @Override + public byte get(long index) { + return in.get(index); + } + + @Override + public byte set(long index, byte value) { + return in.set(index, value); + } + + @Override + public void get(long index, int len, BytesRef ref) { + in.get(index, len, ref); + } + + @Override + public void set(long index, byte[] buf, int offset, int len) { + in.set(index, buf, offset, len); + } + + } + + private class IntArrayWrapper extends AbstractArrayWrapper implements IntArray { + + private final IntArray in; + + IntArrayWrapper(IntArray in, boolean clearOnResize) { + super(clearOnResize); + this.in = in; + } + + @Override + protected BigArray getDelegate() { + return in; + } + + @Override + protected void randomizeContent(long from, long to) { + for (long i = from; i < to; ++i) { + set(i, random.nextInt()); + } + } + + @Override + public int get(long index) { + return in.get(index); + } + + @Override + public int set(long index, int value) { + return in.set(index, value); + } + + @Override + public int increment(long index, int inc) { + return in.increment(index, inc); + } + + } + + private class LongArrayWrapper extends AbstractArrayWrapper implements LongArray { + + private final LongArray in; + + LongArrayWrapper(LongArray in, boolean clearOnResize) { + super(clearOnResize); + this.in = in; + } + + @Override + protected BigArray getDelegate() { + return in; + } + + @Override + protected void randomizeContent(long from, long to) { + for (long i = from; i < to; ++i) { + set(i, random.nextLong()); + } + } + + @Override + public long get(long index) { + return in.get(index); + } + + @Override + public long set(long index, long value) { + return in.set(index, value); + } + + @Override + public long increment(long index, long inc) { + return in.increment(index, inc); + } + + @Override + public void fill(long fromIndex, long toIndex, long value) { + in.fill(fromIndex, toIndex, value); + } + + } + + private class FloatArrayWrapper extends AbstractArrayWrapper implements FloatArray { + + private final FloatArray in; + + FloatArrayWrapper(FloatArray in, boolean clearOnResize) { + super(clearOnResize); + this.in = in; + } + + @Override + protected BigArray getDelegate() { + return in; + } + + @Override + protected void randomizeContent(long from, long to) { + for (long i = from; i < to; ++i) { + set(i, (random.nextFloat() - 0.5f) * 1000); + } + } + + @Override + public float get(long index) { + return in.get(index); + } + + @Override + public float set(long index, float value) { + return in.set(index, value); + } + + @Override + public float increment(long index, float inc) { + return in.increment(index, inc); + } + + @Override + public void fill(long fromIndex, long toIndex, float value) { + in.fill(fromIndex, toIndex, value); + } + + } + + private class DoubleArrayWrapper extends AbstractArrayWrapper implements DoubleArray { + + private final DoubleArray in; + + DoubleArrayWrapper(DoubleArray in, boolean clearOnResize) { + super(clearOnResize); + this.in = in; + } + + @Override + protected BigArray getDelegate() { + return in; + } + + @Override + protected void randomizeContent(long from, long to) { + for (long i = from; i < to; ++i) { + set(i, (random.nextDouble() - 0.5) * 1000); + } + } + + @Override + public double get(long index) { + return in.get(index); + } + + @Override + public double set(long index, double value) { + return in.set(index, value); + } + + @Override + public double increment(long index, double inc) { + return in.increment(index, inc); + } + + @Override + public void fill(long fromIndex, long toIndex, double value) { + in.fill(fromIndex, toIndex, value); + } + + } + + private class ObjectArrayWrapper extends AbstractArrayWrapper implements ObjectArray { + + private final ObjectArray in; + + ObjectArrayWrapper(ObjectArray in) { + super(false); + this.in = in; + } + + @Override + protected BigArray getDelegate() { + return in; + } + + @Override + public T get(long index) { + return in.get(index); + } + + @Override + public T set(long index, T value) { + return in.set(index, value); + } + + @Override + protected void randomizeContent(long from, long to) { + // will be cleared anyway + } + + } + +} diff --git a/src/test/java/org/elasticsearch/test/cache/recycler/MockBigArraysModule.java b/src/test/java/org/elasticsearch/test/cache/recycler/MockBigArraysModule.java new file mode 100644 index 00000000000..e5b48a318d7 --- /dev/null +++ b/src/test/java/org/elasticsearch/test/cache/recycler/MockBigArraysModule.java @@ -0,0 +1,32 @@ +/* + * 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.test.cache.recycler; + +import org.elasticsearch.common.inject.AbstractModule; +import org.elasticsearch.common.util.BigArrays; + +public class MockBigArraysModule extends AbstractModule { + + @Override + protected void configure() { + bind(BigArrays.class).to(MockBigArrays.class).asEagerSingleton(); + } + +}