diff --git a/src/main/java/org/elasticsearch/common/lucene/ReaderContextAware.java b/src/main/java/org/elasticsearch/common/lucene/ReaderContextAware.java index e580909e990..e30f0ad7075 100644 --- a/src/main/java/org/elasticsearch/common/lucene/ReaderContextAware.java +++ b/src/main/java/org/elasticsearch/common/lucene/ReaderContextAware.java @@ -20,10 +20,12 @@ package org.elasticsearch.common.lucene; import org.apache.lucene.index.LeafReaderContext; +import java.io.IOException; + /** * */ public interface ReaderContextAware { - public void setNextReader(LeafReaderContext reader); + public void setNextReader(LeafReaderContext reader) throws IOException; } diff --git a/src/main/java/org/elasticsearch/common/lucene/search/function/ScoreFunction.java b/src/main/java/org/elasticsearch/common/lucene/search/function/ScoreFunction.java index 91b73e970ed..5d3a0f280e2 100644 --- a/src/main/java/org/elasticsearch/common/lucene/search/function/ScoreFunction.java +++ b/src/main/java/org/elasticsearch/common/lucene/search/function/ScoreFunction.java @@ -19,18 +19,16 @@ package org.elasticsearch.common.lucene.search.function; -import org.apache.lucene.index.LeafReaderContext; 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; - public abstract void setNextReader(LeafReaderContext context); - public abstract double score(int docId, float subQueryScore); public abstract Explanation explainScore(int docId, float subQueryScore); diff --git a/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java b/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java index 8b4801d4457..ab96c2613fb 100644 --- a/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java +++ b/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java @@ -88,7 +88,7 @@ public class ScriptScoreFunction extends ScoreFunction { } @Override - public void setNextReader(LeafReaderContext ctx) { + public void setNextReader(LeafReaderContext ctx) throws IOException { script.setNextReader(ctx); } diff --git a/src/main/java/org/elasticsearch/common/lucene/search/function/WeightFactorFunction.java b/src/main/java/org/elasticsearch/common/lucene/search/function/WeightFactorFunction.java index fba7e0ae194..21978eca96d 100644 --- a/src/main/java/org/elasticsearch/common/lucene/search/function/WeightFactorFunction.java +++ b/src/main/java/org/elasticsearch/common/lucene/search/function/WeightFactorFunction.java @@ -24,6 +24,8 @@ import org.apache.lucene.search.ComplexExplanation; import org.apache.lucene.search.Explanation; import org.elasticsearch.ElasticsearchIllegalArgumentException; +import java.io.IOException; + /** * */ @@ -53,7 +55,7 @@ public class WeightFactorFunction extends ScoreFunction { } @Override - public void setNextReader(LeafReaderContext context) { + public void setNextReader(LeafReaderContext context) throws IOException { scoreFunction.setNextReader(context); } diff --git a/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/BytesRefFieldComparatorSource.java b/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/BytesRefFieldComparatorSource.java index 6fd64e5ce66..85f90a74a2d 100644 --- a/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/BytesRefFieldComparatorSource.java +++ b/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/BytesRefFieldComparatorSource.java @@ -58,7 +58,7 @@ public class BytesRefFieldComparatorSource extends IndexFieldData.XFieldComparat return SortField.Type.STRING; } - protected SortedBinaryDocValues getValues(LeafReaderContext context) { + protected SortedBinaryDocValues getValues(LeafReaderContext context) throws IOException { return indexFieldData.load(context).getBytesValues(); } diff --git a/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/DoubleValuesComparatorSource.java b/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/DoubleValuesComparatorSource.java index bae774c1681..8696a487f64 100644 --- a/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/DoubleValuesComparatorSource.java +++ b/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/DoubleValuesComparatorSource.java @@ -56,7 +56,7 @@ public class DoubleValuesComparatorSource extends IndexFieldData.XFieldComparato return SortField.Type.DOUBLE; } - protected SortedNumericDoubleValues getValues(LeafReaderContext context) { + protected SortedNumericDoubleValues getValues(LeafReaderContext context) throws IOException { return indexFieldData.load(context).getDoubleValues(); } diff --git a/src/main/java/org/elasticsearch/percolator/QueryCollector.java b/src/main/java/org/elasticsearch/percolator/QueryCollector.java index f82ba28f002..ce6a75d0f88 100644 --- a/src/main/java/org/elasticsearch/percolator/QueryCollector.java +++ b/src/main/java/org/elasticsearch/percolator/QueryCollector.java @@ -63,7 +63,7 @@ abstract class QueryCollector extends SimpleCollector { List aggregatorLeafCollectors; - QueryCollector(ESLogger logger, PercolateContext context, boolean isNestedDoc) { + QueryCollector(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException { this.logger = logger; this.queries = context.percolateQueries(); 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); } - 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); } - 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); } - 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); } @@ -167,7 +167,7 @@ abstract class QueryCollector extends SimpleCollector { final int size; 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); this.limit = context.limit; this.size = context.size(); @@ -228,7 +228,7 @@ abstract class QueryCollector extends SimpleCollector { private final TopScoreDocCollector topDocsCollector; private LeafCollector topDocsLeafCollector; - MatchAndSort(ESLogger logger, PercolateContext context, boolean isNestedDoc) { + MatchAndSort(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException { super(logger, context, isNestedDoc); // TODO: Use TopFieldCollector.create(...) for ascending and descending scoring? topDocsCollector = TopScoreDocCollector.create(context.size(), false); @@ -289,7 +289,7 @@ abstract class QueryCollector extends SimpleCollector { 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); this.limit = context.limit; this.size = context.size(); @@ -358,7 +358,7 @@ abstract class QueryCollector extends SimpleCollector { private long counter = 0; - Count(ESLogger logger, PercolateContext context, boolean isNestedDoc) { + Count(ESLogger logger, PercolateContext context, boolean isNestedDoc) throws IOException { super(logger, context, isNestedDoc); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java b/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java index a32d33d2035..b0b5a303db9 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.aggregations; import com.google.common.collect.ImmutableMap; + import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.ConstantScoreQuery; 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.SimpleCollector; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.XCollector; @@ -77,7 +79,12 @@ public class AggregationPhase implements SearchPhase { context.aggregations().aggregationContext(aggregationContext); List 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++) { if (!(aggregators[i] instanceof GlobalAggregator)) { Aggregator aggregator = aggregators[i]; @@ -132,7 +139,11 @@ public class AggregationPhase implements SearchPhase { List aggregations = new ArrayList<>(aggregators.length); for (Aggregator aggregator : context.aggregations().aggregators()) { - aggregations.add(aggregator.buildAggregation(0)); + try { + aggregations.add(aggregator.buildAggregation(0)); + } catch (IOException e) { + throw new AggregationExecutionException("Failed to build aggregation [" + aggregator.name() + "]", e); + } } context.queryResult().aggregations(new InternalAggregations(aggregations)); diff --git a/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java b/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java index 637458c525a..50f2586fa5e 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/Aggregator.java @@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; + import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Scorer; import org.elasticsearch.ElasticsearchParseException; @@ -27,6 +28,7 @@ import org.elasticsearch.common.ParseField; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.util.BigArrays; 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.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext; @@ -34,52 +36,36 @@ import org.elasticsearch.search.internal.SearchContext.Lifetime; import org.elasticsearch.search.query.QueryPhaseExecutionException; 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 { + /** + * 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 COLLECTABLE_AGGREGATOR = new Predicate() { @Override public boolean apply(Aggregator aggregator) { return aggregator.shouldCollect(); } }; - private final Map 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 Map 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 { @@ -163,9 +149,8 @@ public abstract class Aggregator extends BucketCollector implements Releasable { protected final AggregationContext context; protected final BigArrays bigArrays; protected final int depth; - protected final long estimatedBucketCount; + private final Map metaData; - protected final BucketAggregationMode bucketAggregationMode; protected final AggregatorFactories factories; protected final Aggregator[] subAggregators; protected BucketCollector collectableSubAggregators; @@ -177,25 +162,21 @@ public abstract class Aggregator extends BucketCollector implements Releasable { * Constructs a new Aggregator. * * @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 estimatedBucketsCount When served as a sub-aggregator, indicate how many buckets the parent aggregator will generate. * @param context The aggregation context * @param parent The parent aggregator (may be {@code null} for top level aggregators) * @param metaData The metaData associated with this aggregator */ - protected Aggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map metaData) { + protected Aggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, Map metaData) throws IOException { this.name = name; this.metaData = metaData; this.parent = parent; - this.estimatedBucketCount = estimatedBucketsCount; this.context = context; this.bigArrays = context.bigArrays(); this.depth = parent == null ? 0 : 1 + parent.depth(); - this.bucketAggregationMode = bucketAggregationMode; assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead"; this.factories = factories; - this.subAggregators = factories.createSubAggregators(this, estimatedBucketsCount); + this.subAggregators = factories.createSubAggregators(this); context.searchContext().addReleasable(this, Lifetime.PHASE); // Register a safeguard to highlight any invalid construction logic (call to this constructor without subsequent preCollection call) collectableSubAggregators = new BucketCollector() { @@ -208,6 +189,11 @@ public abstract class Aggregator extends BucketCollector implements Releasable { badState(); } + @Override + public void preCollection() throws IOException { + badState(); + } + @Override public void postCollection() throws IOException { badState(); @@ -225,7 +211,17 @@ public abstract class Aggregator extends BucketCollector implements Releasable { }; } - protected void preCollection() { + public Map 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 collectables = Iterables.filter(Arrays.asList(subAggregators), COLLECTABLE_AGGREGATOR); List nextPassCollectors = new ArrayList<>(); List thisPassCollectors = new ArrayList<>(); @@ -247,6 +243,8 @@ public abstract class Aggregator extends BucketCollector implements Releasable { thisPassCollectors.add(recordingWrapper); } collectableSubAggregators = BucketCollector.wrap(thisPassCollectors); + collectableSubAggregators.preCollection(); + doPreCollection(); } /** @@ -279,11 +277,6 @@ public abstract class Aggregator extends BucketCollector implements Releasable { return name; } - /** Return the estimated number of buckets. */ - public final long estimatedBucketCount() { - return estimatedBucketCount; - } - /** Return the depth of this aggregator in the aggregation tree. */ public final int depth() { return depth; @@ -320,14 +313,6 @@ public abstract class Aggregator extends BucketCollector implements Releasable { 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 * 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 */ - public abstract InternalAggregation buildAggregation(long owningBucketOrdinal); + public abstract InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException; @Override - public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) { + public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) throws IOException { results.add(buildAggregation(bucketOrdinal)); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java index 560a9c9f67a..8d23df5a85c 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java @@ -18,12 +18,7 @@ */ package org.elasticsearch.search.aggregations; -import org.apache.lucene.index.LeafReaderContext; 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 java.io.IOException; @@ -49,8 +44,8 @@ public class AggregatorFactories { this.factories = factories; } - private static Aggregator createAndRegisterContextAware(AggregationContext context, AggregatorFactory factory, Aggregator parent, long estimatedBucketsCount) { - final Aggregator aggregator = factory.create(context, parent, estimatedBucketsCount); + private static Aggregator createAndRegisterContextAware(AggregationContext context, AggregatorFactory factory, Aggregator parent, boolean collectsFromSingleBucket) throws IOException { + final Aggregator aggregator = factory.create(context, parent, collectsFromSingleBucket); if (aggregator.shouldCollect()) { context.registerReaderContextAware(aggregator); } @@ -64,97 +59,26 @@ public class AggregatorFactories { /** * 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()]; for (int i = 0; i < factories.length; ++i) { - final AggregatorFactory factory = factories[i]; - final Aggregator first = createAndRegisterContextAware(parent.context(), factory, parent, estimatedBucketsCount); - if (first.bucketAggregationMode() == BucketAggregationMode.MULTI_BUCKETS) { - // This aggregator already supports multiple bucket ordinals, can be used directly - aggregators[i] = first; - continue; - } - // 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 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(); + // TODO: sometimes even sub aggregations always get called with bucket 0, eg. if + // you have a terms agg under a top-level filter agg. We should have a way to + // propagate the fact that only bucket 0 will be collected with single-bucket + // aggs + final boolean collectsFromSingleBucket = false; + aggregators[i] = createAndRegisterContextAware(parent.context(), factories[i], parent, collectsFromSingleBucket); } 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 Aggregator[] aggregators = new Aggregator[factories.length]; 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; } @@ -185,7 +109,7 @@ public class AggregatorFactories { } @Override - public Aggregator[] createSubAggregators(Aggregator parent, long estimatedBucketsCount) { + public Aggregator[] createSubAggregators(Aggregator parent) { return EMPTY_AGGREGATORS; } diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java index 353f5698bed..8681ffa6d29 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java @@ -18,8 +18,13 @@ */ 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 java.io.IOException; import java.util.Map; /** @@ -72,20 +77,20 @@ public abstract class AggregatorFactory { return parent; } + protected abstract Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException; + /** * Creates the aggregator * * @param context The aggregation context * @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 - * may generate (this is an estimation only). For top level factories, this will always be 0 + * @param collectsFromSingleBucket If true then the created aggregator will only be collected with 0 as a bucket ordinal. + * Some factories can take advantage of this in order to return more optimized implementations. * * @return The created aggregator */ - protected abstract Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map metaData); - - public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) { - Aggregator aggregator = createInternal(context, parent, expectedBucketsCount, this.metaData); + public final Aggregator create(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket) throws IOException { + Aggregator aggregator = createInternal(context, parent, collectsFromSingleBucket, this.metaData); return aggregator; } @@ -95,4 +100,98 @@ public abstract class AggregatorFactory { public void setMetaData(Map metaData) { this.metaData = metaData; } + + /** + * Utility method. Given an {@link AggregatorFactory} that creates {@link Aggregator}s that only know how + * to collect bucket 0, 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 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); + } + } + }; + } + } diff --git a/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java b/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java index 98aa1d799b0..3e607e9b5e1 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java +++ b/src/main/java/org/elasticsearch/search/aggregations/BucketCollector.java @@ -20,9 +20,9 @@ package org.elasticsearch.search.aggregations; import com.google.common.collect.Iterables; + import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.common.lucene.ReaderContextAware; -import org.elasticsearch.search.aggregations.Aggregator.BucketAggregationMode; import java.io.IOException; @@ -52,6 +52,9 @@ public abstract class BucketCollector implements ReaderContextAware { public void setNextReader(LeafReaderContext reader) { // no-op } + public void preCollection() throws IOException { + // no-op + } @Override public void postCollection() throws IOException { // no-op @@ -83,12 +86,19 @@ public abstract class BucketCollector implements ReaderContextAware { } @Override - public void setNextReader(LeafReaderContext reader) { + public void setNextReader(LeafReaderContext reader) throws IOException { for (BucketCollector collector : collectors) { collector.setNextReader(reader); } } + @Override + public void preCollection() throws IOException { + for (BucketCollector collector : collectors) { + collector.preCollection(); + } + } + @Override public void postCollection() throws IOException { for (BucketCollector collector : collectors) { @@ -97,7 +107,7 @@ public abstract class BucketCollector implements ReaderContextAware { } @Override - public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) { + public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) throws IOException { for (BucketCollector collector : collectors) { collector.gatherAnalysis(results, bucketOrdinal); } @@ -120,6 +130,11 @@ public abstract class BucketCollector implements ReaderContextAware { */ public abstract void collect(int docId, long bucketOrdinal) throws IOException; + /** + * Pre collection callback. + */ + public abstract void preCollection() throws IOException; + /** * Post collection callback. */ @@ -130,5 +145,5 @@ public abstract class BucketCollector implements ReaderContextAware { * @param analysisCollector * @param bucketOrdinal */ - public abstract void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal); + public abstract void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) throws IOException; } diff --git a/src/main/java/org/elasticsearch/search/aggregations/FilteringBucketCollector.java b/src/main/java/org/elasticsearch/search/aggregations/FilteringBucketCollector.java index fa2a5ccae3c..60efbce5f1a 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/FilteringBucketCollector.java +++ b/src/main/java/org/elasticsearch/search/aggregations/FilteringBucketCollector.java @@ -30,16 +30,16 @@ import org.elasticsearch.common.util.LongHash; import java.io.IOException; /** - * Filters a collection stream of docIds and related buckets using a sorted + * Filters a collection stream of docIds and related buckets using a sorted * list of valid bucket ordinals. */ public class FilteringBucketCollector extends BucketCollector implements Releasable { - + private final LongHash denseMap; private final BucketCollector delegate; - + /** - * + * * @param the valid BucketOrds * @param delegate The collector that will be called for any buckets listed in sortedBucketOrds */ @@ -52,7 +52,7 @@ public class FilteringBucketCollector extends BucketCollector implements Releasa } @Override - public final void setNextReader(LeafReaderContext reader) { + public final void setNextReader(LeafReaderContext reader) throws IOException { delegate.setNextReader(reader); } @@ -64,6 +64,11 @@ public class FilteringBucketCollector extends BucketCollector implements Releasa } } + @Override + public void preCollection() throws IOException { + delegate.preCollection(); + } + @Override public final void postCollection() throws IOException { delegate.postCollection(); @@ -75,7 +80,7 @@ public class FilteringBucketCollector extends BucketCollector implements Releasa } @Override - public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal){ + public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) throws IOException{ long ordinal = denseMap.find(bucketOrdinal); if (ordinal >= 0) { delegate.gatherAnalysis(analysisCollector, ordinal); diff --git a/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java index 6cee7368cba..cabad486ec8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java @@ -31,11 +31,11 @@ import java.util.Map; */ public abstract class NonCollectingAggregator extends Aggregator { - protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, AggregatorFactories subFactories, Map metaData) { - super(name, BucketAggregationMode.MULTI_BUCKETS, subFactories, 0, context, parent, metaData); + protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, AggregatorFactories subFactories, Map metaData) throws IOException { + super(name, subFactories, context, parent, metaData); } - protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, Map metaData) { + protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { this(name, context, parent, AggregatorFactories.EMPTY, metaData); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/RecordingPerReaderBucketCollector.java b/src/main/java/org/elasticsearch/search/aggregations/RecordingPerReaderBucketCollector.java index 711819e5073..245045bf8f5 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/RecordingPerReaderBucketCollector.java +++ b/src/main/java/org/elasticsearch/search/aggregations/RecordingPerReaderBucketCollector.java @@ -149,6 +149,11 @@ public class RecordingPerReaderBucketCollector extends RecordingBucketCollector collector.postCollection(); } + @Override + public void preCollection() throws IOException { + // nothing to do + } + @Override public void postCollection() throws IOException { recordingComplete = true; diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java index 0e3f247813d..b6ffd4487eb 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java @@ -35,10 +35,10 @@ public abstract class BucketsAggregator extends Aggregator { private IntArray docCounts; - public BucketsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, - long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map metaData) { - super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent, metaData); - docCounts = bigArrays.newIntArray(estimatedBucketsCount, true); + public BucketsAggregator(String name, AggregatorFactories factories, + AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, factories, context, parent, metaData); + docCounts = bigArrays.newIntArray(1, true); } /** @@ -48,11 +48,18 @@ public abstract class BucketsAggregator extends Aggregator { return docCounts.size(); } + /** + * Ensure there are at least maxBucketOrd 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) */ protected final void collectBucket(int doc, long bucketOrd) throws IOException { - docCounts = bigArrays.grow(docCounts, bucketOrd + 1); + grow(bucketOrd + 1); 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). */ - protected final InternalAggregations bucketAggregations(long bucketOrd) { + protected final InternalAggregations bucketAggregations(long bucketOrd) throws IOException { final ArrayList childAggs = new ArrayList<>(); final long bucketDocCount = bucketDocCount(bucketOrd); if (bucketDocCount == 0L) { diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java index 01ede852d8a..e232e1a1a62 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/DeferringBucketCollector.java @@ -49,14 +49,14 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa private FilteringBucketCollector filteredCollector; - public DeferringBucketCollector (BucketCollector deferred, AggregationContext context) { + public DeferringBucketCollector(BucketCollector deferred, AggregationContext context) { this.deferred = deferred; this.recording = new RecordingPerReaderBucketCollector(context); this.context = context; } @Override - public void setNextReader(LeafReaderContext reader) { + public void setNextReader(LeafReaderContext reader) throws IOException { recording.setNextReader(reader); } @@ -65,6 +65,11 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa recording.collect(docId, bucketOrdinal); } + @Override + public void preCollection() throws IOException { + recording.preCollection(); + } + @Override public void postCollection() throws IOException { recording.postCollection(); @@ -82,7 +87,7 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa BucketCollector subs = new BucketCollector() { @Override - public void setNextReader(LeafReaderContext reader) { + public void setNextReader(LeafReaderContext reader) throws IOException { // Need to set AggregationContext otherwise ValueSources in aggs // don't read any values context.setNextReader(reader); @@ -94,13 +99,18 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa deferred.collect(docId, bucketOrdinal); } + @Override + public void preCollection() throws IOException { + deferred.preCollection(); + } + @Override public void postCollection() throws IOException { deferred.postCollection(); } @Override - public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) { + public void gatherAnalysis(BucketAnalysisCollector results, long bucketOrdinal) throws IOException { deferred.gatherAnalysis(results, bucketOrdinal); } }; @@ -121,7 +131,7 @@ public class DeferringBucketCollector extends BucketCollector implements Releasa } @Override - public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) { + public void gatherAnalysis(BucketAnalysisCollector analysisCollector, long bucketOrdinal) throws IOException { filteredCollector.gatherAnalysis(analysisCollector, bucketOrdinal); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/SingleBucketAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/SingleBucketAggregator.java index c8a03ac6adb..9f3ed2db62b 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/SingleBucketAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/SingleBucketAggregator.java @@ -22,6 +22,7 @@ import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.support.AggregationContext; +import java.io.IOException; import java.util.Map; /** @@ -30,8 +31,8 @@ import java.util.Map; public abstract class SingleBucketAggregator extends BucketsAggregator { protected SingleBucketAggregator(String name, AggregatorFactories factories, - AggregationContext aggregationContext, Aggregator parent, Map metaData) { - super(name, BucketAggregationMode.MULTI_BUCKETS, factories, parent == null ? 1 : parent.estimatedBucketCount(), aggregationContext, parent, metaData); + AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, metaData); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java index eb6168bab52..83d9a74fab8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java @@ -72,7 +72,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement public ParentToChildrenAggregator(String name, AggregatorFactories factories, AggregationContext aggregationContext, Aggregator parent, String parentType, Filter childFilter, Filter parentFilter, - ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long maxOrd, Map metaData) { + ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long maxOrd, Map metaData) throws IOException { super(name, factories, aggregationContext, parent, metaData); this.parentType = parentType; // these two filters are cached in the parser @@ -85,13 +85,13 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { - return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalChildren(name, 0, buildEmptySubAggregations(), getMetaData()); + return new InternalChildren(name, 0, buildEmptySubAggregations(), metaData()); } @Override @@ -185,7 +185,7 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement Releasables.close(parentOrdToBuckets, parentOrdToOtherBuckets); } - public static class Factory extends ValuesSourceAggregatorFactory> { + public static class Factory extends ValuesSourceAggregatorFactory { private final String parentType; private final Filter parentFilter; @@ -199,19 +199,19 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator implement } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { return new NonCollectingAggregator(name, aggregationContext, parent, metaData) { @Override public InternalAggregation buildEmptyAggregation() { - return new InternalChildren(name, 0, buildEmptySubAggregations(), getMetaData()); + return new InternalChildren(name, 0, buildEmptySubAggregations(), metaData()); } }; } @Override - protected Aggregator create(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected Aggregator doCreateInternal(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { long maxOrd = valuesSource.globalMaxOrd(aggregationContext.searchContext().searcher(), parentType); return new ParentToChildrenAggregator(name, factories, aggregationContext, parent, parentType, childFilter, parentFilter, valuesSource, maxOrd, metaData); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java index 9df08bb48fd..bae4036d832 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java @@ -43,7 +43,7 @@ public class FilterAggregator extends SingleBucketAggregator { AggregatorFactories factories, AggregationContext aggregationContext, Aggregator parent, - Map metaData) { + Map metaData) throws IOException { super(name, factories, aggregationContext, parent, metaData); this.filter = filter; } @@ -65,13 +65,13 @@ public class FilterAggregator extends SingleBucketAggregator { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { - return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalFilter(name, 0, buildEmptySubAggregations(), getMetaData()); + return new InternalFilter(name, 0, buildEmptySubAggregations(), metaData()); } public static class Factory extends AggregatorFactory { @@ -84,7 +84,7 @@ public class FilterAggregator extends SingleBucketAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map metaData) { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { return new FilterAggregator(name, filter, factories, context, parent, metaData); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregator.java index bc30e3a5861..da925c9511d 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregator.java @@ -53,9 +53,8 @@ public class FiltersAggregator extends BucketsAggregator { private boolean keyed; public FiltersAggregator(String name, AggregatorFactories factories, List filters, boolean keyed, AggregationContext aggregationContext, - Aggregator parent, Map metaData) { - super(name, BucketAggregationMode.MULTI_BUCKETS, factories, filters.size() * (parent == null ? 1 : parent.estimatedBucketCount()), - aggregationContext, parent, metaData); + Aggregator parent, Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, metaData); this.keyed = keyed; this.filters = filters.toArray(new KeyedFilter[filters.size()]); this.bits = new Bits[this.filters.length]; @@ -87,7 +86,7 @@ public class FiltersAggregator extends BucketsAggregator { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { List buckets = Lists.newArrayListWithCapacity(filters.length); for (int i = 0; i < filters.length; 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); buckets.add(bucket); } - return new InternalFilters(name, buckets, keyed, getMetaData()); + return new InternalFilters(name, buckets, keyed, metaData()); } @Override @@ -106,7 +105,7 @@ public class FiltersAggregator extends BucketsAggregator { InternalFilters.Bucket bucket = new InternalFilters.Bucket(filters[i].key, 0, subAggs, keyed); 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) { @@ -125,7 +124,7 @@ public class FiltersAggregator extends BucketsAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map metaData) { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { return new FiltersAggregator(name, factories, filters, keyed, context, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java index 8b4674550a8..a1bd76b5b6a 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java @@ -41,8 +41,6 @@ import java.util.Map; public class GeoHashGridAggregator extends BucketsAggregator { - private static final int INITIAL_CAPACITY = 50; // TODO sizing - private final int requiredSize; private final int shardSize; private final ValuesSource.Numeric valuesSource; @@ -50,12 +48,12 @@ public class GeoHashGridAggregator extends BucketsAggregator { private SortedNumericDocValues values; public GeoHashGridAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, - int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - super(name, BucketAggregationMode.PER_BUCKET, factories, INITIAL_CAPACITY, aggregationContext, parent, metaData); + int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, metaData); this.valuesSource = valuesSource; this.requiredSize = requiredSize; this.shardSize = shardSize; - bucketOrds = new LongHash(INITIAL_CAPACITY, aggregationContext.bigArrays()); + bucketOrds = new LongHash(1, aggregationContext.bigArrays()); } @Override @@ -102,7 +100,7 @@ public class GeoHashGridAggregator extends BucketsAggregator { } @Override - public InternalGeoHashGrid buildAggregation(long owningBucketOrdinal) { + public InternalGeoHashGrid buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; final int size = (int) Math.min(bucketOrds.size(), shardSize); @@ -125,12 +123,12 @@ public class GeoHashGridAggregator extends BucketsAggregator { bucket.aggregations = bucketAggregations(bucket.bucketOrd); list[i] = bucket; } - return new InternalGeoHashGrid(name, requiredSize, Arrays.asList(list), getMetaData()); + return new InternalGeoHashGrid(name, requiredSize, Arrays.asList(list), metaData()); } @Override public InternalGeoHashGrid buildEmptyAggregation() { - return new InternalGeoHashGrid(name, requiredSize, Collections.emptyList(), getMetaData()); + return new InternalGeoHashGrid(name, requiredSize, Collections.emptyList(), metaData()); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java index e78bf3cda86..f620bbe8fa7 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java @@ -103,7 +103,7 @@ public class GeoHashGridParser implements Aggregator.Parser { } - private static class GeoGridFactory extends ValuesSourceAggregatorFactory> { + private static class GeoGridFactory extends ValuesSourceAggregatorFactory { private int precision; private int requiredSize; @@ -117,7 +117,7 @@ public class GeoHashGridParser implements Aggregator.Parser { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize, Collections.emptyList(), metaData); return new NonCollectingAggregator(name, aggregationContext, parent, metaData) { public InternalAggregation buildEmptyAggregation() { @@ -127,7 +127,10 @@ public class GeoHashGridParser implements Aggregator.Parser { } @Override - protected Aggregator create(final ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + if (collectsFromSingleBucket == false) { + return asMultiBucketAggregator(this, aggregationContext, parent); + } final CellValues cellIdValues = new CellValues(valuesSource, precision); ValuesSource.Numeric cellIdSource = new CellIdSource(cellIdValues, valuesSource.metaData()); return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent, metaData); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java index d6ec410d167..ec2aaf52b01 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.aggregations.bucket.global; import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; import org.elasticsearch.search.aggregations.support.AggregationContext; @@ -31,7 +32,7 @@ import java.util.Map; */ public class GlobalAggregator extends SingleBucketAggregator { - public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext, Map metaData) { + public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext, Map metaData) throws IOException { super(name, subFactories, aggregationContext, null, metaData); } @@ -46,9 +47,9 @@ public class GlobalAggregator extends SingleBucketAggregator { } @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"; - return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); + return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); } @Override @@ -63,11 +64,14 @@ public class GlobalAggregator extends SingleBucketAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map metaData) { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { if (parent != null) { throw new AggregationExecutionException("Aggregation [" + parent.name() + "] cannot have a global " + "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); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java index 5453c1e1f08..2377b5b0bec 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java @@ -59,10 +59,10 @@ public class HistogramAggregator extends BucketsAggregator { public HistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, InternalOrder order, boolean keyed, long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - long initialCapacity, InternalHistogram.Factory histogramFactory, - AggregationContext aggregationContext, Aggregator parent, Map metaData) { + InternalHistogram.Factory histogramFactory, + AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - super(name, BucketAggregationMode.PER_BUCKET, factories, initialCapacity, aggregationContext, parent, metaData); + super(name, factories, aggregationContext, parent, metaData); this.rounding = rounding; this.order = order; this.keyed = keyed; @@ -72,7 +72,7 @@ public class HistogramAggregator extends BucketsAggregator { this.formatter = formatter; this.histogramFactory = histogramFactory; - bucketOrds = new LongHash(initialCapacity, aggregationContext.bigArrays()); + bucketOrds = new LongHash(1, aggregationContext.bigArrays()); } @Override @@ -111,7 +111,7 @@ public class HistogramAggregator extends BucketsAggregator { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; List buckets = new ArrayList<>((int) bucketOrds.size()); 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 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 public InternalAggregation buildEmptyAggregation() { 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 @@ -137,7 +137,7 @@ public class HistogramAggregator extends BucketsAggregator { Releasables.close(bucketOrds); } - public static class Factory extends ValuesSourceAggregatorFactory> { + public static class Factory extends ValuesSourceAggregatorFactory { private final Rounding rounding; private final InternalOrder order; @@ -160,18 +160,15 @@ public class HistogramAggregator extends BucketsAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, null, config.formatter(), 0, histogramFactory, aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, null, config.formatter(), histogramFactory, aggregationContext, parent, metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - // 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 - long estimatedBucketCount = 50; - if (hasParentBucketAggregator(parent)) { - estimatedBucketCount = 8; + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + if (collectsFromSingleBucket == false) { + return asMultiBucketAggregator(this, aggregationContext, parent); } - // 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. // 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()); 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); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java index 41bea16e717..4115cf56ad6 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java @@ -41,7 +41,7 @@ public class MissingAggregator extends SingleBucketAggregator { private Bits docsWithValue; public MissingAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, - AggregationContext aggregationContext, Aggregator parent, Map metaData) { + AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { super(name, factories, aggregationContext, parent, metaData); this.valuesSource = valuesSource; } @@ -63,28 +63,28 @@ public class MissingAggregator extends SingleBucketAggregator { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { - return new InternalMissing(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + return new InternalMissing(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalMissing(name, 0, buildEmptySubAggregations(), getMetaData()); + return new InternalMissing(name, 0, buildEmptySubAggregations(), metaData()); } - public static class Factory extends ValuesSourceAggregatorFactory> { + public static class Factory extends ValuesSourceAggregatorFactory { public Factory(String name, ValuesSourceConfig valueSourceConfig) { super(name, InternalMissing.TYPE.name(), valueSourceConfig); } @Override - protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { return new MissingAggregator(name, factories, null, aggregationContext, parent, metaData); } @Override - protected MissingAggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected MissingAggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { return new MissingAggregator(name, factories, valuesSource, aggregationContext, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java index eaaac90d3fa..f16e707f311 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java @@ -51,7 +51,7 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo private DocIdSetIterator childDocs; private BitSet parentDocs; - public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parentAggregator, Map metaData, FilterCachingPolicy filterCachingPolicy) { + public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parentAggregator, Map metaData, FilterCachingPolicy filterCachingPolicy) throws IOException { super(name, factories, aggregationContext, parentAggregator, metaData); this.parentAggregator = parentAggregator; childFilter = aggregationContext.searchContext().filterCache().cache(objectMapper.nestedTypeFilter(), null, filterCachingPolicy); @@ -117,13 +117,13 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { - return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalNested(name, 0, buildEmptySubAggregations(), getMetaData()); + return new InternalNested(name, 0, buildEmptySubAggregations(), metaData()); } private static Filter findClosestNestedPath(Aggregator parent) { @@ -149,7 +149,7 @@ public class NestedAggregator extends SingleBucketAggregator implements ReaderCo } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map metaData) { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { MapperService.SmartNameObjectMapper mapper = context.searchContext().smartNameObjectMapper(path); if (mapper == null) { 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 { - public Unmapped(String name, AggregationContext context, Aggregator parent, Map metaData) { + public Unmapped(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { super(name, context, parent, metaData); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalNested(name, 0, buildEmptySubAggregations(), getMetaData()); + return new InternalNested(name, 0, buildEmptySubAggregations(), metaData()); } } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java index 8c4ffb45ec9..aedc0af8930 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java @@ -49,7 +49,7 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R // TODO: Add LongIntPagedHashMap? private final LongIntOpenHashMap bucketOrdToLastCollectedParentDoc; - public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parent, Map metaData) { + public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { super(name, factories, aggregationContext, parent, metaData); if (objectMapper == null) { parentFilter = SearchContext.current().bitsetFilterCache().getBitDocIdSetFilter(NonNestedDocsFilter.INSTANCE); @@ -117,13 +117,13 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { - return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), getMetaData()); + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { + return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalReverseNested(name, 0, buildEmptySubAggregations(), getMetaData()); + return new InternalReverseNested(name, 0, buildEmptySubAggregations(), metaData()); } Filter getParentFilter() { @@ -140,7 +140,7 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map metaData) { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { // Early validation NestedAggregator closestNestedAggregator = findClosestNestedAggregator(parent); if (closestNestedAggregator == null) { @@ -168,13 +168,13 @@ public class ReverseNestedAggregator extends SingleBucketAggregator implements R private final static class Unmapped extends NonCollectingAggregator { - public Unmapped(String name, AggregationContext context, Aggregator parent, Map metaData) { + public Unmapped(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { super(name, context, parent, metaData); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalReverseNested(name, 0, buildEmptySubAggregations(), getMetaData()); + return new InternalReverseNested(name, 0, buildEmptySubAggregations(), metaData()); } } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java index 335e736d952..b9f0cf42bc7 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java @@ -98,9 +98,9 @@ public class RangeAggregator extends BucketsAggregator { boolean keyed, AggregationContext aggregationContext, Aggregator parent, - Map metaData) { + Map 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; this.valuesSource = valuesSource; this.formatter = format != null ? format.formatter() : null; @@ -196,17 +196,17 @@ public class RangeAggregator extends BucketsAggregator { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { List buckets = Lists.newArrayListWithCapacity(ranges.length); for (int i = 0; i < ranges.length; i++) { Range range = ranges[i]; final long bucketOrd = subBucketOrdinal(owningBucketOrdinal, i); 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); } // 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 @@ -220,7 +220,7 @@ public class RangeAggregator extends BucketsAggregator { buckets.add(bucket); } // 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) { @@ -258,7 +258,7 @@ public class RangeAggregator extends BucketsAggregator { AggregationContext context, Aggregator parent, InternalRange.Factory factory, - Map metaData) { + Map metaData) throws IOException { super(name, context, parent, metaData); this.ranges = ranges; @@ -278,11 +278,11 @@ public class RangeAggregator extends BucketsAggregator { for (RangeAggregator.Range range : ranges) { 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> { + public static class Factory extends ValuesSourceAggregatorFactory { private final InternalRange.Factory rangeFactory; private final List ranges; @@ -296,12 +296,12 @@ public class RangeAggregator extends BucketsAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, aggregationContext, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceParser.java index d05c1162768..b9b0479540d 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceParser.java @@ -153,7 +153,7 @@ public class GeoDistanceParser implements Aggregator.Parser { return new GeoDistanceFactory(aggregationName, vsParser.config(), InternalGeoDistance.FACTORY, origin, unit, distanceType, ranges, keyed); } - private static class GeoDistanceFactory extends ValuesSourceAggregatorFactory> { + private static class GeoDistanceFactory extends ValuesSourceAggregatorFactory { private final GeoPoint origin; private final DistanceUnit unit; @@ -175,12 +175,12 @@ public class GeoDistanceParser implements Aggregator.Parser { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { return new Unmapped(name, ranges, keyed, null, aggregationContext, parent, rangeFactory, metaData); } @Override - protected Aggregator create(final ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit); aggregationContext.registerReaderContextAware(distanceSource); return new RangeAggregator(name, factories, distanceSource, null, rangeFactory, ranges, keyed, aggregationContext, parent, metaData); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java index 1a55d584497..a365e734901 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java @@ -44,11 +44,11 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri protected final SignificantTermsAggregatorFactory termsAggFactory; 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, - SignificantTermsAggregatorFactory termsAggFactory, Map metaData) { + SignificantTermsAggregatorFactory termsAggFactory, Map 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; } @@ -59,7 +59,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri } @Override - public SignificantStringTerms buildAggregation(long owningBucketOrdinal) { + public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; if (globalOrds == null) { // no context in this reader return buildEmptyAggregation(); @@ -114,7 +114,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri 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 @@ -123,7 +123,7 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri ContextIndexSearcher searcher = context.searchContext().searcher(); IndexReader topReader = searcher.getIndexReader(); int supersetSize = topReader.numDocs(); - return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.emptyList(), getMetaData()); + return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.emptyList(), metaData()); } @Override @@ -135,9 +135,9 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri 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 metaData) { - super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory, metaData); - bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays()); + public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, Map metaData) throws IOException { + super(name, factories, valuesSource, 1, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory, metaData); + bucketOrds = new LongHash(1, aggregationContext.bigArrays()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java index 041f808eb7a..571f7054808 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java @@ -41,10 +41,10 @@ import java.util.Map; public class SignificantLongTermsAggregator extends LongTermsAggregator { public SignificantLongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, - long estimatedBucketCount, BucketCountThresholds bucketCountThresholds, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, IncludeExclude.LongFilter includeExclude, Map metaData) { + BucketCountThresholds bucketCountThresholds, + AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, IncludeExclude.LongFilter includeExclude, Map 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; } @@ -58,7 +58,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator { } @Override - public SignificantLongTerms buildAggregation(long owningBucketOrdinal) { + public SignificantLongTerms buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize()); @@ -93,7 +93,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator { bucket.aggregations = bucketAggregations(bucket.bucketOrd); 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 @@ -102,7 +102,7 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator { ContextIndexSearcher searcher = context.searchContext().searcher(); IndexReader topReader = searcher.getIndexReader(); int supersetSize = topReader.numDocs(); - return new SignificantLongTerms(0, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.emptyList(), getMetaData()); + return new SignificantLongTerms(0, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.emptyList(), metaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java index 901784c0140..6b174473bb1 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java @@ -43,11 +43,11 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator { protected final SignificantTermsAggregatorFactory termsAggFactory; public SignificantStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, - long estimatedBucketCount, BucketCountThresholds bucketCountThresholds, + BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, - SignificantTermsAggregatorFactory termsAggFactory, Map metaData) { + SignificantTermsAggregatorFactory termsAggFactory, Map 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; } @@ -58,7 +58,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator { } @Override - public SignificantStringTerms buildAggregation(long owningBucketOrdinal) { + public SignificantStringTerms buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; final int size = (int) Math.min(bucketOrds.size(), bucketCountThresholds.getShardSize()); @@ -98,7 +98,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator { 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 @@ -107,7 +107,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator { ContextIndexSearcher searcher = context.searchContext().searcher(); IndexReader topReader = searcher.getIndexReader(); int supersetSize = topReader.numDocs(); - return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.emptyList(), getMetaData()); + return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.emptyList(), metaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java index d604bbe85a0..edda310d90b 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java @@ -30,10 +30,13 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lucene.index.FilterableTermsEnum; import org.elasticsearch.common.lucene.index.FreqTermsEnum; 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.terms.TermsAggregator; -import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregatorFactory; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -47,7 +50,7 @@ import java.util.Map; /** * */ -public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFactory> implements Releasable { +public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFactory implements Releasable { public SignificanceHeuristic getSignificanceHeuristic() { return significanceHeuristic; @@ -58,10 +61,10 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac MAP(new ParseField("map")) { @Override - Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, + Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) { - return new SignificantStringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData); + AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) throws IOException { + return new SignificantStringTermsAggregator(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData); } @Override @@ -73,13 +76,13 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac GLOBAL_ORDINALS(new ParseField("global_ordinals")) { @Override - Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, + Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) { + AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) throws IOException { ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource; IndexSearcher indexSearcher = aggregationContext.searchContext().searcher(); 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 @@ -91,10 +94,10 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac GLOBAL_ORDINALS_HASH(new ParseField("global_ordinals_hash")) { @Override - Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, long estimatedBucketCount, + Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) { - return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData); + AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) throws IOException { + return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggregatorFactory, metaData); } @Override @@ -118,9 +121,9 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac 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, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData); + AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) throws IOException; abstract boolean needsGlobalOrdinals(); @@ -159,7 +162,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), metaData); return new NonCollectingAggregator(name, aggregationContext, parent, metaData) { @Override @@ -170,10 +173,12 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac } @Override - protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - numberOfAggregatorsCreated++; + protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + if (collectsFromSingleBucket == false) { + return asMultiBucketAggregator(this, aggregationContext, parent); + } - long estimatedBucketCount = TermsAggregatorFactory.estimatedBucketCount(valuesSource, parent); + numberOfAggregatorsCreated++; if (valuesSource instanceof ValuesSource.Bytes) { ExecutionMode execution = null; @@ -184,7 +189,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac execution = ExecutionMode.MAP; } if (execution == null) { - if (Aggregator.hasParentBucketAggregator(parent)) { + if (Aggregator.descendsFromBucketAggregator(parent)) { execution = ExecutionMode.GLOBAL_ORDINALS_HASH; } else { execution = ExecutionMode.GLOBAL_ORDINALS; @@ -192,7 +197,7 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac } assert execution != null; 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) { 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() + diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java index ad1fec669bc..8ceb035ed58 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java @@ -24,6 +24,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.support.AggregationContext; +import java.io.IOException; import java.util.Collections; import java.util.Map; @@ -33,9 +34,10 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator { protected final boolean showTermDocCountError; public AbstractStringTermsAggregator(String name, AggregatorFactories factories, - long estimatedBucketsCount, AggregationContext context, Aggregator parent, - Terms.Order order, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map metaData) { - super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketsCount, context, parent, bucketCountThresholds, order, subAggCollectMode, metaData); + AggregationContext context, Aggregator parent, + Terms.Order order, BucketCountThresholds bucketCountThresholds, + SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map metaData) throws IOException { + super(name, factories, context, parent, bucketCountThresholds, order, subAggCollectMode, metaData); this.showTermDocCountError = showTermDocCountError; } @@ -46,7 +48,7 @@ abstract class AbstractStringTermsAggregator extends TermsAggregator { @Override public InternalAggregation buildEmptyAggregation() { - return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.emptyList(), showTermDocCountError, 0, 0, getMetaData()); + return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.emptyList(), showTermDocCountError, 0, 0, metaData()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java index 5bfbb38be49..3eab05e97fb 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java @@ -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.format.ValueFormat; +import java.io.IOException; import java.util.Arrays; import java.util.Map; @@ -38,9 +39,9 @@ import java.util.Map; */ public class DoubleTermsAggregator extends LongTermsAggregator { - public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount, - Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map metaData) { - super(name, factories, valuesSource, format, estimatedBucketCount, order, bucketCountThresholds, aggregationContext, parent, collectionMode, showTermDocCountError, longFilter, metaData); + 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 metaData) throws IOException { + super(name, factories, valuesSource, format, order, bucketCountThresholds, aggregationContext, parent, collectionMode, showTermDocCountError, longFilter, metaData); } @Override @@ -49,7 +50,7 @@ public class DoubleTermsAggregator extends LongTermsAggregator { } @Override - public DoubleTerms buildAggregation(long owningBucketOrdinal) { + public DoubleTerms buildAggregation(long owningBucketOrdinal) throws IOException { final LongTerms terms = (LongTerms) super.buildAggregation(owningBucketOrdinal); return convertToDouble(terms); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java index 30b58c13f57..c25175bbbae 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java @@ -19,8 +19,8 @@ package org.elasticsearch.search.aggregations.bucket.terms; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.RandomAccessOrds; import org.apache.lucene.index.SortedDocValues; 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.util.IntArray; import org.elasticsearch.common.util.LongHash; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.fielddata.AbstractRandomAccessOrds; import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalMapping; @@ -72,12 +71,13 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr 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, - IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) { - super(name, factories, maxOrd, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData); + IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData); this.valuesSource = valuesSource; this.includeExclude = includeExclude; + grow(maxOrd); } protected long getBucketOrd(long termOrd) { @@ -143,7 +143,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { if (globalOrds == null) { // no context in this reader return buildEmptyAggregation(); } @@ -201,7 +201,7 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr 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 */ @@ -271,12 +271,11 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr 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, - Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) { - // Set maxOrd to estimatedBucketCount! To be conservative with memory. - super(name, factories, valuesSource, estimatedBucketCount, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); - bucketOrds = new LongHash(estimatedBucketCount, aggregationContext.bigArrays()); + Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) throws IOException { + super(name, factories, valuesSource, 1, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); + bucketOrds = new LongHash(1, aggregationContext.bigArrays()); } protected Collector newCollector(final RandomAccessOrds ords) { @@ -340,9 +339,9 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr private RandomAccessOrds segmentOrds; - public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, long estimatedBucketCount, - long maxOrd, Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) { - super(name, factories, valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); + 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 metaData) throws IOException { + super(name, factories, valuesSource, maxOrd, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); assert factories == null || factories.count() == 0; this.segmentDocCounts = bigArrays.newIntArray(maxOrd + 1, true); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java index f727bce5e96..3959f313a3f 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java @@ -51,14 +51,14 @@ public class LongTermsAggregator extends TermsAggregator { private SortedNumericDocValues values; private LongFilter longFilter; - public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, long estimatedBucketCount, - Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map metaData) { - super(name, BucketAggregationMode.PER_BUCKET, factories, estimatedBucketCount, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode, metaData); + 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 metaData) throws IOException { + super(name, factories, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode, metaData); this.valuesSource = valuesSource; this.showTermDocCountError = showTermDocCountError; this.formatter = format != null ? format.formatter() : null; 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 - public InternalAggregation buildAggregation(long owningBucketOrdinal) { + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; 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; } - 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 public InternalAggregation buildEmptyAggregation() { - return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.emptyList(), showTermDocCountError, 0, 0, getMetaData()); + return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.emptyList(), showTermDocCountError, 0, 0, metaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java index 175ce4773e6..b070b3dc3dc 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java @@ -34,7 +34,6 @@ import org.elasticsearch.search.aggregations.support.ValuesSource; import java.io.IOException; import java.util.Arrays; -import java.util.Collections; import java.util.Map; /** @@ -48,14 +47,14 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator { private SortedBinaryDocValues values; 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, - IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) { + IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map 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.includeExclude = includeExclude; - bucketOrds = new BytesRefHash(estimatedBucketCount, aggregationContext.bigArrays()); + bucketOrds = new BytesRefHash(1, aggregationContext.bigArrays()); previous = new BytesRefBuilder(); } @@ -97,7 +96,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator { } @Override - public InternalAggregation buildAggregation(long owningBucketOrdinal) { + public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; if (bucketCountThresholds.getMinDocCount() == 0 && (order != InternalOrder.COUNT_DESC || bucketOrds.size() < bucketCountThresholds.getRequiredSize())) { @@ -157,7 +156,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator { 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 diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java index 474d4215981..6ede9c524bf 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java @@ -135,8 +135,8 @@ public abstract class TermsAggregator extends BucketsAggregator { protected Set aggsUsedForSorting = new HashSet<>(); 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 metaData) { - super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent, metaData); + public TermsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode subAggCollectMode, Map metaData) throws IOException { + super(name, factories, context, parent, metaData); this.bucketCountThresholds = bucketCountThresholds; this.order = InternalOrder.validate(order, this); this.subAggCollectMode = subAggCollectMode; diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java index 1215cde1f1d..08d02d0636b 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java @@ -29,22 +29,23 @@ import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import java.io.IOException; import java.util.Map; /** * */ -public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory> { +public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory { public enum ExecutionMode { MAP(new ParseField("map")) { @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, - AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map metaData) { - return new StringTermsAggregator(name, factories, valuesSource, estimatedBucketCount, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); + AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map metaData) throws IOException { + return new StringTermsAggregator(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); } @Override @@ -56,10 +57,10 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) { - return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); + AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map metaData) throws IOException { + return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); } @Override @@ -71,10 +72,10 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) { - return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, estimatedBucketCount, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); + AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map metaData) throws IOException { + return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxOrd, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); } @Override @@ -85,13 +86,13 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) { + AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map metaData) throws IOException { 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 @@ -115,9 +116,9 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData); + IncludeExclude includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map metaData) throws IOException; abstract boolean needsGlobalOrdinals(); @@ -145,7 +146,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), metaData); return new NonCollectingAggregator(name, aggregationContext, parent, factories, metaData) { { @@ -159,34 +160,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) { - long estimatedBucketCount = estimatedBucketCount(valuesSource, parent); - + protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + if (collectsFromSingleBucket == false) { + return asMultiBucketAggregator(this, aggregationContext, parent); + } if (valuesSource instanceof ValuesSource.Bytes) { ExecutionMode execution = null; if (executionHint != null) { @@ -215,7 +193,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) { - this(name, estimatedBucketsCount, BucketAggregationMode.MULTI_BUCKETS, context, parent, metaData); - } - protected MetricsAggregator(String name, long estimatedBucketsCount, BucketAggregationMode bucketAggregationMode, AggregationContext context, Aggregator parent, Map metaData) { - super(name, bucketAggregationMode, AggregatorFactories.EMPTY, estimatedBucketsCount, context, parent, metaData); + protected MetricsAggregator(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, AggregatorFactories.EMPTY, context, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericMetricsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericMetricsAggregator.java index c8220ee46a1..66adf3ed74e 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericMetricsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericMetricsAggregator.java @@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.metrics; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.support.AggregationContext; +import java.io.IOException; import java.util.Map; /** @@ -28,14 +29,14 @@ import java.util.Map; */ public abstract class NumericMetricsAggregator extends MetricsAggregator { - private NumericMetricsAggregator(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + private NumericMetricsAggregator(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); } public static abstract class SingleValue extends NumericMetricsAggregator { - protected SingleValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + protected SingleValue(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); } public abstract double metric(long owningBucketOrd); @@ -43,8 +44,8 @@ public abstract class NumericMetricsAggregator extends MetricsAggregator { public static abstract class MultiValue extends NumericMetricsAggregator { - protected MultiValue(String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + protected MultiValue(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); } public abstract boolean hasMetric(String name); diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregator.java index 0522c83147a..d91006903ee 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregator.java @@ -48,15 +48,14 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue { private DoubleArray sums; private ValueFormatter formatter; - public AvgAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + public AvgAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, + AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name,context, parent, metaData); this.valuesSource = valuesSource; this.formatter = formatter; if (valuesSource != null) { - final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - counts = bigArrays.newLongArray(initialSize, true); - sums = bigArrays.newDoubleArray(initialSize, true); + counts = bigArrays.newLongArray(1, true); + sums = bigArrays.newDoubleArray(1, true); } } @@ -93,30 +92,30 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) { 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 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> { + public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { public Factory(String name, String type, ValuesSourceConfig valuesSourceConfig) { super(name, type, valuesSourceConfig); } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new AvgAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new AvgAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new AvgAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new AvgAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java index 205a9e43b69..820000a5bee 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java @@ -64,14 +64,13 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue private Collector collector; private ValueFormatter formatter; - public CardinalityAggregator(String name, long estimatedBucketsCount, ValuesSource valuesSource, boolean rehash, - int precision, - @Nullable ValueFormatter formatter, AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + public CardinalityAggregator(String name, ValuesSource valuesSource, boolean rehash, int precision, @Nullable ValueFormatter formatter, + AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); this.valuesSource = valuesSource; this.rehash = rehash; 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; } @@ -157,12 +156,12 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue // this Aggregator (and its HLL++ counters) is released. HyperLogLogPlusPlus copy = new HyperLogLogPlusPlus(precision, BigArrays.NON_RECYCLING_INSTANCE, 1); copy.merge(0, counts, owningBucketOrdinal); - return new InternalCardinality(name, copy, formatter, getMetaData()); + return new InternalCardinality(name, copy, formatter, metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalCardinality(name, null, formatter, getMetaData()); + return new InternalCardinality(name, null, formatter, metaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregatorFactory.java index c52c4ad4fb1..332d9f9c1a7 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregatorFactory.java @@ -21,15 +21,16 @@ package org.elasticsearch.search.aggregations.metrics.cardinality; import org.elasticsearch.search.aggregations.AggregationExecutionException; 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.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import java.io.IOException; import java.util.Map; -final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory> { +final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory { private final long precisionThreshold; private final boolean rehash; @@ -45,18 +46,16 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory metaData) { - return new CardinalityAggregator(name, parent == null ? 1 : parent.estimatedBucketCount(), null, true, precision(parent), - config.formatter(), context, parent, metaData); + protected Aggregator createUnmapped(AggregationContext context, Aggregator parent, Map metaData) throws IOException { + return new CardinalityAggregator(name, null, true, precision(parent), config.formatter(), context, parent, metaData); } @Override - protected Aggregator create(ValuesSource valuesSource, long expectedBucketsCount, AggregationContext context, Aggregator parent, Map metaData) { + protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { if (!(valuesSource instanceof ValuesSource.Numeric) && !rehash) { 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), - config.formatter(), context, parent, metaData); + return new CardinalityAggregator(name, valuesSource, rehash, precision(parent), config.formatter(), context, parent, metaData); } /* @@ -67,8 +66,8 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory metaData) { - super(name, estimatedBucketsCount, aggregationContext, parent, metaData); + protected GeoBoundsAggregator(String name, AggregationContext aggregationContext, + Aggregator parent, ValuesSource.GeoPoint valuesSource, boolean wrapLongitude, Map metaData) throws IOException { + super(name, aggregationContext, parent, metaData); this.valuesSource = valuesSource; this.wrapLongitude = wrapLongitude; if (valuesSource != null) { - final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - tops = bigArrays.newDoubleArray(initialSize, false); + tops = bigArrays.newDoubleArray(1, false); 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); - posLefts = bigArrays.newDoubleArray(initialSize, false); + posLefts = bigArrays.newDoubleArray(1, false); 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); - negLefts = bigArrays.newDoubleArray(initialSize, false); + negLefts = bigArrays.newDoubleArray(1, false); 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); } } @@ -90,13 +89,13 @@ public final class GeoBoundsAggregator extends MetricsAggregator { double posRight = posRights.get(owningBucketOrdinal); double negLeft = negLefts.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 public InternalAggregation buildEmptyAggregation() { 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 @@ -160,7 +159,7 @@ public final class GeoBoundsAggregator extends MetricsAggregator { Releasables.close(tops, bottoms, posLefts, posRights, negLefts, negRights); } - public static class Factory extends ValuesSourceAggregatorFactory> { + public static class Factory extends ValuesSourceAggregatorFactory { private final boolean wrapLongitude; @@ -170,14 +169,14 @@ public final class GeoBoundsAggregator extends MetricsAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new GeoBoundsAggregator(name, 0, aggregationContext, parent, null, wrapLongitude, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new GeoBoundsAggregator(name, aggregationContext, parent, null, wrapLongitude, metaData); } @Override - protected Aggregator create(ValuesSource.GeoPoint valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, - Aggregator parent, Map metaData) { - return new GeoBoundsAggregator(name, expectedBucketsCount, aggregationContext, parent, valuesSource, wrapLongitude, metaData); + protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, + Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new GeoBoundsAggregator(name, aggregationContext, parent, valuesSource, wrapLongitude, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java index 9214f47ab60..8f21ce5ed66 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java @@ -48,14 +48,13 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue { private DoubleArray maxes; private ValueFormatter formatter; - public MaxAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + public MaxAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, + AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); this.valuesSource = valuesSource; this.formatter = formatter; if (valuesSource != null) { - final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - maxes = bigArrays.newDoubleArray(initialSize, false); + maxes = bigArrays.newDoubleArray(1, false); maxes.fill(0, maxes.size(), Double.NEGATIVE_INFINITY); } } @@ -92,31 +91,31 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) { 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(); - return new InternalMax(name, maxes.get(owningBucketOrdinal), formatter, getMetaData()); + return new InternalMax(name, maxes.get(owningBucketOrdinal), formatter, metaData()); } @Override 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> { + public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { public Factory(String name, ValuesSourceConfig valuesSourceConfig) { super(name, InternalMax.TYPE.name(), valuesSourceConfig); } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new MaxAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new MaxAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new MaxAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new MaxAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java index 26d2268f22c..4c099af0959 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java @@ -48,13 +48,12 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue { private DoubleArray mins; private ValueFormatter formatter; - public MinAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + public MinAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, + AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); this.valuesSource = valuesSource; if (valuesSource != null) { - final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - mins = bigArrays.newDoubleArray(initialSize, false); + mins = bigArrays.newDoubleArray(1, false); mins.fill(0, mins.size(), Double.POSITIVE_INFINITY); } this.formatter = formatter; @@ -92,31 +91,31 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) { 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(); - return new InternalMin(name, mins.get(owningBucketOrdinal), formatter, getMetaData()); + return new InternalMin(name, mins.get(owningBucketOrdinal), formatter, metaData()); } @Override 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> { + public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { public Factory(String name, ValuesSourceConfig valuesSourceConfig) { super(name, InternalMin.TYPE.name(), valuesSourceConfig); } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new MinAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new MinAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new MinAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new MinAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesAggregator.java index 82dbba6dccc..c3c2545f6f5 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesAggregator.java @@ -49,14 +49,14 @@ public abstract class AbstractPercentilesAggregator extends NumericMetricsAggreg protected final boolean keyed; protected ValueFormatter formatter; - public AbstractPercentilesAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, AggregationContext context, - Aggregator parent, double[] keys, double compression, boolean keyed, - @Nullable ValueFormatter formatter, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + public AbstractPercentilesAggregator(String name, ValuesSource.Numeric valuesSource, AggregationContext context, + Aggregator parent, double[] keys, double compression, boolean keyed, + @Nullable ValueFormatter formatter, Map metaData) throws IOException { + super(name, context, parent, metaData); this.valuesSource = valuesSource; this.keyed = keyed; this.formatter = formatter; - this.states = bigArrays.newObjectArray(estimatedBucketsCount); + this.states = bigArrays.newObjectArray(1); this.keys = keys; this.compression = compression; } @@ -106,4 +106,4 @@ public abstract class AbstractPercentilesAggregator extends NumericMetricsAggreg Releasables.close(states); } -} \ No newline at end of file +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregator.java index 07dfcae9f64..0383e33e8a7 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregator.java @@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import java.io.IOException; import java.util.Map; /** @@ -36,10 +37,10 @@ import java.util.Map; */ 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, - Map metaData) { - super(name, estimatedBucketsCount, valuesSource, context, parent, percents, compression, keyed, formatter, metaData); + Map metaData) throws IOException { + super(name, valuesSource, context, parent, percents, compression, keyed, formatter, metaData); } @Override @@ -48,13 +49,13 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator { if (state == null) { return buildEmptyAggregation(); } else { - return new InternalPercentileRanks(name, keys, state, keyed, formatter, getMetaData()); + return new InternalPercentileRanks(name, keys, state, keyed, formatter, metaData()); } } @Override 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 @@ -67,7 +68,7 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator { } } - public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly> { + public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { private final double[] values; private final double compression; @@ -82,14 +83,14 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new PercentileRanksAggregator(name, 0, null, aggregationContext, parent, values, compression, keyed, config.formatter(), + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new PercentileRanksAggregator(name, null, aggregationContext, parent, values, compression, keyed, config.formatter(), metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new PercentileRanksAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, values, compression, + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new PercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, compression, keyed, config.formatter(), metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregator.java index b34fc7f2129..4dd99b73cd9 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregator.java @@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import java.io.IOException; import java.util.Map; /** @@ -36,10 +37,10 @@ import java.util.Map; */ 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, - Map metaData) { - super(name, estimatedBucketsCount, valuesSource, context, parent, percents, compression, keyed, formatter, metaData); + Map metaData) throws IOException { + super(name, valuesSource, context, parent, percents, compression, keyed, formatter, metaData); } @Override @@ -48,7 +49,7 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator { if (state == null) { return buildEmptyAggregation(); } 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 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> { + public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { private final double[] percents; private final double compression; @@ -82,14 +83,14 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new PercentilesAggregator(name, 0, null, aggregationContext, parent, percents, compression, keyed, config.formatter(), + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new PercentilesAggregator(name, null, aggregationContext, parent, percents, compression, keyed, config.formatter(), metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new PercentilesAggregator(name, expectedBucketsCount, valuesSource, aggregationContext, parent, percents, compression, + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new PercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, compression, keyed, config.formatter(), metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregator.java index 5e914864fda..e5d5de1ae89 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregator.java @@ -52,8 +52,8 @@ public class ScriptedMetricAggregator extends MetricsAggregator { protected ScriptedMetricAggregator(String name, String scriptLang, ScriptType initScriptType, String initScript, ScriptType mapScriptType, String mapScript, ScriptType combineScriptType, String combineScript, ScriptType reduceScriptType, - String reduceScript, Map params, Map reduceParams, AggregationContext context, Aggregator parent, Map metaData) { - super(name, 1, BucketAggregationMode.PER_BUCKET, context, parent, metaData); + String reduceScript, Map params, Map reduceParams, AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); this.scriptService = context.searchContext().scriptService(); this.scriptLang = scriptLang; this.reduceScriptType = reduceScriptType; @@ -86,12 +86,13 @@ public class ScriptedMetricAggregator extends MetricsAggregator { } @Override - public void setNextReader(LeafReaderContext reader) { + public void setNextReader(LeafReaderContext reader) throws IOException { mapScript.setNextReader(reader); } @Override public void collect(int docId, long bucketOrdinal) throws IOException { + assert bucketOrdinal == 0 : bucketOrdinal; mapScript.setNextDocId(docId); mapScript.run(); } @@ -104,12 +105,12 @@ public class ScriptedMetricAggregator extends MetricsAggregator { } else { 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 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 { @@ -143,7 +144,10 @@ public class ScriptedMetricAggregator extends MetricsAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, long expectedBucketsCount, Map metaData) { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + if (collectsFromSingleBucket == false) { + return asMultiBucketAggregator(this, context, parent); + } Map params = null; if (this.params != null) { params = deepCopyParams(this.params, context.searchContext()); diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java index 980d808dcad..2bfba589949 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java @@ -52,17 +52,16 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue { private DoubleArray maxes; private ValueFormatter formatter; - public StatsAggegator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + public StatsAggegator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, + AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); this.valuesSource = valuesSource; if (valuesSource != null) { - final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - counts = bigArrays.newLongArray(initialSize, true); - sums = bigArrays.newDoubleArray(initialSize, true); - mins = bigArrays.newDoubleArray(initialSize, false); + counts = bigArrays.newLongArray(1, true); + sums = bigArrays.newDoubleArray(1, true); + mins = bigArrays.newDoubleArray(1, false); 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); } this.formatter = formatter; @@ -134,32 +133,32 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) { 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(); return new InternalStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal), mins.get(owningBucketOrdinal), - maxes.get(owningBucketOrdinal), formatter, getMetaData()); + maxes.get(owningBucketOrdinal), formatter, metaData()); } @Override 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> { + public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { public Factory(String name, ValuesSourceConfig valuesSourceConfig) { super(name, InternalStats.TYPE.name(), valuesSourceConfig); } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new StatsAggegator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new StatsAggegator(name, null, config.formatter(), aggregationContext, parent, metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new StatsAggegator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new StatsAggegator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java index e8a4b7e008a..e9bf1a8081b 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java @@ -53,20 +53,19 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue private DoubleArray sumOfSqrs; private ValueFormatter formatter; - public ExtendedStatsAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, - @Nullable ValueFormatter formatter, AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + public ExtendedStatsAggregator(String name, ValuesSource.Numeric valuesSource, + @Nullable ValueFormatter formatter, AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); this.valuesSource = valuesSource; this.formatter = formatter; if (valuesSource != null) { - final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - counts = bigArrays.newLongArray(initialSize, true); - sums = bigArrays.newDoubleArray(initialSize, true); - mins = bigArrays.newDoubleArray(initialSize, false); + counts = bigArrays.newLongArray(1, true); + sums = bigArrays.newDoubleArray(1, true); + mins = bigArrays.newDoubleArray(1, false); 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); - sumOfSqrs = bigArrays.newDoubleArray(initialSize, true); + sumOfSqrs = bigArrays.newDoubleArray(1, true); } } @@ -149,16 +148,16 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) { 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(); 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 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 @@ -166,20 +165,20 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue Releasables.close(counts, maxes, mins, sumOfSqrs, sums); } - public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly> { + public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { public Factory(String name, ValuesSourceConfig valuesSourceConfig) { super(name, InternalExtendedStats.TYPE.name(), valuesSourceConfig); } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new ExtendedStatsAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new ExtendedStatsAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new ExtendedStatsAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new ExtendedStatsAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java index 496c45719c3..d7708aaadc4 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java @@ -46,14 +46,13 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue { private DoubleArray sums; private ValueFormatter formatter; - public SumAggregator(String name, long estimatedBucketsCount, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + public SumAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, + AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); this.valuesSource = valuesSource; this.formatter = formatter; if (valuesSource != null) { - final long initialSize = estimatedBucketsCount < 2 ? 1 : estimatedBucketsCount; - sums = bigArrays.newDoubleArray(initialSize, true); + sums = bigArrays.newDoubleArray(1, true); } } @@ -87,30 +86,30 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) { 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 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> { + public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { public Factory(String name, ValuesSourceConfig valuesSourceConfig) { super(name, InternalSum.TYPE.name(), valuesSourceConfig); } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new SumAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new SumAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); } @Override - protected Aggregator create(ValuesSource.Numeric valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new SumAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new SumAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java index 823ce8f5236..c9d54d822bf 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java @@ -59,10 +59,10 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware private Scorer currentScorer; private LeafReaderContext currentContext; - public TopHitsAggregator(FetchPhase fetchPhase, SubSearchContext subSearchContext, String name, long estimatedBucketsCount, AggregationContext context, Aggregator parent, Map metaData) { - super(name, estimatedBucketsCount, context, parent, metaData); + public TopHitsAggregator(FetchPhase fetchPhase, SubSearchContext subSearchContext, String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { + super(name, context, parent, metaData); this.fetchPhase = fetchPhase; - topDocsCollectors = new LongObjectPagedHashMap<>(estimatedBucketsCount, context.bigArrays()); + topDocsCollectors = new LongObjectPagedHashMap<>(1, context.bigArrays()); this.subSearchContext = subSearchContext; context.registerScorerAware(this); } @@ -167,8 +167,8 @@ public class TopHitsAggregator extends MetricsAggregator implements ScorerAware } @Override - public Aggregator createInternal(AggregationContext aggregationContext, Aggregator parent, long expectedBucketsCount, Map metaData) { - return new TopHitsAggregator(fetchPhase, subSearchContext, name, expectedBucketsCount, aggregationContext, parent, metaData); + public Aggregator createInternal(AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new TopHitsAggregator(fetchPhase, subSearchContext, name, aggregationContext, parent, metaData); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java index 10febb775b4..2e9517894a2 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java @@ -50,15 +50,13 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue { LongArray counts; private ValueFormatter formatter; - public ValueCountAggregator(String name, long expectedBucketsCount, ValuesSource valuesSource, @Nullable ValueFormatter formatter, - AggregationContext aggregationContext, Aggregator parent, Map metaData) { - super(name, 0, aggregationContext, parent, metaData); + public ValueCountAggregator(String name, ValuesSource valuesSource, @Nullable ValueFormatter formatter, + AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + super(name, aggregationContext, parent, metaData); this.valuesSource = valuesSource; this.formatter = formatter; if (valuesSource != null) { - // expectedBucketsCount == 0 means it's a top level bucket - final long initialSize = expectedBucketsCount < 2 ? 1 : expectedBucketsCount; - counts = bigArrays.newLongArray(initialSize, true); + counts = bigArrays.newLongArray(1, true); } } @@ -87,15 +85,15 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) { if (valuesSource == null) { - return new InternalValueCount(name, 0, formatter, getMetaData()); + return new InternalValueCount(name, 0, formatter, metaData()); } assert owningBucketOrdinal < counts.size(); - return new InternalValueCount(name, counts.get(owningBucketOrdinal), formatter, getMetaData()); + return new InternalValueCount(name, counts.get(owningBucketOrdinal), formatter, metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalValueCount(name, 0l, formatter, getMetaData()); + return new InternalValueCount(name, 0l, formatter, metaData()); } @Override @@ -103,20 +101,20 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue { Releasables.close(counts); } - public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly> { + public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { public Factory(String name, ValuesSourceConfig config) { super(name, InternalValueCount.TYPE.name(), config); } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new ValueCountAggregator(name, 0, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + return new ValueCountAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); } @Override - protected Aggregator create(VS valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent, Map metaData) { - return new ValueCountAggregator(name, expectedBucketsCount, valuesSource, config.formatter(), aggregationContext, parent, + protected Aggregator doCreateInternal(VS valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + return new ValueCountAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java b/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java index d64edd5d31d..99eaccfc4f4 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java @@ -19,6 +19,7 @@ package org.elasticsearch.search.aggregations.support; import com.carrotsearch.hppc.ObjectObjectOpenHashMap; + import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.IndexReaderContext; 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.internal.SearchContext; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -44,7 +46,7 @@ import java.util.List; /** * */ -@SuppressWarnings({"unchecked", "ForLoopReplaceableByForEach"}) +@SuppressWarnings({"unchecked"}) public class AggregationContext implements ReaderContextAware, ScorerAware { 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; for (ReaderContextAware aware : readerAwares) { 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. */ - public VS valuesSource(ValuesSourceConfig config, int depth) { + public VS valuesSource(ValuesSourceConfig 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.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); } - private ValuesSource.Numeric numericScript(ValuesSourceConfig config) { + private ValuesSource.Numeric numericScript(ValuesSourceConfig config) throws IOException { setScorerIfNeeded(config.script); setReaderIfNeeded(config.script); scorerAwares.add(config.script); @@ -152,7 +154,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware { return source; } - private ValuesSource.Numeric numericField(ObjectObjectOpenHashMap fieldDataSources, ValuesSourceConfig config) { + private ValuesSource.Numeric numericField(ObjectObjectOpenHashMap fieldDataSources, ValuesSourceConfig config) throws IOException { final ConfigCacheKey cacheKey = new ConfigCacheKey(config); ValuesSource.Numeric dataSource = (ValuesSource.Numeric) fieldDataSources.get(cacheKey); if (dataSource == null) { @@ -172,7 +174,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware { return dataSource; } - private ValuesSource bytesField(ObjectObjectOpenHashMap fieldDataSources, ValuesSourceConfig config) { + private ValuesSource bytesField(ObjectObjectOpenHashMap fieldDataSources, ValuesSourceConfig config) throws IOException { final ConfigCacheKey cacheKey = new ConfigCacheKey(config); ValuesSource dataSource = fieldDataSources.get(cacheKey); if (dataSource == null) { @@ -202,7 +204,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware { return dataSource; } - private ValuesSource.Bytes bytesScript(ValuesSourceConfig config) { + private ValuesSource.Bytes bytesScript(ValuesSourceConfig config) throws IOException { setScorerIfNeeded(config.script); setReaderIfNeeded(config.script); scorerAwares.add(config.script); @@ -211,7 +213,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware { return source; } - private ValuesSource.GeoPoint geoPointField(ObjectObjectOpenHashMap fieldDataSources, ValuesSourceConfig config) { + private ValuesSource.GeoPoint geoPointField(ObjectObjectOpenHashMap fieldDataSources, ValuesSourceConfig config) throws IOException { final ConfigCacheKey cacheKey = new ConfigCacheKey(config); ValuesSource.GeoPoint dataSource = (ValuesSource.GeoPoint) fieldDataSources.get(cacheKey); if (dataSource == null) { @@ -224,7 +226,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware { return dataSource; } - public void registerReaderContextAware(ReaderContextAware readerContextAware) { + public void registerReaderContextAware(ReaderContextAware readerContextAware) throws IOException { setReaderIfNeeded(readerContextAware); readerAwares.add(readerContextAware); } @@ -234,7 +236,7 @@ public class AggregationContext implements ReaderContextAware, ScorerAware { scorerAwares.add(scorerAware); } - private void setReaderIfNeeded(ReaderContextAware readerAware) { + private void setReaderIfNeeded(ReaderContextAware readerAware) throws IOException { if (reader != null) { readerAware.setNextReader(reader); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java index 1c80d9bd298..f877a521f9f 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java @@ -21,14 +21,15 @@ package org.elasticsearch.search.aggregations.support; import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.support.format.ValueFormat; +import java.io.IOException; import java.util.Map; /** * */ -public abstract class ValuesSourceAggregatorFactory> extends AggregatorFactory { +public abstract class ValuesSourceAggregatorFactory extends AggregatorFactory { - public static abstract class LeafOnly> extends ValuesSourceAggregatorFactory { + public static abstract class LeafOnly extends ValuesSourceAggregatorFactory { protected LeafOnly(String name, String type, ValuesSourceConfig valuesSourceConfig) { super(name, type, valuesSourceConfig); @@ -48,12 +49,12 @@ public abstract class ValuesSourceAggregatorFactory metaData) { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { 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()); - return create(vs, expectedBucketsCount, context, parent, (M)metaData); + return doCreateInternal(vs, context, parent, collectsFromSingleBucket, metaData); } @Override @@ -63,9 +64,9 @@ public abstract class ValuesSourceAggregatorFactory 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 metaData) throws IOException; private void resolveValuesSourceConfigFromAncestors(String aggName, AggregatorFactory parent, Class requiredValuesSourceType) { ValuesSourceConfig config; diff --git a/src/main/java/org/elasticsearch/search/fetch/script/ScriptFieldsFetchSubPhase.java b/src/main/java/org/elasticsearch/search/fetch/script/ScriptFieldsFetchSubPhase.java index beb9e22b659..7a53898c1a7 100644 --- a/src/main/java/org/elasticsearch/search/fetch/script/ScriptFieldsFetchSubPhase.java +++ b/src/main/java/org/elasticsearch/search/fetch/script/ScriptFieldsFetchSubPhase.java @@ -21,6 +21,7 @@ package org.elasticsearch.search.fetch.script; import com.google.common.collect.ImmutableMap; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.search.SearchHitField; 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.SearchContext; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -70,7 +72,11 @@ public class ScriptFieldsFetchSubPhase implements FetchSubPhase { @Override public void hitExecute(SearchContext context, HitContext hitContext) throws ElasticsearchException { for (ScriptFieldsContext.ScriptField scriptField : context.scriptFields().fields()) { - scriptField.script().setNextReader(hitContext.readerContext()); + try { + scriptField.script().setNextReader(hitContext.readerContext()); + } catch (IOException e) { + throw new ElasticsearchIllegalStateException("IOException while calling setNextReader", e); + } scriptField.script().setNextDocId(hitContext.docId()); Object value; diff --git a/src/main/java/org/elasticsearch/search/sort/ScriptSortParser.java b/src/main/java/org/elasticsearch/search/sort/ScriptSortParser.java index 4035f60f4e2..2531d5f29eb 100644 --- a/src/main/java/org/elasticsearch/search/sort/ScriptSortParser.java +++ b/src/main/java/org/elasticsearch/search/sort/ScriptSortParser.java @@ -43,6 +43,7 @@ import org.elasticsearch.search.MultiValueMode; import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.internal.SearchContext; +import java.io.IOException; import java.util.Map; /** @@ -154,7 +155,7 @@ public class ScriptSortParser implements SortParser { case STRING_SORT_TYPE: fieldComparatorSource = new BytesRefFieldComparatorSource(null, null, sortMode, nested) { @Override - protected SortedBinaryDocValues getValues(LeafReaderContext context) { + protected SortedBinaryDocValues getValues(LeafReaderContext context) throws IOException { searchScript.setNextReader(context); final BinaryDocValues values = new BinaryDocValues() { final BytesRefBuilder spare = new BytesRefBuilder(); @@ -177,7 +178,7 @@ public class ScriptSortParser implements SortParser { // TODO: should we rather sort missing values last? fieldComparatorSource = new DoubleValuesComparatorSource(null, Double.MAX_VALUE, sortMode, nested) { @Override - protected SortedNumericDoubleValues getValues(LeafReaderContext context) { + protected SortedNumericDoubleValues getValues(LeafReaderContext context) throws IOException { searchScript.setNextReader(context); final NumericDoubleValues values = new NumericDoubleValues() { @Override