Aggregations: simplifications.

BucketAggregationMode used to be part of the framework, now it's only an
implementation detail of the terms, histogram, geohash grid and scripted
aggregators.

Aggregator.estimatedBucketCount() was a complicated way to do the initial sizing
of the data structures, but it did not work very well in practice and was rather
likely to lead to over-sized data-structures causing OOMEs. It's removed now and
all data-structures start with a size of 1 and grow exponentially.

Aggregator.preCollection() is now symetric with postCollection(): it exists on
all aggregation objects where postCollection() also is and recursively calls its
children.

Fixed other minor issues related to generics and exceptions.

Close #9097
This commit is contained in:
Adrien Grand 2014-12-30 09:08:18 +01:00
parent b8be8e432e
commit d9165dfe73
62 changed files with 673 additions and 621 deletions

View File

@ -20,10 +20,12 @@ package org.elasticsearch.common.lucene;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import java.io.IOException;
/** /**
* *
*/ */
public interface ReaderContextAware { public interface ReaderContextAware {
public void setNextReader(LeafReaderContext reader); public void setNextReader(LeafReaderContext reader) throws IOException;
} }

View File

@ -19,18 +19,16 @@
package org.elasticsearch.common.lucene.search.function; package org.elasticsearch.common.lucene.search.function;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Explanation; import org.apache.lucene.search.Explanation;
import org.elasticsearch.common.lucene.ReaderContextAware;
/** /**
* *
*/ */
public abstract class ScoreFunction { public abstract class ScoreFunction implements ReaderContextAware {
private final CombineFunction scoreCombiner; private final CombineFunction scoreCombiner;
public abstract void setNextReader(LeafReaderContext context);
public abstract double score(int docId, float subQueryScore); public abstract double score(int docId, float subQueryScore);
public abstract Explanation explainScore(int docId, float subQueryScore); public abstract Explanation explainScore(int docId, float subQueryScore);

View File

@ -88,7 +88,7 @@ public class ScriptScoreFunction extends ScoreFunction {
} }
@Override @Override
public void setNextReader(LeafReaderContext ctx) { public void setNextReader(LeafReaderContext ctx) throws IOException {
script.setNextReader(ctx); script.setNextReader(ctx);
} }

View File

@ -24,6 +24,8 @@ import org.apache.lucene.search.ComplexExplanation;
import org.apache.lucene.search.Explanation; import org.apache.lucene.search.Explanation;
import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.ElasticsearchIllegalArgumentException;
import java.io.IOException;
/** /**
* *
*/ */
@ -53,7 +55,7 @@ public class WeightFactorFunction extends ScoreFunction {
} }
@Override @Override
public void setNextReader(LeafReaderContext context) { public void setNextReader(LeafReaderContext context) throws IOException {
scoreFunction.setNextReader(context); scoreFunction.setNextReader(context);
} }

View File

@ -58,7 +58,7 @@ public class BytesRefFieldComparatorSource extends IndexFieldData.XFieldComparat
return SortField.Type.STRING; return SortField.Type.STRING;
} }
protected SortedBinaryDocValues getValues(LeafReaderContext context) { protected SortedBinaryDocValues getValues(LeafReaderContext context) throws IOException {
return indexFieldData.load(context).getBytesValues(); return indexFieldData.load(context).getBytesValues();
} }

View File

@ -56,7 +56,7 @@ public class DoubleValuesComparatorSource extends IndexFieldData.XFieldComparato
return SortField.Type.DOUBLE; return SortField.Type.DOUBLE;
} }
protected SortedNumericDoubleValues getValues(LeafReaderContext context) { protected SortedNumericDoubleValues getValues(LeafReaderContext context) throws IOException {
return indexFieldData.load(context).getDoubleValues(); return indexFieldData.load(context).getDoubleValues();
} }

View File

