mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-02-06 13:08:29 +00:00
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:
parent
b8be8e432e
commit
d9165dfe73
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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));
|
||||||
|
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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) {
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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() +
|
||||||
|
@ -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());
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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());
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
@ -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
|
||||||
|
Loading…
x
Reference in New Issue
Block a user