mirror of https://github.com/apache/lucene.git
LUCENE-4615: do not reuse facet aggregation arrays by default; added ReusingFacetArrays
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1420159 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
17f8b6cf36
commit
7dd14db96b
|
@ -85,7 +85,11 @@ Changes in backwards compatibility policy
|
|||
non-point shapes. If you want to call makeDistanceValueSource() based on
|
||||
shape centers, you need to do this yourself in another spatial field.
|
||||
(David Smiley)
|
||||
|
||||
|
||||
* LUCENE-4615: Replace IntArrayAllocator and FloatArrayAllocator by ArraysPool.
|
||||
FacetArrays no longer takes those allocators; if you need to reuse the arrays,
|
||||
you should use ReusingFacetArrays. (Shai Erera, Gilad Barkai)
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-4226: New experimental StoredFieldsFormat that compresses chunks of
|
||||
|
|
|
@ -2,18 +2,15 @@ package org.apache.lucene.facet.search;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
import org.apache.lucene.facet.search.params.FacetSearchParams;
|
||||
import org.apache.lucene.facet.search.results.FacetResult;
|
||||
import org.apache.lucene.facet.search.results.FacetResultNode;
|
||||
import org.apache.lucene.facet.search.sampling.RandomSampler;
|
||||
import org.apache.lucene.facet.search.sampling.RepeatableSampler;
|
||||
import org.apache.lucene.facet.search.sampling.Sampler;
|
||||
import org.apache.lucene.facet.search.sampling.SamplingAccumulator;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -59,14 +56,14 @@ public final class AdaptiveFacetsAccumulator extends StandardFacetsAccumulator {
|
|||
}
|
||||
|
||||
/**
|
||||
* Create an {@link AdaptiveFacetsAccumulator}
|
||||
* @see StandardFacetsAccumulator#StandardFacetsAccumulator(FacetSearchParams, IndexReader, TaxonomyReader,
|
||||
* IntArrayAllocator, FloatArrayAllocator)
|
||||
* Create an {@link AdaptiveFacetsAccumulator}
|
||||
*
|
||||
* @see StandardFacetsAccumulator#StandardFacetsAccumulator(FacetSearchParams,
|
||||
* IndexReader, TaxonomyReader, FacetArrays)
|
||||
*/
|
||||
public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
|
||||
TaxonomyReader taxonomyReader, IntArrayAllocator intArrayAllocator,
|
||||
FloatArrayAllocator floatArrayAllocator) {
|
||||
super(searchParams, indexReader, taxonomyReader, intArrayAllocator, floatArrayAllocator);
|
||||
TaxonomyReader taxonomyReader, FacetArrays facetArrays) {
|
||||
super(searchParams, indexReader, taxonomyReader, facetArrays);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,110 @@
|
|||
package org.apache.lucene.facet.search;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.ArrayBlockingQueue;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* A thread-safe pool of {@code int[]} and {@code float[]} arrays. One specifies
|
||||
* the maximum number of arrays in the constructor. Calls to
|
||||
* {@link #allocateFloatArray()} or {@link #allocateIntArray()} take an array
|
||||
* from the pool, and if one is not available, allocate a new one. When you are
|
||||
* done using the array, you should {@link #free(int[]) free} it.
|
||||
* <p>
|
||||
* This class is used by {@link ReusingFacetArrays} for temporal facet
|
||||
* aggregation arrays, which can be reused across searches instead of being
|
||||
* allocated afresh on every search.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class ArraysPool {
|
||||
|
||||
private final ArrayBlockingQueue<int[]> intsPool;
|
||||
private final ArrayBlockingQueue<float[]> floatsPool;
|
||||
|
||||
public final int arrayLength;
|
||||
|
||||
/**
|
||||
* Specifies the max number of arrays to pool, as well as the length of each
|
||||
* array to allocate.
|
||||
*
|
||||
* @param arrayLength the size of the arrays to allocate
|
||||
* @param maxArrays the maximum number of arrays to pool, from each type
|
||||
*
|
||||
* @throws IllegalArgumentException if maxArrays is set to 0.
|
||||
*/
|
||||
public ArraysPool(int arrayLength, int maxArrays) {
|
||||
if (maxArrays == 0) {
|
||||
throw new IllegalArgumentException(
|
||||
"maxArrays cannot be 0 - don't use this class if you don't intend to pool arrays");
|
||||
}
|
||||
this.arrayLength = arrayLength;
|
||||
this.intsPool = new ArrayBlockingQueue<int[]>(maxArrays);
|
||||
this.floatsPool = new ArrayBlockingQueue<float[]>(maxArrays);
|
||||
}
|
||||
|
||||
/**
|
||||
* Allocates a new {@code int[]}. If there's an available array in the pool,
|
||||
* it is used, otherwise a new array is allocated.
|
||||
*/
|
||||
public final int[] allocateIntArray() {
|
||||
int[] arr = intsPool.poll();
|
||||
if (arr == null) {
|
||||
return new int[arrayLength];
|
||||
}
|
||||
Arrays.fill(arr, 0); // reset array
|
||||
return arr;
|
||||
}
|
||||
|
||||
/**
|
||||
* Allocates a new {@code float[]}. If there's an available array in the pool,
|
||||
* it is used, otherwise a new array is allocated.
|
||||
*/
|
||||
public final float[] allocateFloatArray() {
|
||||
float[] arr = floatsPool.poll();
|
||||
if (arr == null) {
|
||||
return new float[arrayLength];
|
||||
}
|
||||
Arrays.fill(arr, 0f); // reset array
|
||||
return arr;
|
||||
}
|
||||
|
||||
/**
|
||||
* Frees a no-longer-needed array. If there's room in the pool, the array is
|
||||
* added to it, otherwise discarded.
|
||||
*/
|
||||
public final void free(int[] arr) {
|
||||
if (arr != null) {
|
||||
// use offer - if there isn't room, we don't want to wait
|
||||
intsPool.offer(arr);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Frees a no-longer-needed array. If there's room in the pool, the array is
|
||||
* added to it, otherwise discarded.
|
||||
*/
|
||||
public final void free(float[] arr) {
|
||||
if (arr != null) {
|
||||
// use offer - if there isn't room, we don't want to wait
|
||||
floatsPool.offer(arr);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -18,74 +18,66 @@ package org.apache.lucene.facet.search;
|
|||
*/
|
||||
|
||||
/**
|
||||
* Provider of arrays used for facet operations such as counting.
|
||||
* Provider of arrays used for facets aggregation. Returns either an
|
||||
* {@code int[]} or {@code float[]} of the specified array length. When the
|
||||
* arrays are no longer needed, you should call {@link #free()}, so that e.g.
|
||||
* they will be reclaimed.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE:</b> if you need to reuse the allocated arrays between search
|
||||
* requests, use {@link ReusingFacetArrays}.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE:</b> this class is not thread safe. You typically allocate it per
|
||||
* search.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class FacetArrays {
|
||||
|
||||
private int[] intArray;
|
||||
private float[] floatArray;
|
||||
private IntArrayAllocator intArrayAllocator;
|
||||
private FloatArrayAllocator floatArrayAllocator;
|
||||
private int arraysLength;
|
||||
private int[] ints;
|
||||
private float[] floats;
|
||||
|
||||
public final int arrayLength;
|
||||
|
||||
/** Arrays will be allocated at the specified length. */
|
||||
public FacetArrays(int arrayLength) {
|
||||
this.arrayLength = arrayLength;
|
||||
}
|
||||
|
||||
protected float[] newFloatArray() {
|
||||
return new float[arrayLength];
|
||||
}
|
||||
|
||||
protected int[] newIntArray() {
|
||||
return new int[arrayLength];
|
||||
}
|
||||
|
||||
protected void doFree(float[] floats, int[] ints) {
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a FacetArrays with certain array allocators.
|
||||
* @param intArrayAllocator allocator for int arrays.
|
||||
* @param floatArrayAllocator allocator for float arrays.
|
||||
* Notifies that the arrays obtained from {@link #getIntArray()}
|
||||
* or {@link #getFloatArray()} are no longer needed and can be freed.
|
||||
*/
|
||||
public FacetArrays(IntArrayAllocator intArrayAllocator,
|
||||
FloatArrayAllocator floatArrayAllocator) {
|
||||
this.intArrayAllocator = intArrayAllocator;
|
||||
this.floatArrayAllocator = floatArrayAllocator;
|
||||
public final void free() {
|
||||
doFree(floats, ints);
|
||||
ints = null;
|
||||
floats = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Notify allocators that they can free arrays allocated
|
||||
* on behalf of this FacetArrays object.
|
||||
*/
|
||||
public void free() {
|
||||
if (intArrayAllocator!=null) {
|
||||
intArrayAllocator.free(intArray);
|
||||
// Should give up handle to the array now
|
||||
// that it is freed.
|
||||
intArray = null;
|
||||
public final int[] getIntArray() {
|
||||
if (ints == null) {
|
||||
ints = newIntArray();
|
||||
}
|
||||
if (floatArrayAllocator!=null) {
|
||||
floatArrayAllocator.free(floatArray);
|
||||
// Should give up handle to the array now
|
||||
// that it is freed.
|
||||
floatArray = null;
|
||||
return ints;
|
||||
}
|
||||
|
||||
public final float[] getFloatArray() {
|
||||
if (floats == null) {
|
||||
floats = newFloatArray();
|
||||
}
|
||||
arraysLength = 0;
|
||||
return floats;
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain an int array, e.g. for facet counting.
|
||||
*/
|
||||
public int[] getIntArray() {
|
||||
if (intArray == null) {
|
||||
intArray = intArrayAllocator.allocate();
|
||||
arraysLength = intArray.length;
|
||||
}
|
||||
return intArray;
|
||||
}
|
||||
|
||||
/** Obtain a float array, e.g. for evaluating facet association values. */
|
||||
public float[] getFloatArray() {
|
||||
if (floatArray == null) {
|
||||
floatArray = floatArrayAllocator.allocate();
|
||||
arraysLength = floatArray.length;
|
||||
}
|
||||
return floatArray;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the arrays length
|
||||
*/
|
||||
public int getArraysLength() {
|
||||
return arraysLength;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -154,7 +154,7 @@ public abstract class FacetResultsHandler {
|
|||
* offset in input arrays where partition starts
|
||||
*/
|
||||
protected boolean isSelfPartition (int ordinal, FacetArrays facetArrays, int offset) {
|
||||
int partitionSize = facetArrays.getArraysLength();
|
||||
int partitionSize = facetArrays.arrayLength;
|
||||
return ordinal / partitionSize == offset / partitionSize;
|
||||
}
|
||||
|
||||
|
|
|
@ -1,68 +0,0 @@
|
|||
package org.apache.lucene.facet.search;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* An FloatArrayAllocator is an object which manages float array objects
|
||||
* of a certain size. These float arrays are needed temporarily during
|
||||
* faceted search (see {@link FacetsAccumulator} and can be reused across searches
|
||||
* instead of being allocated afresh on every search.
|
||||
* <P>
|
||||
* An FloatArrayAllocator is thread-safe.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class FloatArrayAllocator extends TemporaryObjectAllocator<float[]> {
|
||||
|
||||
// An FloatArrayAllocater deals with integer arrays of a fixed size, size.
|
||||
private int size;
|
||||
|
||||
/**
|
||||
* Construct an allocator for float arrays of size <CODE>size</CODE>,
|
||||
* keeping around a pool of up to <CODE>maxArrays</CODE> old arrays.
|
||||
* <P>
|
||||
* Note that the pool size only restricts the number of arrays that hang
|
||||
* around when not needed, but <I>not</I> the maximum number of arrays
|
||||
* that are allocated when actually is use: If a number of concurrent
|
||||
* threads ask for an allocation, all of them will get a counter array,
|
||||
* even if their number is greater than maxArrays. If an application wants
|
||||
* to limit the number of concurrent threads making allocations, it needs
|
||||
* to do so on its own - for example by blocking new threads until the
|
||||
* existing ones have finished.
|
||||
* <P>
|
||||
* In particular, when maxArrays=0, this object behaves as a trivial
|
||||
* allocator, always allocating a new array and never reusing an old one.
|
||||
*/
|
||||
public FloatArrayAllocator(int size, int maxArrays) {
|
||||
super(maxArrays);
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float[] create() {
|
||||
return new float[size];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear(float[] array) {
|
||||
Arrays.fill(array, 0);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,68 +0,0 @@
|
|||
package org.apache.lucene.facet.search;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* An IntArrayAllocator is an object which manages counter array objects
|
||||
* of a certain length. These counter arrays are needed temporarily during
|
||||
* faceted search (see {@link FacetsAccumulator} and can be reused across searches
|
||||
* instead of being allocated afresh on every search.
|
||||
* <P>
|
||||
* An IntArrayAllocator is thread-safe.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class IntArrayAllocator extends TemporaryObjectAllocator<int[]> {
|
||||
|
||||
// An IntArrayAllocater deals with integer arrays of a fixed length.
|
||||
private int length;
|
||||
|
||||
/**
|
||||
* Construct an allocator for counter arrays of length <CODE>length</CODE>,
|
||||
* keeping around a pool of up to <CODE>maxArrays</CODE> old arrays.
|
||||
* <P>
|
||||
* Note that the pool size only restricts the number of arrays that hang
|
||||
* around when not needed, but <I>not</I> the maximum number of arrays
|
||||
* that are allocated when actually is use: If a number of concurrent
|
||||
* threads ask for an allocation, all of them will get a counter array,
|
||||
* even if their number is greater than maxArrays. If an application wants
|
||||
* to limit the number of concurrent threads making allocations, it needs
|
||||
* to do so on its own - for example by blocking new threads until the
|
||||
* existing ones have finished.
|
||||
* <P>
|
||||
* In particular, when maxArrays=0, this object behaves as a trivial
|
||||
* allocator, always allocating a new array and never reusing an old one.
|
||||
*/
|
||||
public IntArrayAllocator(int length, int maxArrays) {
|
||||
super(maxArrays);
|
||||
this.length = length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] create() {
|
||||
return new int[length];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear(int[] array) {
|
||||
Arrays.fill(array, 0);
|
||||
}
|
||||
|
||||
}
|
|
@ -62,8 +62,7 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
|
|||
|
||||
private static final Logger logger = Logger.getLogger(StandardFacetsAccumulator.class.getName());
|
||||
|
||||
protected final IntArrayAllocator intArrayAllocator;
|
||||
protected final FloatArrayAllocator floatArrayAllocator;
|
||||
protected final FacetArrays facetArrays;
|
||||
|
||||
protected int partitionSize;
|
||||
protected int maxPartitions;
|
||||
|
@ -74,20 +73,14 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
|
|||
private Object accumulateGuard;
|
||||
|
||||
public StandardFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
|
||||
TaxonomyReader taxonomyReader, IntArrayAllocator intArrayAllocator,
|
||||
FloatArrayAllocator floatArrayAllocator) {
|
||||
|
||||
TaxonomyReader taxonomyReader, FacetArrays facetArrays) {
|
||||
super(searchParams,indexReader,taxonomyReader);
|
||||
int realPartitionSize = intArrayAllocator == null || floatArrayAllocator == null
|
||||
? PartitionsUtils.partitionSize(searchParams, taxonomyReader) : -1; // -1 if not needed.
|
||||
this.intArrayAllocator = intArrayAllocator != null
|
||||
? intArrayAllocator
|
||||
// create a default one if null was provided
|
||||
: new IntArrayAllocator(realPartitionSize, 1);
|
||||
this.floatArrayAllocator = floatArrayAllocator != null
|
||||
? floatArrayAllocator
|
||||
// create a default one if null provided
|
||||
: new FloatArrayAllocator(realPartitionSize, 1);
|
||||
|
||||
if (facetArrays == null) {
|
||||
throw new IllegalArgumentException("facetArrays cannot be null");
|
||||
}
|
||||
|
||||
this.facetArrays = facetArrays;
|
||||
// can only be computed later when docids size is known
|
||||
isUsingComplements = false;
|
||||
partitionSize = PartitionsUtils.partitionSize(searchParams, taxonomyReader);
|
||||
|
@ -95,10 +88,10 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
|
|||
accumulateGuard = new Object();
|
||||
}
|
||||
|
||||
public StandardFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
|
||||
TaxonomyReader taxonomyReader) {
|
||||
|
||||
this(searchParams, indexReader, taxonomyReader, null, null);
|
||||
public StandardFacetsAccumulator(FacetSearchParams searchParams,
|
||||
IndexReader indexReader, TaxonomyReader taxonomyReader) {
|
||||
this(searchParams, indexReader, taxonomyReader, new FacetArrays(
|
||||
PartitionsUtils.partitionSize(searchParams, taxonomyReader)));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -152,8 +145,6 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
|
|||
|
||||
docids = actualDocsToAccumulate(docids);
|
||||
|
||||
FacetArrays facetArrays = new FacetArrays(intArrayAllocator, floatArrayAllocator);
|
||||
|
||||
HashMap<FacetRequest, IntermediateFacetResult> fr2tmpRes = new HashMap<FacetRequest, IntermediateFacetResult>();
|
||||
|
||||
try {
|
||||
|
|
|
@ -1,114 +0,0 @@
|
|||
package org.apache.lucene.facet.search;
|
||||
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* An TemporaryObjectAllocator is an object which manages large, reusable,
|
||||
* temporary objects needed during multiple concurrent computations. The idea
|
||||
* is to remember some of the previously allocated temporary objects, and
|
||||
* reuse them if possible to avoid constant allocation and garbage-collection
|
||||
* of these objects.
|
||||
* <P>
|
||||
* This technique is useful for temporary counter arrays in faceted search
|
||||
* (see {@link FacetsAccumulator}), which can be reused across searches instead
|
||||
* of being allocated afresh on every search.
|
||||
* <P>
|
||||
* A TemporaryObjectAllocator is thread-safe.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class TemporaryObjectAllocator<T> {
|
||||
|
||||
// In the "pool" we hold up to "maxObjects" old objects, and if the pool
|
||||
// is not empty, we return one of its objects rather than allocating a new
|
||||
// one.
|
||||
ConcurrentLinkedQueue<T> pool = new ConcurrentLinkedQueue<T>();
|
||||
int maxObjects;
|
||||
|
||||
/**
|
||||
* Construct an allocator for objects of a certain type, keeping around a
|
||||
* pool of up to <CODE>maxObjects</CODE> old objects.
|
||||
* <P>
|
||||
* Note that the pool size only restricts the number of objects that hang
|
||||
* around when not needed, but <I>not</I> the maximum number of objects
|
||||
* that are allocated when actually is use: If a number of concurrent
|
||||
* threads ask for an allocation, all of them will get an object, even if
|
||||
* their number is greater than maxObjects. If an application wants to
|
||||
* limit the number of concurrent threads making allocations, it needs to
|
||||
* do so on its own - for example by blocking new threads until the
|
||||
* existing ones have finished. If more than maxObjects are freed, only
|
||||
* maxObjects of them will be kept in the pool - the rest will not and
|
||||
* will eventually be garbage-collected by Java.
|
||||
* <P>
|
||||
* In particular, when maxObjects=0, this object behaves as a trivial
|
||||
* allocator, always allocating a new array and never reusing an old one.
|
||||
*/
|
||||
public TemporaryObjectAllocator(int maxObjects) {
|
||||
this.maxObjects = maxObjects;
|
||||
}
|
||||
|
||||
/**
|
||||
* Subclasses must override this method to actually create a new object
|
||||
* of the desired type.
|
||||
*
|
||||
*/
|
||||
protected abstract T create();
|
||||
|
||||
/**
|
||||
* Subclasses must override this method to clear an existing object of
|
||||
* the desired type, to prepare it for reuse. Note that objects will be
|
||||
* cleared just before reuse (on allocation), not when freed.
|
||||
*/
|
||||
protected abstract void clear(T object);
|
||||
|
||||
/**
|
||||
* Allocate a new object. If there's a previously allocated object in our
|
||||
* pool, we return it immediately. Otherwise, a new object is allocated.
|
||||
* <P>
|
||||
* Don't forget to call {@link #free(Object)} when you're done with the object,
|
||||
* to return it to the pool. If you don't, memory is <I>not</I> leaked,
|
||||
* but the pool will remain empty and a new object will be allocated each
|
||||
* time (just like the maxArrays=0 case).
|
||||
*/
|
||||
public final T allocate() {
|
||||
T object = pool.poll();
|
||||
if (object==null) {
|
||||
return create();
|
||||
}
|
||||
clear(object);
|
||||
return object;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a no-longer-needed object back to the pool. If we already have
|
||||
* enough objects in the pool (maxObjects as specified in the constructor),
|
||||
* the array will not be saved, and Java will eventually garbage collect
|
||||
* it.
|
||||
* <P>
|
||||
* In particular, when maxArrays=0, the given array is never saved and
|
||||
* free does nothing.
|
||||
*/
|
||||
public final void free(T object) {
|
||||
if (pool.size() < maxObjects && object != null) {
|
||||
pool.add(object);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -59,7 +59,7 @@ public class TopKFacetResultsHandler extends FacetResultsHandler {
|
|||
if (ordinal != TaxonomyReader.INVALID_ORDINAL) {
|
||||
double value = 0;
|
||||
if (isSelfPartition(ordinal, facetArrays, offset)) {
|
||||
int partitionSize = facetArrays.getArraysLength();
|
||||
int partitionSize = facetArrays.arrayLength;
|
||||
value = facetRequest.getValueOf(facetArrays, ordinal % partitionSize);
|
||||
}
|
||||
|
||||
|
@ -121,7 +121,7 @@ public class TopKFacetResultsHandler extends FacetResultsHandler {
|
|||
*/
|
||||
private int heapDescendants(int ordinal, Heap<FacetResultNode> pq,
|
||||
MutableFacetResultNode parentResultNode, FacetArrays facetArrays, int offset) throws IOException {
|
||||
int partitionSize = facetArrays.getArraysLength();
|
||||
int partitionSize = facetArrays.arrayLength;
|
||||
int endOffset = offset + partitionSize;
|
||||
ParallelTaxonomyArrays childrenArray = taxonomyReader.getParallelTaxonomyArrays();
|
||||
int[] children = childrenArray.children();
|
||||
|
|
|
@ -119,7 +119,7 @@ public class TopKInEachNodeHandler extends FacetResultsHandler {
|
|||
// this will grow into the returned IntermediateFacetResult
|
||||
IntToObjectMap<AACO> AACOsOfOnePartition = new IntToObjectMap<AACO>();
|
||||
|
||||
int partitionSize = arrays.getArraysLength(); // all partitions, except, possibly, the last,
|
||||
int partitionSize = arrays.arrayLength; // all partitions, except, possibly, the last,
|
||||
// have the same length. Hence modulo is OK.
|
||||
|
||||
int depth = facetRequest.getDepth();
|
||||
|
|
|
@ -4,12 +4,9 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
import org.apache.lucene.facet.search.FacetArrays;
|
||||
import org.apache.lucene.facet.search.FacetResultsHandler;
|
||||
import org.apache.lucene.facet.search.FacetsAccumulator;
|
||||
import org.apache.lucene.facet.search.FloatArrayAllocator;
|
||||
import org.apache.lucene.facet.search.IntArrayAllocator;
|
||||
import org.apache.lucene.facet.search.SamplingWrapper;
|
||||
import org.apache.lucene.facet.search.ScoredDocIDs;
|
||||
import org.apache.lucene.facet.search.StandardFacetsAccumulator;
|
||||
|
@ -18,6 +15,7 @@ import org.apache.lucene.facet.search.results.FacetResult;
|
|||
import org.apache.lucene.facet.search.results.FacetResultNode;
|
||||
import org.apache.lucene.facet.search.sampling.Sampler.SampleResult;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -61,17 +59,10 @@ public class SamplingAccumulator extends StandardFacetsAccumulator {
|
|||
private double samplingRatio = -1d;
|
||||
private final Sampler sampler;
|
||||
|
||||
/**
|
||||
* Constructor...
|
||||
*/
|
||||
public SamplingAccumulator(
|
||||
Sampler sampler,
|
||||
FacetSearchParams searchParams,
|
||||
public SamplingAccumulator(Sampler sampler, FacetSearchParams searchParams,
|
||||
IndexReader indexReader, TaxonomyReader taxonomyReader,
|
||||
IntArrayAllocator intArrayAllocator,
|
||||
FloatArrayAllocator floatArrayAllocator) {
|
||||
super(searchParams, indexReader, taxonomyReader, intArrayAllocator,
|
||||
floatArrayAllocator);
|
||||
FacetArrays facetArrays) {
|
||||
super(searchParams, indexReader, taxonomyReader, facetArrays);
|
||||
this.sampler = sampler;
|
||||
}
|
||||
|
||||
|
|
|
@ -1,11 +1,7 @@
|
|||
package org.apache.lucene.facet.search;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.facet.search.FacetArrays;
|
||||
import org.apache.lucene.facet.search.FloatArrayAllocator;
|
||||
import org.apache.lucene.facet.search.IntArrayAllocator;
|
||||
import org.junit.Test;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -27,26 +23,39 @@ import org.apache.lucene.facet.search.IntArrayAllocator;
|
|||
public class TestFacetArrays extends LuceneTestCase {
|
||||
|
||||
@Test
|
||||
public void testSimple() {
|
||||
FacetArrays arrays = new FacetArrays(new IntArrayAllocator(1, 1), new FloatArrayAllocator(1, 1));
|
||||
|
||||
int[] intArray = arrays.getIntArray();
|
||||
// Set the element, then free
|
||||
intArray[0] = 1;
|
||||
arrays.free();
|
||||
|
||||
// We should expect a cleared array back
|
||||
intArray = arrays.getIntArray();
|
||||
assertEquals("Expected a cleared array back, but the array is still filled", 0, intArray[0]);
|
||||
|
||||
float[] floatArray = arrays.getFloatArray();
|
||||
// Set the element, then free
|
||||
floatArray[0] = 1.0f;
|
||||
arrays.free();
|
||||
|
||||
// We should expect a cleared array back
|
||||
floatArray = arrays.getFloatArray();
|
||||
assertEquals("Expected a cleared array back, but the array is still filled", 0.0f, floatArray[0], 0.0);
|
||||
public void testFacetArrays() {
|
||||
for (boolean reusing : new boolean[] { false, true }) {
|
||||
final FacetArrays arrays;
|
||||
if (reusing) {
|
||||
arrays = new ReusingFacetArrays(new ArraysPool(1, 1));
|
||||
} else {
|
||||
arrays = new FacetArrays(1);
|
||||
}
|
||||
|
||||
int[] intArray = arrays.getIntArray();
|
||||
// Set the element, then free
|
||||
intArray[0] = 1;
|
||||
arrays.free();
|
||||
|
||||
// We should expect a cleared array back
|
||||
int[] newIntArray = arrays.getIntArray();
|
||||
assertEquals("Expected a cleared array back, but the array is still filled", 0, newIntArray[0]);
|
||||
|
||||
float[] floatArray = arrays.getFloatArray();
|
||||
// Set the element, then free
|
||||
floatArray[0] = 1.0f;
|
||||
arrays.free();
|
||||
|
||||
// We should expect a cleared array back
|
||||
float[] newFloatArray = arrays.getFloatArray();
|
||||
assertEquals("Expected a cleared array back, but the array is still filled", 0.0f, newFloatArray[0], 0.0);
|
||||
|
||||
if (reusing) {
|
||||
// same instance should be returned after free()
|
||||
assertSame("ReusingFacetArrays did not reuse the array!", intArray, newIntArray);
|
||||
assertSame("ReusingFacetArrays did not reuse the array!", floatArray, newFloatArray);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -163,9 +163,8 @@ public class TestTopKInEachNodeResultHandler extends LuceneTestCase {
|
|||
facetSearchParams.addFacetRequest(doctor);
|
||||
facetSearchParams.addFacetRequest(cfrb20);
|
||||
|
||||
IntArrayAllocator iaa = new IntArrayAllocator(PartitionsUtils.partitionSize(facetSearchParams,tr), 1);
|
||||
FloatArrayAllocator faa = new FloatArrayAllocator(PartitionsUtils.partitionSize(facetSearchParams,tr), 1);
|
||||
FacetsAccumulator fctExtrctr = new StandardFacetsAccumulator(facetSearchParams, is.getIndexReader(), tr, iaa, faa);
|
||||
FacetArrays facetArrays = new FacetArrays(PartitionsUtils.partitionSize(facetSearchParams,tr));
|
||||
FacetsAccumulator fctExtrctr = new StandardFacetsAccumulator(facetSearchParams, is.getIndexReader(), tr, facetArrays);
|
||||
fctExtrctr.setComplementThreshold(FacetsAccumulator.DISABLE_COMPLEMENT);
|
||||
long start = System.currentTimeMillis();
|
||||
|
||||
|
|
Loading…
Reference in New Issue