@ -63,7 +63,7 @@ abstract class QueryCollector extends SimpleCollector {
List<LeafCollector> aggregatorLeafCollectors; List<LeafCollector> aggregatorLeafCollectors;
QueryCollector(ESLogger logger, PercolateContext context, boolean isNestedDoc) { QueryCollector(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
this.logger = logger; this.logger = logger;
this.queries = context.percolateQueries(); this.queries = context.percolateQueries();
this.searcher = context.docSearcher(); this.searcher = context.docSearcher();
@ -126,19 +126,19 @@ abstract class QueryCollector extends SimpleCollector {
} }
static Match match(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) { static Match match(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {
return new Match(logger, context, highlightPhase, isNestedDoc); return new Match(logger, context, highlightPhase, isNestedDoc);
} }
static Count count(ESLogger logger, PercolateContext context, boolean isNestedDoc) { static Count count(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
return new Count(logger, context, isNestedDoc); return new Count(logger, context, isNestedDoc);
} }
static MatchAndScore matchAndScore(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) { static MatchAndScore matchAndScore(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {
return new MatchAndScore(logger, context, highlightPhase, isNestedDoc); return new MatchAndScore(logger, context, highlightPhase, isNestedDoc);
} }
static MatchAndSort matchAndSort(ESLogger logger, PercolateContext context, boolean isNestedDoc) { static MatchAndSort matchAndSort(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
return new MatchAndSort(logger, context, isNestedDoc); return new MatchAndSort(logger, context, isNestedDoc);
} }
@ -167,7 +167,7 @@ abstract class QueryCollector extends SimpleCollector {
final int size; final int size;
long counter = 0; long counter = 0;
Match(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) { Match(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {
super(logger, context, isNestedDoc); super(logger, context, isNestedDoc);
this.limit = context.limit; this.limit = context.limit;
this.size = context.size(); this.size = context.size();
@ -228,7 +228,7 @@ abstract class QueryCollector extends SimpleCollector {
private final TopScoreDocCollector topDocsCollector; private final TopScoreDocCollector topDocsCollector;
private LeafCollector topDocsLeafCollector; private LeafCollector topDocsLeafCollector;
MatchAndSort(ESLogger logger, PercolateContext context, boolean isNestedDoc) { MatchAndSort(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
super(logger, context, isNestedDoc); super(logger, context, isNestedDoc);
// TODO: Use TopFieldCollector.create(...) for ascending and descending scoring? // TODO: Use TopFieldCollector.create(...) for ascending and descending scoring?
topDocsCollector = TopScoreDocCollector.create(context.size(), false); topDocsCollector = TopScoreDocCollector.create(context.size(), false);
@ -289,7 +289,7 @@ abstract class QueryCollector extends SimpleCollector {
private Scorer scorer; private Scorer scorer;
MatchAndScore(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) { MatchAndScore(ESLogger logger, PercolateContext context, HighlightPhase highlightPhase, boolean isNestedDoc) throws IOException {
super(logger, context, isNestedDoc); super(logger, context, isNestedDoc);
this.limit = context.limit; this.limit = context.limit;
this.size = context.size(); this.size = context.size();
@ -358,7 +358,7 @@ abstract class QueryCollector extends SimpleCollector {
private long counter = 0; private long counter = 0;
Count(ESLogger logger, PercolateContext context, boolean isNestedDoc) { Count(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException {
super(logger, context, isNestedDoc); super(logger, context, isNestedDoc);
} }

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations; package org.elasticsearch.search.aggregations;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Filter; import org.apache.lucene.search.Filter;
@ -27,6 +28,7 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector; import org.apache.lucene.search.SimpleCollector;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.lucene.search.XCollector; import org.elasticsearch.common.lucene.search.XCollector;
@ -77,7 +79,12 @@ public class AggregationPhase implements SearchPhase {
context.aggregations().aggregationContext(aggregationContext); context.aggregations().aggregationContext(aggregationContext);
List<Aggregator> collectors = new ArrayList<>(); List<Aggregator> collectors = new ArrayList<>();
Aggregator[] aggregators = context.aggregations().factories().createTopLevelAggregators(aggregationContext); Aggregator[] aggregators;
try {
aggregators = context.aggregations().factories().createTopLevelAggregators(aggregationContext);
} catch (IOException e) {
throw new AggregationInitializationException("Could not initialize aggregators", e);
}
for (int i = 0; i < aggregators.length; i++) { for (int i = 0; i < aggregators.length; i++) {
if (!(aggregators[i] instanceof GlobalAggregator)) { if (!(aggregators[i] instanceof GlobalAggregator)) {
Aggregator aggregator = aggregators[i]; Aggregator aggregator = aggregators[i];
@ -132,7 +139,11 @@ public class AggregationPhase implements SearchPhase {
List<InternalAggregation> aggregations = new ArrayList<>(aggregators.length); List<InternalAggregation> aggregations = new ArrayList<>(aggregators.length);
for (Aggregator aggregator : context.aggregations().aggregators()) { for (Aggregator aggregator : context.aggregations().aggregators()) {
try {
aggregations.add(aggregator.buildAggregation(0)); aggregations.add(aggregator.buildAggregation(0));
} catch (IOException e) {
throw new AggregationExecutionException("Failed to build aggregation [" + aggregator.name() + "]", e);
}
} }
context.queryResult().aggregations(new InternalAggregations(aggregations)); context.queryResult().aggregations(new InternalAggregations(aggregations));

View File

@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Scorer;
import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ElasticsearchParseException;
@ -27,6 +28,7 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
@ -34,53 +36,37 @@ import org.elasticsearch.search.internal.SearchContext.Lifetime;
import org.elasticsearch.search.query.QueryPhaseExecutionException; import org.elasticsearch.search.query.QueryPhaseExecutionException;
import java.io.IOException; import java.io.IOException;
import java.util.*; import java.util.ArrayList;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public abstract class Aggregator extends BucketCollector implements Releasable { public abstract class Aggregator extends BucketCollector implements Releasable {
/**
* Returns whether one of the parents is a {@link BucketsAggregator}.
*/
public static boolean descendsFromBucketAggregator(Aggregator parent) {
while (parent != null) {
if (parent instanceof BucketsAggregator) {
return true;
}
parent = parent.parent;
}
return false;
}
private static final Predicate<Aggregator> COLLECTABLE_AGGREGATOR = new Predicate<Aggregator>() { private static final Predicate<Aggregator> COLLECTABLE_AGGREGATOR = new Predicate<Aggregator>() {
@Override @Override
public boolean apply(Aggregator aggregator) { public boolean apply(Aggregator aggregator) {
return aggregator.shouldCollect(); return aggregator.shouldCollect();
} }
}; };
private final Map<String, Object> metaData;
/**
* Returns whether any of the parent aggregators has {@link BucketAggregationMode#PER_BUCKET} as a bucket aggregation mode.
*/
public static boolean hasParentBucketAggregator(Aggregator parent) {
if (parent == null) {
return false;
} else if (parent.bucketAggregationMode() == BucketAggregationMode.PER_BUCKET) {
return true;
} else {
return hasParentBucketAggregator(parent.parent());
}
}
public static final ParseField COLLECT_MODE = new ParseField("collect_mode"); public static final ParseField COLLECT_MODE = new ParseField("collect_mode");
public Map<String, Object> getMetaData() {
return this.metaData;
}
/**
* Defines the nature of the aggregator's aggregation execution when nested in other aggregators and the buckets they create.
*/
public static enum BucketAggregationMode {
/**
* In this mode, a new aggregator instance will be created per bucket (created by the parent aggregator)
*/
PER_BUCKET,
/**
* In this mode, a single aggregator instance will be created per parent aggregator, that will handle the aggregations of all its buckets.
*/
MULTI_BUCKETS
}
public enum SubAggCollectionMode { public enum SubAggCollectionMode {
/** /**
@ -163,9 +149,8 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
protected final AggregationContext context; protected final AggregationContext context;
protected final BigArrays bigArrays; protected final BigArrays bigArrays;
protected final int depth; protected final int depth;
protected final long estimatedBucketCount; private final Map<String, Object> metaData;
protected final BucketAggregationMode bucketAggregationMode;
protected final AggregatorFactories factories; protected final AggregatorFactories factories;
protected final Aggregator[] subAggregators; protected final Aggregator[] subAggregators;
protected BucketCollector collectableSubAggregators; protected BucketCollector collectableSubAggregators;
@ -177,25 +162,21 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
* Constructs a new Aggregator. * Constructs a new Aggregator.
* *
* @param name The name of the aggregation * @param name The name of the aggregation
* @param bucketAggregationMode The nature of execution as a sub-aggregator (see {@link BucketAggregationMode})
* @param factories The factories for all the sub-aggregators under this aggregator * @param factories The factories for all the sub-aggregators under this aggregator
* @param estimatedBucketsCount When served as a sub-aggregator, indicate how many buckets the parent aggregator will generate.
* @param context The aggregation context * @param context The aggregation context
* @param parent The parent aggregator (may be {@code null} for top level aggregators) * @param parent The parent aggregator (may be {@code null} for top level aggregators)
* @param metaData The metaData associated with this aggregator * @param metaData The metaData associated with this aggregator
*/ */
protected Aggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { protected Aggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
this.name = name; this.name = name;
this.metaData = metaData; this.metaData = metaData;
this.parent = parent; this.parent = parent;
this.estimatedBucketCount = estimatedBucketsCount;
this.context = context; this.context = context;
this.bigArrays = context.bigArrays(); this.bigArrays = context.bigArrays();
this.depth = parent == null ? 0 : 1 + parent.depth(); 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"; assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead";
this.factories = factories; this.factories = factories;
this.subAggregators = factories.createSubAggregators(this, estimatedBucketsCount); this.subAggregators = factories.createSubAggregators(this);
context.searchContext().addReleasable(this, Lifetime.PHASE); context.searchContext().addReleasable(this, Lifetime.PHASE);
// Register a safeguard to highlight any invalid construction logic (call to this constructor without subsequent preCollection call) // Register a safeguard to highlight any invalid construction logic (call to this constructor without subsequent preCollection call)
collectableSubAggregators = new BucketCollector() { collectableSubAggregators = new BucketCollector() {
@ -208,6 +189,11 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
badState(); badState();
} }
@Override
public void preCollection() throws IOException {
badState();
}
@Override @Override
public void postCollection() throws IOException { public void postCollection() throws IOException {
badState(); badState();
@ -225,7 +211,17 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
}; };
} }
protected void preCollection() { public Map<String, Object> metaData() {
return this.metaData;
}
/**
* Can be overriden by aggregator implementation to be called back when the collection phase starts.
*/
protected void doPreCollection() throws IOException {
}
public final void preCollection() throws IOException {
Iterable<Aggregator> collectables = Iterables.filter(Arrays.asList(subAggregators), COLLECTABLE_AGGREGATOR); Iterable<Aggregator> collectables = Iterables.filter(Arrays.asList(subAggregators), COLLECTABLE_AGGREGATOR);
List<BucketCollector> nextPassCollectors = new ArrayList<>(); List<BucketCollector> nextPassCollectors = new ArrayList<>();
List<BucketCollector> thisPassCollectors = new ArrayList<>(); List<BucketCollector> thisPassCollectors = new ArrayList<>();
@ -247,6 +243,8 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
thisPassCollectors.add(recordingWrapper); thisPassCollectors.add(recordingWrapper);
} }
collectableSubAggregators = BucketCollector.wrap(thisPassCollectors); collectableSubAggregators = BucketCollector.wrap(thisPassCollectors);
collectableSubAggregators.preCollection();
doPreCollection();
} }
/** /**
@ -279,11 +277,6 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
return name; return name;
} }
/** Return the estimated number of buckets. */
public final long estimatedBucketCount() {
return estimatedBucketCount;
}
/** Return the depth of this aggregator in the aggregation tree. */ /** Return the depth of this aggregator in the aggregation tree. */
public final int depth() { public final int depth() {
return depth; return depth;
@ -320,14 +313,6 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
return context; return context;
} }
/**
* @return The bucket aggregation mode of this aggregator. This mode defines the nature in which the aggregation is executed
* @see BucketAggregationMode
*/
public BucketAggregationMode bucketAggregationMode() {
return bucketAggregationMode;
}
/** /**
* @return Whether this aggregator is in the state where it can collect documents. Some aggregators can do their aggregations without * @return Whether this aggregator is in the state where it can collect documents. Some aggregators can do their aggregations without
* actually collecting documents, for example, an aggregator that computes stats over unmapped fields doesn't need to collect * actually collecting documents, for example, an aggregator that computes stats over unmapped fields doesn't need to collect
@ -363,10 +348,10 @@ public abstract class Aggregator extends BucketCollector implements Releasable {
/** /**
* @return The aggregated & built aggregation * @return The aggregated & built aggregation
*/ */
public abstract InternalAggregation buildAggregation(long owningBucketOrdinal); public abstract InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException;
@Override @Override
public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) { public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) throws IOException {
results.add(buildAggregation(bucketOrdinal)); results.add(buildAggregation(bucketOrdinal));
} }

View File

@ -18,12 +18,7 @@
*/ */
package org.elasticsearch.search.aggregations; package org.elasticsearch.search.aggregations;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.ElasticsearchIllegalArgumentException; import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.search.aggregations.Aggregator.BucketAggregationMode;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException; import java.io.IOException;
@ -49,8 +44,8 @@ public class AggregatorFactories {
this.factories = factories; this.factories = factories;
} }
private static Aggregator createAndRegisterContextAware(AggregationContext context, AggregatorFactory factory, Aggregator parent, long estimatedBucketsCount) { private static Aggregator createAndRegisterContextAware(AggregationContext context, AggregatorFactory factory, Aggregator parent, boolean collectsFromSingleBucket) throws IOException {
final Aggregator aggregator = factory.create(context, parent, estimatedBucketsCount); final Aggregator aggregator = factory.create(context, parent, collectsFromSingleBucket);
if (aggregator.shouldCollect()) { if (aggregator.shouldCollect()) {
context.registerReaderContextAware(aggregator); context.registerReaderContextAware(aggregator);
} }
@ -64,97 +59,26 @@ public class AggregatorFactories {
/** /**
* Create all aggregators so that they can be consumed with multiple buckets. * Create all aggregators so that they can be consumed with multiple buckets.
*/ */
public Aggregator[] createSubAggregators(Aggregator parent, final long estimatedBucketsCount) { public Aggregator[] createSubAggregators(Aggregator parent) throws IOException {
Aggregator[] aggregators = new Aggregator[count()]; Aggregator[] aggregators = new Aggregator[count()];
for (int i = 0; i < factories.length; ++i) { for (int i = 0; i < factories.length; ++i) {
final AggregatorFactory factory = factories[i]; // TODO: sometimes even sub aggregations always get called with bucket 0, eg. if
final Aggregator first = createAndRegisterContextAware(parent.context(), factory, parent, estimatedBucketsCount); // you have a terms agg under a top-level filter agg. We should have a way to
if (first.bucketAggregationMode() == BucketAggregationMode.MULTI_BUCKETS) { // propagate the fact that only bucket 0 will be collected with single-bucket
// This aggregator already supports multiple bucket ordinals, can be used directly // aggs
aggregators[i] = first; final boolean collectsFromSingleBucket = false;
continue; aggregators[i] = createAndRegisterContextAware(parent.context(), factories[i], parent, collectsFromSingleBucket);
}
// the aggregator doesn't support multiple ordinals, let's wrap it so that it does.
aggregators[i] = new Aggregator(first.name(), BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, 1, first.context(), first.parent(), first.getMetaData()) {
ObjectArray<Aggregator> aggregators;
{
// if estimated count is zero, we at least create a single aggregator.
// 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);
aggregators.set(0, first);
}
@Override
public boolean shouldCollect() {
return first.shouldCollect();
}
@Override
protected void doPostCollection() throws IOException {
for (long i = 0; i < aggregators.size(); ++i) {
final Aggregator aggregator = aggregators.get(i);
if (aggregator != null) {
aggregator.postCollection();
}
}
}
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
aggregators = bigArrays.grow(aggregators, owningBucketOrdinal + 1);
Aggregator aggregator = aggregators.get(owningBucketOrdinal);
if (aggregator == null) {
aggregator = createAndRegisterContextAware(parent.context(), factory, parent, estimatedBucketsCount);
aggregators.set(owningBucketOrdinal, aggregator);
}
aggregator.collect(doc, 0);
}
@Override
public void setNextReader(LeafReaderContext reader) {
}
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
throw new ElasticsearchIllegalStateException("Invalid context - aggregation must use addResults() to collect child results");
}
@Override
public InternalAggregation buildEmptyAggregation() {
return first.buildEmptyAggregation();
}
@Override
public void doClose() {
Releasables.close(aggregators);
}
@Override
public void gatherAnalysis(BucketAnalysisCollector results, long owningBucketOrdinal) {
// The bucket ordinal may be out of range in case of eg. a terms/filter/terms where
// the filter matches no document in the highest buckets of the first terms agg
if (owningBucketOrdinal >= aggregators.size() || aggregators.get(owningBucketOrdinal) == null) {
results.add(first.buildEmptyAggregation());
} else {
aggregators.get(owningBucketOrdinal).gatherAnalysis(results,0);
}
}
};
aggregators[i].preCollection();
} }
return aggregators; return aggregators;
} }
public Aggregator[] createTopLevelAggregators(AggregationContext ctx) { public Aggregator[] createTopLevelAggregators(AggregationContext ctx) throws IOException {
// These aggregators are going to be used with a single bucket ordinal, no need to wrap the PER_BUCKET ones // These aggregators are going to be used with a single bucket ordinal, no need to wrap the PER_BUCKET ones
Aggregator[] aggregators = new Aggregator[factories.length]; Aggregator[] aggregators = new Aggregator[factories.length];
for (int i = 0; i < factories.length; i++) { for (int i = 0; i < factories.length; i++) {
aggregators[i] = createAndRegisterContextAware(ctx, factories[i], null, 0); // top-level aggs only get called with bucket 0
final boolean collectsFromSingleBucket = true;
aggregators[i] = createAndRegisterContextAware(ctx, factories[i], null, collectsFromSingleBucket);
} }
return aggregators; return aggregators;
} }
@ -185,7 +109,7 @@ public class AggregatorFactories {
} }
@Override @Override
public Aggregator[] createSubAggregators(Aggregator parent, long estimatedBucketsCount) { public Aggregator[] createSubAggregators(Aggregator parent) {
return EMPTY_AGGREGATORS; return EMPTY_AGGREGATORS;
} }

View File

@ -18,8 +18,13 @@
*/ */
package org.elasticsearch.search.aggregations; package org.elasticsearch.search.aggregations;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.Map; import java.util.Map;
/** /**
@ -72,20 +77,20 @@ public abstract class AggregatorFactory {
return parent; return parent;
} }
protected abstract Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException;
/** /**
* Creates the aggregator * Creates the aggregator
* *
* @param context The aggregation context * @param context The aggregation context
* @param parent The parent aggregator (if this is a top level factory, the parent will be {@code null}) * @param parent The parent aggregator (if this is a top level factory, the parent will be {@code null})
* @param expectedBucketsCount If this is a sub-factory of another factory, this will indicate the number of bucket the parent aggregator * @param collectsFromSingleBucket If true then the created aggregator will only be collected with <tt>0</tt> as a bucket ordinal.
* may generate (this is an estimation only). For top level factories, this will always be 0 * Some factories can take advantage of this in order to return more optimized implementations.
* *
* @return The created aggregator * @return The created aggregator
*/ */
protected abstract Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData); public final Aggregator create(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket) throws IOException {
Aggregator aggregator = createInternal(context, parent, collectsFromSingleBucket, this.metaData);
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
Aggregator aggregator = createInternal(context, parent, expectedBucketsCount, this.metaData);
return aggregator; return aggregator;
} }
@ -95,4 +100,98 @@ public abstract class AggregatorFactory {
public void setMetaData(Map<String, Object> metaData) { public void setMetaData(Map<String, Object> metaData) {
this.metaData = metaData; this.metaData = metaData;
} }
/**
* Utility method. Given an {@link AggregatorFactory} that creates {@link Aggregator}s that only know how
* to collect bucket <tt>0</tt>, this returns an aggregator that can collect any bucket.
*/
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final AggregationContext context, Aggregator parent) throws IOException {
final Aggregator first = factory.create(context, parent, true);
return new Aggregator(first.name(), AggregatorFactories.EMPTY, first.context(), first.parent(), first.metaData()) {
ObjectArray<Aggregator> aggregators;
LeafReaderContext readerContext;
{
aggregators = bigArrays.newObjectArray(1);
aggregators.set(0, first);
}
@Override
public boolean shouldCollect() {
return first.shouldCollect();
}
@Override
protected void doPreCollection() throws IOException {
for (long i = 0; i < aggregators.size(); ++i) {
final Aggregator aggregator = aggregators.get(i);
if (aggregator != null) {
aggregator.preCollection();
}
}
}
@Override
protected void doPostCollection() throws IOException {
for (long i = 0; i < aggregators.size(); ++i) {
final Aggregator aggregator = aggregators.get(i);
if (aggregator != null) {
aggregator.postCollection();
}
}
}
@Override
public void collect(int doc, long owningBucketOrdinal) throws IOException {
aggregators = bigArrays.grow(aggregators, owningBucketOrdinal + 1);
Aggregator aggregator = aggregators.get(owningBucketOrdinal);
if (aggregator == null) {
aggregator = factory.create(context, parent, true);
aggregator.preCollection();
aggregator.setNextReader(readerContext);
aggregators.set(owningBucketOrdinal, aggregator);
}
aggregator.collect(doc, 0);
}
@Override
public void setNextReader(LeafReaderContext context) throws IOException {
this.readerContext = context;
for (long i = 0; i < aggregators.size(); ++i) {
final Aggregator aggregator = aggregators.get(i);
if (aggregator != null) {
aggregator.setNextReader(context);
}
}
}
@Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
throw new ElasticsearchIllegalStateException("Invalid context - aggregation must use addResults() to collect child results");
}
@Override
public InternalAggregation buildEmptyAggregation() {
return first.buildEmptyAggregation();
}
@Override
public void doClose() {
Releasables.close(aggregators);
}
@Override
public void gatherAnalysis(BucketAnalysisCollector results, long owningBucketOrdinal) throws IOException {
// The bucket ordinal may be out of range in case of eg. a terms/filter/terms where
// the filter matches no document in the highest buckets of the first terms agg
if (owningBucketOrdinal >= aggregators.size() || aggregators.get(owningBucketOrdinal) == null) {
results.add(first.buildEmptyAggregation());
} else {
aggregators.get(owningBucketOrdinal).gatherAnalysis(results,0);
}
}
};
}
} }

View File

@ -20,9 +20,9 @@
package org.elasticsearch.search.aggregations; package org.elasticsearch.search.aggregations;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.lucene.ReaderContextAware; import org.elasticsearch.common.lucene.ReaderContextAware;
import org.elasticsearch.search.aggregations.Aggregator.BucketAggregationMode;
import java.io.IOException; import java.io.IOException;
@ -52,6 +52,9 @@ public abstract class BucketCollector implements ReaderContextAware {
public void setNextReader(LeafReaderContext reader) { public void setNextReader(LeafReaderContext reader) {
// no-op // no-op
} }
public void preCollection() throws IOException {
// no-op
}
@Override @Override
public void postCollection() throws IOException { public void postCollection() throws IOException {
// no-op // no-op
@ -83,12 +86,19 @@ public abstract class BucketCollector implements ReaderContextAware {
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public void setNextReader(LeafReaderContext reader) throws IOException {
for (BucketCollector collector : collectors) { for (BucketCollector collector : collectors) {
collector.setNextReader(reader); collector.setNextReader(reader);
} }
} }
@Override
public void preCollection() throws IOException {
for (BucketCollector collector : collectors) {
collector.preCollection();
}
}
@Override @Override
public void postCollection() throws IOException { public void postCollection() throws IOException {
for (BucketCollector collector : collectors) { for (BucketCollector collector : collectors) {
@ -97,7 +107,7 @@ public abstract class BucketCollector implements ReaderContextAware {
} }
@Override @Override
public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) { public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) throws IOException {
for (BucketCollector collector : collectors) { for (BucketCollector collector : collectors) {
collector.gatherAnalysis(results, bucketOrdinal); collector.gatherAnalysis(results, bucketOrdinal);
} }
@ -120,6 +130,11 @@ public abstract class BucketCollector implements ReaderContextAware {
*/ */
public abstract void collect(int docId, long bucketOrdinal) throws IOException; public abstract void collect(int docId, long bucketOrdinal) throws IOException;
/**
* Pre collection callback.
*/
public abstract void preCollection() throws IOException;
/** /**
* Post collection callback. * Post collection callback.
*/ */
@ -130,5 +145,5 @@ public abstract class BucketCollector implements ReaderContextAware {
* @param analysisCollector * @param analysisCollector
* @param bucketOrdinal * @param bucketOrdinal
*/ */
public abstract void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal); public abstract void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) throws IOException;
} }

View File

@ -52,7 +52,7 @@ public class FilteringBucketCollector extends BucketCollector implements Releasa
} }
@Override @Override
public final void setNextReader(LeafReaderContext reader) { public final void setNextReader(LeafReaderContext reader) throws IOException {
delegate.setNextReader(reader); delegate.setNextReader(reader);
} }
@ -64,6 +64,11 @@ public class FilteringBucketCollector extends BucketCollector implements Releasa
} }
} }
@Override
public void preCollection() throws IOException {
delegate.preCollection();
}
@Override @Override
public final void postCollection() throws IOException { public final void postCollection() throws IOException {
delegate.postCollection(); delegate.postCollection();
@ -75,7 +80,7 @@ public class FilteringBucketCollector extends BucketCollector implements Releasa
} }
@Override @Override
public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal){ public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) throws IOException{
long ordinal = denseMap.find(bucketOrdinal); long ordinal = denseMap.find(bucketOrdinal);
if (ordinal >= 0) { if (ordinal >= 0) {
delegate.gatherAnalysis(analysisCollector, ordinal); delegate.gatherAnalysis(analysisCollector, ordinal);

View File

@ -31,11 +31,11 @@ import java.util.Map;
*/ */
public abstract class NonCollectingAggregator extends Aggregator { public abstract class NonCollectingAggregator extends Aggregator {
protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, AggregatorFactories subFactories, Map<String, Object> metaData) { protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, AggregatorFactories subFactories, Map<String, Object> metaData) throws IOException {
super(name, BucketAggregationMode.MULTI_BUCKETS, subFactories, 0, context, parent, metaData); super(name, subFactories, context, parent, metaData);
} }
protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
this(name, context, parent, AggregatorFactories.EMPTY, metaData); this(name, context, parent, AggregatorFactories.EMPTY, metaData);
} }

View File

@ -149,6 +149,11 @@ public class RecordingPerReaderBucketCollector extends RecordingBucketCollector
collector.postCollection(); collector.postCollection();
} }
@Override
public void preCollection() throws IOException {
// nothing to do
}
@Override @Override
public void postCollection() throws IOException { public void postCollection() throws IOException {
recordingComplete = true; recordingComplete = true;

View File

@ -35,10 +35,10 @@ public abstract class BucketsAggregator extends Aggregator {
private IntArray docCounts; private IntArray docCounts;
public BucketsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, public BucketsAggregator(String name, AggregatorFactories factories,
long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent, metaData); super(name, factories, context, parent, metaData);
docCounts = bigArrays.newIntArray(estimatedBucketsCount, true); docCounts = bigArrays.newIntArray(1, true);
} }
/** /**
@ -48,11 +48,18 @@ public abstract class BucketsAggregator extends Aggregator {
return docCounts.size(); return docCounts.size();
} }
/**
* Ensure there are at least <code>maxBucketOrd</code> buckets available.
*/
protected final void grow(long maxBucketOrd) {
docCounts = bigArrays.grow(docCounts, maxBucketOrd);
}
/** /**
* Utility method to collect the given doc in the given bucket (identified by the bucket ordinal) * 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 { protected final void collectBucket(int doc, long bucketOrd) throws IOException {
docCounts = bigArrays.grow(docCounts, bucketOrd + 1); grow(bucketOrd + 1);
collectExistingBucket(doc, bucketOrd); collectExistingBucket(doc, bucketOrd);
} }
@ -101,7 +108,7 @@ public abstract class BucketsAggregator extends Aggregator {
/** /**
* Required method to build the child aggregations of the given bucket (identified by the bucket ordinal). * Required method to build the child aggregations of the given bucket (identified by the bucket ordinal).
*/ */
protected final InternalAggregations bucketAggregations(long bucketOrd) { protected final InternalAggregations bucketAggregations(long bucketOrd) throws IOException {
final ArrayList<InternalAggregation> childAggs = new ArrayList<>(); final ArrayList<InternalAggregation> childAggs = new ArrayList<>();
final long bucketDocCount = bucketDocCount(bucketOrd); final long bucketDocCount = bucketDocCount(bucketOrd);
if (bucketDocCount == 0L) { if (bucketDocCount == 0L) {

View File

@ -49,14 +49,14 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa
private FilteringBucketCollector filteredCollector; private FilteringBucketCollector filteredCollector;
public DeferringBucketCollector (BucketCollector deferred, AggregationContext context) { public DeferringBucketCollector(BucketCollector deferred, AggregationContext context) {
this.deferred = deferred; this.deferred = deferred;
this.recording = new RecordingPerReaderBucketCollector(context); this.recording = new RecordingPerReaderBucketCollector(context);
this.context = context; this.context = context;
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public void setNextReader(LeafReaderContext reader) throws IOException {
recording.setNextReader(reader); recording.setNextReader(reader);
} }
@ -65,6 +65,11 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa
recording.collect(docId, bucketOrdinal); recording.collect(docId, bucketOrdinal);
} }
@Override
public void preCollection() throws IOException {
recording.preCollection();
}
@Override @Override
public void postCollection() throws IOException { public void postCollection() throws IOException {
recording.postCollection(); recording.postCollection();
@ -82,7 +87,7 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa
BucketCollector subs = new BucketCollector() { BucketCollector subs = new BucketCollector() {
@Override @Override
public void setNextReader(LeafReaderContext reader) { public void setNextReader(LeafReaderContext reader) throws IOException {
// Need to set AggregationContext otherwise ValueSources in aggs // Need to set AggregationContext otherwise ValueSources in aggs
// don't read any values // don't read any values
context.setNextReader(reader); context.setNextReader(reader);
@ -94,13 +99,18 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa
deferred.collect(docId, bucketOrdinal); deferred.collect(docId, bucketOrdinal);
} }
@Override
public void preCollection() throws IOException {
deferred.preCollection();
}
@Override @Override
public void postCollection() throws IOException { public void postCollection() throws IOException {
deferred.postCollection(); deferred.postCollection();
} }
@Override @Override
public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) { public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) throws IOException {
deferred.gatherAnalysis(results, bucketOrdinal); deferred.gatherAnalysis(results, bucketOrdinal);
} }
}; };
@ -121,7 +131,7 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa
} }
@Override @Override
public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) { public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) throws IOException {
filteredCollector.gatherAnalysis(analysisCollector, bucketOrdinal); filteredCollector.gatherAnalysis(analysisCollector, bucketOrdinal);
} }

View File

@ -22,6 +22,7 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.Map; import java.util.Map;
/** /**
@ -30,8 +31,8 @@ import java.util.Map;
public abstract class SingleBucketAggregator extends BucketsAggregator { public abstract class SingleBucketAggregator extends BucketsAggregator {
protected SingleBucketAggregator(String name, AggregatorFactories factories, protected SingleBucketAggregator(String name, AggregatorFactories factories,
AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, BucketAggregationMode.MULTI_BUCKETS, factories, parent == null ? 1 : parent.estimatedBucketCount(), aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
} }
@Override @Override

View File

@ -72,7 +72,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
public ParentToChildrenAggregator(String name, AggregatorFactories factories, AggregationContext aggregationContext, public ParentToChildrenAggregator(String name, AggregatorFactories factories, AggregationContext aggregationContext,
Aggregator parent, String parentType, Filter childFilter, Filter parentFilter, Aggregator parent, String parentType, Filter childFilter, Filter parentFilter,
ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long maxOrd, Map<String, Object> metaData) { ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long maxOrd, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
this.parentType = parentType; this.parentType = parentType;
// these two filters are cached in the parser // these two filters are cached in the parser
@ -85,13 +85,13 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalChildren(name, 0, buildEmptySubAggregations(), getMetaData()); return new InternalChildren(name, 0, buildEmptySubAggregations(), metaData());
} }
@Override @Override
@ -185,7 +185,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
Releasables.close(parentOrdToBuckets, parentOrdToOtherBuckets); Releasables.close(parentOrdToBuckets, parentOrdToOtherBuckets);
} }
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Bytes.WithOrdinals.ParentChild, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Bytes.WithOrdinals.ParentChild> {
private final String parentType; private final String parentType;
private final Filter parentFilter; private final Filter parentFilter;
@ -199,19 +199,19 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new NonCollectingAggregator(name, aggregationContext, parent, metaData) { return new NonCollectingAggregator(name, aggregationContext, parent, metaData) {
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalChildren(name, 0, buildEmptySubAggregations(), getMetaData()); return new InternalChildren(name, 0, buildEmptySubAggregations(), metaData());
} }
}; };
} }
@Override @Override
protected Aggregator create(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
long maxOrd = valuesSource.globalMaxOrd(aggregationContext.searchContext().searcher(), parentType); long maxOrd = valuesSource.globalMaxOrd(aggregationContext.searchContext().searcher(), parentType);
return new ParentToChildrenAggregator(name, factories, aggregationContext, parent, parentType, childFilter, parentFilter, valuesSource, maxOrd, metaData); return new ParentToChildrenAggregator(name, factories, aggregationContext, parent, parentType, childFilter, parentFilter, valuesSource, maxOrd, metaData);
} }

View File

@ -43,7 +43,7 @@ public class FilterAggregator extends SingleBucketAggregator {
AggregatorFactories factories, AggregatorFactories factories,
AggregationContext aggregationContext, AggregationContext aggregationContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metaData) { Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
this.filter = filter; this.filter = filter;
} }
@ -65,13 +65,13 @@ public class FilterAggregator extends SingleBucketAggregator {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalFilter(name, 0, buildEmptySubAggregations(), getMetaData()); return new InternalFilter(name, 0, buildEmptySubAggregations(), metaData());
} }
public static class Factory extends AggregatorFactory { public static class Factory extends AggregatorFactory {
@ -84,7 +84,7 @@ public class FilterAggregator extends SingleBucketAggregator {
} }
@Override @Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) { public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new FilterAggregator(name, filter, factories, context, parent, metaData); return new FilterAggregator(name, filter, factories, context, parent, metaData);
} }

View File

@ -53,9 +53,8 @@ public class FiltersAggregator extends BucketsAggregator {
private boolean keyed; private boolean keyed;
public FiltersAggregator(String name, AggregatorFactories factories, List<KeyedFilter> filters, boolean keyed, AggregationContext aggregationContext, public FiltersAggregator(String name, AggregatorFactories factories, List<KeyedFilter> filters, boolean keyed, AggregationContext aggregationContext,
Aggregator parent, Map<String, Object> metaData) { Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, BucketAggregationMode.MULTI_BUCKETS, factories, filters.size() * (parent == null ? 1 : parent.estimatedBucketCount()), super(name, factories, aggregationContext, parent, metaData);
aggregationContext, parent, metaData);
this.keyed = keyed; this.keyed = keyed;
this.filters = filters.toArray(new KeyedFilter[filters.size()]); this.filters = filters.toArray(new KeyedFilter[filters.size()]);
this.bits = new Bits[this.filters.length]; this.bits = new Bits[this.filters.length];
@ -87,7 +86,7 @@ public class FiltersAggregator extends BucketsAggregator {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
List<InternalFilters.Bucket> buckets = Lists.newArrayListWithCapacity(filters.length); List<InternalFilters.Bucket> buckets = Lists.newArrayListWithCapacity(filters.length);
for (int i = 0; i < filters.length; i++) { for (int i = 0; i < filters.length; i++) {
KeyedFilter filter = filters[i]; KeyedFilter filter = filters[i];
@ -95,7 +94,7 @@ public class FiltersAggregator extends BucketsAggregator {
InternalFilters.Bucket bucket = new InternalFilters.Bucket(filter.key, bucketDocCount(bucketOrd), bucketAggregations(bucketOrd), keyed); InternalFilters.Bucket bucket = new InternalFilters.Bucket(filter.key, bucketDocCount(bucketOrd), bucketAggregations(bucketOrd), keyed);
buckets.add(bucket); buckets.add(bucket);
} }
return new InternalFilters(name, buckets, keyed, getMetaData()); return new InternalFilters(name, buckets, keyed, metaData());
} }
@Override @Override
@ -106,7 +105,7 @@ public class FiltersAggregator extends BucketsAggregator {
InternalFilters.Bucket bucket = new InternalFilters.Bucket(filters[i].key, 0, subAggs, keyed); InternalFilters.Bucket bucket = new InternalFilters.Bucket(filters[i].key, 0, subAggs, keyed);
buckets.add(bucket); buckets.add(bucket);
} }
return new InternalFilters(name, buckets, keyed, getMetaData()); return new InternalFilters(name, buckets, keyed, metaData());
} }
private final long bucketOrd(long owningBucketOrdinal, int filterOrd) { private final long bucketOrd(long owningBucketOrdinal, int filterOrd) {
@ -125,7 +124,7 @@ public class FiltersAggregator extends BucketsAggregator {
} }
@Override @Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) { public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new FiltersAggregator(name, factories, filters, keyed, context, parent, metaData); return new FiltersAggregator(name, factories, filters, keyed, context, parent, metaData);
} }
} }

View File

@ -41,8 +41,6 @@ import java.util.Map;
public class GeoHashGridAggregator extends BucketsAggregator { public class GeoHashGridAggregator extends BucketsAggregator {
private static final int INITIAL_CAPACITY = 50; // TODO sizing
private final int requiredSize; private final int requiredSize;
private final int shardSize; private final int shardSize;
private final ValuesSource.Numeric valuesSource; private final ValuesSource.Numeric valuesSource;
@ -50,12 +48,12 @@ public class GeoHashGridAggregator extends BucketsAggregator {
private SortedNumericDocValues values; private SortedNumericDocValues values;
public GeoHashGridAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, public GeoHashGridAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource,
int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, BucketAggregationMode.PER_BUCKET, factories, INITIAL_CAPACITY, aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.requiredSize = requiredSize; this.requiredSize = requiredSize;
this.shardSize = shardSize; this.shardSize = shardSize;
bucketOrds = new LongHash(INITIAL_CAPACITY, aggregationContext.bigArrays()); bucketOrds = new LongHash(1, aggregationContext.bigArrays());
} }
@Override @Override
@ -102,7 +100,7 @@ public class GeoHashGridAggregator extends BucketsAggregator {
} }
@Override @Override
public InternalGeoHashGrid buildAggregation(long owningBucketOrdinal) { public InternalGeoHashGrid buildAggregation(long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0; assert owningBucketOrdinal == 0;
final int size = (int) Math.min(bucketOrds.size(), shardSize); final int size = (int) Math.min(bucketOrds.size(), shardSize);
@ -125,12 +123,12 @@ public class GeoHashGridAggregator extends BucketsAggregator {
bucket.aggregations = bucketAggregations(bucket.bucketOrd); bucket.aggregations = bucketAggregations(bucket.bucketOrd);
list[i] = bucket; list[i] = bucket;
} }
return new InternalGeoHashGrid(name, requiredSize, Arrays.asList(list), getMetaData()); return new InternalGeoHashGrid(name, requiredSize, Arrays.asList(list), metaData());
} }
@Override @Override
public InternalGeoHashGrid buildEmptyAggregation() { public InternalGeoHashGrid buildEmptyAggregation() {
return new InternalGeoHashGrid(name, requiredSize, Collections.<InternalGeoHashGrid.Bucket>emptyList(), getMetaData()); return new InternalGeoHashGrid(name, requiredSize, Collections.<InternalGeoHashGrid.Bucket>emptyList(), metaData());
} }

View File

@ -103,7 +103,7 @@ public class GeoHashGridParser implements Aggregator.Parser {
} }
private static class GeoGridFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, Map<String, Object>> { private static class GeoGridFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> {
private int precision; private int precision;
private int requiredSize; private int requiredSize;
@ -117,7 +117,7 @@ public class GeoHashGridParser implements Aggregator.Parser {
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize, Collections.<InternalGeoHashGrid.Bucket>emptyList(), metaData); final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize, Collections.<InternalGeoHashGrid.Bucket>emptyList(), metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, metaData) { return new NonCollectingAggregator(name, aggregationContext, parent, metaData) {
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
@ -127,7 +127,10 @@ public class GeoHashGridParser implements Aggregator.Parser {
} }
@Override @Override
protected Aggregator create(final ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
final CellValues cellIdValues = new CellValues(valuesSource, precision); final CellValues cellIdValues = new CellValues(valuesSource, precision);
ValuesSource.Numeric cellIdSource = new CellIdSource(cellIdValues, valuesSource.metaData()); ValuesSource.Numeric cellIdSource = new CellIdSource(cellIdValues, valuesSource.metaData());
return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent, metaData); return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent, metaData);

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.bucket.global; package org.elasticsearch.search.aggregations.bucket.global;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.*;
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
@ -31,7 +32,7 @@ import java.util.Map;
*/ */
public class GlobalAggregator extends SingleBucketAggregator { public class GlobalAggregator extends SingleBucketAggregator {
public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext, Map<String, Object> metaData) { public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext, Map<String, Object> metaData) throws IOException {
super(name, subFactories, aggregationContext, null, metaData); super(name, subFactories, aggregationContext, null, metaData);
} }
@ -46,9 +47,9 @@ public class GlobalAggregator extends SingleBucketAggregator {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0 : "global aggregator can only be a top level aggregator"; assert owningBucketOrdinal == 0 : "global aggregator can only be a top level aggregator";
return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData());
} }
@Override @Override
@ -63,11 +64,14 @@ public class GlobalAggregator extends SingleBucketAggregator {
} }
@Override @Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) { public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
if (parent != null) { if (parent != null) {
throw new AggregationExecutionException("Aggregation [" + parent.name() + "] cannot have a global " + throw new AggregationExecutionException("Aggregation [" + parent.name() + "] cannot have a global " +
"sub-aggregation [" + name + "]. Global aggregations can only be defined as top level aggregations"); "sub-aggregation [" + name + "]. Global aggregations can only be defined as top level aggregations");
} }
if (collectsFromSingleBucket == false) {
throw new ElasticsearchIllegalStateException();
}
return new GlobalAggregator(name, factories, context, metaData); return new GlobalAggregator(name, factories, context, metaData);
} }

View File

@ -59,10 +59,10 @@ public class HistogramAggregator extends BucketsAggregator {
public HistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, InternalOrder order, public HistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, InternalOrder order,
boolean keyed, long minDocCount, @Nullable ExtendedBounds extendedBounds, boolean keyed, long minDocCount, @Nullable ExtendedBounds extendedBounds,
@Nullable ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, @Nullable ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
long initialCapacity, InternalHistogram.Factory<?> histogramFactory, InternalHistogram.Factory<?> histogramFactory,
AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, BucketAggregationMode.PER_BUCKET, factories, initialCapacity, aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
this.rounding = rounding; this.rounding = rounding;
this.order = order; this.order = order;
this.keyed = keyed; this.keyed = keyed;
@ -72,7 +72,7 @@ public class HistogramAggregator extends BucketsAggregator {
this.formatter = formatter; this.formatter = formatter;
this.histogramFactory = histogramFactory; this.histogramFactory = histogramFactory;
bucketOrds = new LongHash(initialCapacity, aggregationContext.bigArrays()); bucketOrds = new LongHash(1, aggregationContext.bigArrays());
} }
@Override @Override
@ -111,7 +111,7 @@ public class HistogramAggregator extends BucketsAggregator {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0; assert owningBucketOrdinal == 0;
List<InternalHistogram.Bucket> buckets = new ArrayList<>((int) bucketOrds.size()); List<InternalHistogram.Bucket> buckets = new ArrayList<>((int) bucketOrds.size());
for (long i = 0; i < bucketOrds.size(); i++) { for (long i = 0; i < bucketOrds.size(); i++) {
@ -123,13 +123,13 @@ public class HistogramAggregator extends BucketsAggregator {
// value source will be null for unmapped fields // value source will be null for unmapped fields
InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null; InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null;
return histogramFactory.create(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, getMetaData()); return histogramFactory.create(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null; InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null;
return histogramFactory.create(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed, getMetaData()); return histogramFactory.create(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed, metaData());
} }
@Override @Override
@ -137,7 +137,7 @@ public class HistogramAggregator extends BucketsAggregator {
Releasables.close(bucketOrds); Releasables.close(bucketOrds);
} }
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {
private final Rounding rounding; private final Rounding rounding;
private final InternalOrder order; private final InternalOrder order;
@ -160,18 +160,15 @@ public class HistogramAggregator extends BucketsAggregator {
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, null, config.formatter(), 0, histogramFactory, aggregationContext, parent, metaData); return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, null, config.formatter(), histogramFactory, aggregationContext, parent, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
// todo if we'll keep track of min/max values in IndexFieldData, we could use the max here to come up with a better estimation for the buckets count if (collectsFromSingleBucket == false) {
long estimatedBucketCount = 50; return asMultiBucketAggregator(this, aggregationContext, parent);
if (hasParentBucketAggregator(parent)) {
estimatedBucketCount = 8;
} }
// we need to round the bounds given by the user and we have to do it for every aggregator we crate // we need to round the bounds given by the user and we have to do it for every aggregator we crate
// as the rounding is not necessarily an idempotent operation. // as the rounding is not necessarily an idempotent operation.
// todo we need to think of a better structure to the factory/agtor code so we won't need to do that // todo we need to think of a better structure to the factory/agtor code so we won't need to do that
@ -181,7 +178,7 @@ public class HistogramAggregator extends BucketsAggregator {
extendedBounds.processAndValidate(name, aggregationContext.searchContext(), config.parser()); extendedBounds.processAndValidate(name, aggregationContext.searchContext(), config.parser());
roundedBounds = extendedBounds.round(rounding); roundedBounds = extendedBounds.round(rounding);
} }
return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource, config.formatter(), estimatedBucketCount, histogramFactory, aggregationContext, parent, metaData); return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource, config.formatter(), histogramFactory, aggregationContext, parent, metaData);
} }
} }

View File

@ -41,7 +41,7 @@ public class MissingAggregator extends SingleBucketAggregator {
private Bits docsWithValue; private Bits docsWithValue;
public MissingAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, public MissingAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
} }
@ -63,28 +63,28 @@ public class MissingAggregator extends SingleBucketAggregator {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
return new InternalMissing(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); return new InternalMissing(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalMissing(name, 0, buildEmptySubAggregations(), getMetaData()); return new InternalMissing(name, 0, buildEmptySubAggregations(), metaData());
} }
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource> {
public Factory(String name, ValuesSourceConfig valueSourceConfig) { public Factory(String name, ValuesSourceConfig valueSourceConfig) {
super(name, InternalMissing.TYPE.name(), valueSourceConfig); super(name, InternalMissing.TYPE.name(), valueSourceConfig);
} }
@Override @Override
protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new MissingAggregator(name, factories, null, aggregationContext, parent, metaData); return new MissingAggregator(name, factories, null, aggregationContext, parent, metaData);
} }
@Override @Override
protected MissingAggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected MissingAggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new MissingAggregator(name, factories, valuesSource, aggregationContext, parent, metaData); return new MissingAggregator(name, factories, valuesSource, aggregationContext, parent, metaData);
} }
} }

View File

@ -51,7 +51,7 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
private DocIdSetIterator childDocs; private DocIdSetIterator childDocs;
private BitSet parentDocs; private BitSet parentDocs;
public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parentAggregator, Map<String, Object> metaData, FilterCachingPolicy filterCachingPolicy) { public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parentAggregator, Map<String, Object> metaData, FilterCachingPolicy filterCachingPolicy) throws IOException {
super(name, factories, aggregationContext, parentAggregator, metaData); super(name, factories, aggregationContext, parentAggregator, metaData);
this.parentAggregator = parentAggregator; this.parentAggregator = parentAggregator;
childFilter = aggregationContext.searchContext().filterCache().cache(objectMapper.nestedTypeFilter(), null, filterCachingPolicy); childFilter = aggregationContext.searchContext().filterCache().cache(objectMapper.nestedTypeFilter(), null, filterCachingPolicy);
@ -117,13 +117,13 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalNested(name, 0, buildEmptySubAggregations(), getMetaData()); return new InternalNested(name, 0, buildEmptySubAggregations(), metaData());
} }
private static Filter findClosestNestedPath(Aggregator parent) { private static Filter findClosestNestedPath(Aggregator parent) {
@ -149,7 +149,7 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
} }
@Override @Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) { public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
MapperService.SmartNameObjectMapper mapper = context.searchContext().smartNameObjectMapper(path); MapperService.SmartNameObjectMapper mapper = context.searchContext().smartNameObjectMapper(path);
if (mapper == null) { if (mapper == null) {
return new Unmapped(name, context, parent, metaData); return new Unmapped(name, context, parent, metaData);
@ -166,13 +166,13 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo
private final static class Unmapped extends NonCollectingAggregator { private final static class Unmapped extends NonCollectingAggregator {
public Unmapped(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { public Unmapped(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, context, parent, metaData); super(name, context, parent, metaData);
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalNested(name, 0, buildEmptySubAggregations(), getMetaData()); return new InternalNested(name, 0, buildEmptySubAggregations(), metaData());
} }
} }
} }

View File

@ -49,7 +49,7 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
// TODO: Add LongIntPagedHashMap? // TODO: Add LongIntPagedHashMap?
private final LongIntOpenHashMap bucketOrdToLastCollectedParentDoc; private final LongIntOpenHashMap bucketOrdToLastCollectedParentDoc;
public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, factories, aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
if (objectMapper == null) { if (objectMapper == null) {
parentFilter = SearchContext.current().bitsetFilterCache().getBitDocIdSetFilter(NonNestedDocsFilter.INSTANCE); parentFilter = SearchContext.current().bitsetFilterCache().getBitDocIdSetFilter(NonNestedDocsFilter.INSTANCE);
@ -117,13 +117,13 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalReverseNested(name, 0, buildEmptySubAggregations(), getMetaData()); return new InternalReverseNested(name, 0, buildEmptySubAggregations(), metaData());
} }
Filter getParentFilter() { Filter getParentFilter() {
@ -140,7 +140,7 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
} }
@Override @Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) { public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
// Early validation // Early validation
NestedAggregator closestNestedAggregator = findClosestNestedAggregator(parent); NestedAggregator closestNestedAggregator = findClosestNestedAggregator(parent);
if (closestNestedAggregator == null) { if (closestNestedAggregator == null) {
@ -168,13 +168,13 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R
private final static class Unmapped extends NonCollectingAggregator { private final static class Unmapped extends NonCollectingAggregator {
public Unmapped(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { public Unmapped(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, context, parent, metaData); super(name, context, parent, metaData);
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalReverseNested(name, 0, buildEmptySubAggregations(), getMetaData()); return new InternalReverseNested(name, 0, buildEmptySubAggregations(), metaData());
} }
} }
} }

View File

@ -98,9 +98,9 @@ public class RangeAggregator extends BucketsAggregator {
boolean keyed, boolean keyed,
AggregationContext aggregationContext, AggregationContext aggregationContext,
Aggregator parent, Aggregator parent,
Map<String, Object> metaData) { Map<String, Object> metaData) throws IOException {
super(name, BucketAggregationMode.MULTI_BUCKETS, factories, ranges.size() * (parent == null ? 1 : parent.estimatedBucketCount()), aggregationContext, parent, metaData); super(name, factories, aggregationContext, parent, metaData);
assert valuesSource != null; assert valuesSource != null;
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = format != null ? format.formatter() : null; this.formatter = format != null ? format.formatter() : null;
@ -196,17 +196,17 @@ public class RangeAggregator extends BucketsAggregator {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
List<org.elasticsearch.search.aggregations.bucket.range.Range.Bucket> buckets = Lists.newArrayListWithCapacity(ranges.length); List<org.elasticsearch.search.aggregations.bucket.range.Range.Bucket> buckets = Lists.newArrayListWithCapacity(ranges.length);
for (int i = 0; i < ranges.length; i++) { for (int i = 0; i < ranges.length; i++) {
Range range = ranges[i]; Range range = ranges[i];
final long bucketOrd = subBucketOrdinal(owningBucketOrdinal, i); final long bucketOrd = subBucketOrdinal(owningBucketOrdinal, i);
org.elasticsearch.search.aggregations.bucket.range.Range.Bucket bucket = org.elasticsearch.search.aggregations.bucket.range.Range.Bucket bucket =
rangeFactory.createBucket(range.key, range.from, range.to, bucketDocCount(bucketOrd),bucketAggregations(bucketOrd), keyed, formatter); rangeFactory.createBucket(range.key, range.from, range.to, bucketDocCount(bucketOrd), bucketAggregations(bucketOrd), keyed, formatter);
buckets.add(bucket); buckets.add(bucket);
} }
// value source can be null in the case of unmapped fields // value source can be null in the case of unmapped fields
return rangeFactory.create(name, buckets, formatter, keyed, getMetaData()); return rangeFactory.create(name, buckets, formatter, keyed, metaData());
} }
@Override @Override
@ -220,7 +220,7 @@ public class RangeAggregator extends BucketsAggregator {
buckets.add(bucket); buckets.add(bucket);
} }
// value source can be null in the case of unmapped fields // value source can be null in the case of unmapped fields
return rangeFactory.create(name, buckets, formatter, keyed, getMetaData()); return rangeFactory.create(name, buckets, formatter, keyed, metaData());
} }
private static final void sortRanges(final Range[] ranges) { private static final void sortRanges(final Range[] ranges) {
@ -258,7 +258,7 @@ public class RangeAggregator extends BucketsAggregator {
AggregationContext context, AggregationContext context,
Aggregator parent, Aggregator parent,
InternalRange.Factory factory, InternalRange.Factory factory,
Map<String, Object> metaData) { Map<String, Object> metaData) throws IOException {
super(name, context, parent, metaData); super(name, context, parent, metaData);
this.ranges = ranges; this.ranges = ranges;
@ -278,11 +278,11 @@ public class RangeAggregator extends BucketsAggregator {
for (RangeAggregator.Range range : ranges) { for (RangeAggregator.Range range : ranges) {
buckets.add(factory.createBucket(range.key, range.from, range.to, 0, subAggs, keyed, formatter)); buckets.add(factory.createBucket(range.key, range.from, range.to, 0, subAggs, keyed, formatter));
} }
return factory.create(name, buckets, formatter, keyed, getMetaData()); return factory.create(name, buckets, formatter, keyed, metaData());
} }
} }
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {
private final InternalRange.Factory rangeFactory; private final InternalRange.Factory rangeFactory;
private final List<Range> ranges; private final List<Range> ranges;
@ -296,12 +296,12 @@ public class RangeAggregator extends BucketsAggregator {
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, metaData); return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, aggregationContext, parent, metaData); return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, aggregationContext, parent, metaData);
} }
} }

View File

@ -153,7 +153,7 @@ public class GeoDistanceParser implements Aggregator.Parser {
return new GeoDistanceFactory(aggregationName, vsParser.config(), InternalGeoDistance.FACTORY, origin, unit, distanceType, ranges, keyed); return new GeoDistanceFactory(aggregationName, vsParser.config(), InternalGeoDistance.FACTORY, origin, unit, distanceType, ranges, keyed);
} }
private static class GeoDistanceFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, Map<String, Object>> { private static class GeoDistanceFactory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> {
private final GeoPoint origin; private final GeoPoint origin;
private final DistanceUnit unit; private final DistanceUnit unit;
@ -175,12 +175,12 @@ public class GeoDistanceParser implements Aggregator.Parser {
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new Unmapped(name, ranges, keyed, null, aggregationContext, parent, rangeFactory, metaData); return new Unmapped(name, ranges, keyed, null, aggregationContext, parent, rangeFactory, metaData);
} }
@Override @Override
protected Aggregator create(final ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit); DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit);
aggregationContext.registerReaderContextAware(distanceSource); aggregationContext.registerReaderContextAware(distanceSource);
return new RangeAggregator(name, factories, distanceSource, null, rangeFactory, ranges, keyed, aggregationContext, parent, metaData); return new RangeAggregator(name, factories, distanceSource, null, rangeFactory, ranges, keyed, aggregationContext, parent, metaData);

View File

@ -44,11 +44,11 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
protected final SignificantTermsAggregatorFactory termsAggFactory; protected final SignificantTermsAggregatorFactory termsAggFactory;
public GlobalOrdinalsSignificantTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, public GlobalOrdinalsSignificantTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
long estimatedBucketCount, long maxOrd, BucketCountThresholds bucketCountThresholds, long maxOrd, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent,
SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) { SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, estimatedBucketCount, maxOrd, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData); super(name, factories, valuesSource, maxOrd, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData);
this.termsAggFactory = termsAggFactory; this.termsAggFactory = termsAggFactory;
} }
@ -59,7 +59,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
} }
@Override @Override
public SignificantStringTerms buildAggregation(long owningBucketOrdinal) { public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0; assert owningBucketOrdinal == 0;
if (globalOrds == null) { // no context in this reader if (globalOrds == null) { // no context in this reader
return buildEmptyAggregation(); return buildEmptyAggregation();
@ -114,7 +114,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
list[i] = bucket; list[i] = bucket;
} }
return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), getMetaData()); return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), metaData());
} }
@Override @Override
@ -123,7 +123,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
ContextIndexSearcher searcher = context.searchContext().searcher(); ContextIndexSearcher searcher = context.searchContext().searcher();
IndexReader topReader = searcher.getIndexReader(); IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs(); int supersetSize = topReader.numDocs();
return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList(), getMetaData()); return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList(), metaData());
} }
@Override @Override
@ -135,9 +135,9 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri
private final LongHash bucketOrds; private final LongHash bucketOrds;
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) { public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory, metaData); super(name, factories, valuesSource, 1, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory, metaData);
bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays()); bucketOrds = new LongHash(1, aggregationContext.bigArrays());
} }
@Override @Override

View File

@ -41,10 +41,10 @@ import java.util.Map;
public class SignificantLongTermsAggregator extends LongTermsAggregator { public class SignificantLongTermsAggregator extends LongTermsAggregator {
public SignificantLongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, public SignificantLongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format,
long estimatedBucketCount, BucketCountThresholds bucketCountThresholds, BucketCountThresholds bucketCountThresholds,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, IncludeExclude.LongFilter includeExclude, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, IncludeExclude.LongFilter includeExclude, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, format, estimatedBucketCount, null, bucketCountThresholds, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, includeExclude, metaData); super(name, factories, valuesSource, format, null, bucketCountThresholds, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, includeExclude, metaData);
this.termsAggFactory = termsAggFactory; this.termsAggFactory = termsAggFactory;
} }
@ -58,7 +58,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
} }
@Override @Override
public SignificantLongTerms buildAggregation(long owningBucketOrdinal) { public SignificantLongTerms buildAggregation(long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0; assert owningBucketOrdinal == 0;
final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize()); final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize());
@ -93,7 +93,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
bucket.aggregations = bucketAggregations(bucket.bucketOrd); bucket.aggregations = bucketAggregations(bucket.bucketOrd);
list[i] = bucket; list[i] = bucket;
} }
return new SignificantLongTerms(subsetSize, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), getMetaData()); return new SignificantLongTerms(subsetSize, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), metaData());
} }
@Override @Override
@ -102,7 +102,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
ContextIndexSearcher searcher = context.searchContext().searcher(); ContextIndexSearcher searcher = context.searchContext().searcher();
IndexReader topReader = searcher.getIndexReader(); IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs(); int supersetSize = topReader.numDocs();
return new SignificantLongTerms(0, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList(), getMetaData()); return new SignificantLongTerms(0, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList(), metaData());
} }
@Override @Override

View File

@ -43,11 +43,11 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
protected final SignificantTermsAggregatorFactory termsAggFactory; protected final SignificantTermsAggregatorFactory termsAggFactory;
public SignificantStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, public SignificantStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
long estimatedBucketCount, BucketCountThresholds bucketCountThresholds, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent,
SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) { SignificantTermsAggregatorFactory termsAggFactory, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, estimatedBucketCount, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData); super(name, factories, valuesSource, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData);
this.termsAggFactory = termsAggFactory; this.termsAggFactory = termsAggFactory;
} }
@ -58,7 +58,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
} }
@Override @Override
public SignificantStringTerms buildAggregation(long owningBucketOrdinal) { public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0; assert owningBucketOrdinal == 0;
final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize()); final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize());
@ -98,7 +98,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
list[i] = bucket; list[i] = bucket;
} }
return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), getMetaData()); return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), metaData());
} }
@Override @Override
@ -107,7 +107,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
ContextIndexSearcher searcher = context.searchContext().searcher(); ContextIndexSearcher searcher = context.searchContext().searcher();
IndexReader topReader = searcher.getIndexReader(); IndexReader topReader = searcher.getIndexReader();
int supersetSize = topReader.numDocs(); int supersetSize = topReader.numDocs();
return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList(), getMetaData()); return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.<InternalSignificantTerms.Bucket>emptyList(), metaData());
} }
@Override @Override

View File

@ -30,10 +30,13 @@ import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lucene.index.FilterableTermsEnum; import org.elasticsearch.common.lucene.index.FilterableTermsEnum;
import org.elasticsearch.common.lucene.index.FreqTermsEnum; import org.elasticsearch.common.lucene.index.FreqTermsEnum;
import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -47,7 +50,7 @@ import java.util.Map;
/** /**
* *
*/ */
public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, Map<String, Object>> implements Releasable { public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource> implements Releasable {
public SignificanceHeuristic getSignificanceHeuristic() { public SignificanceHeuristic getSignificanceHeuristic() {
return significanceHeuristic; return significanceHeuristic;
@ -58,10 +61,10 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
MAP(new ParseField("map")) { MAP(new ParseField("map")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException {
return new SignificantStringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData); return new SignificantStringTermsAggregator(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
} }
@Override @Override
@ -73,13 +76,13 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
GLOBAL_ORDINALS(new ParseField("global_ordinals")) { GLOBAL_ORDINALS(new ParseField("global_ordinals")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException {
ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource; ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource;
IndexSearcher indexSearcher = aggregationContext.searchContext().searcher(); IndexSearcher indexSearcher = aggregationContext.searchContext().searcher();
long maxOrd = valueSourceWithOrdinals.globalMaxOrd(indexSearcher); long maxOrd = valueSourceWithOrdinals.globalMaxOrd(indexSearcher);
return new GlobalOrdinalsSignificantTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData); return new GlobalOrdinalsSignificantTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
} }
@Override @Override
@ -91,10 +94,10 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
GLOBAL_ORDINALS_HASH(new ParseField("global_ordinals_hash")) { GLOBAL_ORDINALS_HASH(new ParseField("global_ordinals_hash")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException {
return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData); return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData);
} }
@Override @Override
@ -118,9 +121,9 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
this.parseField = parseField; this.parseField = parseField;
} }
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData); AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map<String, Object> metaData) throws IOException;
abstract boolean needsGlobalOrdinals(); abstract boolean needsGlobalOrdinals();
@ -159,7 +162,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), metaData); final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, metaData) { return new NonCollectingAggregator(name, aggregationContext, parent, metaData) {
@Override @Override
@ -170,10 +173,12 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
} }
@Override @Override
protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
numberOfAggregatorsCreated++; if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
long estimatedBucketCount = TermsAggregatorFactory.estimatedBucketCount(valuesSource, parent); numberOfAggregatorsCreated++;
if (valuesSource instanceof ValuesSource.Bytes) { if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null; ExecutionMode execution = null;
@ -184,7 +189,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
execution = ExecutionMode.MAP; execution = ExecutionMode.MAP;
} }
if (execution == null) { if (execution == null) {
if (Aggregator.hasParentBucketAggregator(parent)) { if (Aggregator.descendsFromBucketAggregator(parent)) {
execution = ExecutionMode.GLOBAL_ORDINALS_HASH; execution = ExecutionMode.GLOBAL_ORDINALS_HASH;
} else { } else {
execution = ExecutionMode.GLOBAL_ORDINALS; execution = ExecutionMode.GLOBAL_ORDINALS;
@ -192,7 +197,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
} }
assert execution != null; assert execution != null;
valuesSource.setNeedsGlobalOrdinals(execution.needsGlobalOrdinals()); valuesSource.setNeedsGlobalOrdinals(execution.needsGlobalOrdinals());
return execution.create(name, factories, valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, this, metaData); return execution.create(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, this, metaData);
} }
@ -210,7 +215,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
if (includeExclude != null) { if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter(); longFilter = includeExclude.convertToLongFilter();
} }
return new SignificantLongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), estimatedBucketCount, bucketCountThresholds, aggregationContext, parent, this, longFilter, metaData); return new SignificantLongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), bucketCountThresholds, aggregationContext, parent, this, longFilter, metaData);
} }
throw new AggregationExecutionException("sigfnificant_terms aggregation cannot be applied to field [" + config.fieldContext().field() + throw new AggregationExecutionException("sigfnificant_terms aggregation cannot be applied to field [" + config.fieldContext().field() +

View File

@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.Map; import java.util.Map;
@ -33,9 +34,10 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
protected final boolean showTermDocCountError; protected final boolean showTermDocCountError;
public AbstractStringTermsAggregator(String name, AggregatorFactories factories, public AbstractStringTermsAggregator(String name, AggregatorFactories factories,
long estimatedBucketsCount, AggregationContext context, Aggregator parent, AggregationContext context, Aggregator parent,
Terms.Order order, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) { Terms.Order order, BucketCountThresholds bucketCountThresholds,
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketsCount, context, parent, bucketCountThresholds, order, subAggCollectMode, metaData); SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
super(name, factories, context, parent, bucketCountThresholds, order, subAggCollectMode, metaData);
this.showTermDocCountError = showTermDocCountError; this.showTermDocCountError = showTermDocCountError;
} }
@ -46,7 +48,7 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator {
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, getMetaData()); return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, metaData());
} }
} }

View File

@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.format.ValueFormat; import org.elasticsearch.search.aggregations.support.format.ValueFormat;
import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Map; import java.util.Map;
@ -38,9 +39,9 @@ import java.util.Map;
*/ */
public class DoubleTermsAggregator extends LongTermsAggregator { public class DoubleTermsAggregator extends LongTermsAggregator {
public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount, public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format,
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map<String, Object> metaData) { Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, format, estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, collectionMode, showTermDocCountError, longFilter, metaData); super(name, factories, valuesSource, format, order, bucketCountThresholds, aggregationContext, parent, collectionMode, showTermDocCountError, longFilter, metaData);
} }
@Override @Override
@ -49,7 +50,7 @@ public class DoubleTermsAggregator extends LongTermsAggregator {
} }
@Override @Override
public DoubleTerms buildAggregation(long owningBucketOrdinal) { public DoubleTerms buildAggregation(long owningBucketOrdinal) throws IOException {
final LongTerms terms = (LongTerms) super.buildAggregation(owningBucketOrdinal); final LongTerms terms = (LongTerms) super.buildAggregation(owningBucketOrdinal);
return convertToDouble(terms); return convertToDouble(terms);
} }

View File

@ -19,8 +19,8 @@
package org.elasticsearch.search.aggregations.bucket.terms; package org.elasticsearch.search.aggregations.bucket.terms;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.RandomAccessOrds; import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedDocValues; import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.ArrayUtil;
@ -34,7 +34,6 @@ import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.text.Text; import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.util.IntArray; import org.elasticsearch.common.util.IntArray;
import org.elasticsearch.common.util.LongHash; import org.elasticsearch.common.util.LongHash;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.AbstractRandomAccessOrds; import org.elasticsearch.index.fielddata.AbstractRandomAccessOrds;
import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalMapping; import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalMapping;
@ -72,12 +71,13 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
protected Collector collector; protected Collector collector;
public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) { IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
super(name, factories, maxOrd, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData); super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.includeExclude = includeExclude; this.includeExclude = includeExclude;
grow(maxOrd);
} }
protected long getBucketOrd(long termOrd) { protected long getBucketOrd(long termOrd) {
@ -143,7 +143,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
if (globalOrds == null) { // no context in this reader if (globalOrds == null) { // no context in this reader
return buildEmptyAggregation(); return buildEmptyAggregation();
} }
@ -201,7 +201,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
bucket.docCountError = 0; bucket.docCountError = 0;
} }
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, getMetaData()); return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, metaData());
} }
/** This is used internally only, just for compare using global ordinal instead of term bytes in the PQ */ /** This is used internally only, just for compare using global ordinal instead of term bytes in the PQ */
@ -271,12 +271,11 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
private final LongHash bucketOrds; private final LongHash bucketOrds;
public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext,
Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) { Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
// Set maxOrd to estimatedBucketCount! To be conservative with memory. super(name, factories, valuesSource, 1, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError, metaData);
super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); bucketOrds = new LongHash(1, aggregationContext.bigArrays());
bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays());
} }
protected Collector newCollector(final RandomAccessOrds ords) { protected Collector newCollector(final RandomAccessOrds ords) {
@ -340,9 +339,9 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
private RandomAccessOrds segmentOrds; private RandomAccessOrds segmentOrds;
public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource,
long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) { long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
super(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); super(name, factories, valuesSource, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, metaData);
assert factories == null || factories.count() == 0; assert factories == null || factories.count() == 0;
this.segmentDocCounts = bigArrays.newIntArray(maxOrd + 1, true); this.segmentDocCounts = bigArrays.newIntArray(maxOrd + 1, true);
} }

View File

@ -51,14 +51,14 @@ public class LongTermsAggregator extends TermsAggregator {
private SortedNumericDocValues values; private SortedNumericDocValues values;
private LongFilter longFilter; private LongFilter longFilter;
public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount, public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format,
Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map<String, Object> metaData) { Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map<String, Object> metaData) throws IOException {
super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode, metaData); super(name, factories, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.showTermDocCountError = showTermDocCountError; this.showTermDocCountError = showTermDocCountError;
this.formatter = format != null ? format.formatter() : null; this.formatter = format != null ? format.formatter() : null;
this.longFilter = longFilter; this.longFilter = longFilter;
bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays()); bucketOrds = new LongHash(1, aggregationContext.bigArrays());
} }
@ -103,7 +103,7 @@ public class LongTermsAggregator extends TermsAggregator {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0; assert owningBucketOrdinal == 0;
if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) { if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
@ -157,13 +157,13 @@ public class LongTermsAggregator extends TermsAggregator {
list[i].docCountError = 0; list[i].docCountError = 0;
} }
return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, getMetaData()); return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, getMetaData()); return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.<InternalTerms.Bucket>emptyList(), showTermDocCountError, 0, 0, metaData());
} }
@Override @Override

View File

@ -34,7 +34,6 @@ import org.elasticsearch.search.aggregations.support.ValuesSource;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Map; import java.util.Map;
/** /**
@ -48,14 +47,14 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
private SortedBinaryDocValues values; private SortedBinaryDocValues values;
private final BytesRefBuilder previous; private final BytesRefBuilder previous;
public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource,
Terms.Order order, BucketCountThresholds bucketCountThresholds, Terms.Order order, BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) { IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
super(name, factories, estimatedBucketCount, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData); super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.includeExclude = includeExclude; this.includeExclude = includeExclude;
bucketOrds = new BytesRefHash(estimatedBucketCount, aggregationContext.bigArrays()); bucketOrds = new BytesRefHash(1, aggregationContext.bigArrays());
previous = new BytesRefBuilder(); previous = new BytesRefBuilder();
} }
@ -97,7 +96,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
} }
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException {
assert owningBucketOrdinal == 0; assert owningBucketOrdinal == 0;
if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) { if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) {
@ -157,7 +156,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
bucket.docCountError = 0; bucket.docCountError = 0;
} }
return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, getMetaData()); return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, metaData());
} }
@Override @Override

View File

@ -135,8 +135,8 @@ public abstract class TermsAggregator extends BucketsAggregator {
protected Set<Aggregator> aggsUsedForSorting = new HashSet<>(); protected Set<Aggregator> aggsUsedForSorting = new HashSet<>();
protected SubAggCollectionMode subAggCollectMode; protected SubAggCollectionMode subAggCollectMode;
public TermsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode subAggCollectMode, Map<String, Object> metaData) { public TermsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode subAggCollectMode, Map<String, Object> metaData) throws IOException {
super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent, metaData); super(name, factories, context, parent, metaData);
this.bucketCountThresholds = bucketCountThresholds; this.bucketCountThresholds = bucketCountThresholds;
this.order = InternalOrder.validate(order, this); this.order = InternalOrder.validate(order, this);
this.subAggCollectMode = subAggCollectMode; this.subAggCollectMode = subAggCollectMode;

View File

@ -29,22 +29,23 @@ import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map; import java.util.Map;
/** /**
* *
*/ */
public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, Map<String, Object>> { public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource> {
public enum ExecutionMode { public enum ExecutionMode {
MAP(new ParseField("map")) { MAP(new ParseField("map")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
return new StringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return new StringTermsAggregator(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
@Override @Override
@ -56,10 +57,10 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
GLOBAL_ORDINALS(new ParseField("global_ordinals")) { GLOBAL_ORDINALS(new ParseField("global_ordinals")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
@Override @Override
@ -71,10 +72,10 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
GLOBAL_ORDINALS_HASH(new ParseField("global_ordinals_hash")) { GLOBAL_ORDINALS_HASH(new ParseField("global_ordinals_hash")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
@Override @Override
@ -85,13 +86,13 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
GLOBAL_ORDINALS_LOW_CARDINALITY(new ParseField("global_ordinals_low_cardinality")) { GLOBAL_ORDINALS_LOW_CARDINALITY(new ParseField("global_ordinals_low_cardinality")) {
@Override @Override
Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude,
AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException {
if (includeExclude != null || factories.count() > 0) { if (includeExclude != null || factories.count() > 0) {
return GLOBAL_ORDINALS.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return GLOBAL_ORDINALS.create(name, factories, valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
@Override @Override
@ -115,9 +116,9 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
this.parseField = parseField; this.parseField = parseField;
} }
abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource,
long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds, long maxOrd, Terms.Order order, TermsAggregator.BucketCountThresholds bucketCountThresholds,
IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData); IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map<String, Object> metaData) throws IOException;
abstract boolean needsGlobalOrdinals(); abstract boolean needsGlobalOrdinals();
@ -145,7 +146,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), metaData); final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), metaData);
return new NonCollectingAggregator(name, aggregationContext, parent, factories, metaData) { return new NonCollectingAggregator(name, aggregationContext, parent, factories, metaData) {
{ {
@ -159,34 +160,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
}; };
} }
public static long estimatedBucketCount(ValuesSource valuesSource, Aggregator parent) {
long estimatedBucketCount = valuesSource.metaData().maxAtomicUniqueValuesCount();
if (estimatedBucketCount < 0) {
// there isn't an estimation available.. 50 should be a good start
estimatedBucketCount = 50;
}
// adding an upper bound on the estimation as some atomic field data in the future (binary doc values) and not
// going to know their exact cardinality and will return upper bounds in AtomicFieldData.getNumberUniqueValues()
// that may be largely over-estimated.. the value chosen here is arbitrary just to play nice with typical CPU cache
//
// Another reason is that it may be faster to resize upon growth than to start directly with the appropriate size.
// And that all values are not necessarily visited by the matches.
estimatedBucketCount = Math.min(estimatedBucketCount, 512);
if (Aggregator.hasParentBucketAggregator(parent)) {
// There is a parent that creates buckets, potentially with a very long tail of buckets with few documents
// Let's be conservative with memory in that case
estimatedBucketCount = Math.min(estimatedBucketCount, 8);
}
return estimatedBucketCount;
}
@Override @Override
protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
long estimatedBucketCount = estimatedBucketCount(valuesSource, parent); if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, aggregationContext, parent);
}
if (valuesSource instanceof ValuesSource.Bytes) { if (valuesSource instanceof ValuesSource.Bytes) {
ExecutionMode execution = null; ExecutionMode execution = null;
if (executionHint != null) { if (executionHint != null) {
@ -215,7 +193,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
// if there is a parent bucket aggregator the number of instances of this aggregator is going // if there is a parent bucket aggregator the number of instances of this aggregator is going
// to be unbounded and most instances may only aggregate few documents, so use hashed based // to be unbounded and most instances may only aggregate few documents, so use hashed based
// global ordinals to keep the bucket ords dense. // global ordinals to keep the bucket ords dense.
if (Aggregator.hasParentBucketAggregator(parent)) { if (Aggregator.descendsFromBucketAggregator(parent)) {
execution = ExecutionMode.GLOBAL_ORDINALS_HASH; execution = ExecutionMode.GLOBAL_ORDINALS_HASH;
} else { } else {
if (factories == AggregatorFactories.EMPTY) { if (factories == AggregatorFactories.EMPTY) {
@ -234,7 +212,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
assert execution != null; assert execution != null;
valuesSource.setNeedsGlobalOrdinals(execution.needsGlobalOrdinals()); valuesSource.setNeedsGlobalOrdinals(execution.needsGlobalOrdinals());
return execution.create(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); return execution.create(name, factories, valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData);
} }
if ((includeExclude != null) && (includeExclude.isRegexBased())) { if ((includeExclude != null) && (includeExclude.isRegexBased())) {
@ -249,13 +227,13 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
longFilter = includeExclude.convertToDoubleFilter(); longFilter = includeExclude.convertToDoubleFilter();
} }
return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), return new DoubleTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(),
estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode,
showTermDocCountError, longFilter, metaData); showTermDocCountError, longFilter, metaData);
} }
if (includeExclude != null) { if (includeExclude != null) {
longFilter = includeExclude.convertToLongFilter(); longFilter = includeExclude.convertToLongFilter();
} }
return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), estimatedBucketCount, return new LongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(),
order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, longFilter, metaData); order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, longFilter, metaData);
} }

View File

@ -23,15 +23,12 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.Map; import java.util.Map;
public abstract class MetricsAggregator extends Aggregator { public abstract class MetricsAggregator extends Aggregator {
protected MetricsAggregator(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { protected MetricsAggregator(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
this(name, estimatedBucketsCount, BucketAggregationMode.MULTI_BUCKETS, context, parent, metaData); super(name, AggregatorFactories.EMPTY, context, parent, metaData);
}
protected MetricsAggregator(String name, long estimatedBucketsCount, BucketAggregationMode bucketAggregationMode, AggregationContext context, Aggregator parent, Map<String, Object> metaData) {
super(name, bucketAggregationMode, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent, metaData);
} }
} }

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.metrics;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import java.io.IOException;
import java.util.Map; import java.util.Map;
/** /**
@ -28,14 +29,14 @@ import java.util.Map;
*/ */
public abstract class NumericMetricsAggregator extends MetricsAggregator { public abstract class NumericMetricsAggregator extends MetricsAggregator {
private NumericMetricsAggregator(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { private NumericMetricsAggregator(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
} }
public static abstract class SingleValue extends NumericMetricsAggregator { public static abstract class SingleValue extends NumericMetricsAggregator {
protected SingleValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { protected SingleValue(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
} }
public abstract double metric(long owningBucketOrd); public abstract double metric(long owningBucketOrd);
@ -43,8 +44,8 @@ public abstract class NumericMetricsAggregator extends MetricsAggregator {
public static abstract class MultiValue extends NumericMetricsAggregator { public static abstract class MultiValue extends NumericMetricsAggregator {
protected MultiValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { protected MultiValue(String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
} }
public abstract boolean hasMetric(String name); public abstract boolean hasMetric(String name);

View File

@ -48,15 +48,14 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
private DoubleArray sums; private DoubleArray sums;
private ValueFormatter formatter; private ValueFormatter formatter;
public AvgAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public AvgAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name,context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
if (valuesSource != null) { if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; counts = bigArrays.newLongArray(1, true);
counts = bigArrays.newLongArray(initialSize, true); sums = bigArrays.newDoubleArray(1, true);
sums = bigArrays.newDoubleArray(initialSize, true);
} }
} }
@ -93,30 +92,30 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue {
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null || owningBucketOrdinal >= counts.size()) { if (valuesSource == null || owningBucketOrdinal >= counts.size()) {
return new InternalAvg(name, 0l, 0, formatter, getMetaData()); return new InternalAvg(name, 0l, 0, formatter, metaData());
} }
return new InternalAvg(name, sums.get(owningBucketOrdinal), counts.get(owningBucketOrdinal), formatter, getMetaData()); return new InternalAvg(name, sums.get(owningBucketOrdinal), counts.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalAvg(name, 0.0, 0l, formatter, getMetaData()); return new InternalAvg(name, 0.0, 0l, formatter, metaData());
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name, String type, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) { public Factory(String name, String type, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, type, valuesSourceConfig); super(name, type, valuesSourceConfig);
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new AvgAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); return new AvgAggregator(name, null, config.formatter(), aggregationContext, parent, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new AvgAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); return new AvgAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData);
} }
} }

View File

@ -64,14 +64,13 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
private Collector collector; private Collector collector;
private ValueFormatter formatter; private ValueFormatter formatter;
public CardinalityAggregator(String name, long estimatedBucketsCount, ValuesSource valuesSource, boolean rehash, public CardinalityAggregator(String name, ValuesSource valuesSource, boolean rehash, int precision, @Nullable ValueFormatter formatter,
int precision, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
@Nullable ValueFormatter formatter, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { super(name, context, parent, metaData);
super(name, estimatedBucketsCount, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.rehash = rehash; this.rehash = rehash;
this.precision = precision; this.precision = precision;
this.counts = valuesSource == null ? null : new HyperLogLogPlusPlus(precision, bigArrays, estimatedBucketsCount); this.counts = valuesSource == null ? null : new HyperLogLogPlusPlus(precision, bigArrays, 1);
this.formatter = formatter; this.formatter = formatter;
} }
@ -157,12 +156,12 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue
// this Aggregator (and its HLL++ counters) is released. // this Aggregator (and its HLL++ counters) is released.
HyperLogLogPlusPlus copy = new HyperLogLogPlusPlus(precision, BigArrays.NON_RECYCLING_INSTANCE, 1); HyperLogLogPlusPlus copy = new HyperLogLogPlusPlus(precision, BigArrays.NON_RECYCLING_INSTANCE, 1);
copy.merge(0, counts, owningBucketOrdinal); copy.merge(0, counts, owningBucketOrdinal);
return new InternalCardinality(name, copy, formatter, getMetaData()); return new InternalCardinality(name, copy, formatter, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalCardinality(name, null, formatter, getMetaData()); return new InternalCardinality(name, null, formatter, metaData());
} }
@Override @Override

View File

@ -21,15 +21,16 @@ package org.elasticsearch.search.aggregations.metrics.cardinality;
import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.Aggregator.BucketAggregationMode; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import java.io.IOException;
import java.util.Map; import java.util.Map;
final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, Map<String, Object>> { final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource> {
private final long precisionThreshold; private final long precisionThreshold;
private final boolean rehash; private final boolean rehash;
@ -45,18 +46,16 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<V
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext context, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new CardinalityAggregator(name, parent == null ? 1 : parent.estimatedBucketCount(), null, true, precision(parent), return new CardinalityAggregator(name, null, true, precision(parent), config.formatter(), context, parent, metaData);
config.formatter(), context, parent, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
if (!(valuesSource instanceof ValuesSource.Numeric) && !rehash) { if (!(valuesSource instanceof ValuesSource.Numeric) && !rehash) {
throw new AggregationExecutionException("Turning off rehashing for cardinality aggregation [" + name + "] on non-numeric values in not allowed"); throw new AggregationExecutionException("Turning off rehashing for cardinality aggregation [" + name + "] on non-numeric values in not allowed");
} }
return new CardinalityAggregator(name, parent == null ? 1 : parent.estimatedBucketCount(), valuesSource, rehash, precision(parent), return new CardinalityAggregator(name, valuesSource, rehash, precision(parent), config.formatter(), context, parent, metaData);
config.formatter(), context, parent, metaData);
} }
/* /*
@ -67,8 +66,8 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<V
private int defaultPrecision(Aggregator parent) { private int defaultPrecision(Aggregator parent) {
int precision = HyperLogLogPlusPlus.DEFAULT_PRECISION; int precision = HyperLogLogPlusPlus.DEFAULT_PRECISION;
while (parent != null) { while (parent != null) {
if (parent.bucketAggregationMode() == BucketAggregationMode.PER_BUCKET) { if (parent instanceof SingleBucketAggregator == false) {
// if the parent is a per-bucket aggregator, we substract 5 to the precision, // if the parent creates buckets, we substract 5 to the precision,
// which will effectively divide the memory usage of each counter by 32 // which will effectively divide the memory usage of each counter by 32
precision -= 5; precision -= 5;
} }

View File

@ -47,24 +47,23 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
private DoubleArray negRights; private DoubleArray negRights;
private MultiGeoPointValues values; private MultiGeoPointValues values;
protected GeoBoundsAggregator(String name, long estimatedBucketsCount, protected GeoBoundsAggregator(String name, AggregationContext aggregationContext,
AggregationContext aggregationContext, Aggregator parent, ValuesSource.GeoPoint valuesSource, boolean wrapLongitude, Map<String, Object> metaData) { Aggregator parent, ValuesSource.GeoPoint valuesSource, boolean wrapLongitude, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, aggregationContext, parent, metaData); super(name, aggregationContext, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.wrapLongitude = wrapLongitude; this.wrapLongitude = wrapLongitude;
if (valuesSource != null) { if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; tops = bigArrays.newDoubleArray(1, false);
tops = bigArrays.newDoubleArray(initialSize, false);
tops.fill(0, tops.size(), Double.NEGATIVE_INFINITY); tops.fill(0, tops.size(), Double.NEGATIVE_INFINITY);
bottoms = bigArrays.newDoubleArray(initialSize, false); bottoms = bigArrays.newDoubleArray(1, false);
bottoms.fill(0, bottoms.size(), Double.POSITIVE_INFINITY); bottoms.fill(0, bottoms.size(), Double.POSITIVE_INFINITY);
posLefts = bigArrays.newDoubleArray(initialSize, false); posLefts = bigArrays.newDoubleArray(1, false);
posLefts.fill(0, posLefts.size(), Double.POSITIVE_INFINITY); posLefts.fill(0, posLefts.size(), Double.POSITIVE_INFINITY);
posRights = bigArrays.newDoubleArray(initialSize, false); posRights = bigArrays.newDoubleArray(1, false);
posRights.fill(0, posRights.size(), Double.NEGATIVE_INFINITY); posRights.fill(0, posRights.size(), Double.NEGATIVE_INFINITY);
negLefts = bigArrays.newDoubleArray(initialSize, false); negLefts = bigArrays.newDoubleArray(1, false);
negLefts.fill(0, negLefts.size(), Double.POSITIVE_INFINITY); negLefts.fill(0, negLefts.size(), Double.POSITIVE_INFINITY);
negRights = bigArrays.newDoubleArray(initialSize, false); negRights = bigArrays.newDoubleArray(1, false);
negRights.fill(0, negRights.size(), Double.NEGATIVE_INFINITY); negRights.fill(0, negRights.size(), Double.NEGATIVE_INFINITY);
} }
} }
@ -90,13 +89,13 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
double posRight = posRights.get(owningBucketOrdinal); double posRight = posRights.get(owningBucketOrdinal);
double negLeft = negLefts.get(owningBucketOrdinal); double negLeft = negLefts.get(owningBucketOrdinal);
double negRight = negRights.get(owningBucketOrdinal); double negRight = negRights.get(owningBucketOrdinal);
return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude, getMetaData()); return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalGeoBounds(name, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, return new InternalGeoBounds(name, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY,
Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, wrapLongitude, getMetaData()); Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, wrapLongitude, metaData());
} }
@Override @Override
@ -160,7 +159,7 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
Releasables.close(tops, bottoms, posLefts, posRights, negLefts, negRights); Releasables.close(tops, bottoms, posLefts, posRights, negLefts, negRights);
} }
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.GeoPoint> {
private final boolean wrapLongitude; private final boolean wrapLongitude;
@ -170,14 +169,14 @@ public final class GeoBoundsAggregator extends MetricsAggregator {
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new GeoBoundsAggregator(name, 0, aggregationContext, parent, null, wrapLongitude, metaData); return new GeoBoundsAggregator(name, aggregationContext, parent, null, wrapLongitude, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext,
Aggregator parent, Map<String, Object> metaData) { Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new GeoBoundsAggregator(name, expectedBucketsCount, aggregationContext, parent, valuesSource, wrapLongitude, metaData); return new GeoBoundsAggregator(name, aggregationContext, parent, valuesSource, wrapLongitude, metaData);
} }
} }

View File

@ -48,14 +48,13 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
private DoubleArray maxes; private DoubleArray maxes;
private ValueFormatter formatter; private ValueFormatter formatter;
public MaxAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public MaxAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
if (valuesSource != null) { if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; maxes = bigArrays.newDoubleArray(1, false);
maxes = bigArrays.newDoubleArray(initialSize, false);
maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY);
} }
} }
@ -92,31 +91,31 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) { if (valuesSource == null) {
return new InternalMax(name, Double.NEGATIVE_INFINITY, formatter, getMetaData()); return new InternalMax(name, Double.NEGATIVE_INFINITY, formatter, metaData());
} }
assert owningBucketOrdinal < maxes.size(); assert owningBucketOrdinal < maxes.size();
return new InternalMax(name, maxes.get(owningBucketOrdinal), formatter, getMetaData()); return new InternalMax(name, maxes.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalMax(name, Double.NEGATIVE_INFINITY, formatter, getMetaData()); return new InternalMax(name, Double.NEGATIVE_INFINITY, formatter, metaData());
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) { public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalMax.TYPE.name(), valuesSourceConfig); super(name, InternalMax.TYPE.name(), valuesSourceConfig);
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new MaxAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); return new MaxAggregator(name, null, config.formatter(), aggregationContext, parent, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new MaxAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); return new MaxAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData);
} }
} }

View File

@ -48,13 +48,12 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
private DoubleArray mins; private DoubleArray mins;
private ValueFormatter formatter; private ValueFormatter formatter;
public MinAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public MinAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
if (valuesSource != null) { if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; mins = bigArrays.newDoubleArray(1, false);
mins = bigArrays.newDoubleArray(initialSize, false);
mins.fill(0, mins.size(), Double.POSITIVE_INFINITY); mins.fill(0, mins.size(), Double.POSITIVE_INFINITY);
} }
this.formatter = formatter; this.formatter = formatter;
@ -92,31 +91,31 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) { if (valuesSource == null) {
return new InternalMin(name, Double.POSITIVE_INFINITY, formatter, getMetaData()); return new InternalMin(name, Double.POSITIVE_INFINITY, formatter, metaData());
} }
assert owningBucketOrdinal < mins.size(); assert owningBucketOrdinal < mins.size();
return new InternalMin(name, mins.get(owningBucketOrdinal), formatter, getMetaData()); return new InternalMin(name, mins.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalMin(name, Double.POSITIVE_INFINITY, formatter, getMetaData()); return new InternalMin(name, Double.POSITIVE_INFINITY, formatter, metaData());
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String,Object>> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) { public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalMin.TYPE.name(), valuesSourceConfig); super(name, InternalMin.TYPE.name(), valuesSourceConfig);
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new MinAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); return new MinAggregator(name, null, config.formatter(), aggregationContext, parent, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new MinAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); return new MinAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData);
} }
} }

View File

@ -49,14 +49,14 @@ public abstract class AbstractPercentilesAggregator extends NumericMetricsAggreg
protected final boolean keyed; protected final boolean keyed;
protected ValueFormatter formatter; protected ValueFormatter formatter;
public AbstractPercentilesAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, public AbstractPercentilesAggregator(String name, ValuesSource.Numeric valuesSource, AggregationContext context,
Aggregator parent, double[] keys, double compression, boolean keyed, Aggregator parent, double[] keys, double compression, boolean keyed,
@Nullable ValueFormatter formatter, Map<String, Object> metaData) { @Nullable ValueFormatter formatter, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.keyed = keyed; this.keyed = keyed;
this.formatter = formatter; this.formatter = formatter;
this.states = bigArrays.newObjectArray(estimatedBucketsCount); this.states = bigArrays.newObjectArray(1);
this.keys = keys; this.keys = keys;
this.compression = compression; this.compression = compression;
} }

View File

@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.Map; import java.util.Map;
/** /**
@ -36,10 +37,10 @@ import java.util.Map;
*/ */
public class PercentileRanksAggregator extends AbstractPercentilesAggregator { public class PercentileRanksAggregator extends AbstractPercentilesAggregator {
public PercentileRanksAggregator(String name, long estimatedBucketsCount, Numeric valuesSource, AggregationContext context, public PercentileRanksAggregator(String name, Numeric valuesSource, AggregationContext context,
Aggregator parent, double[] percents, double compression, boolean keyed, @Nullable ValueFormatter formatter, Aggregator parent, double[] percents, double compression, boolean keyed, @Nullable ValueFormatter formatter,
Map<String, Object> metaData) { Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, valuesSource, context, parent, percents, compression, keyed, formatter, metaData); super(name, valuesSource, context, parent, percents, compression, keyed, formatter, metaData);
} }
@Override @Override
@ -48,13 +49,13 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator {
if (state == null) { if (state == null) {
return buildEmptyAggregation(); return buildEmptyAggregation();
} else { } else {
return new InternalPercentileRanks(name, keys, state, keyed, formatter, getMetaData()); return new InternalPercentileRanks(name, keys, state, keyed, formatter, metaData());
} }
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalPercentileRanks(name, keys, new TDigestState(compression), keyed, formatter, getMetaData()); return new InternalPercentileRanks(name, keys, new TDigestState(compression), keyed, formatter, metaData());
} }
@Override @Override
@ -67,7 +68,7 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator {
} }
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
private final double[] values; private final double[] values;
private final double compression; private final double compression;
@ -82,14 +83,14 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator {
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new PercentileRanksAggregator(name, 0, null, aggregationContext, parent, values, compression, keyed, config.formatter(), return new PercentileRanksAggregator(name, null, aggregationContext, parent, values, compression, keyed, config.formatter(),
metaData); metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new PercentileRanksAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, values, compression, return new PercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, compression,
keyed, config.formatter(), metaData); keyed, config.formatter(), metaData);
} }
} }

View File

@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatter;
import java.io.IOException;
import java.util.Map; import java.util.Map;
/** /**
@ -36,10 +37,10 @@ import java.util.Map;
*/ */
public class PercentilesAggregator extends AbstractPercentilesAggregator { public class PercentilesAggregator extends AbstractPercentilesAggregator {
public PercentilesAggregator(String name, long estimatedBucketsCount, Numeric valuesSource, AggregationContext context, public PercentilesAggregator(String name, Numeric valuesSource, AggregationContext context,
Aggregator parent, double[] percents, double compression, boolean keyed, @Nullable ValueFormatter formatter, Aggregator parent, double[] percents, double compression, boolean keyed, @Nullable ValueFormatter formatter,
Map<String, Object> metaData) { Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, valuesSource, context, parent, percents, compression, keyed, formatter, metaData); super(name, valuesSource, context, parent, percents, compression, keyed, formatter, metaData);
} }
@Override @Override
@ -48,7 +49,7 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator {
if (state == null) { if (state == null) {
return buildEmptyAggregation(); return buildEmptyAggregation();
} else { } else {
return new InternalPercentiles(name, keys, state, keyed, formatter, getMetaData()); return new InternalPercentiles(name, keys, state, keyed, formatter, metaData());
} }
} }
@ -64,10 +65,10 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator {
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalPercentiles(name, keys, new TDigestState(compression), keyed, formatter, getMetaData()); return new InternalPercentiles(name, keys, new TDigestState(compression), keyed, formatter, metaData());
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
private final double[] percents; private final double[] percents;
private final double compression; private final double compression;
@ -82,14 +83,14 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator {
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new PercentilesAggregator(name, 0, null, aggregationContext, parent, percents, compression, keyed, config.formatter(), return new PercentilesAggregator(name, null, aggregationContext, parent, percents, compression, keyed, config.formatter(),
metaData); metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new PercentilesAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, percents, compression, return new PercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, compression,
keyed, config.formatter(), metaData); keyed, config.formatter(), metaData);
} }
} }

View File

@ -52,8 +52,8 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
protected ScriptedMetricAggregator(String name, String scriptLang, ScriptType initScriptType, String initScript, protected ScriptedMetricAggregator(String name, String scriptLang, ScriptType initScriptType, String initScript,
ScriptType mapScriptType, String mapScript, ScriptType combineScriptType, String combineScript, ScriptType reduceScriptType, ScriptType mapScriptType, String mapScript, ScriptType combineScriptType, String combineScript, ScriptType reduceScriptType,
String reduceScript, Map<String, Object> params, Map<String, Object> reduceParams, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { String reduceScript, Map<String, Object> params, Map<String, Object> reduceParams, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, 1, BucketAggregationMode.PER_BUCKET, context, parent, metaData); super(name, context, parent, metaData);
this.scriptService = context.searchContext().scriptService(); this.scriptService = context.searchContext().scriptService();
this.scriptLang = scriptLang; this.scriptLang = scriptLang;
this.reduceScriptType = reduceScriptType; this.reduceScriptType = reduceScriptType;
@ -86,12 +86,13 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
} }
@Override @Override
public void setNextReader(LeafReaderContext reader) { public void setNextReader(LeafReaderContext reader) throws IOException {
mapScript.setNextReader(reader); mapScript.setNextReader(reader);
} }
@Override @Override
public void collect(int docId, long bucketOrdinal) throws IOException { public void collect(int docId, long bucketOrdinal) throws IOException {
assert bucketOrdinal == 0 : bucketOrdinal;
mapScript.setNextDocId(docId); mapScript.setNextDocId(docId);
mapScript.run(); mapScript.run();
} }
@ -104,12 +105,12 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
} else { } else {
aggregation = params.get("_agg"); aggregation = params.get("_agg");
} }
return new InternalScriptedMetric(name, aggregation, scriptLang, reduceScriptType, reduceScript, reduceParams, getMetaData()); return new InternalScriptedMetric(name, aggregation, scriptLang, reduceScriptType, reduceScript, reduceParams, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalScriptedMetric(name, null, scriptLang, reduceScriptType, reduceScript, reduceParams, getMetaData()); return new InternalScriptedMetric(name, null, scriptLang, reduceScriptType, reduceScript, reduceParams, metaData());
} }
public static class Factory extends AggregatorFactory { public static class Factory extends AggregatorFactory {
@ -143,7 +144,10 @@ public class ScriptedMetricAggregator extends MetricsAggregator {
} }
@Override @Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) { public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
if (collectsFromSingleBucket == false) {
return asMultiBucketAggregator(this, context, parent);
}
Map<String, Object> params = null; Map<String, Object> params = null;
if (this.params != null) { if (this.params != null) {
params = deepCopyParams(this.params, context.searchContext()); params = deepCopyParams(this.params, context.searchContext());

View File

@ -52,17 +52,16 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue {
private DoubleArray maxes; private DoubleArray maxes;
private ValueFormatter formatter; private ValueFormatter formatter;
public StatsAggegator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public StatsAggegator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
if (valuesSource != null) { if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; counts = bigArrays.newLongArray(1, true);
counts = bigArrays.newLongArray(initialSize, true); sums = bigArrays.newDoubleArray(1, true);
sums = bigArrays.newDoubleArray(initialSize, true); mins = bigArrays.newDoubleArray(1, false);
mins = bigArrays.newDoubleArray(initialSize, false);
mins.fill(0, mins.size(), Double.POSITIVE_INFINITY); mins.fill(0, mins.size(), Double.POSITIVE_INFINITY);
maxes = bigArrays.newDoubleArray(initialSize, false); maxes = bigArrays.newDoubleArray(1, false);
maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY);
} }
this.formatter = formatter; this.formatter = formatter;
@ -134,32 +133,32 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue {
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) { if (valuesSource == null) {
return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, formatter, getMetaData()); return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, formatter, metaData());
} }
assert owningBucketOrdinal < counts.size(); assert owningBucketOrdinal < counts.size();
return new InternalStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal), mins.get(owningBucketOrdinal), return new InternalStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal), mins.get(owningBucketOrdinal),
maxes.get(owningBucketOrdinal), formatter, getMetaData()); maxes.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, formatter, getMetaData()); return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, formatter, metaData());
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) { public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalStats.TYPE.name(), valuesSourceConfig); super(name, InternalStats.TYPE.name(), valuesSourceConfig);
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new StatsAggegator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); return new StatsAggegator(name, null, config.formatter(), aggregationContext, parent, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new StatsAggegator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); return new StatsAggegator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData);
} }
} }

View File

@ -53,20 +53,19 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
private DoubleArray sumOfSqrs; private DoubleArray sumOfSqrs;
private ValueFormatter formatter; private ValueFormatter formatter;
public ExtendedStatsAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, public ExtendedStatsAggregator(String name, ValuesSource.Numeric valuesSource,
@Nullable ValueFormatter formatter, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { @Nullable ValueFormatter formatter, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
if (valuesSource != null) { if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; counts = bigArrays.newLongArray(1, true);
counts = bigArrays.newLongArray(initialSize, true); sums = bigArrays.newDoubleArray(1, true);
sums = bigArrays.newDoubleArray(initialSize, true); mins = bigArrays.newDoubleArray(1, false);
mins = bigArrays.newDoubleArray(initialSize, false);
mins.fill(0, mins.size(), Double.POSITIVE_INFINITY); mins.fill(0, mins.size(), Double.POSITIVE_INFINITY);
maxes = bigArrays.newDoubleArray(initialSize, false); maxes = bigArrays.newDoubleArray(1, false);
maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY);
sumOfSqrs = bigArrays.newDoubleArray(initialSize, true); sumOfSqrs = bigArrays.newDoubleArray(1, true);
} }
} }
@ -149,16 +148,16 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) { if (valuesSource == null) {
return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, formatter, getMetaData()); return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, formatter, metaData());
} }
assert owningBucketOrdinal < counts.size(); assert owningBucketOrdinal < counts.size();
return new InternalExtendedStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal), return new InternalExtendedStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal),
mins.get(owningBucketOrdinal), maxes.get(owningBucketOrdinal), sumOfSqrs.get(owningBucketOrdinal), formatter, getMetaData()); mins.get(owningBucketOrdinal), maxes.get(owningBucketOrdinal), sumOfSqrs.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, formatter, getMetaData()); return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, formatter, metaData());
} }
@Override @Override
@ -166,20 +165,20 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue
Releasables.close(counts, maxes, mins, sumOfSqrs, sums); Releasables.close(counts, maxes, mins, sumOfSqrs, sums);
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) { public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalExtendedStats.TYPE.name(), valuesSourceConfig); super(name, InternalExtendedStats.TYPE.name(), valuesSourceConfig);
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new ExtendedStatsAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); return new ExtendedStatsAggregator(name, null, config.formatter(), aggregationContext, parent, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new ExtendedStatsAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, return new ExtendedStatsAggregator(name, valuesSource, config.formatter(), aggregationContext, parent,
metaData); metaData);
} }
} }

View File

@ -46,14 +46,13 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
private DoubleArray sums; private DoubleArray sums;
private ValueFormatter formatter; private ValueFormatter formatter;
public SumAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, public SumAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter,
AggregationContext context, Aggregator parent, Map<String, Object> metaData) { AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
if (valuesSource != null) { if (valuesSource != null) {
final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; sums = bigArrays.newDoubleArray(1, true);
sums = bigArrays.newDoubleArray(initialSize, true);
} }
} }
@ -87,30 +86,30 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue {
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) { if (valuesSource == null) {
return new InternalSum(name, 0, formatter, getMetaData()); return new InternalSum(name, 0, formatter, metaData());
} }
return new InternalSum(name, sums.get(owningBucketOrdinal), formatter, getMetaData()); return new InternalSum(name, sums.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalSum(name, 0.0, formatter, getMetaData()); return new InternalSum(name, 0.0, formatter, metaData());
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric, Map<String, Object>> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {
public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) { public Factory(String name, ValuesSourceConfig<ValuesSource.Numeric> valuesSourceConfig) {
super(name, InternalSum.TYPE.name(), valuesSourceConfig); super(name, InternalSum.TYPE.name(), valuesSourceConfig);
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new SumAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); return new SumAggregator(name, null, config.formatter(), aggregationContext, parent, metaData);
} }
@Override @Override
protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new SumAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); return new SumAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData);
} }
} }

View File

@ -59,10 +59,10 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware
private Scorer currentScorer; private Scorer currentScorer;
private LeafReaderContext currentContext; private LeafReaderContext currentContext;
public TopHitsAggregator(FetchPhase fetchPhase, SubSearchContext subSearchContext, String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map<String, Object> metaData) { public TopHitsAggregator(FetchPhase fetchPhase, SubSearchContext subSearchContext, String name, AggregationContext context, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, estimatedBucketsCount, context, parent, metaData); super(name, context, parent, metaData);
this.fetchPhase = fetchPhase; this.fetchPhase = fetchPhase;
topDocsCollectors = new LongObjectPagedHashMap<>(estimatedBucketsCount, context.bigArrays()); topDocsCollectors = new LongObjectPagedHashMap<>(1, context.bigArrays());
this.subSearchContext = subSearchContext; this.subSearchContext = subSearchContext;
context.registerScorerAware(this); context.registerScorerAware(this);
} }
@ -167,8 +167,8 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware
} }
@Override @Override
public Aggregator createInternal(AggregationContext aggregationContext, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) { public Aggregator createInternal(AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new TopHitsAggregator(fetchPhase, subSearchContext, name, expectedBucketsCount, aggregationContext, parent, metaData); return new TopHitsAggregator(fetchPhase, subSearchContext, name, aggregationContext, parent, metaData);
} }
@Override @Override

View File

@ -50,15 +50,13 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
LongArray counts; LongArray counts;
private ValueFormatter formatter; private ValueFormatter formatter;
public ValueCountAggregator(String name, long expectedBucketsCount, ValuesSource valuesSource, @Nullable ValueFormatter formatter, public ValueCountAggregator(String name, ValuesSource valuesSource, @Nullable ValueFormatter formatter,
AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
super(name, 0, aggregationContext, parent, metaData); super(name, aggregationContext, parent, metaData);
this.valuesSource = valuesSource; this.valuesSource = valuesSource;
this.formatter = formatter; this.formatter = formatter;
if (valuesSource != null) { if (valuesSource != null) {
// expectedBucketsCount == 0 means it's a top level bucket counts = bigArrays.newLongArray(1, true);
final long initialSize = expectedBucketsCount < 2 ? 1 : expectedBucketsCount;
counts = bigArrays.newLongArray(initialSize, true);
} }
} }
@ -87,15 +85,15 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
@Override @Override
public InternalAggregation buildAggregation(long owningBucketOrdinal) { public InternalAggregation buildAggregation(long owningBucketOrdinal) {
if (valuesSource == null) { if (valuesSource == null) {
return new InternalValueCount(name, 0, formatter, getMetaData()); return new InternalValueCount(name, 0, formatter, metaData());
} }
assert owningBucketOrdinal < counts.size(); assert owningBucketOrdinal < counts.size();
return new InternalValueCount(name, counts.get(owningBucketOrdinal), formatter, getMetaData()); return new InternalValueCount(name, counts.get(owningBucketOrdinal), formatter, metaData());
} }
@Override @Override
public InternalAggregation buildEmptyAggregation() { public InternalAggregation buildEmptyAggregation() {
return new InternalValueCount(name, 0l, formatter, getMetaData()); return new InternalValueCount(name, 0l, formatter, metaData());
} }
@Override @Override
@ -103,20 +101,20 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue {
Releasables.close(counts); Releasables.close(counts);
} }
public static class Factory<VS extends ValuesSource> extends ValuesSourceAggregatorFactory.LeafOnly<VS, Map<String, Object>> { public static class Factory<VS extends ValuesSource> extends ValuesSourceAggregatorFactory.LeafOnly<VS> {
public Factory(String name, ValuesSourceConfig<VS> config) { public Factory(String name, ValuesSourceConfig<VS> config) {
super(name, InternalValueCount.TYPE.name(), config); super(name, InternalValueCount.TYPE.name(), config);
} }
@Override @Override
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException {
return new ValueCountAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); return new ValueCountAggregator(name, null, config.formatter(), aggregationContext, parent, metaData);
} }
@Override @Override
protected Aggregator create(VS valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) { protected Aggregator doCreateInternal(VS valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
return new ValueCountAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, return new ValueCountAggregator(name, valuesSource, config.formatter(), aggregationContext, parent,
metaData); metaData);
} }

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.support; package org.elasticsearch.search.aggregations.support;
import com.carrotsearch.hppc.ObjectObjectOpenHashMap; import com.carrotsearch.hppc.ObjectObjectOpenHashMap;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Scorer;
@ -37,6 +38,7 @@ import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
@ -44,7 +46,7 @@ import java.util.List;
/** /**
* *
*/ */
@SuppressWarnings({"unchecked", "ForLoopReplaceableByForEach"}) @SuppressWarnings({"unchecked"})
public class AggregationContext implements ReaderContextAware, ScorerAware { public class AggregationContext implements ReaderContextAware, ScorerAware {
private final SearchContext searchContext; private final SearchContext searchContext;
@ -88,7 +90,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
} }
} }
public void setNextReader(LeafReaderContext reader) { public void setNextReader(LeafReaderContext reader) throws IOException {
this.reader = reader; this.reader = reader;
for (ReaderContextAware aware : readerAwares) { for (ReaderContextAware aware : readerAwares) {
aware.setNextReader(reader); aware.setNextReader(reader);
@ -111,7 +113,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
} }
/** Get a value source given its configuration and the depth of the aggregator in the aggregation tree. */ /** Get a value source given its configuration and the depth of the aggregator in the aggregation tree. */
public <VS extends ValuesSource> VS valuesSource(ValuesSourceConfig<VS> config, int depth) { public <VS extends ValuesSource> VS valuesSource(ValuesSourceConfig<VS> config, int depth) throws IOException {
assert config.valid() : "value source config is invalid - must have either a field context or a script or marked as unmapped"; assert config.valid() : "value source config is invalid - must have either a field context or a script or marked as unmapped";
assert !config.unmapped : "value source should not be created for unmapped fields"; assert !config.unmapped : "value source should not be created for unmapped fields";
@ -143,7 +145,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
return (VS) bytesField(fieldDataSources, config); return (VS) bytesField(fieldDataSources, config);
} }
private ValuesSource.Numeric numericScript(ValuesSourceConfig<?> config) { private ValuesSource.Numeric numericScript(ValuesSourceConfig<?> config) throws IOException {
setScorerIfNeeded(config.script); setScorerIfNeeded(config.script);
setReaderIfNeeded(config.script); setReaderIfNeeded(config.script);
scorerAwares.add(config.script); scorerAwares.add(config.script);
@ -152,7 +154,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
return source; return source;
} }
private ValuesSource.Numeric numericField(ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources, ValuesSourceConfig<?> config) { private ValuesSource.Numeric numericField(ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources, ValuesSourceConfig<?> config) throws IOException {
final ConfigCacheKey cacheKey = new ConfigCacheKey(config); final ConfigCacheKey cacheKey = new ConfigCacheKey(config);
ValuesSource.Numeric dataSource = (ValuesSource.Numeric) fieldDataSources.get(cacheKey); ValuesSource.Numeric dataSource = (ValuesSource.Numeric) fieldDataSources.get(cacheKey);
if (dataSource == null) { if (dataSource == null) {
@ -172,7 +174,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
return dataSource; return dataSource;
} }
private ValuesSource bytesField(ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources, ValuesSourceConfig<?> config) { private ValuesSource bytesField(ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources, ValuesSourceConfig<?> config) throws IOException {
final ConfigCacheKey cacheKey = new ConfigCacheKey(config); final ConfigCacheKey cacheKey = new ConfigCacheKey(config);
ValuesSource dataSource = fieldDataSources.get(cacheKey); ValuesSource dataSource = fieldDataSources.get(cacheKey);
if (dataSource == null) { if (dataSource == null) {
@ -202,7 +204,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
return dataSource; return dataSource;
} }
private ValuesSource.Bytes bytesScript(ValuesSourceConfig<?> config) { private ValuesSource.Bytes bytesScript(ValuesSourceConfig<?> config) throws IOException {
setScorerIfNeeded(config.script); setScorerIfNeeded(config.script);
setReaderIfNeeded(config.script); setReaderIfNeeded(config.script);
scorerAwares.add(config.script); scorerAwares.add(config.script);
@ -211,7 +213,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
return source; return source;
} }
private ValuesSource.GeoPoint geoPointField(ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources, ValuesSourceConfig<?> config) { private ValuesSource.GeoPoint geoPointField(ObjectObjectOpenHashMap<ConfigCacheKey, ValuesSource> fieldDataSources, ValuesSourceConfig<?> config) throws IOException {
final ConfigCacheKey cacheKey = new ConfigCacheKey(config); final ConfigCacheKey cacheKey = new ConfigCacheKey(config);
ValuesSource.GeoPoint dataSource = (ValuesSource.GeoPoint) fieldDataSources.get(cacheKey); ValuesSource.GeoPoint dataSource = (ValuesSource.GeoPoint) fieldDataSources.get(cacheKey);
if (dataSource == null) { if (dataSource == null) {
@ -224,7 +226,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
return dataSource; return dataSource;
} }
public void registerReaderContextAware(ReaderContextAware readerContextAware) { public void registerReaderContextAware(ReaderContextAware readerContextAware) throws IOException {
setReaderIfNeeded(readerContextAware); setReaderIfNeeded(readerContextAware);
readerAwares.add(readerContextAware); readerAwares.add(readerContextAware);
} }
@ -234,7 +236,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware {
scorerAwares.add(scorerAware); scorerAwares.add(scorerAware);
} }
private void setReaderIfNeeded(ReaderContextAware readerAware) { private void setReaderIfNeeded(ReaderContextAware readerAware) throws IOException {
if (reader != null) { if (reader != null) {
readerAware.setNextReader(reader); readerAware.setNextReader(reader);
} }

View File

@ -21,14 +21,15 @@ package org.elasticsearch.search.aggregations.support;
import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.*;
import org.elasticsearch.search.aggregations.support.format.ValueFormat; import org.elasticsearch.search.aggregations.support.format.ValueFormat;
import java.io.IOException;
import java.util.Map; import java.util.Map;
/** /**
* *
*/ */
public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource, M extends Map<String, Object>> extends AggregatorFactory { public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource> extends AggregatorFactory {
public static abstract class LeafOnly<VS extends ValuesSource, M extends Map<String, Object>> extends ValuesSourceAggregatorFactory<VS, M> { public static abstract class LeafOnly<VS extends ValuesSource> extends ValuesSourceAggregatorFactory<VS> {
protected LeafOnly(String name, String type, ValuesSourceConfig<VS> valuesSourceConfig) { protected LeafOnly(String name, String type, ValuesSourceConfig<VS> valuesSourceConfig) {
super(name, type, valuesSourceConfig); super(name, type, valuesSourceConfig);
@ -48,12 +49,12 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource, M e
} }
@Override @Override
public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map<String, Object> metaData) { public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException {
if (config.unmapped()) { if (config.unmapped()) {
return createUnmapped(context, parent, (M)metaData); return createUnmapped(context, parent, metaData);
} }
VS vs = context.valuesSource(config, parent == null ? 0 : 1 + parent.depth()); VS vs = context.valuesSource(config, parent == null ? 0 : 1 + parent.depth());
return create(vs, expectedBucketsCount, context, parent, (M)metaData); return doCreateInternal(vs, context, parent, collectsFromSingleBucket, metaData);
} }
@Override @Override
@ -63,9 +64,9 @@ public abstract class ValuesSourceAggregatorFactory<VS extends ValuesSource, M e
} }
} }
protected abstract Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, M metaData); protected abstract Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map<String, Object> metaData) throws IOException;
protected abstract Aggregator create(VS valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, M metaData); protected abstract Aggregator doCreateInternal(VS valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map<String, Object> metaData) throws IOException;
private void resolveValuesSourceConfigFromAncestors(String aggName, AggregatorFactory parent, Class<VS> requiredValuesSourceType) { private void resolveValuesSourceConfigFromAncestors(String aggName, AggregatorFactory parent, Class<VS> requiredValuesSourceType) {
ValuesSourceConfig config; ValuesSourceConfig config;

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.fetch.script;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.SearchParseElement; import org.elasticsearch.search.SearchParseElement;
@ -29,6 +30,7 @@ import org.elasticsearch.search.internal.InternalSearchHit;
import org.elasticsearch.search.internal.InternalSearchHitField; import org.elasticsearch.search.internal.InternalSearchHitField;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -70,7 +72,11 @@ public class ScriptFieldsFetchSubPhase implements FetchSubPhase {
@Override @Override
public void hitExecute(SearchContext context, HitContext hitContext) throws ElasticsearchException { public void hitExecute(SearchContext context, HitContext hitContext) throws ElasticsearchException {
for (ScriptFieldsContext.ScriptField scriptField : context.scriptFields().fields()) { for (ScriptFieldsContext.ScriptField scriptField : context.scriptFields().fields()) {
try {
scriptField.script().setNextReader(hitContext.readerContext()); scriptField.script().setNextReader(hitContext.readerContext());
} catch (IOException e) {
throw new ElasticsearchIllegalStateException("IOException while calling setNextReader", e);
}
scriptField.script().setNextDocId(hitContext.docId()); scriptField.script().setNextDocId(hitContext.docId());
Object value; Object value;

View File

@ -43,6 +43,7 @@ import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.SearchParseException;
import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Map; import java.util.Map;
/** /**
@ -154,7 +155,7 @@ public class ScriptSortParser implements SortParser {
case STRING_SORT_TYPE: case STRING_SORT_TYPE:
fieldComparatorSource = new BytesRefFieldComparatorSource(null, null, sortMode, nested) { fieldComparatorSource = new BytesRefFieldComparatorSource(null, null, sortMode, nested) {
@Override @Override
protected SortedBinaryDocValues getValues(LeafReaderContext context) { protected SortedBinaryDocValues getValues(LeafReaderContext context) throws IOException {
searchScript.setNextReader(context); searchScript.setNextReader(context);
final BinaryDocValues values = new BinaryDocValues() { final BinaryDocValues values = new BinaryDocValues() {
final BytesRefBuilder spare = new BytesRefBuilder(); final BytesRefBuilder spare = new BytesRefBuilder();
@ -177,7 +178,7 @@ public class ScriptSortParser implements SortParser {
// TODO: should we rather sort missing values last? // TODO: should we rather sort missing values last?
fieldComparatorSource = new DoubleValuesComparatorSource(null, Double.MAX_VALUE, sortMode, nested) { fieldComparatorSource = new DoubleValuesComparatorSource(null, Double.MAX_VALUE, sortMode, nested) {
@Override @Override
protected SortedNumericDoubleValues getValues(LeafReaderContext context) { protected SortedNumericDoubleValues getValues(LeafReaderContext context) throws IOException {
searchScript.setNextReader(context); searchScript.setNextReader(context);
final NumericDoubleValues values = new NumericDoubleValues() { final NumericDoubleValues values = new NumericDoubleValues() {
@Override @Override